Commit ce0fdbfe authored by Amit Kapila's avatar Amit Kapila

Allow multiple xacts during table sync in logical replication.

For the initial table data synchronization in logical replication, we use
a single transaction to copy the entire table and then synchronize the
position in the stream with the main apply worker.

There are multiple downsides of this approach: (a) We have to perform the
entire copy operation again if there is any error (network breakdown,
error in the database operation, etc.) while we synchronize the WAL
position between tablesync worker and apply worker; this will be onerous
especially for large copies, (b) Using a single transaction in the
synchronization-phase (where we can receive WAL from multiple
transactions) will have the risk of exceeding the CID limit, (c) The slot
will hold the WAL till the entire sync is complete because we never commit
till the end.

This patch solves all the above downsides by allowing multiple
transactions during the tablesync phase. The initial copy is done in a
single transaction and after that, we commit each transaction as we
receive. To allow recovery after any error or crash, we use a permanent
slot and origin to track the progress. The slot and origin will be removed
once we finish the synchronization of the table. We also remove slot and
origin of tablesync workers if the user performs DROP SUBSCRIPTION .. or
ALTER SUBSCRIPTION .. REFERESH and some of the table syncs are still not
finished.

The commands ALTER SUBSCRIPTION ... REFRESH PUBLICATION and
ALTER SUBSCRIPTION ... SET PUBLICATION ... with refresh option as true
cannot be executed inside a transaction block because they can now drop
the slots for which we have no provision to rollback.

This will also open up the path for logical replication of 2PC
transactions on the subscriber side. Previously, we can't do that because
of the requirement of maintaining a single transaction in tablesync
workers.

Bump catalog version due to change of state in the catalog
(pg_subscription_rel).

Author: Peter Smith, Amit Kapila, and Takamichi Osumi
Reviewed-by: Ajin Cherian, Petr Jelinek, Hou Zhijie and Amit Kapila
Discussion: https://postgr.es/m/CAA4eK1KHJxaZS-fod-0fey=0tq3=Gkn4ho=8N4-5HWiCfu0H1A@mail.gmail.com
parent 3063eb17
......@@ -7673,6 +7673,7 @@ SCRAM-SHA-256$<replaceable>&lt;iteration count&gt;</replaceable>:<replaceable>&l
State code:
<literal>i</literal> = initialize,
<literal>d</literal> = data is being copied,
<literal>f</literal> = finished table copy,
<literal>s</literal> = synchronized,
<literal>r</literal> = ready (normal replication)
</para></entry>
......
......@@ -186,9 +186,10 @@
<para>
Each subscription will receive changes via one replication slot (see
<xref linkend="streaming-replication-slots"/>). Additional temporary
replication slots may be required for the initial data synchronization
of pre-existing table data.
<xref linkend="streaming-replication-slots"/>). Additional replication
slots may be required for the initial data synchronization of
pre-existing table data and those will be dropped at the end of data
synchronization.
</para>
<para>
......@@ -248,13 +249,23 @@
<para>
As mentioned earlier, each (active) subscription receives changes from a
replication slot on the remote (publishing) side. Normally, the remote
replication slot is created automatically when the subscription is created
using <command>CREATE SUBSCRIPTION</command> and it is dropped
automatically when the subscription is dropped using <command>DROP
SUBSCRIPTION</command>. In some situations, however, it can be useful or
necessary to manipulate the subscription and the underlying replication
slot separately. Here are some scenarios:
replication slot on the remote (publishing) side.
</para>
<para>
Additional table synchronization slots are normally transient, created
internally to perform initial table synchronization and dropped
automatically when they are no longer needed. These table synchronization
slots have generated names: <quote><literal>pg_%u_sync_%u_%llu</literal></quote>
(parameters: Subscription <parameter>oid</parameter>,
Table <parameter>relid</parameter>, system identifier <parameter>sysid</parameter>)
</para>
<para>
Normally, the remote replication slot is created automatically when the
subscription is created using <command>CREATE SUBSCRIPTION</command> and it
is dropped automatically when the subscription is dropped using
<command>DROP SUBSCRIPTION</command>. In some situations, however, it can
be useful or necessary to manipulate the subscription and the underlying
replication slot separately. Here are some scenarios:
<itemizedlist>
<listitem>
......@@ -294,8 +305,9 @@
using <command>ALTER SUBSCRIPTION</command> before attempting to drop
the subscription. If the remote database instance no longer exists, no
further action is then necessary. If, however, the remote database
instance is just unreachable, the replication slot should then be
dropped manually; otherwise it would continue to reserve WAL and might
instance is just unreachable, the replication slot (and any still
remaining table synchronization slots) should then be
dropped manually; otherwise it/they would continue to reserve WAL and might
eventually cause the disk to fill up. Such cases should be carefully
investigated.
</para>
......@@ -470,14 +482,17 @@
<para>
The initial data in existing subscribed tables are snapshotted and
copied in a parallel instance of a special kind of apply process.
This process will create its own temporary replication slot and
copy the existing data. Once existing data is copied, the worker
This process will create its own replication slot and copy the existing
data. As soon as the copy is finished the table contents will become
visible to other backends. Once existing data is copied, the worker
enters synchronization mode, which ensures that the table is brought
up to a synchronized state with the main apply process by streaming
any changes that happened during the initial data copy using standard
logical replication. Once the synchronization is done, the control
of the replication of the table is given back to the main apply
process where the replication continues as normal.
logical replication. During this synchronization phase, the changes
are applied and committed in the same order as they happened on the
publisher. Once the synchronization is done, the control of the
replication of the table is given back to the main apply process where
the replication continues as normal.
</para>
</sect2>
</sect1>
......
......@@ -48,6 +48,24 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
(Currently, all subscription owners must be superusers, so the owner checks
will be bypassed in practice. But this might change in the future.)
</para>
<para>
When refreshing a publication we remove the relations that are no longer
part of the publication and we also remove the tablesync slots if there are
any. It is necessary to remove tablesync slots so that the resources
allocated for the subscription on the remote host are released. If due to
network breakdown or some other error, <productname>PostgreSQL</productname>
is unable to remove the slots, an ERROR will be reported. To proceed in this
situation, either the user need to retry the operation or disassociate the
slot from the subscription and drop the subscription as explained in
<xref linkend="sql-dropsubscription"/>.
</para>
<para>
Commands <command>ALTER SUBSCRIPTION ... REFRESH PUBLICATION</command> and
<command>ALTER SUBSCRIPTION ... SET PUBLICATION ...</command> with refresh
option as true cannot be executed inside a transaction block.
</para>
</refsect1>
<refsect1>
......
......@@ -79,7 +79,8 @@ DROP SUBSCRIPTION [ IF EXISTS ] <replaceable class="parameter">name</replaceable
<para>
When dropping a subscription that is associated with a replication slot on
the remote host (the normal state), <command>DROP SUBSCRIPTION</command>
will connect to the remote host and try to drop the replication slot as
will connect to the remote host and try to drop the replication slot (and
any remaining table synchronization slots) as
part of its operation. This is necessary so that the resources allocated
for the subscription on the remote host are released. If this fails,
either because the remote host is not reachable or because the remote
......@@ -89,7 +90,8 @@ DROP SUBSCRIPTION [ IF EXISTS ] <replaceable class="parameter">name</replaceable
executing <literal>ALTER SUBSCRIPTION ... SET (slot_name = NONE)</literal>.
After that, <command>DROP SUBSCRIPTION</command> will no longer attempt any
actions on a remote host. Note that if the remote replication slot still
exists, it should then be dropped manually; otherwise it will continue to
exists, it (and any related table synchronization slots) should then be
dropped manually; otherwise it/they will continue to
reserve WAL and might eventually cause the disk to fill up. See
also <xref linkend="logical-replication-subscription-slot"/>.
</para>
......
......@@ -2432,15 +2432,6 @@ PrepareTransaction(void)
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot PREPARE a transaction that has exported snapshots")));
/*
* Don't allow PREPARE but for transaction that has/might kill logical
* replication workers.
*/
if (XactManipulatesLogicalReplicationWorkers())
ereport(ERROR,
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot PREPARE a transaction that has manipulated logical replication workers")));
/* Prevent cancel/die interrupt while cleaning up */
HOLD_INTERRUPTS();
......@@ -4899,7 +4890,6 @@ CommitSubTransaction(void)
AtEOSubXact_HashTables(true, s->nestingLevel);
AtEOSubXact_PgStat(true, s->nestingLevel);
AtSubCommit_Snapshot(s->nestingLevel);
AtEOSubXact_ApplyLauncher(true, s->nestingLevel);
/*
* We need to restore the upper transaction's read-only state, in case the
......@@ -5059,7 +5049,6 @@ AbortSubTransaction(void)
AtEOSubXact_HashTables(false, s->nestingLevel);
AtEOSubXact_PgStat(false, s->nestingLevel);
AtSubAbort_Snapshot(s->nestingLevel);
AtEOSubXact_ApplyLauncher(false, s->nestingLevel);
}
/*
......
......@@ -29,6 +29,7 @@
#include "utils/array.h"
#include "utils/builtins.h"
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/pg_lsn.h"
#include "utils/rel.h"
#include "utils/syscache.h"
......@@ -337,6 +338,13 @@ GetSubscriptionRelState(Oid subid, Oid relid, XLogRecPtr *sublsn)
char substate;
bool isnull;
Datum d;
Relation rel;
/*
* This is to avoid the race condition with AlterSubscription which tries
* to remove this relstate.
*/
rel = table_open(SubscriptionRelRelationId, AccessShareLock);
/* Try finding the mapping. */
tup = SearchSysCache2(SUBSCRIPTIONRELMAP,
......@@ -363,6 +371,8 @@ GetSubscriptionRelState(Oid subid, Oid relid, XLogRecPtr *sublsn)
/* Cleanup */
ReleaseSysCache(tup);
table_close(rel, AccessShareLock);
return substate;
}
......@@ -403,6 +413,34 @@ RemoveSubscriptionRel(Oid subid, Oid relid)
scan = table_beginscan_catalog(rel, nkeys, skey);
while (HeapTupleIsValid(tup = heap_getnext(scan, ForwardScanDirection)))
{
Form_pg_subscription_rel subrel;
subrel = (Form_pg_subscription_rel) GETSTRUCT(tup);
/*
* We don't allow to drop the relation mapping when the table
* synchronization is in progress unless the caller updates the
* corresponding subscription as well. This is to ensure that we don't
* leave tablesync slots or origins in the system when the
* corresponding table is dropped.
*/
if (!OidIsValid(subid) && subrel->srsubstate != SUBREL_STATE_READY)
{
ereport(ERROR,
(errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("could not drop relation mapping for subscription \"%s\"",
get_subscription_name(subrel->srsubid, false)),
errdetail("Table synchronization for relation \"%s\" is in progress and is in state \"%c\".",
get_rel_name(relid), subrel->srsubstate),
/*
* translator: first %s is a SQL ALTER command and second %s is a
* SQL DROP command
*/
errhint("Use %s to enable subscription if not already enabled or use %s to drop the subscription.",
"ALTER SUBSCRIPTION ... ENABLE",
"DROP SUBSCRIPTION ...")));
}
CatalogTupleDelete(rel, &tup->t_self);
}
table_endscan(scan);
......
This diff is collapsed.
......@@ -982,6 +982,7 @@ libpqrcv_exec(WalReceiverConn *conn, const char *query,
{
PGresult *pgres = NULL;
WalRcvExecResult *walres = palloc0(sizeof(WalRcvExecResult));
char *diag_sqlstate;
if (MyDatabaseId == InvalidOid)
ereport(ERROR,
......@@ -1025,6 +1026,13 @@ libpqrcv_exec(WalReceiverConn *conn, const char *query,
case PGRES_BAD_RESPONSE:
walres->status = WALRCV_ERROR;
walres->err = pchomp(PQerrorMessage(conn->streamConn));
diag_sqlstate = PQresultErrorField(pgres, PG_DIAG_SQLSTATE);
if (diag_sqlstate)
walres->sqlstate = MAKE_SQLSTATE(diag_sqlstate[0],
diag_sqlstate[1],
diag_sqlstate[2],
diag_sqlstate[3],
diag_sqlstate[4]);
break;
}
......
......@@ -73,20 +73,6 @@ typedef struct LogicalRepWorkerId
Oid relid;
} LogicalRepWorkerId;
typedef struct StopWorkersData
{
int nestDepth; /* Sub-transaction nest level */
List *workers; /* List of LogicalRepWorkerId */
struct StopWorkersData *parent; /* This need not be an immediate
* subtransaction parent */
} StopWorkersData;
/*
* Stack of StopWorkersData elements. Each stack element contains the workers
* to be stopped for that subtransaction.
*/
static StopWorkersData *on_commit_stop_workers = NULL;
static void ApplyLauncherWakeup(void);
static void logicalrep_launcher_onexit(int code, Datum arg);
static void logicalrep_worker_onexit(int code, Datum arg);
......@@ -546,51 +532,6 @@ logicalrep_worker_stop(Oid subid, Oid relid)
LWLockRelease(LogicalRepWorkerLock);
}
/*
* Request worker for specified sub/rel to be stopped on commit.
*/
void
logicalrep_worker_stop_at_commit(Oid subid, Oid relid)
{
int nestDepth = GetCurrentTransactionNestLevel();
LogicalRepWorkerId *wid;
MemoryContext oldctx;
/* Make sure we store the info in context that survives until commit. */
oldctx = MemoryContextSwitchTo(TopTransactionContext);
/* Check that previous transactions were properly cleaned up. */
Assert(on_commit_stop_workers == NULL ||
nestDepth >= on_commit_stop_workers->nestDepth);
/*
* Push a new stack element if we don't already have one for the current
* nestDepth.
*/
if (on_commit_stop_workers == NULL ||
nestDepth > on_commit_stop_workers->nestDepth)
{
StopWorkersData *newdata = palloc(sizeof(StopWorkersData));
newdata->nestDepth = nestDepth;
newdata->workers = NIL;
newdata->parent = on_commit_stop_workers;
on_commit_stop_workers = newdata;
}
/*
* Finally add a new worker into the worker list of the current
* subtransaction.
*/
wid = palloc(sizeof(LogicalRepWorkerId));
wid->subid = subid;
wid->relid = relid;
on_commit_stop_workers->workers =
lappend(on_commit_stop_workers->workers, wid);
MemoryContextSwitchTo(oldctx);
}
/*
* Wake up (using latch) any logical replication worker for specified sub/rel.
*/
......@@ -819,109 +760,21 @@ ApplyLauncherShmemInit(void)
}
}
/*
* Check whether current transaction has manipulated logical replication
* workers.
*/
bool
XactManipulatesLogicalReplicationWorkers(void)
{
return (on_commit_stop_workers != NULL);
}
/*
* Wakeup the launcher on commit if requested.
*/
void
AtEOXact_ApplyLauncher(bool isCommit)
{
Assert(on_commit_stop_workers == NULL ||
(on_commit_stop_workers->nestDepth == 1 &&
on_commit_stop_workers->parent == NULL));
if (isCommit)
{
ListCell *lc;
if (on_commit_stop_workers != NULL)
{
List *workers = on_commit_stop_workers->workers;
foreach(lc, workers)
{
LogicalRepWorkerId *wid = lfirst(lc);
logicalrep_worker_stop(wid->subid, wid->relid);
}
}
if (on_commit_launcher_wakeup)
ApplyLauncherWakeup();
}
/*
* No need to pfree on_commit_stop_workers. It was allocated in
* transaction memory context, which is going to be cleaned soon.
*/
on_commit_stop_workers = NULL;
on_commit_launcher_wakeup = false;
}
/*
* On commit, merge the current on_commit_stop_workers list into the
* immediate parent, if present.
* On rollback, discard the current on_commit_stop_workers list.
* Pop out the stack.
*/
void
AtEOSubXact_ApplyLauncher(bool isCommit, int nestDepth)
{
StopWorkersData *parent;
/* Exit immediately if there's no work to do at this level. */
if (on_commit_stop_workers == NULL ||
on_commit_stop_workers->nestDepth < nestDepth)
return;
Assert(on_commit_stop_workers->nestDepth == nestDepth);
parent = on_commit_stop_workers->parent;
if (isCommit)
{
/*
* If the upper stack element is not an immediate parent
* subtransaction, just decrement the notional nesting depth without
* doing any real work. Else, we need to merge the current workers
* list into the parent.
*/
if (!parent || parent->nestDepth < nestDepth - 1)
{
on_commit_stop_workers->nestDepth--;
return;
}
parent->workers =
list_concat(parent->workers, on_commit_stop_workers->workers);
}
else
{
/*
* Abandon everything that was done at this nesting level. Explicitly
* free memory to avoid a transaction-lifespan leak.
*/
list_free_deep(on_commit_stop_workers->workers);
}
/*
* We have taken care of the current subtransaction workers list for both
* abort or commit. So we are ready to pop the stack.
*/
pfree(on_commit_stop_workers);
on_commit_stop_workers = parent;
}
/*
* Request wakeup of the launcher on commit of the transaction.
*
......
This diff is collapsed.
......@@ -807,12 +807,8 @@ apply_handle_stream_stop(StringInfo s)
/* We must be in a valid transaction state */
Assert(IsTransactionState());
/* The synchronization worker runs in single transaction. */
if (!am_tablesync_worker())
{
/* Commit the per-stream transaction */
CommitTransactionCommand();
}
in_streamed_transaction = false;
......@@ -889,8 +885,6 @@ apply_handle_stream_abort(StringInfo s)
/* Cleanup the subxact info */
cleanup_subxact_info();
/* The synchronization worker runs in single transaction */
if (!am_tablesync_worker())
CommitTransactionCommand();
return;
}
......@@ -918,7 +912,6 @@ apply_handle_stream_abort(StringInfo s)
/* write the updated subxact list */
subxact_info_write(MyLogicalRepWorker->subid, xid);
if (!am_tablesync_worker())
CommitTransactionCommand();
}
}
......@@ -1062,8 +1055,7 @@ apply_handle_stream_commit(StringInfo s)
static void
apply_handle_commit_internal(StringInfo s, LogicalRepCommitData *commit_data)
{
/* The synchronization worker runs in single transaction. */
if (IsTransactionState() && !am_tablesync_worker())
if (IsTransactionState())
{
/*
* Update origin state so we can restart streaming from correct
......
......@@ -1786,7 +1786,8 @@ ProcessUtilitySlow(ParseState *pstate,
break;
case T_AlterSubscriptionStmt:
address = AlterSubscription((AlterSubscriptionStmt *) parsetree);
address = AlterSubscription((AlterSubscriptionStmt *) parsetree,
isTopLevel);
break;
case T_DropSubscriptionStmt:
......
......@@ -53,6 +53,6 @@
*/
/* yyyymmddN */
#define CATALOG_VERSION_NO 202102021
#define CATALOG_VERSION_NO 202102121
#endif
......@@ -61,6 +61,8 @@ DECLARE_UNIQUE_INDEX_PKEY(pg_subscription_rel_srrelid_srsubid_index, 6117, on pg
#define SUBREL_STATE_INIT 'i' /* initializing (sublsn NULL) */
#define SUBREL_STATE_DATASYNC 'd' /* data is being synchronized (sublsn
* NULL) */
#define SUBREL_STATE_FINISHEDCOPY 'f' /* tablesync copy phase is completed
* (sublsn NULL) */
#define SUBREL_STATE_SYNCDONE 's' /* synchronization finished in front of
* apply (sublsn set) */
#define SUBREL_STATE_READY 'r' /* ready (sublsn set) */
......
......@@ -20,7 +20,7 @@
extern ObjectAddress CreateSubscription(CreateSubscriptionStmt *stmt,
bool isTopLevel);
extern ObjectAddress AlterSubscription(AlterSubscriptionStmt *stmt);
extern ObjectAddress AlterSubscription(AlterSubscriptionStmt *stmt, bool isTopLevel);
extern void DropSubscription(DropSubscriptionStmt *stmt, bool isTopLevel);
extern ObjectAddress AlterSubscriptionOwner(const char *name, Oid newOwnerId);
......
......@@ -22,9 +22,7 @@ extern Size ApplyLauncherShmemSize(void);
extern void ApplyLauncherShmemInit(void);
extern void ApplyLauncherWakeupAtCommit(void);
extern bool XactManipulatesLogicalReplicationWorkers(void);
extern void AtEOXact_ApplyLauncher(bool isCommit);
extern void AtEOSubXact_ApplyLauncher(bool isCommit, int nestDepth);
extern bool IsLogicalLauncher(void);
......
......@@ -15,6 +15,7 @@
#include "storage/lwlock.h"
#include "storage/shmem.h"
#include "storage/spin.h"
#include "replication/walreceiver.h"
/*
* Behaviour of replication slots, upon release or crash.
......@@ -211,6 +212,8 @@ extern bool ReplicationSlotsCountDBSlots(Oid dboid, int *nslots, int *nactive);
extern void ReplicationSlotsDropDBSlots(Oid dboid);
extern void InvalidateObsoleteReplicationSlots(XLogSegNo oldestSegno);
extern ReplicationSlot *SearchNamedReplicationSlot(const char *name);
extern char *ReplicationSlotNameForTablesync(Oid suboid, Oid relid, char *syncslotname);
extern void ReplicationSlotDropAtPubNode(WalReceiverConn *wrconn, char *slotname, bool missing_ok);
extern void StartupReplicationSlots(void);
extern void CheckPointReplicationSlots(void);
......
......@@ -210,6 +210,7 @@ typedef enum
typedef struct WalRcvExecResult
{
WalRcvExecStatus status;
int sqlstate;
char *err;
Tuplestorestate *tuplestore;
TupleDesc tupledesc;
......
......@@ -77,13 +77,14 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running);
extern void logicalrep_worker_launch(Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_worker_stop_at_commit(Oid subid, Oid relid);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
extern void logicalrep_worker_wakeup_ptr(LogicalRepWorker *worker);
extern int logicalrep_sync_worker_count(Oid subid);
extern void ReplicationOriginNameForTablesync(Oid suboid, Oid relid, char *originname);
extern char *LogicalRepSyncTableStart(XLogRecPtr *origin_startpos);
void process_syncing_tables(XLogRecPtr current_lsn);
void invalidate_syncing_table_states(Datum arg, int cacheid,
uint32 hashvalue);
......
......@@ -201,6 +201,27 @@ ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
(1 row)
DROP SUBSCRIPTION regress_testsub;
CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=postgres' PUBLICATION mypub
WITH (enabled = true, create_slot = false, copy_data = false);
-- fail - ALTER SUBSCRIPTION with refresh is not allowed in a transaction
-- block or function
BEGIN;
ALTER SUBSCRIPTION regress_testsub SET PUBLICATION mypub WITH (refresh = true);
ERROR: ALTER SUBSCRIPTION with refresh cannot run inside a transaction block
END;
BEGIN;
ALTER SUBSCRIPTION regress_testsub REFRESH PUBLICATION;
ERROR: ALTER SUBSCRIPTION ... REFRESH cannot run inside a transaction block
END;
CREATE FUNCTION func() RETURNS VOID AS
$$ ALTER SUBSCRIPTION regress_testsub SET PUBLICATION mypub WITH (refresh = true) $$ LANGUAGE SQL;
SELECT func();
ERROR: ALTER SUBSCRIPTION with refresh cannot be executed from a function
CONTEXT: SQL function "func" statement 1
ALTER SUBSCRIPTION regress_testsub DISABLE;
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
DROP FUNCTION func;
RESET SESSION AUTHORIZATION;
DROP ROLE regress_subscription_user;
DROP ROLE regress_subscription_user2;
......
......@@ -147,6 +147,28 @@ ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=postgres' PUBLICATION mypub
WITH (enabled = true, create_slot = false, copy_data = false);
-- fail - ALTER SUBSCRIPTION with refresh is not allowed in a transaction
-- block or function
BEGIN;
ALTER SUBSCRIPTION regress_testsub SET PUBLICATION mypub WITH (refresh = true);
END;
BEGIN;
ALTER SUBSCRIPTION regress_testsub REFRESH PUBLICATION;
END;
CREATE FUNCTION func() RETURNS VOID AS
$$ ALTER SUBSCRIPTION regress_testsub SET PUBLICATION mypub WITH (refresh = true) $$ LANGUAGE SQL;
SELECT func();
ALTER SUBSCRIPTION regress_testsub DISABLE;
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
DROP FUNCTION func;
RESET SESSION AUTHORIZATION;
DROP ROLE regress_subscription_user;
DROP ROLE regress_subscription_user2;
......
......@@ -3,7 +3,7 @@ use strict;
use warnings;
use PostgresNode;
use TestLib;
use Test::More tests => 7;
use Test::More tests => 8;
# Initialize publisher node
my $node_publisher = get_new_node('publisher');
......@@ -149,7 +149,26 @@ $result = $node_subscriber->safe_psql('postgres',
is($result, qq(20),
'changes for table added after subscription initialized replicated');
# clean up
$node_publisher->safe_psql('postgres', "DROP TABLE tab_rep_next");
$node_subscriber->safe_psql('postgres', "DROP TABLE tab_rep_next");
$node_subscriber->safe_psql('postgres', "DROP SUBSCRIPTION tap_sub");
# Table tap_rep already has the same records on both publisher and subscriber
# at this time. Recreate the subscription which will do the initial copy of
# the table again and fails due to unique constraint violation.
$node_subscriber->safe_psql('postgres',
"CREATE SUBSCRIPTION tap_sub CONNECTION '$publisher_connstr' PUBLICATION tap_pub");
$result = $node_subscriber->poll_query_until('postgres', $started_query)
or die "Timed out while waiting for subscriber to start sync";
# DROP SUBSCRIPTION must clean up slots on the publisher side when the
# subscriber is stuck on data copy for constraint violation.
$node_subscriber->safe_psql('postgres', "DROP SUBSCRIPTION tap_sub");
$result = $node_publisher->safe_psql('postgres', "SELECT count(*) FROM pg_replication_slots");
is($result, qq(0), 'DROP SUBSCRIPTION during error can clean up the slots on the publisher');
$node_subscriber->stop('fast');
$node_publisher->stop('fast');
......@@ -2397,7 +2397,6 @@ StdAnalyzeData
StdRdOptions
Step
StopList
StopWorkersData
StrategyNumber
StreamCtl
StreamXidHash
......@@ -2408,6 +2407,7 @@ SubLink
SubLinkType
SubPlan
SubPlanState
SubRemoveRels
SubTransactionId
SubXactCallback
SubXactCallbackItem
......
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment