Merge pull request #997 from citusdata/add_sync_metadata_to_node

Add sync_metadata_to_node UDF
pull/1012/merge
Eren Başak 2016-12-14 10:06:17 +02:00 committed by GitHub
commit 9d4e586457
20 changed files with 787 additions and 48 deletions

View File

@ -8,7 +8,8 @@ EXTENSION = citus
EXTVERSIONS = 5.0 5.0-1 5.0-2 \ EXTVERSIONS = 5.0 5.0-1 5.0-2 \
5.1-1 5.1-2 5.1-3 5.1-4 5.1-5 5.1-6 5.1-7 5.1-8 \ 5.1-1 5.1-2 5.1-3 5.1-4 5.1-5 5.1-6 5.1-7 5.1-8 \
5.2-1 5.2-2 5.2-3 5.2-4 \ 5.2-1 5.2-2 5.2-3 5.2-4 \
6.0-1 6.0-2 6.0-3 6.0-4 6.0-5 6.0-6 6.0-7 6.0-8 6.0-9 6.0-10 6.0-11 6.0-12 6.0-13 6.0-14 6.0-15 6.0-16 6.0-17 6.0-18 6.0-1 6.0-2 6.0-3 6.0-4 6.0-5 6.0-6 6.0-7 6.0-8 6.0-9 6.0-10 6.0-11 6.0-12 6.0-13 6.0-14 6.0-15 6.0-16 6.0-17 6.0-18 \
6.1-1 6.1-2 6.1-3
# All citus--*.sql files in the source directory # All citus--*.sql files in the source directory
DATA = $(patsubst $(citus_abs_srcdir)/%.sql,%.sql,$(wildcard $(citus_abs_srcdir)/$(EXTENSION)--*--*.sql)) DATA = $(patsubst $(citus_abs_srcdir)/%.sql,%.sql,$(wildcard $(citus_abs_srcdir)/$(EXTENSION)--*--*.sql))
@ -94,6 +95,12 @@ $(EXTENSION)--6.0-17.sql: $(EXTENSION)--6.0-16.sql $(EXTENSION)--6.0-16--6.0-17.
cat $^ > $@ cat $^ > $@
$(EXTENSION)--6.0-18.sql: $(EXTENSION)--6.0-17.sql $(EXTENSION)--6.0-17--6.0-18.sql $(EXTENSION)--6.0-18.sql: $(EXTENSION)--6.0-17.sql $(EXTENSION)--6.0-17--6.0-18.sql
cat $^ > $@ cat $^ > $@
$(EXTENSION)--6.1-1.sql: $(EXTENSION)--6.0-18.sql $(EXTENSION)--6.0-18--6.1-1.sql
cat $^ > $@
$(EXTENSION)--6.1-2.sql: $(EXTENSION)--6.1-1.sql $(EXTENSION)--6.1-1--6.1-2.sql
cat $^ > $@
$(EXTENSION)--6.1-3.sql: $(EXTENSION)--6.1-2.sql $(EXTENSION)--6.1-2--6.1-3.sql
cat $^ > $@
NO_PGXS = 1 NO_PGXS = 1

View File

@ -0,0 +1,12 @@
/* citus--6.0-18--6.1-1.sql */
SET search_path = 'pg_catalog';
CREATE FUNCTION start_metadata_sync_to_node(nodename text, nodeport integer)
RETURNS VOID
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$start_metadata_sync_to_node$$;
COMMENT ON FUNCTION start_metadata_sync_to_node(nodename text, nodeport integer)
IS 'sync metadata to node';
RESET search_path;

View File

@ -0,0 +1,12 @@
/* citus--6.1-1--6.1-2.sql */
SET search_path = 'pg_catalog';
CREATE FUNCTION worker_create_truncate_trigger(table_name regclass)
RETURNS VOID
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$worker_create_truncate_trigger$$;
COMMENT ON FUNCTION worker_create_truncate_trigger(tablename regclass)
IS 'create truncate trigger for distributed table';
RESET search_path;

View File

@ -0,0 +1,12 @@
/* citus--6.1-2--6.1-3.sql */
SET search_path = 'pg_catalog';
CREATE FUNCTION stop_metadata_sync_to_node(nodename text, nodeport integer)
RETURNS VOID
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$stop_metadata_sync_to_node$$;
COMMENT ON FUNCTION stop_metadata_sync_to_node(nodename text, nodeport integer)
IS 'stop metadata sync to node';
RESET search_path;

View File

@ -1,6 +1,6 @@
# Citus extension # Citus extension
comment = 'Citus distributed database' comment = 'Citus distributed database'
default_version = '6.0-18' default_version = '6.1-3'
module_pathname = '$libdir/citus' module_pathname = '$libdir/citus'
relocatable = false relocatable = false
schema = pg_catalog schema = pg_catalog

View File

@ -76,7 +76,6 @@ static void ErrorIfNotSupportedForeignConstraint(Relation relation,
uint32 colocationId); uint32 colocationId);
static void InsertIntoPgDistPartition(Oid relationId, char distributionMethod, static void InsertIntoPgDistPartition(Oid relationId, char distributionMethod,
Var *distributionColumn, uint32 colocationId); Var *distributionColumn, uint32 colocationId);
static void CreateTruncateTrigger(Oid relationId);
static uint32 ColocationId(int shardCount, int replicationFactor, static uint32 ColocationId(int shardCount, int replicationFactor,
Oid distributionColumnType); Oid distributionColumnType);
static uint32 GetNextColocationId(void); static uint32 GetNextColocationId(void);
@ -823,7 +822,7 @@ LocalTableEmpty(Oid tableId)
* CreateTruncateTrigger creates a truncate trigger on table identified by relationId * CreateTruncateTrigger creates a truncate trigger on table identified by relationId
* and assigns citus_truncate_trigger() as handler. * and assigns citus_truncate_trigger() as handler.
*/ */
static void void
CreateTruncateTrigger(Oid relationId) CreateTruncateTrigger(Oid relationId)
{ {
CreateTrigStmt *trigger = NULL; CreateTrigStmt *trigger = NULL;

View File

@ -15,9 +15,11 @@
#include "miscadmin.h" #include "miscadmin.h"
#include "access/htup_details.h" #include "access/htup_details.h"
#include "access/sysattr.h"
#include "access/xact.h" #include "access/xact.h"
#include "catalog/indexing.h" #include "catalog/indexing.h"
#include "catalog/pg_type.h" #include "catalog/pg_type.h"
#include "catalog/pg_namespace.h"
#include "distributed/citus_nodes.h" #include "distributed/citus_nodes.h"
#include "distributed/master_metadata_utility.h" #include "distributed/master_metadata_utility.h"
#include "distributed/master_protocol.h" #include "distributed/master_protocol.h"

View File

@ -26,6 +26,7 @@
#include "access/htup_details.h" #include "access/htup_details.h"
#include "access/skey.h" #include "access/skey.h"
#include "access/stratnum.h" #include "access/stratnum.h"
#include "access/sysattr.h"
#include "access/tupdesc.h" #include "access/tupdesc.h"
#include "catalog/dependency.h" #include "catalog/dependency.h"
#include "catalog/indexing.h" #include "catalog/indexing.h"
@ -37,6 +38,7 @@
#endif #endif
#include "catalog/pg_index.h" #include "catalog/pg_index.h"
#include "catalog/pg_type.h" #include "catalog/pg_type.h"
#include "catalog/pg_namespace.h"
#include "commands/sequence.h" #include "commands/sequence.h"
#include "distributed/citus_ruleutils.h" #include "distributed/citus_ruleutils.h"
#include "distributed/listutils.h" #include "distributed/listutils.h"
@ -56,6 +58,7 @@
#include "utils/palloc.h" #include "utils/palloc.h"
#include "utils/relcache.h" #include "utils/relcache.h"
#include "utils/ruleutils.h" #include "utils/ruleutils.h"
#include "utils/tqual.h"
/* Shard related configuration */ /* Shard related configuration */
@ -66,6 +69,7 @@ int ShardPlacementPolicy = SHARD_PLACEMENT_ROUND_ROBIN;
static Datum WorkerNodeGetDatum(WorkerNode *workerNode, TupleDesc tupleDescriptor); static Datum WorkerNodeGetDatum(WorkerNode *workerNode, TupleDesc tupleDescriptor);
static char * SchemaOwner(Oid schemaId);
/* exports for SQL callable functions */ /* exports for SQL callable functions */
@ -633,7 +637,8 @@ GetTableDDLEvents(Oid relationId)
if (strncmp(schemaName, "public", NAMEDATALEN) != 0) if (strncmp(schemaName, "public", NAMEDATALEN) != 0)
{ {
StringInfo schemaNameDef = makeStringInfo(); StringInfo schemaNameDef = makeStringInfo();
appendStringInfo(schemaNameDef, CREATE_SCHEMA_COMMAND, schemaName); char *ownerName = SchemaOwner(schemaId);
appendStringInfo(schemaNameDef, CREATE_SCHEMA_COMMAND, schemaName, ownerName);
tableDDLEventList = lappend(tableDDLEventList, schemaNameDef->data); tableDDLEventList = lappend(tableDDLEventList, schemaNameDef->data);
} }
@ -855,3 +860,47 @@ WorkerNodeGetDatum(WorkerNode *workerNode, TupleDesc tupleDescriptor)
return workerNodeDatum; return workerNodeDatum;
} }
/*
* SchemaOwner returns the name of the owner of the specified schema.
*/
char *
SchemaOwner(Oid schemaId)
{
const int scanKeyCount = 1;
Relation namespaceRelation = heap_open(NamespaceRelationId, AccessShareLock);
ScanKeyData scanKeyData[scanKeyCount];
SysScanDesc scanDescriptor = NULL;
HeapTuple tuple = NULL;
char *ownerName = NULL;
/* start scan */
ScanKeyInit(&scanKeyData[0],
ObjectIdAttributeNumber,
BTEqualStrategyNumber, F_OIDEQ,
ObjectIdGetDatum(schemaId));
scanDescriptor = systable_beginscan(namespaceRelation, NamespaceOidIndexId, true,
SnapshotSelf, 1, &scanKeyData[0]);
tuple = systable_getnext(scanDescriptor);
if (HeapTupleIsValid(tuple))
{
Form_pg_namespace nsptup = (Form_pg_namespace) GETSTRUCT(tuple);
Oid ownerId = nsptup->nspowner;
ownerName = GetUserNameFromId(ownerId, false);
}
else
{
/* if the schema is not found, then return the name of current user */
ownerName = GetUserNameFromId(GetUserId(), false);
}
systable_endscan(scanDescriptor);
heap_close(namespaceRelation, NoLock);
return ownerName;
}

View File

@ -17,8 +17,12 @@
#include <sys/stat.h> #include <sys/stat.h>
#include <unistd.h> #include <unistd.h>
#include "access/genam.h"
#include "access/heapam.h" #include "access/heapam.h"
#include "access/htup_details.h"
#include "access/xact.h"
#include "catalog/dependency.h" #include "catalog/dependency.h"
#include "catalog/indexing.h"
#include "catalog/pg_foreign_server.h" #include "catalog/pg_foreign_server.h"
#include "distributed/citus_ruleutils.h" #include "distributed/citus_ruleutils.h"
#include "distributed/distribution_column.h" #include "distributed/distribution_column.h"
@ -27,14 +31,121 @@
#include "distributed/metadata_cache.h" #include "distributed/metadata_cache.h"
#include "distributed/metadata_sync.h" #include "distributed/metadata_sync.h"
#include "distributed/multi_join_order.h" #include "distributed/multi_join_order.h"
#include "distributed/pg_dist_node.h"
#include "distributed/worker_manager.h" #include "distributed/worker_manager.h"
#include "distributed/worker_transaction.h"
#include "foreign/foreign.h" #include "foreign/foreign.h"
#include "nodes/pg_list.h" #include "nodes/pg_list.h"
#include "utils/builtins.h" #include "utils/builtins.h"
#include "utils/fmgroids.h"
#include "utils/lsyscache.h" #include "utils/lsyscache.h"
#include "utils/memutils.h" #include "utils/memutils.h"
static char * LocalGroupIdUpdateCommand(uint32 groupId);
static void MarkNodeHasMetadata(char *nodeName, int32 nodePort, bool hasMetadata);
static char * TruncateTriggerCreateCommand(Oid relationId);
PG_FUNCTION_INFO_V1(start_metadata_sync_to_node);
PG_FUNCTION_INFO_V1(stop_metadata_sync_to_node);
/*
* start_metadata_sync_to_node function creates the metadata in a worker for preparing the
* worker for accepting MX-table queries. The function first sets the localGroupId of the
* worker so that the worker knows which tuple in pg_dist_node table represents itself.
* After that, SQL statetemens for re-creating metadata about mx distributed
* tables are sent to the worker. Finally, the hasmetadata column of the target node in
* pg_dist_node is marked as true.
*/
Datum
start_metadata_sync_to_node(PG_FUNCTION_ARGS)
{
text *nodeName = PG_GETARG_TEXT_P(0);
int32 nodePort = PG_GETARG_INT32(1);
char *nodeNameString = text_to_cstring(nodeName);
char *extensionOwner = CitusExtensionOwnerName();
WorkerNode *workerNode = NULL;
char *localGroupIdUpdateCommand = NULL;
List *recreateMetadataSnapshotCommandList = NIL;
List *dropMetadataCommandList = NIL;
List *createMetadataCommandList = NIL;
EnsureSuperUser();
PreventTransactionChain(true, "start_metadata_sync_to_node");
workerNode = FindWorkerNode(nodeNameString, nodePort);
if (workerNode == NULL)
{
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("you cannot sync metadata to a non-existent node"),
errhint("First, add the node with SELECT master_add_node(%s,%d)",
nodeNameString, nodePort)));
}
/* generate and add the local group id's update query */
localGroupIdUpdateCommand = LocalGroupIdUpdateCommand(workerNode->groupId);
/* generate the queries which drop the metadata */
dropMetadataCommandList = MetadataDropCommands();
/* generate the queries which create the metadata from scratch */
createMetadataCommandList = MetadataCreateCommands();
recreateMetadataSnapshotCommandList = lappend(recreateMetadataSnapshotCommandList,
localGroupIdUpdateCommand);
recreateMetadataSnapshotCommandList = list_concat(recreateMetadataSnapshotCommandList,
dropMetadataCommandList);
recreateMetadataSnapshotCommandList = list_concat(recreateMetadataSnapshotCommandList,
createMetadataCommandList);
/*
* Send the snapshot recreation commands in a single remote transaction and
* error out in any kind of failure. Note that it is not required to send
* createMetadataSnapshotCommandList in the same transaction that we send
* nodeDeleteCommand and nodeInsertCommand commands below.
*/
SendCommandListToWorkerInSingleTransaction(nodeNameString, nodePort, extensionOwner,
recreateMetadataSnapshotCommandList);
MarkNodeHasMetadata(nodeNameString, nodePort, true);
PG_RETURN_VOID();
}
/*
* stop_metadata_sync_to_node function sets the hasmetadata column of the specified node
* to false in pg_dist_node table, thus indicating that the specified worker node does not
* receive DDL changes anymore and cannot be used for issuing mx queries.
*/
Datum
stop_metadata_sync_to_node(PG_FUNCTION_ARGS)
{
text *nodeName = PG_GETARG_TEXT_P(0);
int32 nodePort = PG_GETARG_INT32(1);
char *nodeNameString = text_to_cstring(nodeName);
WorkerNode *workerNode = NULL;
EnsureSuperUser();
workerNode = FindWorkerNode(nodeNameString, nodePort);
if (workerNode == NULL)
{
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("node (%s,%d) does not exist", nodeNameString, nodePort)));
}
MarkNodeHasMetadata(nodeNameString, nodePort, false);
PG_RETURN_VOID();
}
/* /*
* ShouldSyncTableMetadata checks if a distributed table has streaming replication model * ShouldSyncTableMetadata checks if a distributed table has streaming replication model
* and hash distribution. In that case the distributed table is considered an MX table, * and hash distribution. In that case the distributed table is considered an MX table,
@ -76,6 +187,7 @@ MetadataCreateCommands(void)
{ {
List *metadataSnapshotCommandList = NIL; List *metadataSnapshotCommandList = NIL;
List *distributedTableList = DistributedTableList(); List *distributedTableList = DistributedTableList();
List *mxTableList = NIL;
List *workerNodeList = WorkerNodeList(); List *workerNodeList = WorkerNodeList();
ListCell *distributedTableCell = NULL; ListCell *distributedTableCell = NULL;
char *nodeListInsertCommand = NULL; char *nodeListInsertCommand = NULL;
@ -85,26 +197,67 @@ MetadataCreateCommands(void)
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList, metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
nodeListInsertCommand); nodeListInsertCommand);
/* iterate over the distributed tables */ /* create the list of mx tables */
foreach(distributedTableCell, distributedTableList) foreach(distributedTableCell, distributedTableList)
{ {
DistTableCacheEntry *cacheEntry = DistTableCacheEntry *cacheEntry =
(DistTableCacheEntry *) lfirst(distributedTableCell); (DistTableCacheEntry *) lfirst(distributedTableCell);
List *clusteredTableDDLEvents = NIL; if (ShouldSyncTableMetadata(cacheEntry->relationId))
{
mxTableList = lappend(mxTableList, cacheEntry);
}
}
/* create the mx tables, but not the metadata */
foreach(distributedTableCell, mxTableList)
{
DistTableCacheEntry *cacheEntry =
(DistTableCacheEntry *) lfirst(distributedTableCell);
Oid relationId = cacheEntry->relationId;
List *commandList = GetTableDDLEvents(relationId);
char *tableOwnerResetCommand = TableOwnerResetCommand(relationId);
metadataSnapshotCommandList = list_concat(metadataSnapshotCommandList,
commandList);
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
tableOwnerResetCommand);
}
/* construct the foreign key constraints after all tables are created */
foreach(distributedTableCell, mxTableList)
{
DistTableCacheEntry *cacheEntry =
(DistTableCacheEntry *) lfirst(distributedTableCell);
List *foreignConstraintCommands =
GetTableForeignConstraintCommands(cacheEntry->relationId);
metadataSnapshotCommandList = list_concat(metadataSnapshotCommandList,
foreignConstraintCommands);
}
/* after all tables are created, create the metadata */
foreach(distributedTableCell, mxTableList)
{
DistTableCacheEntry *cacheEntry =
(DistTableCacheEntry *) lfirst(distributedTableCell);
List *shardIntervalList = NIL; List *shardIntervalList = NIL;
List *shardCreateCommandList = NIL; List *shardCreateCommandList = NIL;
char *metadataCommand = NULL;
char *truncateTriggerCreateCommand = NULL;
Oid clusteredTableId = cacheEntry->relationId; Oid clusteredTableId = cacheEntry->relationId;
/* add only clustered tables */ /* add the table metadata command first*/
if (!ShouldSyncTableMetadata(clusteredTableId)) metadataCommand = DistributionCreateCommand(cacheEntry);
{ metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
continue; metadataCommand);
}
/* add the DDL events first */ /* add the truncate trigger command after the table became distributed */
clusteredTableDDLEvents = GetDistributedTableDDLEvents(cacheEntry); truncateTriggerCreateCommand =
metadataSnapshotCommandList = list_concat(metadataSnapshotCommandList, TruncateTriggerCreateCommand(cacheEntry->relationId);
clusteredTableDDLEvents); metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
truncateTriggerCreateCommand);
/* add the pg_dist_shard{,placement} entries */ /* add the pg_dist_shard{,placement} entries */
shardIntervalList = LoadShardIntervalList(clusteredTableId); shardIntervalList = LoadShardIntervalList(clusteredTableId);
@ -395,24 +548,91 @@ NodeDeleteCommand(uint32 nodeId)
/* /*
* GetDistributedTableDDLEvents returns the full set of DDL commands necessary to * LocalGroupIdUpdateCommand creates the SQL command required to set the local group id
* create this relation on a worker. This includes setting up any sequences, * of a worker and returns the command in a string.
* setting the owner of the table, and inserting into metadata tables.
*/ */
List * static char *
GetDistributedTableDDLEvents(DistTableCacheEntry *cacheEntry) LocalGroupIdUpdateCommand(uint32 groupId)
{ {
char *ownerResetCommand = NULL; StringInfo updateCommand = makeStringInfo();
char *metadataCommand = NULL;
Oid relationId = cacheEntry->relationId;
List *commandList = GetTableDDLEvents(relationId); appendStringInfo(updateCommand, "UPDATE pg_dist_local_group SET groupid = %d",
groupId);
ownerResetCommand = TableOwnerResetCommand(relationId); return updateCommand->data;
commandList = lappend(commandList, ownerResetCommand); }
metadataCommand = DistributionCreateCommand(cacheEntry);
commandList = lappend(commandList, metadataCommand); /*
* MarkNodeHasMetadata function sets the hasmetadata column of the specified worker in
return commandList; * pg_dist_node to true.
*/
static void
MarkNodeHasMetadata(char *nodeName, int32 nodePort, bool hasMetadata)
{
const bool indexOK = false;
const int scanKeyCount = 2;
Relation pgDistNode = NULL;
TupleDesc tupleDescriptor = NULL;
ScanKeyData scanKey[scanKeyCount];
SysScanDesc scanDescriptor = NULL;
HeapTuple heapTuple = NULL;
Datum values[Natts_pg_dist_node];
bool isnull[Natts_pg_dist_node];
bool replace[Natts_pg_dist_node];
pgDistNode = heap_open(DistNodeRelationId(), RowExclusiveLock);
tupleDescriptor = RelationGetDescr(pgDistNode);
ScanKeyInit(&scanKey[0], Anum_pg_dist_node_nodename,
BTEqualStrategyNumber, F_TEXTEQ, CStringGetTextDatum(nodeName));
ScanKeyInit(&scanKey[1], Anum_pg_dist_node_nodeport,
BTEqualStrategyNumber, F_INT8EQ, Int32GetDatum(nodePort));
scanDescriptor = systable_beginscan(pgDistNode, InvalidOid, indexOK,
NULL, scanKeyCount, scanKey);
heapTuple = systable_getnext(scanDescriptor);
if (!HeapTupleIsValid(heapTuple))
{
ereport(ERROR, (errmsg("could not find valid entry for node \"%s:%d\"",
nodeName, nodePort)));
}
memset(replace, 0, sizeof(replace));
values[Anum_pg_dist_node_hasmetadata - 1] = BoolGetDatum(hasMetadata);
isnull[Anum_pg_dist_node_hasmetadata - 1] = false;
replace[Anum_pg_dist_node_hasmetadata - 1] = true;
heapTuple = heap_modify_tuple(heapTuple, tupleDescriptor, values, isnull, replace);
simple_heap_update(pgDistNode, &heapTuple->t_self, heapTuple);
CatalogUpdateIndexes(pgDistNode, heapTuple);
CitusInvalidateRelcacheByRelid(DistNodeRelationId());
CommandCounterIncrement();
systable_endscan(scanDescriptor);
heap_close(pgDistNode, NoLock);
}
/*
* TruncateTriggerCreateCommand creates a SQL query calling worker_create_truncate_trigger
* function, which creates the truncate trigger on the worker.
*/
static char *
TruncateTriggerCreateCommand(Oid relationId)
{
StringInfo triggerCreateCommand = makeStringInfo();
char *tableName = generate_qualified_relation_name(relationId);
appendStringInfo(triggerCreateCommand,
"SELECT worker_create_truncate_trigger(%s)",
quote_literal_cstr(tableName));
return triggerCreateCommand->data;
} }

View File

@ -42,6 +42,8 @@ static void DeletePartitionRow(Oid distributedRelationId);
* not dropped as in the case of "DROP TABLE distributed_table;" command. * not dropped as in the case of "DROP TABLE distributed_table;" command.
* *
* The function errors out if the input relation Oid is not a regular or foreign table. * The function errors out if the input relation Oid is not a regular or foreign table.
* The function is meant to be called only by the coordinator, therefore requires
* superuser privileges.
*/ */
Datum Datum
worker_drop_distributed_table(PG_FUNCTION_ARGS) worker_drop_distributed_table(PG_FUNCTION_ARGS)
@ -55,6 +57,8 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS)
ListCell *shardCell = NULL; ListCell *shardCell = NULL;
char relationKind = '\0'; char relationKind = '\0';
EnsureSuperUser();
/* first check the relation type */ /* first check the relation type */
distributedRelation = relation_open(relationId, AccessShareLock); distributedRelation = relation_open(relationId, AccessShareLock);
relationKind = distributedRelation->rd_rel->relkind; relationKind = distributedRelation->rd_rel->relkind;
@ -96,8 +100,8 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS)
} }
else else
{ {
/* drop the table only */ /* drop the table with cascade since other tables may be referring to it */
performDeletion(&distributedTableObject, DROP_RESTRICT, performDeletion(&distributedTableObject, DROP_CASCADE,
PERFORM_DELETION_INTERNAL); PERFORM_DELETION_INTERNAL);
} }

View File

@ -0,0 +1,46 @@
/*-------------------------------------------------------------------------
*
* worker_create_truncate_trigger_protocol.c
*
* Routines for creating truncate triggers on distributed tables on worker nodes.
*
* Copyright (c) 2016, Citus Data, Inc.
*
* $Id$
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "fmgr.h"
#include "distributed/citus_ruleutils.h"
#include "distributed/master_metadata_utility.h"
#include "distributed/metadata_cache.h"
#include "distributed/metadata_sync.h"
#include "utils/elog.h"
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
PG_FUNCTION_INFO_V1(worker_create_truncate_trigger);
/*
* worker_create_truncate_trigger creates a truncate trigger for the given distributed
* table on current metadata worker. The function is intented to be called by the
* coordinator node during metadata propagation of mx tables or during the upgrades from
* citus version <=5.2 to >=6.1. The function requires superuser permissions.
*/
Datum
worker_create_truncate_trigger(PG_FUNCTION_ARGS)
{
Oid relationId = PG_GETARG_OID(0);
EnsureSuperUser();
/* Create the truncate trigger */
CreateTruncateTrigger(relationId);
PG_RETURN_VOID();
}

View File

@ -80,6 +80,7 @@ extern void DeleteShardRow(uint64 shardId);
extern void UpdateShardPlacementState(uint64 placementId, char shardState); extern void UpdateShardPlacementState(uint64 placementId, char shardState);
extern uint64 DeleteShardPlacementRow(uint64 shardId, char *workerName, uint32 extern uint64 DeleteShardPlacementRow(uint64 shardId, char *workerName, uint32
workerPort); workerPort);
extern void CreateTruncateTrigger(Oid relationId);
/* Remaining metadata utility functions */ /* Remaining metadata utility functions */
extern char * TableOwner(Oid relationId); extern char * TableOwner(Oid relationId);

View File

@ -66,7 +66,7 @@
#define SHARD_CSTORE_TABLE_SIZE_QUERY "SELECT cstore_table_size(%s)" #define SHARD_CSTORE_TABLE_SIZE_QUERY "SELECT cstore_table_size(%s)"
#define DROP_REGULAR_TABLE_COMMAND "DROP TABLE IF EXISTS %s CASCADE" #define DROP_REGULAR_TABLE_COMMAND "DROP TABLE IF EXISTS %s CASCADE"
#define DROP_FOREIGN_TABLE_COMMAND "DROP FOREIGN TABLE IF EXISTS %s CASCADE" #define DROP_FOREIGN_TABLE_COMMAND "DROP FOREIGN TABLE IF EXISTS %s CASCADE"
#define CREATE_SCHEMA_COMMAND "CREATE SCHEMA IF NOT EXISTS %s" #define CREATE_SCHEMA_COMMAND "CREATE SCHEMA IF NOT EXISTS %s AUTHORIZATION %s"
#define CREATE_EMPTY_SHARD_QUERY "SELECT master_create_empty_shard('%s')" #define CREATE_EMPTY_SHARD_QUERY "SELECT master_create_empty_shard('%s')"
#define FINALIZED_SHARD_PLACEMENTS_QUERY \ #define FINALIZED_SHARD_PLACEMENTS_QUERY \
"SELECT nodename, nodeport FROM pg_dist_shard_placement WHERE shardstate = 1 AND shardid = %ld" "SELECT nodename, nodeport FROM pg_dist_shard_placement WHERE shardstate = 1 AND shardid = %ld"

View File

@ -59,7 +59,6 @@ extern DistTableCacheEntry * DistributedTableCacheEntry(Oid distributedRelationI
extern int GetLocalGroupId(void); extern int GetLocalGroupId(void);
extern void CitusInvalidateRelcacheByRelid(Oid relationId); extern void CitusInvalidateRelcacheByRelid(Oid relationId);
extern void CitusInvalidateRelcacheByShardId(int64 shardId); extern void CitusInvalidateRelcacheByShardId(int64 shardId);
extern void CitusInvalidateNodeCache(void);
extern bool CitusHasBeenLoaded(void); extern bool CitusHasBeenLoaded(void);

View File

@ -28,7 +28,6 @@ extern char * TableOwnerResetCommand(Oid distributedRelationId);
extern char * NodeListInsertCommand(List *workerNodeList); extern char * NodeListInsertCommand(List *workerNodeList);
extern List * ShardListInsertCommand(List *shardIntervalList); extern List * ShardListInsertCommand(List *shardIntervalList);
extern char * NodeDeleteCommand(uint32 nodeId); extern char * NodeDeleteCommand(uint32 nodeId);
extern List * GetDistributedTableDDLEvents(DistTableCacheEntry *cacheEntry);
#define DELETE_ALL_NODES "TRUNCATE pg_dist_node" #define DELETE_ALL_NODES "TRUNCATE pg_dist_node"

View File

@ -58,6 +58,9 @@ ALTER EXTENSION citus UPDATE TO '6.0-15';
ALTER EXTENSION citus UPDATE TO '6.0-16'; ALTER EXTENSION citus UPDATE TO '6.0-16';
ALTER EXTENSION citus UPDATE TO '6.0-17'; ALTER EXTENSION citus UPDATE TO '6.0-17';
ALTER EXTENSION citus UPDATE TO '6.0-18'; ALTER EXTENSION citus UPDATE TO '6.0-18';
ALTER EXTENSION citus UPDATE TO '6.1-1';
ALTER EXTENSION citus UPDATE TO '6.1-2';
ALTER EXTENSION citus UPDATE TO '6.1-3';
-- ensure no objects were created outside pg_catalog -- ensure no objects were created outside pg_catalog
SELECT COUNT(*) SELECT COUNT(*)
FROM pg_depend AS pgd, FROM pg_depend AS pgd,

View File

@ -36,9 +36,9 @@ SELECT table_ddl_command_array('not_null_table');
-- ensure tables not in search path are schema-prefixed -- ensure tables not in search path are schema-prefixed
CREATE SCHEMA not_in_path CREATE TABLE simple_table (id bigint); CREATE SCHEMA not_in_path CREATE TABLE simple_table (id bigint);
SELECT table_ddl_command_array('not_in_path.simple_table'); SELECT table_ddl_command_array('not_in_path.simple_table');
table_ddl_command_array table_ddl_command_array
------------------------------------------------------------------------------------------------- ------------------------------------------------------------------------------------------------------------------------
{"CREATE SCHEMA IF NOT EXISTS not_in_path","CREATE TABLE not_in_path.simple_table (id bigint)"} {"CREATE SCHEMA IF NOT EXISTS not_in_path AUTHORIZATION postgres","CREATE TABLE not_in_path.simple_table (id bigint)"}
(1 row) (1 row)
-- even more complex constraints should be preserved... -- even more complex constraints should be preserved...

View File

@ -61,9 +61,10 @@ SELECT unnest(master_metadata_snapshot());
ALTER TABLE public.mx_test_table ADD CONSTRAINT mx_test_table_col_1_key UNIQUE (col_1) ALTER TABLE public.mx_test_table ADD CONSTRAINT mx_test_table_col_1_key UNIQUE (col_1)
ALTER TABLE public.mx_test_table OWNER TO postgres ALTER TABLE public.mx_test_table OWNER TO postgres
INSERT INTO pg_dist_partition (logicalrelid, partmethod, partkey, colocationid, repmodel) VALUES ('public.mx_test_table'::regclass, 'h', column_name_to_column('public.mx_test_table','col_1'), 0, 's') INSERT INTO pg_dist_partition (logicalrelid, partmethod, partkey, colocationid, repmodel) VALUES ('public.mx_test_table'::regclass, 'h', column_name_to_column('public.mx_test_table','col_1'), 0, 's')
SELECT worker_create_truncate_trigger('public.mx_test_table')
INSERT INTO pg_dist_shard_placement (shardid, shardstate, shardlength, nodename, nodeport, placementid) VALUES (1310000, 1, 0, 'localhost', 57637, 100000),(1310001, 1, 0, 'localhost', 57638, 100001),(1310002, 1, 0, 'localhost', 57637, 100002),(1310003, 1, 0, 'localhost', 57638, 100003),(1310004, 1, 0, 'localhost', 57637, 100004),(1310005, 1, 0, 'localhost', 57638, 100005),(1310006, 1, 0, 'localhost', 57637, 100006),(1310007, 1, 0, 'localhost', 57638, 100007) INSERT INTO pg_dist_shard_placement (shardid, shardstate, shardlength, nodename, nodeport, placementid) VALUES (1310000, 1, 0, 'localhost', 57637, 100000),(1310001, 1, 0, 'localhost', 57638, 100001),(1310002, 1, 0, 'localhost', 57637, 100002),(1310003, 1, 0, 'localhost', 57638, 100003),(1310004, 1, 0, 'localhost', 57637, 100004),(1310005, 1, 0, 'localhost', 57638, 100005),(1310006, 1, 0, 'localhost', 57637, 100006),(1310007, 1, 0, 'localhost', 57638, 100007)
INSERT INTO pg_dist_shard (logicalrelid, shardid, shardstorage, shardminvalue, shardmaxvalue) VALUES ('public.mx_test_table'::regclass, 1310000, 't', '-2147483648', '-1610612737'),('public.mx_test_table'::regclass, 1310001, 't', '-1610612736', '-1073741825'),('public.mx_test_table'::regclass, 1310002, 't', '-1073741824', '-536870913'),('public.mx_test_table'::regclass, 1310003, 't', '-536870912', '-1'),('public.mx_test_table'::regclass, 1310004, 't', '0', '536870911'),('public.mx_test_table'::regclass, 1310005, 't', '536870912', '1073741823'),('public.mx_test_table'::regclass, 1310006, 't', '1073741824', '1610612735'),('public.mx_test_table'::regclass, 1310007, 't', '1610612736', '2147483647') INSERT INTO pg_dist_shard (logicalrelid, shardid, shardstorage, shardminvalue, shardmaxvalue) VALUES ('public.mx_test_table'::regclass, 1310000, 't', '-2147483648', '-1610612737'),('public.mx_test_table'::regclass, 1310001, 't', '-1610612736', '-1073741825'),('public.mx_test_table'::regclass, 1310002, 't', '-1073741824', '-536870913'),('public.mx_test_table'::regclass, 1310003, 't', '-536870912', '-1'),('public.mx_test_table'::regclass, 1310004, 't', '0', '536870911'),('public.mx_test_table'::regclass, 1310005, 't', '536870912', '1073741823'),('public.mx_test_table'::regclass, 1310006, 't', '1073741824', '1610612735'),('public.mx_test_table'::regclass, 1310007, 't', '1610612736', '2147483647')
(10 rows) (11 rows)
-- Show that CREATE INDEX commands are included in the metadata snapshot -- Show that CREATE INDEX commands are included in the metadata snapshot
CREATE INDEX mx_index ON mx_test_table(col_2); CREATE INDEX mx_index ON mx_test_table(col_2);
@ -81,9 +82,10 @@ SELECT unnest(master_metadata_snapshot());
ALTER TABLE public.mx_test_table ADD CONSTRAINT mx_test_table_col_1_key UNIQUE (col_1) ALTER TABLE public.mx_test_table ADD CONSTRAINT mx_test_table_col_1_key UNIQUE (col_1)
ALTER TABLE public.mx_test_table OWNER TO postgres ALTER TABLE public.mx_test_table OWNER TO postgres
INSERT INTO pg_dist_partition (logicalrelid, partmethod, partkey, colocationid, repmodel) VALUES ('public.mx_test_table'::regclass, 'h', column_name_to_column('public.mx_test_table','col_1'), 0, 's') INSERT INTO pg_dist_partition (logicalrelid, partmethod, partkey, colocationid, repmodel) VALUES ('public.mx_test_table'::regclass, 'h', column_name_to_column('public.mx_test_table','col_1'), 0, 's')
SELECT worker_create_truncate_trigger('public.mx_test_table')
INSERT INTO pg_dist_shard_placement (shardid, shardstate, shardlength, nodename, nodeport, placementid) VALUES (1310000, 1, 0, 'localhost', 57637, 100000),(1310001, 1, 0, 'localhost', 57638, 100001),(1310002, 1, 0, 'localhost', 57637, 100002),(1310003, 1, 0, 'localhost', 57638, 100003),(1310004, 1, 0, 'localhost', 57637, 100004),(1310005, 1, 0, 'localhost', 57638, 100005),(1310006, 1, 0, 'localhost', 57637, 100006),(1310007, 1, 0, 'localhost', 57638, 100007) INSERT INTO pg_dist_shard_placement (shardid, shardstate, shardlength, nodename, nodeport, placementid) VALUES (1310000, 1, 0, 'localhost', 57637, 100000),(1310001, 1, 0, 'localhost', 57638, 100001),(1310002, 1, 0, 'localhost', 57637, 100002),(1310003, 1, 0, 'localhost', 57638, 100003),(1310004, 1, 0, 'localhost', 57637, 100004),(1310005, 1, 0, 'localhost', 57638, 100005),(1310006, 1, 0, 'localhost', 57637, 100006),(1310007, 1, 0, 'localhost', 57638, 100007)
INSERT INTO pg_dist_shard (logicalrelid, shardid, shardstorage, shardminvalue, shardmaxvalue) VALUES ('public.mx_test_table'::regclass, 1310000, 't', '-2147483648', '-1610612737'),('public.mx_test_table'::regclass, 1310001, 't', '-1610612736', '-1073741825'),('public.mx_test_table'::regclass, 1310002, 't', '-1073741824', '-536870913'),('public.mx_test_table'::regclass, 1310003, 't', '-536870912', '-1'),('public.mx_test_table'::regclass, 1310004, 't', '0', '536870911'),('public.mx_test_table'::regclass, 1310005, 't', '536870912', '1073741823'),('public.mx_test_table'::regclass, 1310006, 't', '1073741824', '1610612735'),('public.mx_test_table'::regclass, 1310007, 't', '1610612736', '2147483647') INSERT INTO pg_dist_shard (logicalrelid, shardid, shardstorage, shardminvalue, shardmaxvalue) VALUES ('public.mx_test_table'::regclass, 1310000, 't', '-2147483648', '-1610612737'),('public.mx_test_table'::regclass, 1310001, 't', '-1610612736', '-1073741825'),('public.mx_test_table'::regclass, 1310002, 't', '-1073741824', '-536870913'),('public.mx_test_table'::regclass, 1310003, 't', '-536870912', '-1'),('public.mx_test_table'::regclass, 1310004, 't', '0', '536870911'),('public.mx_test_table'::regclass, 1310005, 't', '536870912', '1073741823'),('public.mx_test_table'::regclass, 1310006, 't', '1073741824', '1610612735'),('public.mx_test_table'::regclass, 1310007, 't', '1610612736', '2147483647')
(11 rows) (12 rows)
-- Show that schema changes are included in the metadata snapshot -- Show that schema changes are included in the metadata snapshot
CREATE SCHEMA mx_testing_schema; CREATE SCHEMA mx_testing_schema;
@ -96,16 +98,17 @@ SELECT unnest(master_metadata_snapshot());
TRUNCATE pg_dist_node TRUNCATE pg_dist_node
SELECT worker_drop_distributed_table(logicalrelid) FROM pg_dist_partition SELECT worker_drop_distributed_table(logicalrelid) FROM pg_dist_partition
INSERT INTO pg_dist_node (nodeid, groupid, nodename, nodeport, noderack, hasmetadata) VALUES (2, 2, 'localhost', 57638, 'default', FALSE),(1, 1, 'localhost', 57637, 'default', FALSE) INSERT INTO pg_dist_node (nodeid, groupid, nodename, nodeport, noderack, hasmetadata) VALUES (2, 2, 'localhost', 57638, 'default', FALSE),(1, 1, 'localhost', 57637, 'default', FALSE)
CREATE SCHEMA IF NOT EXISTS mx_testing_schema CREATE SCHEMA IF NOT EXISTS mx_testing_schema AUTHORIZATION postgres
CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE
CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 integer DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL) CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 integer DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL)
CREATE INDEX mx_index ON mx_testing_schema.mx_test_table USING btree (col_2) CREATE INDEX mx_index ON mx_testing_schema.mx_test_table USING btree (col_2)
ALTER TABLE mx_testing_schema.mx_test_table ADD CONSTRAINT mx_test_table_col_1_key UNIQUE (col_1) ALTER TABLE mx_testing_schema.mx_test_table ADD CONSTRAINT mx_test_table_col_1_key UNIQUE (col_1)
ALTER TABLE mx_testing_schema.mx_test_table OWNER TO postgres ALTER TABLE mx_testing_schema.mx_test_table OWNER TO postgres
INSERT INTO pg_dist_partition (logicalrelid, partmethod, partkey, colocationid, repmodel) VALUES ('mx_testing_schema.mx_test_table'::regclass, 'h', column_name_to_column('mx_testing_schema.mx_test_table','col_1'), 0, 's') INSERT INTO pg_dist_partition (logicalrelid, partmethod, partkey, colocationid, repmodel) VALUES ('mx_testing_schema.mx_test_table'::regclass, 'h', column_name_to_column('mx_testing_schema.mx_test_table','col_1'), 0, 's')
SELECT worker_create_truncate_trigger('mx_testing_schema.mx_test_table')
INSERT INTO pg_dist_shard_placement (shardid, shardstate, shardlength, nodename, nodeport, placementid) VALUES (1310000, 1, 0, 'localhost', 57637, 100000),(1310001, 1, 0, 'localhost', 57638, 100001),(1310002, 1, 0, 'localhost', 57637, 100002),(1310003, 1, 0, 'localhost', 57638, 100003),(1310004, 1, 0, 'localhost', 57637, 100004),(1310005, 1, 0, 'localhost', 57638, 100005),(1310006, 1, 0, 'localhost', 57637, 100006),(1310007, 1, 0, 'localhost', 57638, 100007) INSERT INTO pg_dist_shard_placement (shardid, shardstate, shardlength, nodename, nodeport, placementid) VALUES (1310000, 1, 0, 'localhost', 57637, 100000),(1310001, 1, 0, 'localhost', 57638, 100001),(1310002, 1, 0, 'localhost', 57637, 100002),(1310003, 1, 0, 'localhost', 57638, 100003),(1310004, 1, 0, 'localhost', 57637, 100004),(1310005, 1, 0, 'localhost', 57638, 100005),(1310006, 1, 0, 'localhost', 57637, 100006),(1310007, 1, 0, 'localhost', 57638, 100007)
INSERT INTO pg_dist_shard (logicalrelid, shardid, shardstorage, shardminvalue, shardmaxvalue) VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't', '-2147483648', '-1610612737'),('mx_testing_schema.mx_test_table'::regclass, 1310001, 't', '-1610612736', '-1073741825'),('mx_testing_schema.mx_test_table'::regclass, 1310002, 't', '-1073741824', '-536870913'),('mx_testing_schema.mx_test_table'::regclass, 1310003, 't', '-536870912', '-1'),('mx_testing_schema.mx_test_table'::regclass, 1310004, 't', '0', '536870911'),('mx_testing_schema.mx_test_table'::regclass, 1310005, 't', '536870912', '1073741823'),('mx_testing_schema.mx_test_table'::regclass, 1310006, 't', '1073741824', '1610612735'),('mx_testing_schema.mx_test_table'::regclass, 1310007, 't', '1610612736', '2147483647') INSERT INTO pg_dist_shard (logicalrelid, shardid, shardstorage, shardminvalue, shardmaxvalue) VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't', '-2147483648', '-1610612737'),('mx_testing_schema.mx_test_table'::regclass, 1310001, 't', '-1610612736', '-1073741825'),('mx_testing_schema.mx_test_table'::regclass, 1310002, 't', '-1073741824', '-536870913'),('mx_testing_schema.mx_test_table'::regclass, 1310003, 't', '-536870912', '-1'),('mx_testing_schema.mx_test_table'::regclass, 1310004, 't', '0', '536870911'),('mx_testing_schema.mx_test_table'::regclass, 1310005, 't', '536870912', '1073741823'),('mx_testing_schema.mx_test_table'::regclass, 1310006, 't', '1073741824', '1610612735'),('mx_testing_schema.mx_test_table'::regclass, 1310007, 't', '1610612736', '2147483647')
(12 rows) (13 rows)
-- Show that append distributed tables are not included in the metadata snapshot -- Show that append distributed tables are not included in the metadata snapshot
CREATE TABLE non_mx_test_table (col_1 int, col_2 text); CREATE TABLE non_mx_test_table (col_1 int, col_2 text);
@ -122,16 +125,17 @@ SELECT unnest(master_metadata_snapshot());
TRUNCATE pg_dist_node TRUNCATE pg_dist_node
SELECT worker_drop_distributed_table(logicalrelid) FROM pg_dist_partition SELECT worker_drop_distributed_table(logicalrelid) FROM pg_dist_partition
INSERT INTO pg_dist_node (nodeid, groupid, nodename, nodeport, noderack, hasmetadata) VALUES (2, 2, 'localhost', 57638, 'default', FALSE),(1, 1, 'localhost', 57637, 'default', FALSE) INSERT INTO pg_dist_node (nodeid, groupid, nodename, nodeport, noderack, hasmetadata) VALUES (2, 2, 'localhost', 57638, 'default', FALSE),(1, 1, 'localhost', 57637, 'default', FALSE)
CREATE SCHEMA IF NOT EXISTS mx_testing_schema CREATE SCHEMA IF NOT EXISTS mx_testing_schema AUTHORIZATION postgres
CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE
CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 integer DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL) CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 integer DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL)
CREATE INDEX mx_index ON mx_testing_schema.mx_test_table USING btree (col_2) CREATE INDEX mx_index ON mx_testing_schema.mx_test_table USING btree (col_2)
ALTER TABLE mx_testing_schema.mx_test_table ADD CONSTRAINT mx_test_table_col_1_key UNIQUE (col_1) ALTER TABLE mx_testing_schema.mx_test_table ADD CONSTRAINT mx_test_table_col_1_key UNIQUE (col_1)
ALTER TABLE mx_testing_schema.mx_test_table OWNER TO postgres ALTER TABLE mx_testing_schema.mx_test_table OWNER TO postgres
INSERT INTO pg_dist_partition (logicalrelid, partmethod, partkey, colocationid, repmodel) VALUES ('mx_testing_schema.mx_test_table'::regclass, 'h', column_name_to_column('mx_testing_schema.mx_test_table','col_1'), 0, 's') INSERT INTO pg_dist_partition (logicalrelid, partmethod, partkey, colocationid, repmodel) VALUES ('mx_testing_schema.mx_test_table'::regclass, 'h', column_name_to_column('mx_testing_schema.mx_test_table','col_1'), 0, 's')
SELECT worker_create_truncate_trigger('mx_testing_schema.mx_test_table')
INSERT INTO pg_dist_shard_placement (shardid, shardstate, shardlength, nodename, nodeport, placementid) VALUES (1310000, 1, 0, 'localhost', 57637, 100000),(1310001, 1, 0, 'localhost', 57638, 100001),(1310002, 1, 0, 'localhost', 57637, 100002),(1310003, 1, 0, 'localhost', 57638, 100003),(1310004, 1, 0, 'localhost', 57637, 100004),(1310005, 1, 0, 'localhost', 57638, 100005),(1310006, 1, 0, 'localhost', 57637, 100006),(1310007, 1, 0, 'localhost', 57638, 100007) INSERT INTO pg_dist_shard_placement (shardid, shardstate, shardlength, nodename, nodeport, placementid) VALUES (1310000, 1, 0, 'localhost', 57637, 100000),(1310001, 1, 0, 'localhost', 57638, 100001),(1310002, 1, 0, 'localhost', 57637, 100002),(1310003, 1, 0, 'localhost', 57638, 100003),(1310004, 1, 0, 'localhost', 57637, 100004),(1310005, 1, 0, 'localhost', 57638, 100005),(1310006, 1, 0, 'localhost', 57637, 100006),(1310007, 1, 0, 'localhost', 57638, 100007)
INSERT INTO pg_dist_shard (logicalrelid, shardid, shardstorage, shardminvalue, shardmaxvalue) VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't', '-2147483648', '-1610612737'),('mx_testing_schema.mx_test_table'::regclass, 1310001, 't', '-1610612736', '-1073741825'),('mx_testing_schema.mx_test_table'::regclass, 1310002, 't', '-1073741824', '-536870913'),('mx_testing_schema.mx_test_table'::regclass, 1310003, 't', '-536870912', '-1'),('mx_testing_schema.mx_test_table'::regclass, 1310004, 't', '0', '536870911'),('mx_testing_schema.mx_test_table'::regclass, 1310005, 't', '536870912', '1073741823'),('mx_testing_schema.mx_test_table'::regclass, 1310006, 't', '1073741824', '1610612735'),('mx_testing_schema.mx_test_table'::regclass, 1310007, 't', '1610612736', '2147483647') INSERT INTO pg_dist_shard (logicalrelid, shardid, shardstorage, shardminvalue, shardmaxvalue) VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't', '-2147483648', '-1610612737'),('mx_testing_schema.mx_test_table'::regclass, 1310001, 't', '-1610612736', '-1073741825'),('mx_testing_schema.mx_test_table'::regclass, 1310002, 't', '-1073741824', '-536870913'),('mx_testing_schema.mx_test_table'::regclass, 1310003, 't', '-536870912', '-1'),('mx_testing_schema.mx_test_table'::regclass, 1310004, 't', '0', '536870911'),('mx_testing_schema.mx_test_table'::regclass, 1310005, 't', '536870912', '1073741823'),('mx_testing_schema.mx_test_table'::regclass, 1310006, 't', '1073741824', '1610612735'),('mx_testing_schema.mx_test_table'::regclass, 1310007, 't', '1610612736', '2147483647')
(12 rows) (13 rows)
-- Show that range distributed tables are not included in the metadata snapshot -- Show that range distributed tables are not included in the metadata snapshot
UPDATE pg_dist_partition SET partmethod='r' WHERE logicalrelid='non_mx_test_table'::regclass; UPDATE pg_dist_partition SET partmethod='r' WHERE logicalrelid='non_mx_test_table'::regclass;
@ -141,15 +145,289 @@ SELECT unnest(master_metadata_snapshot());
TRUNCATE pg_dist_node TRUNCATE pg_dist_node
SELECT worker_drop_distributed_table(logicalrelid) FROM pg_dist_partition SELECT worker_drop_distributed_table(logicalrelid) FROM pg_dist_partition
INSERT INTO pg_dist_node (nodeid, groupid, nodename, nodeport, noderack, hasmetadata) VALUES (2, 2, 'localhost', 57638, 'default', FALSE),(1, 1, 'localhost', 57637, 'default', FALSE) INSERT INTO pg_dist_node (nodeid, groupid, nodename, nodeport, noderack, hasmetadata) VALUES (2, 2, 'localhost', 57638, 'default', FALSE),(1, 1, 'localhost', 57637, 'default', FALSE)
CREATE SCHEMA IF NOT EXISTS mx_testing_schema CREATE SCHEMA IF NOT EXISTS mx_testing_schema AUTHORIZATION postgres
CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE
CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 integer DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL) CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 integer DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL)
CREATE INDEX mx_index ON mx_testing_schema.mx_test_table USING btree (col_2) CREATE INDEX mx_index ON mx_testing_schema.mx_test_table USING btree (col_2)
ALTER TABLE mx_testing_schema.mx_test_table ADD CONSTRAINT mx_test_table_col_1_key UNIQUE (col_1) ALTER TABLE mx_testing_schema.mx_test_table ADD CONSTRAINT mx_test_table_col_1_key UNIQUE (col_1)
ALTER TABLE mx_testing_schema.mx_test_table OWNER TO postgres ALTER TABLE mx_testing_schema.mx_test_table OWNER TO postgres
INSERT INTO pg_dist_partition (logicalrelid, partmethod, partkey, colocationid, repmodel) VALUES ('mx_testing_schema.mx_test_table'::regclass, 'h', column_name_to_column('mx_testing_schema.mx_test_table','col_1'), 0, 's') INSERT INTO pg_dist_partition (logicalrelid, partmethod, partkey, colocationid, repmodel) VALUES ('mx_testing_schema.mx_test_table'::regclass, 'h', column_name_to_column('mx_testing_schema.mx_test_table','col_1'), 0, 's')
SELECT worker_create_truncate_trigger('mx_testing_schema.mx_test_table')
INSERT INTO pg_dist_shard_placement (shardid, shardstate, shardlength, nodename, nodeport, placementid) VALUES (1310000, 1, 0, 'localhost', 57637, 100000),(1310001, 1, 0, 'localhost', 57638, 100001),(1310002, 1, 0, 'localhost', 57637, 100002),(1310003, 1, 0, 'localhost', 57638, 100003),(1310004, 1, 0, 'localhost', 57637, 100004),(1310005, 1, 0, 'localhost', 57638, 100005),(1310006, 1, 0, 'localhost', 57637, 100006),(1310007, 1, 0, 'localhost', 57638, 100007) INSERT INTO pg_dist_shard_placement (shardid, shardstate, shardlength, nodename, nodeport, placementid) VALUES (1310000, 1, 0, 'localhost', 57637, 100000),(1310001, 1, 0, 'localhost', 57638, 100001),(1310002, 1, 0, 'localhost', 57637, 100002),(1310003, 1, 0, 'localhost', 57638, 100003),(1310004, 1, 0, 'localhost', 57637, 100004),(1310005, 1, 0, 'localhost', 57638, 100005),(1310006, 1, 0, 'localhost', 57637, 100006),(1310007, 1, 0, 'localhost', 57638, 100007)
INSERT INTO pg_dist_shard (logicalrelid, shardid, shardstorage, shardminvalue, shardmaxvalue) VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't', '-2147483648', '-1610612737'),('mx_testing_schema.mx_test_table'::regclass, 1310001, 't', '-1610612736', '-1073741825'),('mx_testing_schema.mx_test_table'::regclass, 1310002, 't', '-1073741824', '-536870913'),('mx_testing_schema.mx_test_table'::regclass, 1310003, 't', '-536870912', '-1'),('mx_testing_schema.mx_test_table'::regclass, 1310004, 't', '0', '536870911'),('mx_testing_schema.mx_test_table'::regclass, 1310005, 't', '536870912', '1073741823'),('mx_testing_schema.mx_test_table'::regclass, 1310006, 't', '1073741824', '1610612735'),('mx_testing_schema.mx_test_table'::regclass, 1310007, 't', '1610612736', '2147483647') INSERT INTO pg_dist_shard (logicalrelid, shardid, shardstorage, shardminvalue, shardmaxvalue) VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't', '-2147483648', '-1610612737'),('mx_testing_schema.mx_test_table'::regclass, 1310001, 't', '-1610612736', '-1073741825'),('mx_testing_schema.mx_test_table'::regclass, 1310002, 't', '-1073741824', '-536870913'),('mx_testing_schema.mx_test_table'::regclass, 1310003, 't', '-536870912', '-1'),('mx_testing_schema.mx_test_table'::regclass, 1310004, 't', '0', '536870911'),('mx_testing_schema.mx_test_table'::regclass, 1310005, 't', '536870912', '1073741823'),('mx_testing_schema.mx_test_table'::regclass, 1310006, 't', '1073741824', '1610612735'),('mx_testing_schema.mx_test_table'::regclass, 1310007, 't', '1610612736', '2147483647')
(12 rows) (13 rows)
-- Test start_metadata_sync_to_node UDF
-- Ensure that hasmetadata=false for all nodes
SELECT count(*) FROM pg_dist_node WHERE hasmetadata=true;
count
-------
0
(1 row)
-- Run start_metadata_sync_to_node and check that it marked hasmetadata for that worker
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
start_metadata_sync_to_node
-----------------------------
(1 row)
SELECT nodeid, hasmetadata FROM pg_dist_node WHERE nodename='localhost' AND nodeport=:worker_1_port;
nodeid | hasmetadata
--------+-------------
1 | t
(1 row)
-- Check that the metadata has been copied to the worker
\c - - - :worker_1_port
SELECT * FROM pg_dist_local_group;
groupid
---------
1
(1 row)
SELECT * FROM pg_dist_node ORDER BY nodeid;
nodeid | groupid | nodename | nodeport | noderack | hasmetadata
--------+---------+-----------+----------+----------+-------------
1 | 1 | localhost | 57637 | default | f
2 | 2 | localhost | 57638 | default | f
(2 rows)
SELECT * FROM pg_dist_partition ORDER BY logicalrelid;
logicalrelid | partmethod | partkey | colocationid | repmodel
---------------------------------+------------+------------------------------------------------------------------------------------------------------------------------+--------------+----------
mx_testing_schema.mx_test_table | h | {VAR :varno 1 :varattno 1 :vartype 23 :vartypmod -1 :varcollid 0 :varlevelsup 0 :varnoold 1 :varoattno 1 :location -1} | 0 | s
(1 row)
SELECT * FROM pg_dist_shard ORDER BY shardid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue
---------------------------------+---------+--------------+---------------+---------------
mx_testing_schema.mx_test_table | 1310000 | t | -2147483648 | -1610612737
mx_testing_schema.mx_test_table | 1310001 | t | -1610612736 | -1073741825
mx_testing_schema.mx_test_table | 1310002 | t | -1073741824 | -536870913
mx_testing_schema.mx_test_table | 1310003 | t | -536870912 | -1
mx_testing_schema.mx_test_table | 1310004 | t | 0 | 536870911
mx_testing_schema.mx_test_table | 1310005 | t | 536870912 | 1073741823
mx_testing_schema.mx_test_table | 1310006 | t | 1073741824 | 1610612735
mx_testing_schema.mx_test_table | 1310007 | t | 1610612736 | 2147483647
(8 rows)
SELECT * FROM pg_dist_shard_placement ORDER BY shardid;
shardid | shardstate | shardlength | nodename | nodeport | placementid
---------+------------+-------------+-----------+----------+-------------
1310000 | 1 | 0 | localhost | 57637 | 100000
1310001 | 1 | 0 | localhost | 57638 | 100001
1310002 | 1 | 0 | localhost | 57637 | 100002
1310003 | 1 | 0 | localhost | 57638 | 100003
1310004 | 1 | 0 | localhost | 57637 | 100004
1310005 | 1 | 0 | localhost | 57638 | 100005
1310006 | 1 | 0 | localhost | 57637 | 100006
1310007 | 1 | 0 | localhost | 57638 | 100007
(8 rows)
\d mx_testing_schema.mx_test_table
Table "mx_testing_schema.mx_test_table"
Column | Type | Modifiers
--------+---------+---------------------------------------------------------------------------------
col_1 | integer |
col_2 | text | not null
col_3 | integer | not null default nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass)
Indexes:
"mx_test_table_col_1_key" UNIQUE CONSTRAINT, btree (col_1)
"mx_index" btree (col_2)
-- Check that pg_dist_colocation is not synced
SELECT * FROM pg_dist_colocation ORDER BY colocationid;
colocationid | shardcount | replicationfactor | distributioncolumntype
--------------+------------+-------------------+------------------------
(0 rows)
-- Make sure that truncate trigger has been set for the MX table on worker
SELECT count(*) FROM pg_trigger WHERE tgrelid='mx_testing_schema.mx_test_table'::regclass;
count
-------
1
(1 row)
-- Make sure that start_metadata_sync_to_node considers foreign key constraints
SET citus.shard_replication_factor TO 1;
CREATE SCHEMA mx_testing_schema_2;
CREATE TABLE mx_testing_schema.fk_test_1 (col1 int, col2 text, col3 int, UNIQUE(col1, col3));
CREATE TABLE mx_testing_schema_2.fk_test_2 (col1 int, col2 int, col3 text,
FOREIGN KEY (col1, col2) REFERENCES mx_testing_schema.fk_test_1 (col1, col3));
SELECT create_distributed_table('mx_testing_schema.fk_test_1', 'col1');
create_distributed_table
--------------------------
(1 row)
SELECT create_distributed_table('mx_testing_schema_2.fk_test_2', 'col1');
create_distributed_table
--------------------------
(1 row)
UPDATE
pg_dist_partition SET repmodel='s'
WHERE
logicalrelid='mx_testing_schema.fk_test_1'::regclass
OR logicalrelid='mx_testing_schema_2.fk_test_2'::regclass;
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
start_metadata_sync_to_node
-----------------------------
(1 row)
-- Check that foreign key metadata exists on the worker
\c - - - :worker_1_port
\d mx_testing_schema_2.fk_test_2
Table "mx_testing_schema_2.fk_test_2"
Column | Type | Modifiers
--------+---------+-----------
col1 | integer |
col2 | integer |
col3 | text |
Foreign-key constraints:
"fk_test_2_col1_fkey" FOREIGN KEY (col1, col2) REFERENCES mx_testing_schema.fk_test_1(col1, col3)
\c - - - :master_port
RESET citus.shard_replication_factor;
-- Check that repeated calls to start_metadata_sync_to_node has no side effects
\c - - - :master_port
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
start_metadata_sync_to_node
-----------------------------
(1 row)
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
start_metadata_sync_to_node
-----------------------------
(1 row)
\c - - - :worker_1_port
SELECT * FROM pg_dist_local_group;
groupid
---------
1
(1 row)
SELECT * FROM pg_dist_node ORDER BY nodeid;
nodeid | groupid | nodename | nodeport | noderack | hasmetadata
--------+---------+-----------+----------+----------+-------------
1 | 1 | localhost | 57637 | default | t
2 | 2 | localhost | 57638 | default | f
(2 rows)
SELECT * FROM pg_dist_partition ORDER BY logicalrelid;
logicalrelid | partmethod | partkey | colocationid | repmodel
---------------------------------+------------+------------------------------------------------------------------------------------------------------------------------+--------------+----------
mx_testing_schema.mx_test_table | h | {VAR :varno 1 :varattno 1 :vartype 23 :vartypmod -1 :varcollid 0 :varlevelsup 0 :varnoold 1 :varoattno 1 :location -1} | 0 | s
(1 row)
SELECT * FROM pg_dist_shard ORDER BY shardid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue
---------------------------------+---------+--------------+---------------+---------------
mx_testing_schema.mx_test_table | 1310000 | t | -2147483648 | -1610612737
mx_testing_schema.mx_test_table | 1310001 | t | -1610612736 | -1073741825
mx_testing_schema.mx_test_table | 1310002 | t | -1073741824 | -536870913
mx_testing_schema.mx_test_table | 1310003 | t | -536870912 | -1
mx_testing_schema.mx_test_table | 1310004 | t | 0 | 536870911
mx_testing_schema.mx_test_table | 1310005 | t | 536870912 | 1073741823
mx_testing_schema.mx_test_table | 1310006 | t | 1073741824 | 1610612735
mx_testing_schema.mx_test_table | 1310007 | t | 1610612736 | 2147483647
(8 rows)
SELECT * FROM pg_dist_shard_placement ORDER BY shardid;
shardid | shardstate | shardlength | nodename | nodeport | placementid
---------+------------+-------------+-----------+----------+-------------
1310000 | 1 | 0 | localhost | 57637 | 100000
1310001 | 1 | 0 | localhost | 57638 | 100001
1310002 | 1 | 0 | localhost | 57637 | 100002
1310003 | 1 | 0 | localhost | 57638 | 100003
1310004 | 1 | 0 | localhost | 57637 | 100004
1310005 | 1 | 0 | localhost | 57638 | 100005
1310006 | 1 | 0 | localhost | 57637 | 100006
1310007 | 1 | 0 | localhost | 57638 | 100007
(8 rows)
\d mx_testing_schema.mx_test_table
Table "mx_testing_schema.mx_test_table"
Column | Type | Modifiers
--------+---------+---------------------------------------------------------------------------------
col_1 | integer |
col_2 | text | not null
col_3 | integer | not null default nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass)
Indexes:
"mx_test_table_col_1_key" UNIQUE CONSTRAINT, btree (col_1)
"mx_index" btree (col_2)
SELECT count(*) FROM pg_trigger WHERE tgrelid='mx_testing_schema.mx_test_table'::regclass;
count
-------
1
(1 row)
-- Make sure that start_metadata_sync_to_node cannot be called inside a transaction
\c - - - :master_port
BEGIN;
SELECT start_metadata_sync_to_node('localhost', :worker_2_port);
ERROR: start_metadata_sync_to_node cannot run inside a transaction block
ROLLBACK;
SELECT hasmetadata FROM pg_dist_node WHERE nodeport=:worker_2_port;
hasmetadata
-------------
f
(1 row)
-- Check that stop_metadata_sync_to_node function sets hasmetadata of the node to false
\c - - - :master_port
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
start_metadata_sync_to_node
-----------------------------
(1 row)
SELECT hasmetadata FROM pg_dist_node WHERE nodeport=:worker_1_port;
hasmetadata
-------------
t
(1 row)
SELECT stop_metadata_sync_to_node('localhost', :worker_1_port);
stop_metadata_sync_to_node
----------------------------
(1 row)
SELECT hasmetadata FROM pg_dist_node WHERE nodeport=:worker_1_port;
hasmetadata
-------------
f
(1 row)
-- Cleanup
\c - - - :worker_1_port
DROP TABLE mx_testing_schema.mx_test_table;
DELETE FROM pg_dist_node;
DELETE FROM pg_dist_partition;
DELETE FROM pg_dist_shard;
DELETE FROM pg_dist_shard_placement;
\d mx_testing_schema.mx_test_table
\c - - - :master_port
SELECT stop_metadata_sync_to_node('localhost', :worker_1_port);
stop_metadata_sync_to_node
----------------------------
(1 row)
SELECT stop_metadata_sync_to_node('localhost', :worker_2_port);
stop_metadata_sync_to_node
----------------------------
(1 row)
DROP TABLE mx_testing_schema.mx_test_table CASCADE;
ALTER SEQUENCE pg_catalog.pg_dist_shard_placement_placementid_seq RESTART :last_placement_id; ALTER SEQUENCE pg_catalog.pg_dist_shard_placement_placementid_seq RESTART :last_placement_id;

View File

@ -58,6 +58,9 @@ ALTER EXTENSION citus UPDATE TO '6.0-15';
ALTER EXTENSION citus UPDATE TO '6.0-16'; ALTER EXTENSION citus UPDATE TO '6.0-16';
ALTER EXTENSION citus UPDATE TO '6.0-17'; ALTER EXTENSION citus UPDATE TO '6.0-17';
ALTER EXTENSION citus UPDATE TO '6.0-18'; ALTER EXTENSION citus UPDATE TO '6.0-18';
ALTER EXTENSION citus UPDATE TO '6.1-1';
ALTER EXTENSION citus UPDATE TO '6.1-2';
ALTER EXTENSION citus UPDATE TO '6.1-3';
-- ensure no objects were created outside pg_catalog -- ensure no objects were created outside pg_catalog
SELECT COUNT(*) SELECT COUNT(*)

View File

@ -59,4 +59,97 @@ SELECT unnest(master_metadata_snapshot());
UPDATE pg_dist_partition SET partmethod='r' WHERE logicalrelid='non_mx_test_table'::regclass; UPDATE pg_dist_partition SET partmethod='r' WHERE logicalrelid='non_mx_test_table'::regclass;
SELECT unnest(master_metadata_snapshot()); SELECT unnest(master_metadata_snapshot());
-- Test start_metadata_sync_to_node UDF
-- Ensure that hasmetadata=false for all nodes
SELECT count(*) FROM pg_dist_node WHERE hasmetadata=true;
-- Run start_metadata_sync_to_node and check that it marked hasmetadata for that worker
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
SELECT nodeid, hasmetadata FROM pg_dist_node WHERE nodename='localhost' AND nodeport=:worker_1_port;
-- Check that the metadata has been copied to the worker
\c - - - :worker_1_port
SELECT * FROM pg_dist_local_group;
SELECT * FROM pg_dist_node ORDER BY nodeid;
SELECT * FROM pg_dist_partition ORDER BY logicalrelid;
SELECT * FROM pg_dist_shard ORDER BY shardid;
SELECT * FROM pg_dist_shard_placement ORDER BY shardid;
\d mx_testing_schema.mx_test_table
-- Check that pg_dist_colocation is not synced
SELECT * FROM pg_dist_colocation ORDER BY colocationid;
-- Make sure that truncate trigger has been set for the MX table on worker
SELECT count(*) FROM pg_trigger WHERE tgrelid='mx_testing_schema.mx_test_table'::regclass;
-- Make sure that start_metadata_sync_to_node considers foreign key constraints
SET citus.shard_replication_factor TO 1;
CREATE SCHEMA mx_testing_schema_2;
CREATE TABLE mx_testing_schema.fk_test_1 (col1 int, col2 text, col3 int, UNIQUE(col1, col3));
CREATE TABLE mx_testing_schema_2.fk_test_2 (col1 int, col2 int, col3 text,
FOREIGN KEY (col1, col2) REFERENCES mx_testing_schema.fk_test_1 (col1, col3));
SELECT create_distributed_table('mx_testing_schema.fk_test_1', 'col1');
SELECT create_distributed_table('mx_testing_schema_2.fk_test_2', 'col1');
UPDATE
pg_dist_partition SET repmodel='s'
WHERE
logicalrelid='mx_testing_schema.fk_test_1'::regclass
OR logicalrelid='mx_testing_schema_2.fk_test_2'::regclass;
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
-- Check that foreign key metadata exists on the worker
\c - - - :worker_1_port
\d mx_testing_schema_2.fk_test_2
\c - - - :master_port
RESET citus.shard_replication_factor;
-- Check that repeated calls to start_metadata_sync_to_node has no side effects
\c - - - :master_port
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
\c - - - :worker_1_port
SELECT * FROM pg_dist_local_group;
SELECT * FROM pg_dist_node ORDER BY nodeid;
SELECT * FROM pg_dist_partition ORDER BY logicalrelid;
SELECT * FROM pg_dist_shard ORDER BY shardid;
SELECT * FROM pg_dist_shard_placement ORDER BY shardid;
\d mx_testing_schema.mx_test_table
SELECT count(*) FROM pg_trigger WHERE tgrelid='mx_testing_schema.mx_test_table'::regclass;
-- Make sure that start_metadata_sync_to_node cannot be called inside a transaction
\c - - - :master_port
BEGIN;
SELECT start_metadata_sync_to_node('localhost', :worker_2_port);
ROLLBACK;
SELECT hasmetadata FROM pg_dist_node WHERE nodeport=:worker_2_port;
-- Check that stop_metadata_sync_to_node function sets hasmetadata of the node to false
\c - - - :master_port
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
SELECT hasmetadata FROM pg_dist_node WHERE nodeport=:worker_1_port;
SELECT stop_metadata_sync_to_node('localhost', :worker_1_port);
SELECT hasmetadata FROM pg_dist_node WHERE nodeport=:worker_1_port;
-- Cleanup
\c - - - :worker_1_port
DROP TABLE mx_testing_schema.mx_test_table;
DELETE FROM pg_dist_node;
DELETE FROM pg_dist_partition;
DELETE FROM pg_dist_shard;
DELETE FROM pg_dist_shard_placement;
\d mx_testing_schema.mx_test_table
\c - - - :master_port
SELECT stop_metadata_sync_to_node('localhost', :worker_1_port);
SELECT stop_metadata_sync_to_node('localhost', :worker_2_port);
DROP TABLE mx_testing_schema.mx_test_table CASCADE;
ALTER SEQUENCE pg_catalog.pg_dist_shard_placement_placementid_seq RESTART :last_placement_id; ALTER SEQUENCE pg_catalog.pg_dist_shard_placement_placementid_seq RESTART :last_placement_id;