From e07f2b6f86477629dfd112cff1e09a5f6c804c9f Mon Sep 17 00:00:00 2001 From: Hou Zhijie 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 name RENAME TO < - The failover, - two_phase, and - retain_conflict_info + The failover + and two_phase parameters can only be altered when the subscription is disabled. 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