1
0
mirror of https://github.com/postgres/postgres.git synced 2025-11-09 06:21:09 +03:00

Allow specifying column lists for logical replication

This allows specifying an optional column list when adding a table to
logical replication. The column list may be specified after the table
name, enclosed in parentheses. Columns not included in this list are not
sent to the subscriber, allowing the schema on the subscriber to be a
subset of the publisher schema.

For UPDATE/DELETE publications, the column list needs to cover all
REPLICA IDENTITY columns. For INSERT publications, the column list is
arbitrary and may omit some REPLICA IDENTITY columns. Furthermore, if
the table uses REPLICA IDENTITY FULL, column list is not allowed.

The column list can contain only simple column references. Complex
expressions, function calls etc. are not allowed. This restriction could
be relaxed in the future.

During the initial table synchronization, only columns included in the
column list are copied to the subscriber. If the subscription has
several publications, containing the same table with different column
lists, columns specified in any of the lists will be copied.

This means all columns are replicated if the table has no column list
at all (which is treated as column list with all columns), or when of
the publications is defined as FOR ALL TABLES (possibly IN SCHEMA that
matches the schema of the table).

For partitioned tables, publish_via_partition_root determines whether
the column list for the root or the leaf relation will be used. If the
parameter is 'false' (the default), the list defined for the leaf
relation is used. Otherwise, the column list for the root partition
will be used.

Psql commands \dRp+ and \d <table-name> now display any column lists.

Author: Tomas Vondra, Alvaro Herrera, Rahila Syed
Reviewed-by: Peter Eisentraut, Alvaro Herrera, Vignesh C, Ibrar Ahmed,
Amit Kapila, Hou zj, Peter Smith, Wang wei, Tang, Shi yu
Discussion: https://postgr.es/m/CAH2L28vddB_NFdRVpuyRBJEBWjz4BSyTB=_ektNRH8NJ1jf95g@mail.gmail.com
This commit is contained in:
Tomas Vondra
2022-03-26 00:45:21 +01:00
parent 05843b1aa4
commit 923def9a53
26 changed files with 2833 additions and 92 deletions

View File

@@ -29,16 +29,30 @@
#define TRUNCATE_CASCADE (1<<0)
#define TRUNCATE_RESTART_SEQS (1<<1)
static void logicalrep_write_attrs(StringInfo out, Relation rel);
static void logicalrep_write_attrs(StringInfo out, Relation rel,
Bitmapset *columns);
static void logicalrep_write_tuple(StringInfo out, Relation rel,
TupleTableSlot *slot,
bool binary);
bool binary, Bitmapset *columns);
static void logicalrep_read_attrs(StringInfo in, LogicalRepRelation *rel);
static void logicalrep_read_tuple(StringInfo in, LogicalRepTupleData *tuple);
static void logicalrep_write_namespace(StringInfo out, Oid nspid);
static const char *logicalrep_read_namespace(StringInfo in);
/*
* Check if a column is covered by a column list.
*
* Need to be careful about NULL, which is treated as a column list covering
* all columns.
*/
static bool
column_in_column_list(int attnum, Bitmapset *columns)
{
return (columns == NULL || bms_is_member(attnum, columns));
}
/*
* Write BEGIN to the output stream.
*/
@@ -398,7 +412,7 @@ logicalrep_read_origin(StringInfo in, XLogRecPtr *origin_lsn)
*/
void
logicalrep_write_insert(StringInfo out, TransactionId xid, Relation rel,
TupleTableSlot *newslot, bool binary)
TupleTableSlot *newslot, bool binary, Bitmapset *columns)
{
pq_sendbyte(out, LOGICAL_REP_MSG_INSERT);
@@ -410,7 +424,7 @@ logicalrep_write_insert(StringInfo out, TransactionId xid, Relation rel,
pq_sendint32(out, RelationGetRelid(rel));
pq_sendbyte(out, 'N'); /* new tuple follows */
logicalrep_write_tuple(out, rel, newslot, binary);
logicalrep_write_tuple(out, rel, newslot, binary, columns);
}
/*
@@ -443,7 +457,7 @@ logicalrep_read_insert(StringInfo in, LogicalRepTupleData *newtup)
void
logicalrep_write_update(StringInfo out, TransactionId xid, Relation rel,
TupleTableSlot *oldslot, TupleTableSlot *newslot,
bool binary)
bool binary, Bitmapset *columns)
{
pq_sendbyte(out, LOGICAL_REP_MSG_UPDATE);
@@ -464,11 +478,11 @@ logicalrep_write_update(StringInfo out, TransactionId xid, Relation rel,
pq_sendbyte(out, 'O'); /* old tuple follows */
else
pq_sendbyte(out, 'K'); /* old key follows */
logicalrep_write_tuple(out, rel, oldslot, binary);
logicalrep_write_tuple(out, rel, oldslot, binary, NULL);
}
pq_sendbyte(out, 'N'); /* new tuple follows */
logicalrep_write_tuple(out, rel, newslot, binary);
logicalrep_write_tuple(out, rel, newslot, binary, columns);
}
/*
@@ -537,7 +551,7 @@ logicalrep_write_delete(StringInfo out, TransactionId xid, Relation rel,
else
pq_sendbyte(out, 'K'); /* old key follows */
logicalrep_write_tuple(out, rel, oldslot, binary);
logicalrep_write_tuple(out, rel, oldslot, binary, NULL);
}
/*
@@ -702,7 +716,8 @@ logicalrep_read_sequence(StringInfo in, LogicalRepSequence *seqdata)
* Write relation description to the output stream.
*/
void
logicalrep_write_rel(StringInfo out, TransactionId xid, Relation rel)
logicalrep_write_rel(StringInfo out, TransactionId xid, Relation rel,
Bitmapset *columns)
{
char *relname;
@@ -724,7 +739,7 @@ logicalrep_write_rel(StringInfo out, TransactionId xid, Relation rel)
pq_sendbyte(out, rel->rd_rel->relreplident);
/* send the attribute info */
logicalrep_write_attrs(out, rel);
logicalrep_write_attrs(out, rel, columns);
}
/*
@@ -801,7 +816,7 @@ logicalrep_read_typ(StringInfo in, LogicalRepTyp *ltyp)
*/
static void
logicalrep_write_tuple(StringInfo out, Relation rel, TupleTableSlot *slot,
bool binary)
bool binary, Bitmapset *columns)
{
TupleDesc desc;
Datum *values;
@@ -813,8 +828,14 @@ logicalrep_write_tuple(StringInfo out, Relation rel, TupleTableSlot *slot,
for (i = 0; i < desc->natts; i++)
{
if (TupleDescAttr(desc, i)->attisdropped || TupleDescAttr(desc, i)->attgenerated)
Form_pg_attribute att = TupleDescAttr(desc, i);
if (att->attisdropped || att->attgenerated)
continue;
if (!column_in_column_list(att->attnum, columns))
continue;
nliveatts++;
}
pq_sendint16(out, nliveatts);
@@ -833,6 +854,9 @@ logicalrep_write_tuple(StringInfo out, Relation rel, TupleTableSlot *slot,
if (att->attisdropped || att->attgenerated)
continue;
if (!column_in_column_list(att->attnum, columns))
continue;
if (isnull[i])
{
pq_sendbyte(out, LOGICALREP_COLUMN_NULL);
@@ -954,7 +978,7 @@ logicalrep_read_tuple(StringInfo in, LogicalRepTupleData *tuple)
* Write relation attribute metadata to the stream.
*/
static void
logicalrep_write_attrs(StringInfo out, Relation rel)
logicalrep_write_attrs(StringInfo out, Relation rel, Bitmapset *columns)
{
TupleDesc desc;
int i;
@@ -967,8 +991,14 @@ logicalrep_write_attrs(StringInfo out, Relation rel)
/* send number of live attributes */
for (i = 0; i < desc->natts; i++)
{
if (TupleDescAttr(desc, i)->attisdropped || TupleDescAttr(desc, i)->attgenerated)
Form_pg_attribute att = TupleDescAttr(desc, i);
if (att->attisdropped || att->attgenerated)
continue;
if (!column_in_column_list(att->attnum, columns))
continue;
nliveatts++;
}
pq_sendint16(out, nliveatts);
@@ -987,6 +1017,9 @@ logicalrep_write_attrs(StringInfo out, Relation rel)
if (att->attisdropped || att->attgenerated)
continue;
if (!column_in_column_list(att->attnum, columns))
continue;
/* REPLICA IDENTITY FULL means all columns are sent as part of key. */
if (replidentfull ||
bms_is_member(att->attnum - FirstLowInvalidHeapAttributeNumber,

View File

@@ -113,6 +113,7 @@
#include "storage/ipc.h"
#include "storage/lmgr.h"
#include "utils/acl.h"
#include "utils/array.h"
#include "utils/builtins.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
@@ -702,12 +703,13 @@ fetch_remote_table_info(char *nspname, char *relname,
StringInfoData cmd;
TupleTableSlot *slot;
Oid tableRow[] = {OIDOID, CHAROID, CHAROID};
Oid attrRow[] = {TEXTOID, OIDOID, BOOLOID};
Oid attrRow[] = {INT2OID, TEXTOID, OIDOID, BOOLOID};
Oid qualRow[] = {TEXTOID};
bool isnull;
int natt;
ListCell *lc;
bool first;
Bitmapset *included_cols = NULL;
lrel->nspname = nspname;
lrel->relname = relname;
@@ -748,10 +750,110 @@ fetch_remote_table_info(char *nspname, char *relname,
ExecDropSingleTupleTableSlot(slot);
walrcv_clear_result(res);
/* Now fetch columns. */
/*
* Get column lists for each relation.
*
* For initial synchronization, column lists can be ignored in following
* cases:
*
* 1) one of the subscribed publications for the table hasn't specified
* any column list
*
* 2) one of the subscribed publications has puballtables set to true
*
* 3) one of the subscribed publications is declared as ALL TABLES IN
* SCHEMA that includes this relation
*
* We need to do this before fetching info about column names and types,
* so that we can skip columns that should not be replicated.
*/
if (walrcv_server_version(LogRepWorkerWalRcvConn) >= 150000)
{
WalRcvExecResult *pubres;
TupleTableSlot *slot;
Oid attrsRow[] = {INT2OID};
StringInfoData pub_names;
bool first = true;
initStringInfo(&pub_names);
foreach(lc, MySubscription->publications)
{
if (!first)
appendStringInfo(&pub_names, ", ");
appendStringInfoString(&pub_names, quote_literal_cstr(strVal(lfirst(lc))));
first = false;
}
/*
* Fetch info about column lists for the relation (from all the
* publications). We unnest the int2vector values, because that
* makes it easier to combine lists by simply adding the attnums
* to a new bitmap (without having to parse the int2vector data).
* This preserves NULL values, so that if one of the publications
* has no column list, we'll know that.
*/
resetStringInfo(&cmd);
appendStringInfo(&cmd,
"SELECT DISTINCT unnest"
" FROM pg_publication p"
" LEFT OUTER JOIN pg_publication_rel pr"
" ON (p.oid = pr.prpubid AND pr.prrelid = %u)"
" LEFT OUTER JOIN unnest(pr.prattrs) ON TRUE,"
" LATERAL pg_get_publication_tables(p.pubname) gpt"
" WHERE gpt.relid = %u"
" AND p.pubname IN ( %s )",
lrel->remoteid,
lrel->remoteid,
pub_names.data);
pubres = walrcv_exec(LogRepWorkerWalRcvConn, cmd.data,
lengthof(attrsRow), attrsRow);
if (pubres->status != WALRCV_OK_TUPLES)
ereport(ERROR,
(errcode(ERRCODE_CONNECTION_FAILURE),
errmsg("could not fetch column list info for table \"%s.%s\" from publisher: %s",
nspname, relname, pubres->err)));
/*
* Merge the column lists (from different publications) by creating
* a single bitmap with all the attnums. If we find a NULL value,
* that means one of the publications has no column list for the
* table we're syncing.
*/
slot = MakeSingleTupleTableSlot(pubres->tupledesc, &TTSOpsMinimalTuple);
while (tuplestore_gettupleslot(pubres->tuplestore, true, false, slot))
{
Datum cfval = slot_getattr(slot, 1, &isnull);
/* NULL means empty column list, so we're done. */
if (isnull)
{
bms_free(included_cols);
included_cols = NULL;
break;
}
included_cols = bms_add_member(included_cols,
DatumGetInt16(cfval));
ExecClearTuple(slot);
}
ExecDropSingleTupleTableSlot(slot);
walrcv_clear_result(pubres);
pfree(pub_names.data);
}
/*
* Now fetch column names and types.
*/
resetStringInfo(&cmd);
appendStringInfo(&cmd,
"SELECT a.attname,"
"SELECT a.attnum,"
" a.attname,"
" a.atttypid,"
" a.attnum = ANY(i.indkey)"
" FROM pg_catalog.pg_attribute a"
@@ -779,16 +881,35 @@ fetch_remote_table_info(char *nspname, char *relname,
lrel->atttyps = palloc0(MaxTupleAttributeNumber * sizeof(Oid));
lrel->attkeys = NULL;
/*
* Store the columns as a list of names. Ignore those that are not
* present in the column list, if there is one.
*/
natt = 0;
slot = MakeSingleTupleTableSlot(res->tupledesc, &TTSOpsMinimalTuple);
while (tuplestore_gettupleslot(res->tuplestore, true, false, slot))
{
lrel->attnames[natt] =
TextDatumGetCString(slot_getattr(slot, 1, &isnull));
char *rel_colname;
AttrNumber attnum;
attnum = DatumGetInt16(slot_getattr(slot, 1, &isnull));
Assert(!isnull);
lrel->atttyps[natt] = DatumGetObjectId(slot_getattr(slot, 2, &isnull));
/* If the column is not in the column list, skip it. */
if (included_cols != NULL && !bms_is_member(attnum, included_cols))
{
ExecClearTuple(slot);
continue;
}
rel_colname = TextDatumGetCString(slot_getattr(slot, 2, &isnull));
Assert(!isnull);
if (DatumGetBool(slot_getattr(slot, 3, &isnull)))
lrel->attnames[natt] = rel_colname;
lrel->atttyps[natt] = DatumGetObjectId(slot_getattr(slot, 3, &isnull));
Assert(!isnull);
if (DatumGetBool(slot_getattr(slot, 4, &isnull)))
lrel->attkeys = bms_add_member(lrel->attkeys, natt);
/* Should never happen. */
@@ -931,8 +1052,24 @@ copy_table(Relation rel)
/* Regular table with no row filter */
if (lrel.relkind == RELKIND_RELATION && qual == NIL)
appendStringInfo(&cmd, "COPY %s TO STDOUT",
{
appendStringInfo(&cmd, "COPY %s (",
quote_qualified_identifier(lrel.nspname, lrel.relname));
/*
* XXX Do we need to list the columns in all cases? Maybe we're replicating
* all columns?
*/
for (int i = 0; i < lrel.natts; i++)
{
if (i > 0)
appendStringInfoString(&cmd, ", ");
appendStringInfoString(&cmd, quote_identifier(lrel.attnames[i]));
}
appendStringInfo(&cmd, ") TO STDOUT");
}
else
{
/*

View File

@@ -30,6 +30,7 @@
#include "utils/inval.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
#include "utils/rel.h"
#include "utils/syscache.h"
#include "utils/varlena.h"
@@ -90,7 +91,8 @@ static List *LoadPublications(List *pubnames);
static void publication_invalidation_cb(Datum arg, int cacheid,
uint32 hashvalue);
static void send_relation_and_attrs(Relation relation, TransactionId xid,
LogicalDecodingContext *ctx);
LogicalDecodingContext *ctx,
Bitmapset *columns);
static void send_repl_origin(LogicalDecodingContext *ctx,
RepOriginId origin_id, XLogRecPtr origin_lsn,
bool send_origin);
@@ -148,9 +150,6 @@ typedef struct RelationSyncEntry
*/
ExprState *exprstate[NUM_ROWFILTER_PUBACTIONS];
EState *estate; /* executor state used for row filter */
MemoryContext cache_expr_cxt; /* private context for exprstate and
* estate, if any */
TupleTableSlot *new_slot; /* slot for storing new tuple */
TupleTableSlot *old_slot; /* slot for storing old tuple */
@@ -169,6 +168,19 @@ typedef struct RelationSyncEntry
* having identical TupleDesc.
*/
AttrMap *attrmap;
/*
* Columns included in the publication, or NULL if all columns are
* included implicitly. Note that the attnums in this bitmap are not
* shifted by FirstLowInvalidHeapAttributeNumber.
*/
Bitmapset *columns;
/*
* Private context to store additional data for this entry - state for
* the row filter expressions, column list, etc.
*/
MemoryContext entry_cxt;
} RelationSyncEntry;
/* Map used to remember which relation schemas we sent. */
@@ -200,6 +212,11 @@ static bool pgoutput_row_filter(Relation relation, TupleTableSlot *old_slot,
RelationSyncEntry *entry,
ReorderBufferChangeType *action);
/* column list routines */
static void pgoutput_column_list_init(PGOutputData *data,
List *publications,
RelationSyncEntry *entry);
/*
* Specify output plugin callbacks
*/
@@ -622,11 +639,11 @@ maybe_send_schema(LogicalDecodingContext *ctx,
{
Relation ancestor = RelationIdGetRelation(relentry->publish_as_relid);
send_relation_and_attrs(ancestor, xid, ctx);
send_relation_and_attrs(ancestor, xid, ctx, relentry->columns);
RelationClose(ancestor);
}
send_relation_and_attrs(relation, xid, ctx);
send_relation_and_attrs(relation, xid, ctx, relentry->columns);
if (in_streaming)
set_schema_sent_in_streamed_txn(relentry, topxid);
@@ -639,7 +656,8 @@ maybe_send_schema(LogicalDecodingContext *ctx,
*/
static void
send_relation_and_attrs(Relation relation, TransactionId xid,
LogicalDecodingContext *ctx)
LogicalDecodingContext *ctx,
Bitmapset *columns)
{
TupleDesc desc = RelationGetDescr(relation);
int i;
@@ -662,13 +680,17 @@ send_relation_and_attrs(Relation relation, TransactionId xid,
if (att->atttypid < FirstGenbkiObjectId)
continue;
/* Skip this attribute if it's not present in the column list */
if (columns != NULL && !bms_is_member(att->attnum, columns))
continue;
OutputPluginPrepareWrite(ctx, false);
logicalrep_write_typ(ctx->out, xid, att->atttypid);
OutputPluginWrite(ctx, false);
}
OutputPluginPrepareWrite(ctx, false);
logicalrep_write_rel(ctx->out, xid, relation);
logicalrep_write_rel(ctx->out, xid, relation, columns);
OutputPluginWrite(ctx, false);
}
@@ -722,6 +744,28 @@ pgoutput_row_filter_exec_expr(ExprState *state, ExprContext *econtext)
return DatumGetBool(ret);
}
/*
* Make sure the per-entry memory context exists.
*/
static void
pgoutput_ensure_entry_cxt(PGOutputData *data, RelationSyncEntry *entry)
{
Relation relation;
/* The context may already exist, in which case bail out. */
if (entry->entry_cxt)
return;
relation = RelationIdGetRelation(entry->publish_as_relid);
entry->entry_cxt = AllocSetContextCreate(data->cachectx,
"entry private context",
ALLOCSET_SMALL_SIZES);
MemoryContextCopyAndSetIdentifier(entry->entry_cxt,
RelationGetRelationName(relation));
}
/*
* Initialize the row filter.
*/
@@ -842,21 +886,13 @@ pgoutput_row_filter_init(PGOutputData *data, List *publications,
{
Relation relation = RelationIdGetRelation(entry->publish_as_relid);
Assert(entry->cache_expr_cxt == NULL);
/* Create the memory context for row filters */
entry->cache_expr_cxt = AllocSetContextCreate(data->cachectx,
"Row filter expressions",
ALLOCSET_DEFAULT_SIZES);
MemoryContextCopyAndSetIdentifier(entry->cache_expr_cxt,
RelationGetRelationName(relation));
pgoutput_ensure_entry_cxt(data, entry);
/*
* Now all the filters for all pubactions are known. Combine them when
* their pubactions are the same.
*/
oldctx = MemoryContextSwitchTo(entry->cache_expr_cxt);
oldctx = MemoryContextSwitchTo(entry->entry_cxt);
entry->estate = create_estate_for_relation(relation);
for (idx = 0; idx < NUM_ROWFILTER_PUBACTIONS; idx++)
{
@@ -879,6 +915,105 @@ pgoutput_row_filter_init(PGOutputData *data, List *publications,
}
}
/*
* Initialize the column list.
*/
static void
pgoutput_column_list_init(PGOutputData *data, List *publications,
RelationSyncEntry *entry)
{
ListCell *lc;
/*
* Find if there are any column lists for this relation. If there are,
* build a bitmap merging all the column lists.
*
* All the given publication-table mappings must be checked.
*
* Multiple publications might have multiple column lists for this relation.
*
* FOR ALL TABLES and FOR ALL TABLES IN SCHEMA implies "don't use column
* list" so it takes precedence.
*/
foreach(lc, publications)
{
Publication *pub = lfirst(lc);
HeapTuple cftuple = NULL;
Datum cfdatum = 0;
/*
* Assume there's no column list. Only if we find pg_publication_rel
* entry with a column list we'll switch it to false.
*/
bool pub_no_list = true;
/*
* If the publication is FOR ALL TABLES then it is treated the same as if
* there are no column lists (even if other publications have a list).
*/
if (!pub->alltables)
{
/*
* Check for the presence of a column list in this publication.
*
* Note: If we find no pg_publication_rel row, it's a publication
* defined for a whole schema, so it can't have a column list, just
* like a FOR ALL TABLES publication.
*/
cftuple = SearchSysCache2(PUBLICATIONRELMAP,
ObjectIdGetDatum(entry->publish_as_relid),
ObjectIdGetDatum(pub->oid));
if (HeapTupleIsValid(cftuple))
{
/*
* Lookup the column list attribute.
*
* Note: We update the pub_no_list value directly, because if
* the value is NULL, we have no list (and vice versa).
*/
cfdatum = SysCacheGetAttr(PUBLICATIONRELMAP, cftuple,
Anum_pg_publication_rel_prattrs,
&pub_no_list);
/*
* Build the column list bitmap in the per-entry context.
*
* We need to merge column lists from all publications, so we
* update the same bitmapset. If the column list is null, we
* interpret it as replicating all columns.
*/
if (!pub_no_list) /* when not null */
{
pgoutput_ensure_entry_cxt(data, entry);
entry->columns = pub_collist_to_bitmapset(entry->columns,
cfdatum,
entry->entry_cxt);
}
}
}
/*
* Found a publication with no column list, so we're done. But first
* discard column list we might have from preceding publications.
*/
if (pub_no_list)
{
if (cftuple)
ReleaseSysCache(cftuple);
bms_free(entry->columns);
entry->columns = NULL;
break;
}
ReleaseSysCache(cftuple);
} /* loop all subscribed publications */
}
/*
* Initialize the slot for storing new and old tuples, and build the map that
* will be used to convert the relation's tuples into the ancestor's format.
@@ -1243,7 +1378,7 @@ pgoutput_change(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
OutputPluginPrepareWrite(ctx, true);
logicalrep_write_insert(ctx->out, xid, targetrel, new_slot,
data->binary);
data->binary, relentry->columns);
OutputPluginWrite(ctx, true);
break;
case REORDER_BUFFER_CHANGE_UPDATE:
@@ -1297,11 +1432,13 @@ pgoutput_change(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
{
case REORDER_BUFFER_CHANGE_INSERT:
logicalrep_write_insert(ctx->out, xid, targetrel,
new_slot, data->binary);
new_slot, data->binary,
relentry->columns);
break;
case REORDER_BUFFER_CHANGE_UPDATE:
logicalrep_write_update(ctx->out, xid, targetrel,
old_slot, new_slot, data->binary);
old_slot, new_slot, data->binary,
relentry->columns);
break;
case REORDER_BUFFER_CHANGE_DELETE:
logicalrep_write_delete(ctx->out, xid, targetrel,
@@ -1794,8 +1931,9 @@ get_rel_sync_entry(PGOutputData *data, Relation relation)
entry->new_slot = NULL;
entry->old_slot = NULL;
memset(entry->exprstate, 0, sizeof(entry->exprstate));
entry->cache_expr_cxt = NULL;
entry->entry_cxt = NULL;
entry->publish_as_relid = InvalidOid;
entry->columns = NULL;
entry->attrmap = NULL;
}
@@ -1841,6 +1979,8 @@ get_rel_sync_entry(PGOutputData *data, Relation relation)
entry->schema_sent = false;
list_free(entry->streamed_txns);
entry->streamed_txns = NIL;
bms_free(entry->columns);
entry->columns = NULL;
entry->pubactions.pubinsert = false;
entry->pubactions.pubupdate = false;
entry->pubactions.pubdelete = false;
@@ -1865,17 +2005,18 @@ get_rel_sync_entry(PGOutputData *data, Relation relation)
/*
* Row filter cache cleanups.
*/
if (entry->cache_expr_cxt)
MemoryContextDelete(entry->cache_expr_cxt);
if (entry->entry_cxt)
MemoryContextDelete(entry->entry_cxt);
entry->cache_expr_cxt = NULL;
entry->entry_cxt = NULL;
entry->estate = NULL;
memset(entry->exprstate, 0, sizeof(entry->exprstate));
/*
* Build publication cache. We can't use one provided by relcache as
* relcache considers all publications given relation is in, but here
* we only need to consider ones that the subscriber requested.
* relcache considers all publications that the given relation is in,
* but here we only need to consider ones that the subscriber
* requested.
*/
foreach(lc, data->publications)
{
@@ -1946,6 +2087,9 @@ get_rel_sync_entry(PGOutputData *data, Relation relation)
}
/*
* If the relation is to be published, determine actions to
* publish, and list of columns, if appropriate.
*
* Don't publish changes for partitioned tables, because
* publishing those of its partitions suffices, unless partition
* changes won't be published due to pubviaroot being set.
@@ -2007,6 +2151,9 @@ get_rel_sync_entry(PGOutputData *data, Relation relation)
/* Initialize the row filter */
pgoutput_row_filter_init(data, rel_publications, entry);
/* Initialize the column list */
pgoutput_column_list_init(data, rel_publications, entry);
}
list_free(pubids);