mirror of https://github.com/citusdata/citus.git
Merge pull request #1045 from citusdata/propagate_mx_metadata_changes
Propagate MX Metadata Changespull/1062/head
commit
d608ef3311
|
@ -9,7 +9,7 @@ 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 6.1-4 6.1-5 6.1-6 6.1-7 6.1-8
|
6.1-1 6.1-2 6.1-3 6.1-4 6.1-5 6.1-6 6.1-7 6.1-8 6.1-9
|
||||||
|
|
||||||
# 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))
|
||||||
|
@ -111,6 +111,8 @@ $(EXTENSION)--6.1-7.sql: $(EXTENSION)--6.1-6.sql $(EXTENSION)--6.1-6--6.1-7.sql
|
||||||
cat $^ > $@
|
cat $^ > $@
|
||||||
$(EXTENSION)--6.1-8.sql: $(EXTENSION)--6.1-7.sql $(EXTENSION)--6.1-7--6.1-8.sql
|
$(EXTENSION)--6.1-8.sql: $(EXTENSION)--6.1-7.sql $(EXTENSION)--6.1-7--6.1-8.sql
|
||||||
cat $^ > $@
|
cat $^ > $@
|
||||||
|
$(EXTENSION)--6.1-9.sql: $(EXTENSION)--6.1-8.sql $(EXTENSION)--6.1-8--6.1-9.sql
|
||||||
|
cat $^ > $@
|
||||||
|
|
||||||
NO_PGXS = 1
|
NO_PGXS = 1
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,89 @@
|
||||||
|
/* citus--6.1-8--6.1-9.sql */
|
||||||
|
|
||||||
|
SET search_path = 'pg_catalog';
|
||||||
|
|
||||||
|
CREATE FUNCTION master_drop_distributed_table_metadata(logicalrelid regclass,
|
||||||
|
schema_name text,
|
||||||
|
table_name text)
|
||||||
|
RETURNS void
|
||||||
|
LANGUAGE C STRICT
|
||||||
|
AS 'MODULE_PATHNAME', $$master_drop_distributed_table_metadata$$;
|
||||||
|
COMMENT ON FUNCTION master_drop_distributed_table_metadata(logicalrelid regclass,
|
||||||
|
schema_name text,
|
||||||
|
table_name text)
|
||||||
|
IS 'delete metadata of the distributed table';
|
||||||
|
|
||||||
|
CREATE OR REPLACE FUNCTION pg_catalog.citus_drop_trigger()
|
||||||
|
RETURNS event_trigger
|
||||||
|
LANGUAGE plpgsql
|
||||||
|
SECURITY DEFINER
|
||||||
|
SET search_path = pg_catalog
|
||||||
|
AS $cdbdt$
|
||||||
|
DECLARE
|
||||||
|
v_obj record;
|
||||||
|
sequence_names text[] := '{}';
|
||||||
|
node_names text[] := '{}';
|
||||||
|
node_ports bigint[] := '{}';
|
||||||
|
node_name text;
|
||||||
|
node_port bigint;
|
||||||
|
table_colocation_id integer;
|
||||||
|
BEGIN
|
||||||
|
-- collect set of dropped sequences to drop on workers later
|
||||||
|
SELECT array_agg(object_identity) INTO sequence_names
|
||||||
|
FROM pg_event_trigger_dropped_objects()
|
||||||
|
WHERE object_type = 'sequence';
|
||||||
|
|
||||||
|
-- Must accumulate set of affected nodes before deleting placements, as
|
||||||
|
-- master_drop_all_shards will erase their rows, making it impossible for
|
||||||
|
-- us to know where to drop sequences (which must be dropped after shards,
|
||||||
|
-- since they have default value expressions which depend on sequences).
|
||||||
|
SELECT array_agg(sp.nodename), array_agg(sp.nodeport)
|
||||||
|
INTO node_names, node_ports
|
||||||
|
FROM pg_event_trigger_dropped_objects() AS dobj,
|
||||||
|
pg_dist_shard AS s,
|
||||||
|
pg_dist_shard_placement AS sp
|
||||||
|
WHERE dobj.object_type IN ('table', 'foreign table')
|
||||||
|
AND dobj.objid = s.logicalrelid
|
||||||
|
AND s.shardid = sp.shardid;
|
||||||
|
|
||||||
|
FOR v_obj IN SELECT * FROM pg_event_trigger_dropped_objects() LOOP
|
||||||
|
IF v_obj.object_type NOT IN ('table', 'foreign table') THEN
|
||||||
|
CONTINUE;
|
||||||
|
END IF;
|
||||||
|
|
||||||
|
-- nothing to do if not a distributed table
|
||||||
|
IF NOT EXISTS(SELECT * FROM pg_dist_partition WHERE logicalrelid = v_obj.objid) THEN
|
||||||
|
CONTINUE;
|
||||||
|
END IF;
|
||||||
|
|
||||||
|
-- get colocation group
|
||||||
|
SELECT colocationid INTO table_colocation_id FROM pg_dist_partition WHERE logicalrelid = v_obj.objid;
|
||||||
|
|
||||||
|
-- ensure all shards are dropped
|
||||||
|
PERFORM master_drop_all_shards(v_obj.objid, v_obj.schema_name, v_obj.object_name);
|
||||||
|
|
||||||
|
PERFORM master_drop_distributed_table_metadata(v_obj.objid, v_obj.schema_name, v_obj.object_name);
|
||||||
|
|
||||||
|
-- drop colocation group if all referencing tables are dropped
|
||||||
|
IF NOT EXISTS(SELECT * FROM pg_dist_partition WHERE colocationId = table_colocation_id) THEN
|
||||||
|
DELETE FROM pg_dist_colocation WHERE colocationId = table_colocation_id;
|
||||||
|
END IF;
|
||||||
|
END LOOP;
|
||||||
|
|
||||||
|
IF cardinality(sequence_names) = 0 THEN
|
||||||
|
RETURN;
|
||||||
|
END IF;
|
||||||
|
|
||||||
|
FOR node_name, node_port IN
|
||||||
|
SELECT DISTINCT name, port
|
||||||
|
FROM unnest(node_names, node_ports) AS nodes(name, port)
|
||||||
|
LOOP
|
||||||
|
PERFORM master_drop_sequences(sequence_names, node_name, node_port);
|
||||||
|
END LOOP;
|
||||||
|
END;
|
||||||
|
$cdbdt$;
|
||||||
|
|
||||||
|
COMMENT ON FUNCTION citus_drop_trigger()
|
||||||
|
IS 'perform checks and actions at the end of DROP actions';
|
||||||
|
|
||||||
|
RESET search_path;
|
|
@ -1,6 +1,6 @@
|
||||||
# Citus extension
|
# Citus extension
|
||||||
comment = 'Citus distributed database'
|
comment = 'Citus distributed database'
|
||||||
default_version = '6.1-8'
|
default_version = '6.1-9'
|
||||||
module_pathname = '$libdir/citus'
|
module_pathname = '$libdir/citus'
|
||||||
relocatable = false
|
relocatable = false
|
||||||
schema = pg_catalog
|
schema = pg_catalog
|
||||||
|
|
|
@ -38,9 +38,11 @@
|
||||||
#include "distributed/master_metadata_utility.h"
|
#include "distributed/master_metadata_utility.h"
|
||||||
#include "distributed/master_protocol.h"
|
#include "distributed/master_protocol.h"
|
||||||
#include "distributed/metadata_cache.h"
|
#include "distributed/metadata_cache.h"
|
||||||
|
#include "distributed/metadata_sync.h"
|
||||||
#include "distributed/multi_logical_planner.h"
|
#include "distributed/multi_logical_planner.h"
|
||||||
#include "distributed/pg_dist_colocation.h"
|
#include "distributed/pg_dist_colocation.h"
|
||||||
#include "distributed/pg_dist_partition.h"
|
#include "distributed/pg_dist_partition.h"
|
||||||
|
#include "distributed/worker_transaction.h"
|
||||||
#include "executor/spi.h"
|
#include "executor/spi.h"
|
||||||
#include "nodes/execnodes.h"
|
#include "nodes/execnodes.h"
|
||||||
#include "nodes/nodeFuncs.h"
|
#include "nodes/nodeFuncs.h"
|
||||||
|
@ -79,8 +81,7 @@ static void InsertIntoPgDistPartition(Oid relationId, char distributionMethod,
|
||||||
char replicationModel);
|
char replicationModel);
|
||||||
static void CreateHashDistributedTable(Oid relationId, char *distributionColumnName,
|
static void CreateHashDistributedTable(Oid relationId, char *distributionColumnName,
|
||||||
char *colocateWithTableName,
|
char *colocateWithTableName,
|
||||||
int shardCount, int replicationFactor,
|
int shardCount, int replicationFactor);
|
||||||
char replicationModel);
|
|
||||||
static Oid ColumnType(Oid relationId, char *columnName);
|
static Oid ColumnType(Oid relationId, char *columnName);
|
||||||
|
|
||||||
|
|
||||||
|
@ -172,8 +173,23 @@ create_distributed_table(PG_FUNCTION_ARGS)
|
||||||
/* use configuration values for shard count and shard replication factor */
|
/* use configuration values for shard count and shard replication factor */
|
||||||
CreateHashDistributedTable(relationId, distributionColumnName,
|
CreateHashDistributedTable(relationId, distributionColumnName,
|
||||||
colocateWithTableName, ShardCount,
|
colocateWithTableName, ShardCount,
|
||||||
ShardReplicationFactor,
|
ShardReplicationFactor);
|
||||||
REPLICATION_MODEL_COORDINATOR);
|
|
||||||
|
if (ShouldSyncTableMetadata(relationId))
|
||||||
|
{
|
||||||
|
List *commandList = GetDistributedTableDDLEvents(relationId);
|
||||||
|
ListCell *commandCell = NULL;
|
||||||
|
|
||||||
|
SendCommandToWorkers(WORKERS_WITH_METADATA, DISABLE_DDL_PROPAGATION);
|
||||||
|
|
||||||
|
/* send the commands one by one */
|
||||||
|
foreach(commandCell, commandList)
|
||||||
|
{
|
||||||
|
char *command = (char *) lfirst(commandCell);
|
||||||
|
|
||||||
|
SendCommandToWorkers(WORKERS_WITH_METADATA, command);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
PG_RETURN_VOID();
|
PG_RETURN_VOID();
|
||||||
}
|
}
|
||||||
|
@ -953,17 +969,28 @@ CreateTruncateTrigger(Oid relationId)
|
||||||
static void
|
static void
|
||||||
CreateHashDistributedTable(Oid relationId, char *distributionColumnName,
|
CreateHashDistributedTable(Oid relationId, char *distributionColumnName,
|
||||||
char *colocateWithTableName, int shardCount,
|
char *colocateWithTableName, int shardCount,
|
||||||
int replicationFactor, char replicationModel)
|
int replicationFactor)
|
||||||
{
|
{
|
||||||
Relation distributedRelation = NULL;
|
Relation distributedRelation = NULL;
|
||||||
Relation pgDistColocation = NULL;
|
Relation pgDistColocation = NULL;
|
||||||
uint32 colocationId = INVALID_COLOCATION_ID;
|
uint32 colocationId = INVALID_COLOCATION_ID;
|
||||||
Oid sourceRelationId = InvalidOid;
|
Oid sourceRelationId = InvalidOid;
|
||||||
Oid distributionColumnType = InvalidOid;
|
Oid distributionColumnType = InvalidOid;
|
||||||
|
char replicationModel = 0;
|
||||||
|
|
||||||
/* get an access lock on the relation to prevent DROP TABLE and ALTER TABLE */
|
/* get an access lock on the relation to prevent DROP TABLE and ALTER TABLE */
|
||||||
distributedRelation = relation_open(relationId, AccessShareLock);
|
distributedRelation = relation_open(relationId, AccessShareLock);
|
||||||
|
|
||||||
|
/* all hash-distributed tables with repfactor=1 are treated as MX tables */
|
||||||
|
if (replicationFactor == 1)
|
||||||
|
{
|
||||||
|
replicationModel = REPLICATION_MODEL_STREAMING;
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
replicationModel = REPLICATION_MODEL_COORDINATOR;
|
||||||
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Get an exclusive lock on the colocation system catalog. Therefore, we
|
* Get an exclusive lock on the colocation system catalog. Therefore, we
|
||||||
* can be sure that there will no modifications on the colocation table
|
* can be sure that there will no modifications on the colocation table
|
||||||
|
@ -1004,7 +1031,7 @@ CreateHashDistributedTable(Oid relationId, char *distributionColumnName,
|
||||||
|
|
||||||
/* create distributed table metadata */
|
/* create distributed table metadata */
|
||||||
ConvertToDistributedTable(relationId, distributionColumnName, DISTRIBUTE_BY_HASH,
|
ConvertToDistributedTable(relationId, distributionColumnName, DISTRIBUTE_BY_HASH,
|
||||||
colocationId, REPLICATION_MODEL_COORDINATOR);
|
colocationId, replicationModel);
|
||||||
|
|
||||||
/* create shards */
|
/* create shards */
|
||||||
if (sourceRelationId != InvalidOid)
|
if (sourceRelationId != InvalidOid)
|
||||||
|
|
|
@ -0,0 +1,56 @@
|
||||||
|
/*-------------------------------------------------------------------------
|
||||||
|
*
|
||||||
|
* drop_distributed_table.c
|
||||||
|
* Routines related to dropping distributed relations from a trigger.
|
||||||
|
*
|
||||||
|
* Copyright (c) 2012-2016, Citus Data, Inc.
|
||||||
|
*
|
||||||
|
*-------------------------------------------------------------------------
|
||||||
|
*/
|
||||||
|
|
||||||
|
#include "postgres.h"
|
||||||
|
#include "miscadmin.h"
|
||||||
|
|
||||||
|
#include "distributed/master_metadata_utility.h"
|
||||||
|
#include "distributed/master_protocol.h"
|
||||||
|
#include "distributed/metadata_sync.h"
|
||||||
|
#include "distributed/worker_transaction.h"
|
||||||
|
#include "utils/builtins.h"
|
||||||
|
#include "utils/lsyscache.h"
|
||||||
|
|
||||||
|
|
||||||
|
/* exports for SQL callable functions */
|
||||||
|
PG_FUNCTION_INFO_V1(master_drop_distributed_table_metadata);
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* master_drop_distributed_table_metadata removes the entry of the specified distributed
|
||||||
|
* table from pg_dist_partition and drops the table from the workers if needed.
|
||||||
|
*/
|
||||||
|
Datum
|
||||||
|
master_drop_distributed_table_metadata(PG_FUNCTION_ARGS)
|
||||||
|
{
|
||||||
|
Oid relationId = PG_GETARG_OID(0);
|
||||||
|
text *schemaNameText = PG_GETARG_TEXT_P(1);
|
||||||
|
text *tableNameText = PG_GETARG_TEXT_P(2);
|
||||||
|
bool shouldSyncMetadata = false;
|
||||||
|
|
||||||
|
char *schemaName = text_to_cstring(schemaNameText);
|
||||||
|
char *tableName = text_to_cstring(tableNameText);
|
||||||
|
|
||||||
|
CheckTableSchemaNameForDrop(relationId, &schemaName, &tableName);
|
||||||
|
|
||||||
|
DeletePartitionRow(relationId);
|
||||||
|
|
||||||
|
shouldSyncMetadata = ShouldSyncTableMetadata(relationId);
|
||||||
|
if (shouldSyncMetadata)
|
||||||
|
{
|
||||||
|
char *deleteDistributionCommand = NULL;
|
||||||
|
|
||||||
|
/* drop the distributed table metadata on the workers */
|
||||||
|
deleteDistributionCommand = DistributionDeleteCommand(schemaName, tableName);
|
||||||
|
SendCommandToWorkers(WORKERS_WITH_METADATA, deleteDistributionCommand);
|
||||||
|
}
|
||||||
|
|
||||||
|
PG_RETURN_VOID();
|
||||||
|
}
|
|
@ -37,6 +37,7 @@
|
||||||
#include "distributed/master_metadata_utility.h"
|
#include "distributed/master_metadata_utility.h"
|
||||||
#include "distributed/master_protocol.h"
|
#include "distributed/master_protocol.h"
|
||||||
#include "distributed/metadata_cache.h"
|
#include "distributed/metadata_cache.h"
|
||||||
|
#include "distributed/metadata_sync.h"
|
||||||
#include "distributed/multi_copy.h"
|
#include "distributed/multi_copy.h"
|
||||||
#include "distributed/multi_join_order.h"
|
#include "distributed/multi_join_order.h"
|
||||||
#include "distributed/multi_planner.h"
|
#include "distributed/multi_planner.h"
|
||||||
|
@ -49,6 +50,7 @@
|
||||||
#include "distributed/transaction_management.h"
|
#include "distributed/transaction_management.h"
|
||||||
#include "distributed/transmit.h"
|
#include "distributed/transmit.h"
|
||||||
#include "distributed/worker_protocol.h"
|
#include "distributed/worker_protocol.h"
|
||||||
|
#include "distributed/worker_transaction.h"
|
||||||
#include "executor/executor.h"
|
#include "executor/executor.h"
|
||||||
#include "foreign/foreign.h"
|
#include "foreign/foreign.h"
|
||||||
#include "lib/stringinfo.h"
|
#include "lib/stringinfo.h"
|
||||||
|
@ -107,6 +109,8 @@ static Node * ProcessDropIndexStmt(DropStmt *dropIndexStatement,
|
||||||
const char *dropIndexCommand, bool isTopLevel);
|
const char *dropIndexCommand, bool isTopLevel);
|
||||||
static Node * ProcessAlterTableStmt(AlterTableStmt *alterTableStatement,
|
static Node * ProcessAlterTableStmt(AlterTableStmt *alterTableStatement,
|
||||||
const char *alterTableCommand, bool isTopLevel);
|
const char *alterTableCommand, bool isTopLevel);
|
||||||
|
static Node * WorkerProcessAlterTableStmt(AlterTableStmt *alterTableStatement,
|
||||||
|
const char *alterTableCommand);
|
||||||
static Node * ProcessAlterObjectSchemaStmt(AlterObjectSchemaStmt *alterObjectSchemaStmt,
|
static Node * ProcessAlterObjectSchemaStmt(AlterObjectSchemaStmt *alterObjectSchemaStmt,
|
||||||
const char *alterObjectSchemaCommand,
|
const char *alterObjectSchemaCommand,
|
||||||
bool isTopLevel);
|
bool isTopLevel);
|
||||||
|
@ -147,6 +151,7 @@ static List * CopyGetAttnums(TupleDesc tupDesc, Relation rel, List *attnamelist)
|
||||||
|
|
||||||
static bool warnedUserAbout2PC = false;
|
static bool warnedUserAbout2PC = false;
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Utility for handling citus specific concerns around utility statements.
|
* Utility for handling citus specific concerns around utility statements.
|
||||||
*
|
*
|
||||||
|
@ -167,6 +172,8 @@ multi_ProcessUtility(Node *parsetree,
|
||||||
DestReceiver *dest,
|
DestReceiver *dest,
|
||||||
char *completionTag)
|
char *completionTag)
|
||||||
{
|
{
|
||||||
|
bool schemaNode = SchemaNode();
|
||||||
|
bool propagateChanges = schemaNode && EnableDDLPropagation;
|
||||||
bool commandMustRunAsOwner = false;
|
bool commandMustRunAsOwner = false;
|
||||||
Oid savedUserId = InvalidOid;
|
Oid savedUserId = InvalidOid;
|
||||||
int savedSecurityContext = 0;
|
int savedSecurityContext = 0;
|
||||||
|
@ -222,8 +229,11 @@ multi_ProcessUtility(Node *parsetree,
|
||||||
ErrorIfUnsupportedTruncateStmt((TruncateStmt *) parsetree);
|
ErrorIfUnsupportedTruncateStmt((TruncateStmt *) parsetree);
|
||||||
}
|
}
|
||||||
|
|
||||||
/* ddl commands are propagated to workers only if EnableDDLPropagation is set */
|
/*
|
||||||
if (EnableDDLPropagation)
|
* DDL commands are propagated to workers only if EnableDDLPropagation is
|
||||||
|
* set to true and the current node is the schema node
|
||||||
|
*/
|
||||||
|
if (propagateChanges)
|
||||||
{
|
{
|
||||||
bool isTopLevel = (context == PROCESS_UTILITY_TOPLEVEL);
|
bool isTopLevel = (context == PROCESS_UTILITY_TOPLEVEL);
|
||||||
|
|
||||||
|
@ -289,6 +299,24 @@ multi_ProcessUtility(Node *parsetree,
|
||||||
"move all tables.")));
|
"move all tables.")));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
else if (!schemaNode)
|
||||||
|
{
|
||||||
|
if (IsA(parsetree, AlterTableStmt))
|
||||||
|
{
|
||||||
|
AlterTableStmt *alterTableStmt = (AlterTableStmt *) parsetree;
|
||||||
|
if (alterTableStmt->relkind == OBJECT_TABLE)
|
||||||
|
{
|
||||||
|
/*
|
||||||
|
* When the schema node issues an ALTER TABLE ... ADD FOREIGN KEY
|
||||||
|
* command, the validation step should be skipped on the distributed
|
||||||
|
* table of the worker. Therefore, we check whether the given ALTER
|
||||||
|
* TABLE statement is a FOREIGN KEY constraint and if so disable the
|
||||||
|
* validation step. Note that validation is done on the shard level.
|
||||||
|
*/
|
||||||
|
parsetree = WorkerProcessAlterTableStmt(alterTableStmt, queryString);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Inform the user about potential caveats.
|
* Inform the user about potential caveats.
|
||||||
|
@ -858,6 +886,68 @@ ProcessAlterTableStmt(AlterTableStmt *alterTableStatement, const char *alterTabl
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* WorkerProcessAlterTableStmt checks and processes the alter table statement to be
|
||||||
|
* worked on the distributed table of the worker node. Currently, it only processes
|
||||||
|
* ALTER TABLE ... ADD FOREIGN KEY command to skip the validation step.
|
||||||
|
*/
|
||||||
|
static Node *
|
||||||
|
WorkerProcessAlterTableStmt(AlterTableStmt *alterTableStatement,
|
||||||
|
const char *alterTableCommand)
|
||||||
|
{
|
||||||
|
LOCKMODE lockmode = 0;
|
||||||
|
Oid leftRelationId = InvalidOid;
|
||||||
|
bool isDistributedRelation = false;
|
||||||
|
List *commandList = NIL;
|
||||||
|
ListCell *commandCell = NULL;
|
||||||
|
|
||||||
|
/* first check whether a distributed relation is affected */
|
||||||
|
if (alterTableStatement->relation == NULL)
|
||||||
|
{
|
||||||
|
return (Node *) alterTableStatement;
|
||||||
|
}
|
||||||
|
|
||||||
|
lockmode = AlterTableGetLockLevel(alterTableStatement->cmds);
|
||||||
|
leftRelationId = AlterTableLookupRelation(alterTableStatement, lockmode);
|
||||||
|
if (!OidIsValid(leftRelationId))
|
||||||
|
{
|
||||||
|
return (Node *) alterTableStatement;
|
||||||
|
}
|
||||||
|
|
||||||
|
isDistributedRelation = IsDistributedTable(leftRelationId);
|
||||||
|
if (!isDistributedRelation)
|
||||||
|
{
|
||||||
|
return (Node *) alterTableStatement;
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* We check if there is a ADD FOREIGN CONSTRAINT command in sub commands list.
|
||||||
|
* If there is we assign referenced releation id to rightRelationId and we also
|
||||||
|
* set skip_validation to true to prevent PostgreSQL to verify validity of the
|
||||||
|
* foreign constraint in master. Validity will be checked in workers anyway.
|
||||||
|
*/
|
||||||
|
commandList = alterTableStatement->cmds;
|
||||||
|
|
||||||
|
foreach(commandCell, commandList)
|
||||||
|
{
|
||||||
|
AlterTableCmd *command = (AlterTableCmd *) lfirst(commandCell);
|
||||||
|
AlterTableType alterTableType = command->subtype;
|
||||||
|
|
||||||
|
if (alterTableType == AT_AddConstraint)
|
||||||
|
{
|
||||||
|
Constraint *constraint = (Constraint *) command->def;
|
||||||
|
if (constraint->contype == CONSTR_FOREIGN)
|
||||||
|
{
|
||||||
|
/* foreign constraint validations will be done in shards. */
|
||||||
|
constraint->skip_validation = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return (Node *) alterTableStatement;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* ProcessAlterObjectSchemaStmt processes ALTER ... SET SCHEMA statements for distributed
|
* ProcessAlterObjectSchemaStmt processes ALTER ... SET SCHEMA statements for distributed
|
||||||
* objects. The function first checks if the statement belongs to a distributed objects
|
* objects. The function first checks if the statement belongs to a distributed objects
|
||||||
|
@ -1866,6 +1956,7 @@ ExecuteDistributedDDLCommand(Oid relationId, const char *ddlCommandString,
|
||||||
bool isTopLevel)
|
bool isTopLevel)
|
||||||
{
|
{
|
||||||
List *taskList = NIL;
|
List *taskList = NIL;
|
||||||
|
bool shouldSyncMetadata = ShouldSyncTableMetadata(relationId);
|
||||||
|
|
||||||
if (XactModificationLevel == XACT_MODIFICATION_DATA)
|
if (XactModificationLevel == XACT_MODIFICATION_DATA)
|
||||||
{
|
{
|
||||||
|
@ -1877,6 +1968,12 @@ ExecuteDistributedDDLCommand(Oid relationId, const char *ddlCommandString,
|
||||||
|
|
||||||
ShowNoticeIfNotUsing2PC();
|
ShowNoticeIfNotUsing2PC();
|
||||||
|
|
||||||
|
if (shouldSyncMetadata)
|
||||||
|
{
|
||||||
|
SendCommandToWorkers(WORKERS_WITH_METADATA, DISABLE_DDL_PROPAGATION);
|
||||||
|
SendCommandToWorkers(WORKERS_WITH_METADATA, (char *) ddlCommandString);
|
||||||
|
}
|
||||||
|
|
||||||
taskList = DDLTaskList(relationId, ddlCommandString);
|
taskList = DDLTaskList(relationId, ddlCommandString);
|
||||||
|
|
||||||
ExecuteModifyTasksWithoutResults(taskList);
|
ExecuteModifyTasksWithoutResults(taskList);
|
||||||
|
@ -1900,6 +1997,7 @@ ExecuteDistributedForeignKeyCommand(Oid leftRelationId, Oid rightRelationId,
|
||||||
const char *ddlCommandString, bool isTopLevel)
|
const char *ddlCommandString, bool isTopLevel)
|
||||||
{
|
{
|
||||||
List *taskList = NIL;
|
List *taskList = NIL;
|
||||||
|
bool shouldSyncMetadata = false;
|
||||||
|
|
||||||
if (XactModificationLevel == XACT_MODIFICATION_DATA)
|
if (XactModificationLevel == XACT_MODIFICATION_DATA)
|
||||||
{
|
{
|
||||||
|
@ -1911,6 +2009,18 @@ ExecuteDistributedForeignKeyCommand(Oid leftRelationId, Oid rightRelationId,
|
||||||
|
|
||||||
ShowNoticeIfNotUsing2PC();
|
ShowNoticeIfNotUsing2PC();
|
||||||
|
|
||||||
|
/*
|
||||||
|
* It is sufficient to check only one of the tables for metadata syncing on workers,
|
||||||
|
* since the colocation of two tables implies that either both or none of them have
|
||||||
|
* metadata on workers.
|
||||||
|
*/
|
||||||
|
shouldSyncMetadata = ShouldSyncTableMetadata(leftRelationId);
|
||||||
|
if (shouldSyncMetadata)
|
||||||
|
{
|
||||||
|
SendCommandToWorkers(WORKERS_WITH_METADATA, DISABLE_DDL_PROPAGATION);
|
||||||
|
SendCommandToWorkers(WORKERS_WITH_METADATA, (char *) ddlCommandString);
|
||||||
|
}
|
||||||
|
|
||||||
taskList = ForeignKeyTaskList(leftRelationId, rightRelationId, ddlCommandString);
|
taskList = ForeignKeyTaskList(leftRelationId, rightRelationId, ddlCommandString);
|
||||||
|
|
||||||
ExecuteModifyTasksWithoutResults(taskList);
|
ExecuteModifyTasksWithoutResults(taskList);
|
||||||
|
|
|
@ -196,45 +196,16 @@ master_drop_all_shards(PG_FUNCTION_ARGS)
|
||||||
text *schemaNameText = PG_GETARG_TEXT_P(1);
|
text *schemaNameText = PG_GETARG_TEXT_P(1);
|
||||||
text *relationNameText = PG_GETARG_TEXT_P(2);
|
text *relationNameText = PG_GETARG_TEXT_P(2);
|
||||||
|
|
||||||
char *schemaName = NULL;
|
|
||||||
char *relationName = NULL;
|
|
||||||
bool isTopLevel = true;
|
bool isTopLevel = true;
|
||||||
List *shardIntervalList = NIL;
|
List *shardIntervalList = NIL;
|
||||||
int droppedShardCount = 0;
|
int droppedShardCount = 0;
|
||||||
|
|
||||||
|
char *schemaName = text_to_cstring(schemaNameText);
|
||||||
|
char *relationName = text_to_cstring(relationNameText);
|
||||||
|
|
||||||
PreventTransactionChain(isTopLevel, "DROP distributed table");
|
PreventTransactionChain(isTopLevel, "DROP distributed table");
|
||||||
|
|
||||||
relationName = get_rel_name(relationId);
|
CheckTableSchemaNameForDrop(relationId, &schemaName, &relationName);
|
||||||
|
|
||||||
if (relationName != NULL)
|
|
||||||
{
|
|
||||||
/* ensure proper values are used if the table exists */
|
|
||||||
Oid schemaId = get_rel_namespace(relationId);
|
|
||||||
schemaName = get_namespace_name(schemaId);
|
|
||||||
|
|
||||||
/*
|
|
||||||
* Only allow the owner to drop all shards, this is more akin to DDL
|
|
||||||
* than DELETE.
|
|
||||||
*/
|
|
||||||
EnsureTableOwner(relationId);
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
/* table has been dropped, rely on user-supplied values */
|
|
||||||
schemaName = text_to_cstring(schemaNameText);
|
|
||||||
relationName = text_to_cstring(relationNameText);
|
|
||||||
|
|
||||||
/*
|
|
||||||
* Verify that this only is run as superuser - that's how it's used in
|
|
||||||
* our drop event trigger, and we can't verify permissions for an
|
|
||||||
* already dropped relation.
|
|
||||||
*/
|
|
||||||
if (!superuser())
|
|
||||||
{
|
|
||||||
ereport(ERROR, (errmsg("cannot drop all shards of a dropped table as "
|
|
||||||
"non-superuser")));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
shardIntervalList = LoadShardIntervalList(relationId);
|
shardIntervalList = LoadShardIntervalList(relationId);
|
||||||
droppedShardCount = DropShards(relationId, schemaName, relationName,
|
droppedShardCount = DropShards(relationId, schemaName, relationName,
|
||||||
|
@ -299,6 +270,35 @@ master_drop_sequences(PG_FUNCTION_ARGS)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* CheckTableSchemaNameForDrop errors out if the current user does not
|
||||||
|
* have permission to undistribute the given relation, taking into
|
||||||
|
* account that it may be called from the drop trigger. If the table exists,
|
||||||
|
* the function rewrites the given table and schema name.
|
||||||
|
*/
|
||||||
|
void
|
||||||
|
CheckTableSchemaNameForDrop(Oid relationId, char **schemaName, char **tableName)
|
||||||
|
{
|
||||||
|
char *tempTableName = get_rel_name(relationId);
|
||||||
|
|
||||||
|
if (tempTableName != NULL)
|
||||||
|
{
|
||||||
|
/* ensure proper values are used if the table exists */
|
||||||
|
Oid schemaId = get_rel_namespace(relationId);
|
||||||
|
(*schemaName) = get_namespace_name(schemaId);
|
||||||
|
(*tableName) = tempTableName;
|
||||||
|
|
||||||
|
EnsureTableOwner(relationId);
|
||||||
|
}
|
||||||
|
else if (!superuser())
|
||||||
|
{
|
||||||
|
/* table does not exist, must be called from drop trigger */
|
||||||
|
ereport(ERROR, (errmsg("cannot drop distributed table metadata as a "
|
||||||
|
"non-superuser")));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* DropShards drops all given shards in a relation. The id, name and schema
|
* DropShards drops all given shards in a relation. The id, name and schema
|
||||||
* for the relation are explicitly provided, since this function may be
|
* for the relation are explicitly provided, since this function may be
|
||||||
|
|
|
@ -515,6 +515,49 @@ InsertShardPlacementRow(uint64 shardId, uint64 placementId,
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* DeletePartitionRow removes the row from pg_dist_partition where the logicalrelid
|
||||||
|
* field equals to distributedRelationId. Then, the function invalidates the
|
||||||
|
* metadata cache.
|
||||||
|
*/
|
||||||
|
void
|
||||||
|
DeletePartitionRow(Oid distributedRelationId)
|
||||||
|
{
|
||||||
|
Relation pgDistPartition = NULL;
|
||||||
|
HeapTuple heapTuple = NULL;
|
||||||
|
SysScanDesc scanDescriptor = NULL;
|
||||||
|
ScanKeyData scanKey[1];
|
||||||
|
int scanKeyCount = 1;
|
||||||
|
|
||||||
|
pgDistPartition = heap_open(DistPartitionRelationId(), RowExclusiveLock);
|
||||||
|
|
||||||
|
ScanKeyInit(&scanKey[0], Anum_pg_dist_partition_logicalrelid,
|
||||||
|
BTEqualStrategyNumber, F_OIDEQ, ObjectIdGetDatum(distributedRelationId));
|
||||||
|
|
||||||
|
scanDescriptor = systable_beginscan(pgDistPartition, InvalidOid, false, NULL,
|
||||||
|
scanKeyCount, scanKey);
|
||||||
|
|
||||||
|
heapTuple = systable_getnext(scanDescriptor);
|
||||||
|
if (!HeapTupleIsValid(heapTuple))
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errmsg("could not find valid entry for partition %d",
|
||||||
|
distributedRelationId)));
|
||||||
|
}
|
||||||
|
|
||||||
|
simple_heap_delete(pgDistPartition, &heapTuple->t_self);
|
||||||
|
|
||||||
|
systable_endscan(scanDescriptor);
|
||||||
|
|
||||||
|
/* invalidate the cache */
|
||||||
|
CitusInvalidateRelcacheByRelid(distributedRelationId);
|
||||||
|
|
||||||
|
/* increment the counter so that next command can see the row */
|
||||||
|
CommandCounterIncrement();
|
||||||
|
|
||||||
|
heap_close(pgDistPartition, RowExclusiveLock);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* DeleteShardRow opens the shard system catalog, finds the unique row that has
|
* DeleteShardRow opens the shard system catalog, finds the unique row that has
|
||||||
* the given shardId, and deletes this row.
|
* the given shardId, and deletes this row.
|
||||||
|
|
|
@ -847,6 +847,17 @@ ShardStorageType(Oid relationId)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* SchemaNode function returns true if this node is identified as the
|
||||||
|
* schema/coordinator/master node of the cluster.
|
||||||
|
*/
|
||||||
|
bool
|
||||||
|
SchemaNode(void)
|
||||||
|
{
|
||||||
|
return (GetLocalGroupId() == 0);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* WorkerNodeGetDatum converts the worker node passed to it into its datum
|
* WorkerNodeGetDatum converts the worker node passed to it into its datum
|
||||||
* representation. To do this, the function first creates the heap tuple from
|
* representation. To do this, the function first creates the heap tuple from
|
||||||
|
|
|
@ -271,6 +271,53 @@ MetadataCreateCommands(void)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* GetDistributedTableDDLEvents returns the full set of DDL commands necessary to
|
||||||
|
* create the given distributed table on a worker. The list includes setting up any
|
||||||
|
* sequences, setting the owner of the table, inserting table and shard metadata,
|
||||||
|
* setting the truncate trigger and foreign key constraints.
|
||||||
|
*/
|
||||||
|
List *
|
||||||
|
GetDistributedTableDDLEvents(Oid relationId)
|
||||||
|
{
|
||||||
|
DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(relationId);
|
||||||
|
|
||||||
|
List *shardIntervalList = NIL;
|
||||||
|
List *commandList = NIL;
|
||||||
|
List *foreignConstraintCommands = NIL;
|
||||||
|
List *shardMetadataInsertCommandList = NIL;
|
||||||
|
char *tableOwnerResetCommand = NULL;
|
||||||
|
char *metadataCommand = NULL;
|
||||||
|
char *truncateTriggerCreateCommand = NULL;
|
||||||
|
|
||||||
|
/* commands to create the table */
|
||||||
|
commandList = GetTableDDLEvents(relationId);
|
||||||
|
|
||||||
|
/* command to reset the table owner */
|
||||||
|
tableOwnerResetCommand = TableOwnerResetCommand(relationId);
|
||||||
|
commandList = lappend(commandList, tableOwnerResetCommand);
|
||||||
|
|
||||||
|
/* command to insert pg_dist_partition entry */
|
||||||
|
metadataCommand = DistributionCreateCommand(cacheEntry);
|
||||||
|
commandList = lappend(commandList, metadataCommand);
|
||||||
|
|
||||||
|
/* commands to create the truncate trigger of the mx table */
|
||||||
|
truncateTriggerCreateCommand = TruncateTriggerCreateCommand(relationId);
|
||||||
|
commandList = lappend(commandList, truncateTriggerCreateCommand);
|
||||||
|
|
||||||
|
/* commands to insert pg_dist_shard & pg_dist_shard_placement entries */
|
||||||
|
shardIntervalList = LoadShardIntervalList(relationId);
|
||||||
|
shardMetadataInsertCommandList = ShardListInsertCommand(shardIntervalList);
|
||||||
|
commandList = list_concat(commandList, shardMetadataInsertCommandList);
|
||||||
|
|
||||||
|
/* commands to create foreign key constraints */
|
||||||
|
foreignConstraintCommands = GetTableForeignConstraintCommands(relationId);
|
||||||
|
commandList = list_concat(commandList, foreignConstraintCommands);
|
||||||
|
|
||||||
|
return commandList;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* MetadataDropCommands returns list of queries that are required to
|
* MetadataDropCommands returns list of queries that are required to
|
||||||
* drop all the metadata of the node that are related to clustered tables.
|
* drop all the metadata of the node that are related to clustered tables.
|
||||||
|
@ -547,6 +594,25 @@ NodeDeleteCommand(uint32 nodeId)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* ColocationIdUpdateCommand creates the SQL command to change the colocationId
|
||||||
|
* of the table with the given name to the given colocationId in pg_dist_partition
|
||||||
|
* table.
|
||||||
|
*/
|
||||||
|
char *
|
||||||
|
ColocationIdUpdateCommand(Oid relationId, uint32 colocationId)
|
||||||
|
{
|
||||||
|
StringInfo command = makeStringInfo();
|
||||||
|
char *qualifiedRelationName = generate_qualified_relation_name(relationId);
|
||||||
|
appendStringInfo(command, "UPDATE pg_dist_partition "
|
||||||
|
"SET colocationid = %d "
|
||||||
|
"WHERE logicalrelid = %s::regclass",
|
||||||
|
colocationId, quote_literal_cstr(qualifiedRelationName));
|
||||||
|
|
||||||
|
return command->data;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* LocalGroupIdUpdateCommand creates the SQL command required to set the local group id
|
* LocalGroupIdUpdateCommand creates the SQL command required to set the local group id
|
||||||
* of a worker and returns the command in a string.
|
* of a worker and returns the command in a string.
|
||||||
|
|
|
@ -22,11 +22,13 @@
|
||||||
#include "distributed/listutils.h"
|
#include "distributed/listutils.h"
|
||||||
#include "distributed/master_protocol.h"
|
#include "distributed/master_protocol.h"
|
||||||
#include "distributed/metadata_cache.h"
|
#include "distributed/metadata_cache.h"
|
||||||
|
#include "distributed/metadata_sync.h"
|
||||||
#include "distributed/multi_logical_planner.h"
|
#include "distributed/multi_logical_planner.h"
|
||||||
#include "distributed/pg_dist_colocation.h"
|
#include "distributed/pg_dist_colocation.h"
|
||||||
#include "distributed/resource_lock.h"
|
#include "distributed/resource_lock.h"
|
||||||
#include "distributed/shardinterval_utils.h"
|
#include "distributed/shardinterval_utils.h"
|
||||||
#include "distributed/worker_protocol.h"
|
#include "distributed/worker_protocol.h"
|
||||||
|
#include "distributed/worker_transaction.h"
|
||||||
#include "utils/builtins.h"
|
#include "utils/builtins.h"
|
||||||
#include "utils/fmgroids.h"
|
#include "utils/fmgroids.h"
|
||||||
#include "utils/lsyscache.h"
|
#include "utils/lsyscache.h"
|
||||||
|
@ -617,6 +619,7 @@ UpdateRelationColocationGroup(Oid distributedRelationId, uint32 colocationId)
|
||||||
HeapTuple heapTuple = NULL;
|
HeapTuple heapTuple = NULL;
|
||||||
TupleDesc tupleDescriptor = NULL;
|
TupleDesc tupleDescriptor = NULL;
|
||||||
SysScanDesc scanDescriptor = NULL;
|
SysScanDesc scanDescriptor = NULL;
|
||||||
|
bool shouldSyncMetadata = false;
|
||||||
bool indexOK = true;
|
bool indexOK = true;
|
||||||
int scanKeyCount = 1;
|
int scanKeyCount = 1;
|
||||||
ScanKeyData scanKey[scanKeyCount];
|
ScanKeyData scanKey[scanKeyCount];
|
||||||
|
@ -660,6 +663,15 @@ UpdateRelationColocationGroup(Oid distributedRelationId, uint32 colocationId)
|
||||||
|
|
||||||
systable_endscan(scanDescriptor);
|
systable_endscan(scanDescriptor);
|
||||||
heap_close(pgDistPartition, NoLock);
|
heap_close(pgDistPartition, NoLock);
|
||||||
|
|
||||||
|
shouldSyncMetadata = ShouldSyncTableMetadata(distributedRelationId);
|
||||||
|
if (shouldSyncMetadata)
|
||||||
|
{
|
||||||
|
char *updateColocationIdCommand = ColocationIdUpdateCommand(distributedRelationId,
|
||||||
|
colocationId);
|
||||||
|
|
||||||
|
SendCommandToWorkers(WORKERS_WITH_METADATA, updateColocationIdCommand);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -1377,6 +1377,7 @@ GetLocalGroupId(void)
|
||||||
TupleDesc tupleDescriptor = NULL;
|
TupleDesc tupleDescriptor = NULL;
|
||||||
Oid groupId = InvalidOid;
|
Oid groupId = InvalidOid;
|
||||||
Relation pgDistLocalGroupId = NULL;
|
Relation pgDistLocalGroupId = NULL;
|
||||||
|
Oid localGroupTableOid = InvalidOid;
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Already set the group id, no need to read the heap again.
|
* Already set the group id, no need to read the heap again.
|
||||||
|
@ -1386,7 +1387,13 @@ GetLocalGroupId(void)
|
||||||
return LocalGroupId;
|
return LocalGroupId;
|
||||||
}
|
}
|
||||||
|
|
||||||
pgDistLocalGroupId = heap_open(DistLocalGroupIdRelationId(), AccessShareLock);
|
localGroupTableOid = get_relname_relid("pg_dist_local_group", PG_CATALOG_NAMESPACE);
|
||||||
|
if (localGroupTableOid == InvalidOid)
|
||||||
|
{
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
pgDistLocalGroupId = heap_open(localGroupTableOid, AccessShareLock);
|
||||||
|
|
||||||
scanDescriptor = systable_beginscan(pgDistLocalGroupId,
|
scanDescriptor = systable_beginscan(pgDistLocalGroupId,
|
||||||
InvalidOid, false,
|
InvalidOid, false,
|
||||||
|
|
|
@ -29,9 +29,6 @@
|
||||||
PG_FUNCTION_INFO_V1(worker_drop_distributed_table);
|
PG_FUNCTION_INFO_V1(worker_drop_distributed_table);
|
||||||
|
|
||||||
|
|
||||||
static void DeletePartitionRow(Oid distributedRelationId);
|
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* worker_drop_distributed_table drops the distributed table with the given oid,
|
* worker_drop_distributed_table drops the distributed table with the given oid,
|
||||||
* then, removes the associated rows from pg_dist_partition, pg_dist_shard and
|
* then, removes the associated rows from pg_dist_partition, pg_dist_shard and
|
||||||
|
@ -133,46 +130,3 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS)
|
||||||
|
|
||||||
PG_RETURN_VOID();
|
PG_RETURN_VOID();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
|
||||||
* DeletePartitionRow removes the row from pg_dist_partition where the logicalrelid
|
|
||||||
* field equals to distributedRelationId. Then, the function invalidates the
|
|
||||||
* metadata cache.
|
|
||||||
*/
|
|
||||||
void
|
|
||||||
DeletePartitionRow(Oid distributedRelationId)
|
|
||||||
{
|
|
||||||
Relation pgDistPartition = NULL;
|
|
||||||
HeapTuple heapTuple = NULL;
|
|
||||||
SysScanDesc scanDescriptor = NULL;
|
|
||||||
ScanKeyData scanKey[1];
|
|
||||||
int scanKeyCount = 1;
|
|
||||||
|
|
||||||
pgDistPartition = heap_open(DistPartitionRelationId(), RowExclusiveLock);
|
|
||||||
|
|
||||||
ScanKeyInit(&scanKey[0], Anum_pg_dist_partition_logicalrelid,
|
|
||||||
BTEqualStrategyNumber, F_OIDEQ, ObjectIdGetDatum(distributedRelationId));
|
|
||||||
|
|
||||||
scanDescriptor = systable_beginscan(pgDistPartition, InvalidOid, false, NULL,
|
|
||||||
scanKeyCount, scanKey);
|
|
||||||
|
|
||||||
heapTuple = systable_getnext(scanDescriptor);
|
|
||||||
if (!HeapTupleIsValid(heapTuple))
|
|
||||||
{
|
|
||||||
ereport(ERROR, (errmsg("could not find valid entry for partition %d",
|
|
||||||
distributedRelationId)));
|
|
||||||
}
|
|
||||||
|
|
||||||
simple_heap_delete(pgDistPartition, &heapTuple->t_self);
|
|
||||||
|
|
||||||
systable_endscan(scanDescriptor);
|
|
||||||
|
|
||||||
/* invalidate the cache */
|
|
||||||
CitusInvalidateRelcacheByRelid(distributedRelationId);
|
|
||||||
|
|
||||||
/* increment the counter so that next command can see the row */
|
|
||||||
CommandCounterIncrement();
|
|
||||||
|
|
||||||
heap_close(pgDistPartition, RowExclusiveLock);
|
|
||||||
}
|
|
||||||
|
|
|
@ -76,6 +76,7 @@ extern void DeleteShardRow(uint64 shardId);
|
||||||
extern void InsertShardPlacementRow(uint64 shardId, uint64 placementId,
|
extern void InsertShardPlacementRow(uint64 shardId, uint64 placementId,
|
||||||
char shardState, uint64 shardLength,
|
char shardState, uint64 shardLength,
|
||||||
char *nodeName, uint32 nodePort);
|
char *nodeName, uint32 nodePort);
|
||||||
|
extern void DeletePartitionRow(Oid distributedRelationId);
|
||||||
extern void DeleteShardRow(uint64 shardId);
|
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
|
||||||
|
|
|
@ -91,6 +91,8 @@ extern int ShardMaxSize;
|
||||||
extern int ShardPlacementPolicy;
|
extern int ShardPlacementPolicy;
|
||||||
|
|
||||||
|
|
||||||
|
extern bool SchemaNode(void);
|
||||||
|
|
||||||
/* Function declarations local to the distributed module */
|
/* Function declarations local to the distributed module */
|
||||||
extern bool CStoreTable(Oid relationId);
|
extern bool CStoreTable(Oid relationId);
|
||||||
extern uint64 GetNextShardId(void);
|
extern uint64 GetNextShardId(void);
|
||||||
|
@ -112,6 +114,8 @@ extern bool WorkerCreateShard(Oid relationId, char *nodeName, uint32 nodePort,
|
||||||
List *ddlCommandList, List *foreignConstraintCommadList);
|
List *ddlCommandList, List *foreignConstraintCommadList);
|
||||||
extern Oid ForeignConstraintGetReferencedTableId(char *queryString);
|
extern Oid ForeignConstraintGetReferencedTableId(char *queryString);
|
||||||
extern void CheckHashPartitionedTable(Oid distributedTableId);
|
extern void CheckHashPartitionedTable(Oid distributedTableId);
|
||||||
|
extern void CheckTableSchemaNameForDrop(Oid relationId, char **schemaName,
|
||||||
|
char **tableName);
|
||||||
|
|
||||||
/* Function declarations for generating metadata for shard and placement creation */
|
/* Function declarations for generating metadata for shard and placement creation */
|
||||||
extern Datum master_get_table_metadata(PG_FUNCTION_ARGS);
|
extern Datum master_get_table_metadata(PG_FUNCTION_ARGS);
|
||||||
|
|
|
@ -20,6 +20,7 @@
|
||||||
/* Functions declarations for metadata syncing */
|
/* Functions declarations for metadata syncing */
|
||||||
extern bool ShouldSyncTableMetadata(Oid relationId);
|
extern bool ShouldSyncTableMetadata(Oid relationId);
|
||||||
extern List * MetadataCreateCommands(void);
|
extern List * MetadataCreateCommands(void);
|
||||||
|
extern List * GetDistributedTableDDLEvents(Oid relationId);
|
||||||
extern List * MetadataDropCommands(void);
|
extern List * MetadataDropCommands(void);
|
||||||
extern char * DistributionCreateCommand(DistTableCacheEntry *cacheEntry);
|
extern char * DistributionCreateCommand(DistTableCacheEntry *cacheEntry);
|
||||||
extern char * DistributionDeleteCommand(char *schemaName,
|
extern char * DistributionDeleteCommand(char *schemaName,
|
||||||
|
@ -28,11 +29,13 @@ 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 char * ColocationIdUpdateCommand(Oid relationId, uint32 colocationId);
|
||||||
|
|
||||||
|
|
||||||
#define DELETE_ALL_NODES "TRUNCATE pg_dist_node"
|
#define DELETE_ALL_NODES "TRUNCATE pg_dist_node"
|
||||||
#define REMOVE_ALL_CLUSTERED_TABLES_COMMAND \
|
#define REMOVE_ALL_CLUSTERED_TABLES_COMMAND \
|
||||||
"SELECT worker_drop_distributed_table(logicalrelid) FROM pg_dist_partition"
|
"SELECT worker_drop_distributed_table(logicalrelid) FROM pg_dist_partition"
|
||||||
|
#define DISABLE_DDL_PROPAGATION "SET citus.enable_ddl_propagation TO 'off'"
|
||||||
|
|
||||||
|
|
||||||
#endif /* METADATA_SYNC_H */
|
#endif /* METADATA_SYNC_H */
|
||||||
|
|
|
@ -371,6 +371,8 @@ SELECT create_distributed_table('table2_groupB', 'id');
|
||||||
|
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
UPDATE pg_dist_partition SET repmodel='c' WHERE logicalrelid='table1_groupB'::regclass;
|
||||||
|
UPDATE pg_dist_partition SET repmodel='c' WHERE logicalrelid='table2_groupB'::regclass;
|
||||||
-- revert back to default shard replication factor
|
-- revert back to default shard replication factor
|
||||||
SET citus.shard_replication_factor to DEFAULT;
|
SET citus.shard_replication_factor to DEFAULT;
|
||||||
-- change partition column type
|
-- change partition column type
|
||||||
|
|
|
@ -137,3 +137,53 @@ SELECT master_create_distributed_table('supplier_single_shard', 's_suppkey', 'ap
|
||||||
|
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
-- Show that when a hash distributed table with replication factor=1 is created, it
|
||||||
|
-- automatically marked as streaming replicated
|
||||||
|
SET citus.shard_replication_factor TO 1;
|
||||||
|
CREATE TABLE mx_table_test (col1 int, col2 text);
|
||||||
|
SELECT create_distributed_table('mx_table_test', 'col1');
|
||||||
|
create_distributed_table
|
||||||
|
--------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT repmodel FROM pg_dist_partition WHERE logicalrelid='mx_table_test'::regclass;
|
||||||
|
repmodel
|
||||||
|
----------
|
||||||
|
s
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
DROP TABLE mx_table_test;
|
||||||
|
-- Show that it is not possible to create an mx table with the old
|
||||||
|
-- master_create_distributed_table function
|
||||||
|
CREATE TABLE mx_table_test (col1 int, col2 text);
|
||||||
|
SELECT master_create_distributed_table('mx_table_test', 'col1', 'hash');
|
||||||
|
master_create_distributed_table
|
||||||
|
---------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT repmodel FROM pg_dist_partition WHERE logicalrelid='mx_table_test'::regclass;
|
||||||
|
repmodel
|
||||||
|
----------
|
||||||
|
c
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
DROP TABLE mx_table_test;
|
||||||
|
-- Show that when replication factor > 1 the table is created as coordinator-replicated
|
||||||
|
SET citus.shard_replication_factor TO 2;
|
||||||
|
CREATE TABLE mx_table_test (col1 int, col2 text);
|
||||||
|
SELECT create_distributed_table('mx_table_test', 'col1');
|
||||||
|
create_distributed_table
|
||||||
|
--------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT repmodel FROM pg_dist_partition WHERE logicalrelid='mx_table_test'::regclass;
|
||||||
|
repmodel
|
||||||
|
----------
|
||||||
|
c
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
DROP TABLE mx_table_test;
|
||||||
|
SET citus.shard_replication_factor TO default;
|
||||||
|
|
|
@ -66,6 +66,7 @@ ALTER EXTENSION citus UPDATE TO '6.1-5';
|
||||||
ALTER EXTENSION citus UPDATE TO '6.1-6';
|
ALTER EXTENSION citus UPDATE TO '6.1-6';
|
||||||
ALTER EXTENSION citus UPDATE TO '6.1-7';
|
ALTER EXTENSION citus UPDATE TO '6.1-7';
|
||||||
ALTER EXTENSION citus UPDATE TO '6.1-8';
|
ALTER EXTENSION citus UPDATE TO '6.1-8';
|
||||||
|
ALTER EXTENSION citus UPDATE TO '6.1-9';
|
||||||
-- 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,
|
||||||
|
|
|
@ -1,7 +1,8 @@
|
||||||
--
|
--
|
||||||
-- MULTI_METADATA_SNAPSHOT
|
-- MULTI_METADATA_SYNC
|
||||||
--
|
--
|
||||||
-- Tests for metadata snapshot functions.
|
-- Tests for metadata snapshot functions, metadata syncing functions and propagation of
|
||||||
|
-- metadata changes to MX tables.
|
||||||
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1310000;
|
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1310000;
|
||||||
ALTER SEQUENCE pg_catalog.pg_dist_jobid_seq RESTART 1310000;
|
ALTER SEQUENCE pg_catalog.pg_dist_jobid_seq RESTART 1310000;
|
||||||
SELECT nextval('pg_catalog.pg_dist_shard_placement_placementid_seq') AS last_placement_id
|
SELECT nextval('pg_catalog.pg_dist_shard_placement_placementid_seq') AS last_placement_id
|
||||||
|
@ -250,6 +251,7 @@ SELECT count(*) FROM pg_trigger WHERE tgrelid='mx_testing_schema.mx_test_table':
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- Make sure that start_metadata_sync_to_node considers foreign key constraints
|
-- Make sure that start_metadata_sync_to_node considers foreign key constraints
|
||||||
|
\c - - - :master_port
|
||||||
SET citus.shard_replication_factor TO 1;
|
SET citus.shard_replication_factor TO 1;
|
||||||
CREATE SCHEMA mx_testing_schema_2;
|
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.fk_test_1 (col1 int, col2 text, col3 int, UNIQUE(col1, col3));
|
||||||
|
@ -267,11 +269,6 @@ SELECT create_distributed_table('mx_testing_schema_2.fk_test_2', 'col1');
|
||||||
|
|
||||||
(1 row)
|
(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);
|
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||||
start_metadata_sync_to_node
|
start_metadata_sync_to_node
|
||||||
|
@ -292,6 +289,8 @@ Foreign-key constraints:
|
||||||
"fk_test_2_col1_fkey" FOREIGN KEY (col1, col2) REFERENCES mx_testing_schema.fk_test_1(col1, col3)
|
"fk_test_2_col1_fkey" FOREIGN KEY (col1, col2) REFERENCES mx_testing_schema.fk_test_1(col1, col3)
|
||||||
|
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
|
DROP TABLE mx_testing_schema_2.fk_test_2;
|
||||||
|
DROP TABLE mx_testing_schema.fk_test_1;
|
||||||
RESET citus.shard_replication_factor;
|
RESET citus.shard_replication_factor;
|
||||||
-- Check that repeated calls to start_metadata_sync_to_node has no side effects
|
-- Check that repeated calls to start_metadata_sync_to_node has no side effects
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
|
@ -382,6 +381,60 @@ SELECT hasmetadata FROM pg_dist_node WHERE nodeport=:worker_2_port;
|
||||||
f
|
f
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
-- Check that the distributed table can be queried from the worker
|
||||||
|
\c - - - :master_port
|
||||||
|
SET citus.shard_replication_factor TO 1;
|
||||||
|
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||||
|
start_metadata_sync_to_node
|
||||||
|
-----------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
CREATE TABLE mx_query_test (a int, b text, c int);
|
||||||
|
SELECT create_distributed_table('mx_query_test', 'a');
|
||||||
|
create_distributed_table
|
||||||
|
--------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT repmodel FROM pg_dist_partition WHERE logicalrelid='mx_query_test'::regclass;
|
||||||
|
repmodel
|
||||||
|
----------
|
||||||
|
s
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
INSERT INTO mx_query_test VALUES (1, 'one', 1);
|
||||||
|
INSERT INTO mx_query_test VALUES (2, 'two', 4);
|
||||||
|
INSERT INTO mx_query_test VALUES (3, 'three', 9);
|
||||||
|
INSERT INTO mx_query_test VALUES (4, 'four', 16);
|
||||||
|
INSERT INTO mx_query_test VALUES (5, 'five', 24);
|
||||||
|
\c - - - :worker_1_port
|
||||||
|
SELECT * FROM mx_query_test ORDER BY a;
|
||||||
|
a | b | c
|
||||||
|
---+-------+----
|
||||||
|
1 | one | 1
|
||||||
|
2 | two | 4
|
||||||
|
3 | three | 9
|
||||||
|
4 | four | 16
|
||||||
|
5 | five | 24
|
||||||
|
(5 rows)
|
||||||
|
|
||||||
|
INSERT INTO mx_query_test VALUES (6, 'six', 36);
|
||||||
|
UPDATE mx_query_test SET c = 25 WHERE a = 5;
|
||||||
|
\c - - - :master_port
|
||||||
|
SELECT * FROM mx_query_test ORDER BY a;
|
||||||
|
a | b | c
|
||||||
|
---+-------+----
|
||||||
|
1 | one | 1
|
||||||
|
2 | two | 4
|
||||||
|
3 | three | 9
|
||||||
|
4 | four | 16
|
||||||
|
5 | five | 25
|
||||||
|
6 | six | 36
|
||||||
|
(6 rows)
|
||||||
|
|
||||||
|
\c - - - :master_port
|
||||||
|
DROP TABLE mx_query_test;
|
||||||
-- Check that stop_metadata_sync_to_node function sets hasmetadata of the node to false
|
-- Check that stop_metadata_sync_to_node function sets hasmetadata of the node to false
|
||||||
\c - - - :master_port
|
\c - - - :master_port
|
||||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||||
|
@ -408,8 +461,411 @@ SELECT hasmetadata FROM pg_dist_node WHERE nodeport=:worker_1_port;
|
||||||
f
|
f
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
-- Test DDL propagation in MX tables
|
||||||
|
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||||
|
start_metadata_sync_to_node
|
||||||
|
-----------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SET citus.shard_count = 5;
|
||||||
|
SET citus.multi_shard_commit_protocol TO '2pc';
|
||||||
|
CREATE SCHEMA mx_test_schema_1;
|
||||||
|
CREATE SCHEMA mx_test_schema_2;
|
||||||
|
-- Create MX tables
|
||||||
|
SET citus.shard_replication_factor TO 1;
|
||||||
|
CREATE TABLE mx_test_schema_1.mx_table_1 (col1 int UNIQUE, col2 text);
|
||||||
|
CREATE INDEX mx_index_1 ON mx_test_schema_1.mx_table_1 (col1);
|
||||||
|
CREATE TABLE mx_test_schema_2.mx_table_2 (col1 int, col2 text);
|
||||||
|
CREATE INDEX mx_index_2 ON mx_test_schema_2.mx_table_2 (col2);
|
||||||
|
ALTER TABLE mx_test_schema_2.mx_table_2 ADD CONSTRAINT mx_fk_constraint FOREIGN KEY(col1) REFERENCES mx_test_schema_1.mx_table_1(col1);
|
||||||
|
\d mx_test_schema_1.mx_table_1
|
||||||
|
Table "mx_test_schema_1.mx_table_1"
|
||||||
|
Column | Type | Modifiers
|
||||||
|
--------+---------+-----------
|
||||||
|
col1 | integer |
|
||||||
|
col2 | text |
|
||||||
|
Indexes:
|
||||||
|
"mx_table_1_col1_key" UNIQUE CONSTRAINT, btree (col1)
|
||||||
|
"mx_index_1" btree (col1)
|
||||||
|
Referenced by:
|
||||||
|
TABLE "mx_test_schema_2.mx_table_2" CONSTRAINT "mx_fk_constraint" FOREIGN KEY (col1) REFERENCES mx_test_schema_1.mx_table_1(col1)
|
||||||
|
|
||||||
|
\d mx_test_schema_2.mx_table_2
|
||||||
|
Table "mx_test_schema_2.mx_table_2"
|
||||||
|
Column | Type | Modifiers
|
||||||
|
--------+---------+-----------
|
||||||
|
col1 | integer |
|
||||||
|
col2 | text |
|
||||||
|
Indexes:
|
||||||
|
"mx_index_2" btree (col2)
|
||||||
|
Foreign-key constraints:
|
||||||
|
"mx_fk_constraint" FOREIGN KEY (col1) REFERENCES mx_test_schema_1.mx_table_1(col1)
|
||||||
|
|
||||||
|
SELECT create_distributed_table('mx_test_schema_1.mx_table_1', 'col1');
|
||||||
|
create_distributed_table
|
||||||
|
--------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT create_distributed_table('mx_test_schema_2.mx_table_2', 'col1');
|
||||||
|
create_distributed_table
|
||||||
|
--------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- Check that created tables are marked as streaming replicated tables
|
||||||
|
SELECT
|
||||||
|
logicalrelid, repmodel
|
||||||
|
FROM
|
||||||
|
pg_dist_partition
|
||||||
|
WHERE
|
||||||
|
logicalrelid = 'mx_test_schema_1.mx_table_1'::regclass
|
||||||
|
OR logicalrelid = 'mx_test_schema_2.mx_table_2'::regclass
|
||||||
|
ORDER BY
|
||||||
|
logicalrelid;
|
||||||
|
logicalrelid | repmodel
|
||||||
|
-----------------------------+----------
|
||||||
|
mx_test_schema_1.mx_table_1 | s
|
||||||
|
mx_test_schema_2.mx_table_2 | s
|
||||||
|
(2 rows)
|
||||||
|
|
||||||
|
-- See the shards and placements of the mx tables
|
||||||
|
SELECT
|
||||||
|
logicalrelid, shardid, nodename, nodeport
|
||||||
|
FROM
|
||||||
|
pg_dist_shard NATURAL JOIN pg_dist_shard_placement
|
||||||
|
WHERE
|
||||||
|
logicalrelid = 'mx_test_schema_1.mx_table_1'::regclass
|
||||||
|
OR logicalrelid = 'mx_test_schema_2.mx_table_2'::regclass
|
||||||
|
ORDER BY
|
||||||
|
logicalrelid, shardid;
|
||||||
|
logicalrelid | shardid | nodename | nodeport
|
||||||
|
-----------------------------+---------+-----------+----------
|
||||||
|
mx_test_schema_1.mx_table_1 | 1310104 | localhost | 57637
|
||||||
|
mx_test_schema_1.mx_table_1 | 1310105 | localhost | 57638
|
||||||
|
mx_test_schema_1.mx_table_1 | 1310106 | localhost | 57637
|
||||||
|
mx_test_schema_1.mx_table_1 | 1310107 | localhost | 57638
|
||||||
|
mx_test_schema_1.mx_table_1 | 1310108 | localhost | 57637
|
||||||
|
mx_test_schema_2.mx_table_2 | 1310109 | localhost | 57637
|
||||||
|
mx_test_schema_2.mx_table_2 | 1310110 | localhost | 57638
|
||||||
|
mx_test_schema_2.mx_table_2 | 1310111 | localhost | 57637
|
||||||
|
mx_test_schema_2.mx_table_2 | 1310112 | localhost | 57638
|
||||||
|
mx_test_schema_2.mx_table_2 | 1310113 | localhost | 57637
|
||||||
|
(10 rows)
|
||||||
|
|
||||||
|
|
||||||
|
-- Check that metadata of MX tables exist on the metadata worker
|
||||||
|
\c - - - :worker_1_port
|
||||||
|
-- Check that tables are created
|
||||||
|
\d mx_test_schema_1.mx_table_1
|
||||||
|
Table "mx_test_schema_1.mx_table_1"
|
||||||
|
Column | Type | Modifiers
|
||||||
|
--------+---------+-----------
|
||||||
|
col1 | integer |
|
||||||
|
col2 | text |
|
||||||
|
Indexes:
|
||||||
|
"mx_table_1_col1_key" UNIQUE CONSTRAINT, btree (col1)
|
||||||
|
"mx_index_1" btree (col1)
|
||||||
|
Referenced by:
|
||||||
|
TABLE "mx_test_schema_2.mx_table_2" CONSTRAINT "mx_fk_constraint" FOREIGN KEY (col1) REFERENCES mx_test_schema_1.mx_table_1(col1)
|
||||||
|
|
||||||
|
\d mx_test_schema_2.mx_table_2
|
||||||
|
Table "mx_test_schema_2.mx_table_2"
|
||||||
|
Column | Type | Modifiers
|
||||||
|
--------+---------+-----------
|
||||||
|
col1 | integer |
|
||||||
|
col2 | text |
|
||||||
|
Indexes:
|
||||||
|
"mx_index_2" btree (col2)
|
||||||
|
Foreign-key constraints:
|
||||||
|
"mx_fk_constraint" FOREIGN KEY (col1) REFERENCES mx_test_schema_1.mx_table_1(col1)
|
||||||
|
|
||||||
|
-- Check that table metadata are created
|
||||||
|
SELECT
|
||||||
|
logicalrelid, repmodel
|
||||||
|
FROM
|
||||||
|
pg_dist_partition
|
||||||
|
WHERE
|
||||||
|
logicalrelid = 'mx_test_schema_1.mx_table_1'::regclass
|
||||||
|
OR logicalrelid = 'mx_test_schema_2.mx_table_2'::regclass;
|
||||||
|
logicalrelid | repmodel
|
||||||
|
-----------------------------+----------
|
||||||
|
mx_test_schema_1.mx_table_1 | s
|
||||||
|
mx_test_schema_2.mx_table_2 | s
|
||||||
|
(2 rows)
|
||||||
|
|
||||||
|
-- Check that shard and placement data are created
|
||||||
|
SELECT
|
||||||
|
logicalrelid, shardid, nodename, nodeport
|
||||||
|
FROM
|
||||||
|
pg_dist_shard NATURAL JOIN pg_dist_shard_placement
|
||||||
|
WHERE
|
||||||
|
logicalrelid = 'mx_test_schema_1.mx_table_1'::regclass
|
||||||
|
OR logicalrelid = 'mx_test_schema_2.mx_table_2'::regclass
|
||||||
|
ORDER BY
|
||||||
|
logicalrelid, shardid;
|
||||||
|
logicalrelid | shardid | nodename | nodeport
|
||||||
|
-----------------------------+---------+-----------+----------
|
||||||
|
mx_test_schema_1.mx_table_1 | 1310104 | localhost | 57637
|
||||||
|
mx_test_schema_1.mx_table_1 | 1310105 | localhost | 57638
|
||||||
|
mx_test_schema_1.mx_table_1 | 1310106 | localhost | 57637
|
||||||
|
mx_test_schema_1.mx_table_1 | 1310107 | localhost | 57638
|
||||||
|
mx_test_schema_1.mx_table_1 | 1310108 | localhost | 57637
|
||||||
|
mx_test_schema_2.mx_table_2 | 1310109 | localhost | 57637
|
||||||
|
mx_test_schema_2.mx_table_2 | 1310110 | localhost | 57638
|
||||||
|
mx_test_schema_2.mx_table_2 | 1310111 | localhost | 57637
|
||||||
|
mx_test_schema_2.mx_table_2 | 1310112 | localhost | 57638
|
||||||
|
mx_test_schema_2.mx_table_2 | 1310113 | localhost | 57637
|
||||||
|
(10 rows)
|
||||||
|
|
||||||
|
-- Check that metadata of MX tables don't exist on the non-metadata worker
|
||||||
|
\c - - - :worker_2_port
|
||||||
|
\d mx_test_schema_1.mx_table_1
|
||||||
|
\d mx_test_schema_2.mx_table_2
|
||||||
|
SELECT * FROM pg_dist_partition;
|
||||||
|
logicalrelid | partmethod | partkey | colocationid | repmodel
|
||||||
|
--------------+------------+---------+--------------+----------
|
||||||
|
(0 rows)
|
||||||
|
|
||||||
|
SELECT * FROM pg_dist_shard;
|
||||||
|
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue
|
||||||
|
--------------+---------+--------------+---------------+---------------
|
||||||
|
(0 rows)
|
||||||
|
|
||||||
|
SELECT * FROM pg_dist_shard_placement;
|
||||||
|
shardid | shardstate | shardlength | nodename | nodeport | placementid
|
||||||
|
---------+------------+-------------+----------+----------+-------------
|
||||||
|
(0 rows)
|
||||||
|
|
||||||
|
-- Check that CREATE INDEX statement is propagated
|
||||||
|
\c - - - :master_port
|
||||||
|
SET citus.multi_shard_commit_protocol TO '2pc';
|
||||||
|
CREATE INDEX mx_index_3 ON mx_test_schema_2.mx_table_2 USING hash (col1);
|
||||||
|
WARNING: hash indexes are not WAL-logged and their use is discouraged
|
||||||
|
CREATE UNIQUE INDEX mx_index_4 ON mx_test_schema_2.mx_table_2(col1);
|
||||||
|
\c - - - :worker_1_port
|
||||||
|
\d mx_test_schema_2.mx_table_2
|
||||||
|
Table "mx_test_schema_2.mx_table_2"
|
||||||
|
Column | Type | Modifiers
|
||||||
|
--------+---------+-----------
|
||||||
|
col1 | integer |
|
||||||
|
col2 | text |
|
||||||
|
Indexes:
|
||||||
|
"mx_index_4" UNIQUE, btree (col1)
|
||||||
|
"mx_index_2" btree (col2)
|
||||||
|
"mx_index_3" hash (col1)
|
||||||
|
Foreign-key constraints:
|
||||||
|
"mx_fk_constraint" FOREIGN KEY (col1) REFERENCES mx_test_schema_1.mx_table_1(col1)
|
||||||
|
|
||||||
|
-- Check that DROP INDEX statement is propagated
|
||||||
|
\c - - - :master_port
|
||||||
|
SET citus.multi_shard_commit_protocol TO '2pc';
|
||||||
|
DROP INDEX mx_test_schema_2.mx_index_3;
|
||||||
|
\c - - - :worker_1_port
|
||||||
|
\d mx_test_schema_2.mx_table_2
|
||||||
|
Table "mx_test_schema_2.mx_table_2"
|
||||||
|
Column | Type | Modifiers
|
||||||
|
--------+---------+-----------
|
||||||
|
col1 | integer |
|
||||||
|
col2 | text |
|
||||||
|
Indexes:
|
||||||
|
"mx_index_4" UNIQUE, btree (col1)
|
||||||
|
"mx_index_2" btree (col2)
|
||||||
|
Foreign-key constraints:
|
||||||
|
"mx_fk_constraint" FOREIGN KEY (col1) REFERENCES mx_test_schema_1.mx_table_1(col1)
|
||||||
|
|
||||||
|
-- Check that ALTER TABLE statements are propagated
|
||||||
|
\c - - - :master_port
|
||||||
|
SET citus.multi_shard_commit_protocol TO '2pc';
|
||||||
|
ALTER TABLE mx_test_schema_1.mx_table_1 ADD COLUMN col3 NUMERIC;
|
||||||
|
ALTER TABLE mx_test_schema_1.mx_table_1 ALTER COLUMN col3 SET DATA TYPE INT;
|
||||||
|
ALTER TABLE
|
||||||
|
mx_test_schema_1.mx_table_1
|
||||||
|
ADD CONSTRAINT
|
||||||
|
mx_fk_constraint
|
||||||
|
FOREIGN KEY
|
||||||
|
(col1)
|
||||||
|
REFERENCES
|
||||||
|
mx_test_schema_2.mx_table_2(col1);
|
||||||
|
\c - - - :worker_1_port
|
||||||
|
\d mx_test_schema_1.mx_table_1
|
||||||
|
Table "mx_test_schema_1.mx_table_1"
|
||||||
|
Column | Type | Modifiers
|
||||||
|
--------+---------+-----------
|
||||||
|
col1 | integer |
|
||||||
|
col2 | text |
|
||||||
|
col3 | integer |
|
||||||
|
Indexes:
|
||||||
|
"mx_table_1_col1_key" UNIQUE CONSTRAINT, btree (col1)
|
||||||
|
"mx_index_1" btree (col1)
|
||||||
|
Foreign-key constraints:
|
||||||
|
"mx_fk_constraint" FOREIGN KEY (col1) REFERENCES mx_test_schema_2.mx_table_2(col1)
|
||||||
|
Referenced by:
|
||||||
|
TABLE "mx_test_schema_2.mx_table_2" CONSTRAINT "mx_fk_constraint" FOREIGN KEY (col1) REFERENCES mx_test_schema_1.mx_table_1(col1)
|
||||||
|
|
||||||
|
-- Check that foreign key constraint with NOT VALID works as well
|
||||||
|
\c - - - :master_port
|
||||||
|
SET citus.multi_shard_commit_protocol TO '2pc';
|
||||||
|
ALTER TABLE mx_test_schema_1.mx_table_1 DROP CONSTRAINT mx_fk_constraint;
|
||||||
|
ALTER TABLE
|
||||||
|
mx_test_schema_1.mx_table_1
|
||||||
|
ADD CONSTRAINT
|
||||||
|
mx_fk_constraint_2
|
||||||
|
FOREIGN KEY
|
||||||
|
(col1)
|
||||||
|
REFERENCES
|
||||||
|
mx_test_schema_2.mx_table_2(col1)
|
||||||
|
NOT VALID;
|
||||||
|
\c - - - :worker_1_port
|
||||||
|
\d mx_test_schema_1.mx_table_1
|
||||||
|
Table "mx_test_schema_1.mx_table_1"
|
||||||
|
Column | Type | Modifiers
|
||||||
|
--------+---------+-----------
|
||||||
|
col1 | integer |
|
||||||
|
col2 | text |
|
||||||
|
col3 | integer |
|
||||||
|
Indexes:
|
||||||
|
"mx_table_1_col1_key" UNIQUE CONSTRAINT, btree (col1)
|
||||||
|
"mx_index_1" btree (col1)
|
||||||
|
Foreign-key constraints:
|
||||||
|
"mx_fk_constraint_2" FOREIGN KEY (col1) REFERENCES mx_test_schema_2.mx_table_2(col1) NOT VALID
|
||||||
|
Referenced by:
|
||||||
|
TABLE "mx_test_schema_2.mx_table_2" CONSTRAINT "mx_fk_constraint" FOREIGN KEY (col1) REFERENCES mx_test_schema_1.mx_table_1(col1)
|
||||||
|
|
||||||
|
-- Check that mark_tables_colocated call propagates the changes to the workers
|
||||||
|
\c - - - :master_port
|
||||||
|
SELECT nextval('pg_catalog.pg_dist_colocationid_seq') AS last_colocation_id \gset
|
||||||
|
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 10000;
|
||||||
|
SET citus.shard_count TO 7;
|
||||||
|
SET citus.shard_replication_factor TO 1;
|
||||||
|
CREATE TABLE mx_colocation_test_1 (a int);
|
||||||
|
SELECT create_distributed_table('mx_colocation_test_1', 'a');
|
||||||
|
create_distributed_table
|
||||||
|
--------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
CREATE TABLE mx_colocation_test_2 (a int);
|
||||||
|
SELECT create_distributed_table('mx_colocation_test_2', 'a');
|
||||||
|
create_distributed_table
|
||||||
|
--------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- Check the colocation IDs of the created tables
|
||||||
|
SELECT
|
||||||
|
logicalrelid, colocationid
|
||||||
|
FROM
|
||||||
|
pg_dist_partition
|
||||||
|
WHERE
|
||||||
|
logicalrelid = 'mx_colocation_test_1'::regclass
|
||||||
|
OR logicalrelid = 'mx_colocation_test_2'::regclass
|
||||||
|
ORDER BY logicalrelid;
|
||||||
|
logicalrelid | colocationid
|
||||||
|
----------------------+--------------
|
||||||
|
mx_colocation_test_1 | 10000
|
||||||
|
mx_colocation_test_2 | 10000
|
||||||
|
(2 rows)
|
||||||
|
|
||||||
|
|
||||||
|
-- Reset the colocation IDs of the test tables
|
||||||
|
DELETE FROM
|
||||||
|
pg_dist_colocation
|
||||||
|
WHERE EXISTS (
|
||||||
|
SELECT 1
|
||||||
|
FROM pg_dist_partition
|
||||||
|
WHERE
|
||||||
|
colocationid = pg_dist_partition.colocationid
|
||||||
|
AND pg_dist_partition.logicalrelid = 'mx_colocation_test_1'::regclass);
|
||||||
|
UPDATE
|
||||||
|
pg_dist_partition
|
||||||
|
SET
|
||||||
|
colocationid = 0
|
||||||
|
WHERE
|
||||||
|
logicalrelid = 'mx_colocation_test_1'::regclass
|
||||||
|
OR logicalrelid = 'mx_colocation_test_2'::regclass;
|
||||||
|
-- Mark tables colocated and see the changes on the master and the worker
|
||||||
|
SELECT mark_tables_colocated('mx_colocation_test_1', ARRAY['mx_colocation_test_2']);
|
||||||
|
mark_tables_colocated
|
||||||
|
-----------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT
|
||||||
|
logicalrelid, colocationid
|
||||||
|
FROM
|
||||||
|
pg_dist_partition
|
||||||
|
WHERE
|
||||||
|
logicalrelid = 'mx_colocation_test_1'::regclass
|
||||||
|
OR logicalrelid = 'mx_colocation_test_2'::regclass;
|
||||||
|
logicalrelid | colocationid
|
||||||
|
----------------------+--------------
|
||||||
|
mx_colocation_test_1 | 10001
|
||||||
|
mx_colocation_test_2 | 10001
|
||||||
|
(2 rows)
|
||||||
|
|
||||||
|
\c - - - :worker_1_port
|
||||||
|
SELECT
|
||||||
|
logicalrelid, colocationid
|
||||||
|
FROM
|
||||||
|
pg_dist_partition
|
||||||
|
WHERE
|
||||||
|
logicalrelid = 'mx_colocation_test_1'::regclass
|
||||||
|
OR logicalrelid = 'mx_colocation_test_2'::regclass;
|
||||||
|
logicalrelid | colocationid
|
||||||
|
----------------------+--------------
|
||||||
|
mx_colocation_test_1 | 10001
|
||||||
|
mx_colocation_test_2 | 10001
|
||||||
|
(2 rows)
|
||||||
|
|
||||||
|
\c - - - :master_port
|
||||||
|
-- Check that DROP TABLE on MX tables works
|
||||||
|
DROP TABLE mx_colocation_test_1;
|
||||||
|
DROP TABLE mx_colocation_test_2;
|
||||||
|
\d mx_colocation_test_1
|
||||||
|
\d mx_colocation_test_2
|
||||||
|
\c - - - :worker_1_port
|
||||||
|
\d mx_colocation_test_1
|
||||||
|
\d mx_colocation_test_2
|
||||||
|
|
||||||
|
-- Check that dropped MX table can be recreated again
|
||||||
|
\c - - - :master_port
|
||||||
|
SET citus.shard_count TO 7;
|
||||||
|
SET citus.shard_replication_factor TO 1;
|
||||||
|
CREATE TABLE mx_temp_drop_test (a int);
|
||||||
|
SELECT create_distributed_table('mx_temp_drop_test', 'a');
|
||||||
|
create_distributed_table
|
||||||
|
--------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT logicalrelid, repmodel FROM pg_dist_partition WHERE logicalrelid = 'mx_temp_drop_test'::regclass;
|
||||||
|
logicalrelid | repmodel
|
||||||
|
-------------------+----------
|
||||||
|
mx_temp_drop_test | s
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
DROP TABLE mx_temp_drop_test;
|
||||||
|
CREATE TABLE mx_temp_drop_test (a int);
|
||||||
|
SELECT create_distributed_table('mx_temp_drop_test', 'a');
|
||||||
|
create_distributed_table
|
||||||
|
--------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT logicalrelid, repmodel FROM pg_dist_partition WHERE logicalrelid = 'mx_temp_drop_test'::regclass;
|
||||||
|
logicalrelid | repmodel
|
||||||
|
-------------------+----------
|
||||||
|
mx_temp_drop_test | s
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
DROP TABLE mx_temp_drop_test;
|
||||||
-- Cleanup
|
-- Cleanup
|
||||||
\c - - - :worker_1_port
|
\c - - - :worker_1_port
|
||||||
|
DROP TABLE mx_test_schema_2.mx_table_2 CASCADE;
|
||||||
|
NOTICE: drop cascades to constraint mx_fk_constraint_2 on table mx_test_schema_1.mx_table_1
|
||||||
|
DROP TABLE mx_test_schema_1.mx_table_1 CASCADE;
|
||||||
DROP TABLE mx_testing_schema.mx_test_table;
|
DROP TABLE mx_testing_schema.mx_test_table;
|
||||||
DELETE FROM pg_dist_node;
|
DELETE FROM pg_dist_node;
|
||||||
DELETE FROM pg_dist_partition;
|
DELETE FROM pg_dist_partition;
|
||||||
|
@ -429,5 +885,12 @@ SELECT stop_metadata_sync_to_node('localhost', :worker_2_port);
|
||||||
|
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
DROP TABLE mx_testing_schema.mx_test_table CASCADE;
|
DROP TABLE mx_test_schema_2.mx_table_2 CASCADE;
|
||||||
|
NOTICE: drop cascades to constraint mx_fk_constraint_2 on table mx_test_schema_1.mx_table_1
|
||||||
|
DROP TABLE mx_test_schema_1.mx_table_1 CASCADE;
|
||||||
|
DROP TABLE mx_testing_schema.mx_test_table;
|
||||||
|
RESET citus.shard_count;
|
||||||
|
RESET citus.shard_replication_factor;
|
||||||
|
RESET citus.multi_shard_commit_protocol;
|
||||||
|
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART :last_colocation_id;
|
||||||
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;
|
|
@ -139,7 +139,7 @@ test: multi_data_types
|
||||||
test: multi_repartition_udt
|
test: multi_repartition_udt
|
||||||
test: multi_repartitioned_subquery_udf
|
test: multi_repartitioned_subquery_udf
|
||||||
test: multi_modifying_xacts
|
test: multi_modifying_xacts
|
||||||
test: multi_metadata_snapshot
|
test: multi_metadata_sync
|
||||||
test: multi_transaction_recovery
|
test: multi_transaction_recovery
|
||||||
|
|
||||||
# ---------
|
# ---------
|
||||||
|
|
|
@ -175,6 +175,9 @@ SELECT create_distributed_table('table1_groupB', 'id');
|
||||||
CREATE TABLE table2_groupB ( id int );
|
CREATE TABLE table2_groupB ( id int );
|
||||||
SELECT create_distributed_table('table2_groupB', 'id');
|
SELECT create_distributed_table('table2_groupB', 'id');
|
||||||
|
|
||||||
|
UPDATE pg_dist_partition SET repmodel='c' WHERE logicalrelid='table1_groupB'::regclass;
|
||||||
|
UPDATE pg_dist_partition SET repmodel='c' WHERE logicalrelid='table2_groupB'::regclass;
|
||||||
|
|
||||||
-- revert back to default shard replication factor
|
-- revert back to default shard replication factor
|
||||||
SET citus.shard_replication_factor to DEFAULT;
|
SET citus.shard_replication_factor to DEFAULT;
|
||||||
|
|
||||||
|
|
|
@ -111,3 +111,28 @@ CREATE TABLE supplier_single_shard
|
||||||
s_comment varchar(101) not null
|
s_comment varchar(101) not null
|
||||||
);
|
);
|
||||||
SELECT master_create_distributed_table('supplier_single_shard', 's_suppkey', 'append');
|
SELECT master_create_distributed_table('supplier_single_shard', 's_suppkey', 'append');
|
||||||
|
|
||||||
|
-- Show that when a hash distributed table with replication factor=1 is created, it
|
||||||
|
-- automatically marked as streaming replicated
|
||||||
|
SET citus.shard_replication_factor TO 1;
|
||||||
|
|
||||||
|
CREATE TABLE mx_table_test (col1 int, col2 text);
|
||||||
|
SELECT create_distributed_table('mx_table_test', 'col1');
|
||||||
|
SELECT repmodel FROM pg_dist_partition WHERE logicalrelid='mx_table_test'::regclass;
|
||||||
|
DROP TABLE mx_table_test;
|
||||||
|
|
||||||
|
-- Show that it is not possible to create an mx table with the old
|
||||||
|
-- master_create_distributed_table function
|
||||||
|
CREATE TABLE mx_table_test (col1 int, col2 text);
|
||||||
|
SELECT master_create_distributed_table('mx_table_test', 'col1', 'hash');
|
||||||
|
SELECT repmodel FROM pg_dist_partition WHERE logicalrelid='mx_table_test'::regclass;
|
||||||
|
DROP TABLE mx_table_test;
|
||||||
|
|
||||||
|
-- Show that when replication factor > 1 the table is created as coordinator-replicated
|
||||||
|
SET citus.shard_replication_factor TO 2;
|
||||||
|
CREATE TABLE mx_table_test (col1 int, col2 text);
|
||||||
|
SELECT create_distributed_table('mx_table_test', 'col1');
|
||||||
|
SELECT repmodel FROM pg_dist_partition WHERE logicalrelid='mx_table_test'::regclass;
|
||||||
|
DROP TABLE mx_table_test;
|
||||||
|
|
||||||
|
SET citus.shard_replication_factor TO default;
|
||||||
|
|
|
@ -66,6 +66,7 @@ ALTER EXTENSION citus UPDATE TO '6.1-5';
|
||||||
ALTER EXTENSION citus UPDATE TO '6.1-6';
|
ALTER EXTENSION citus UPDATE TO '6.1-6';
|
||||||
ALTER EXTENSION citus UPDATE TO '6.1-7';
|
ALTER EXTENSION citus UPDATE TO '6.1-7';
|
||||||
ALTER EXTENSION citus UPDATE TO '6.1-8';
|
ALTER EXTENSION citus UPDATE TO '6.1-8';
|
||||||
|
ALTER EXTENSION citus UPDATE TO '6.1-9';
|
||||||
|
|
||||||
-- ensure no objects were created outside pg_catalog
|
-- ensure no objects were created outside pg_catalog
|
||||||
SELECT COUNT(*)
|
SELECT COUNT(*)
|
||||||
|
|
|
@ -1,155 +0,0 @@
|
||||||
--
|
|
||||||
-- MULTI_METADATA_SNAPSHOT
|
|
||||||
--
|
|
||||||
|
|
||||||
-- Tests for metadata snapshot functions.
|
|
||||||
|
|
||||||
|
|
||||||
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1310000;
|
|
||||||
ALTER SEQUENCE pg_catalog.pg_dist_jobid_seq RESTART 1310000;
|
|
||||||
|
|
||||||
SELECT nextval('pg_catalog.pg_dist_shard_placement_placementid_seq') AS last_placement_id
|
|
||||||
\gset
|
|
||||||
ALTER SEQUENCE pg_catalog.pg_dist_shard_placement_placementid_seq RESTART 100000;
|
|
||||||
|
|
||||||
-- Create the necessary test utility function
|
|
||||||
CREATE FUNCTION master_metadata_snapshot()
|
|
||||||
RETURNS text[]
|
|
||||||
LANGUAGE C STRICT
|
|
||||||
AS 'citus';
|
|
||||||
|
|
||||||
COMMENT ON FUNCTION master_metadata_snapshot()
|
|
||||||
IS 'commands to create the metadata snapshot';
|
|
||||||
|
|
||||||
-- Show that none of the existing tables are qualified to be MX tables
|
|
||||||
SELECT * FROM pg_dist_partition WHERE partmethod='h' AND repmodel='s';
|
|
||||||
|
|
||||||
-- Show that, with no MX tables, metadata snapshot contains only the delete commands and
|
|
||||||
-- pg_dist_node entries
|
|
||||||
SELECT unnest(master_metadata_snapshot());
|
|
||||||
|
|
||||||
-- Create a test table with constraints and SERIAL
|
|
||||||
CREATE TABLE mx_test_table (col_1 int UNIQUE, col_2 text NOT NULL, col_3 SERIAL);
|
|
||||||
SELECT master_create_distributed_table('mx_test_table', 'col_1', 'hash');
|
|
||||||
SELECT master_create_worker_shards('mx_test_table', 8, 1);
|
|
||||||
|
|
||||||
-- Set the replication model of the test table to streaming replication so that it is
|
|
||||||
-- considered as an MX table
|
|
||||||
UPDATE pg_dist_partition SET repmodel='s' WHERE logicalrelid='mx_test_table'::regclass;
|
|
||||||
|
|
||||||
-- Show that the created MX table is included in the metadata snapshot
|
|
||||||
SELECT unnest(master_metadata_snapshot());
|
|
||||||
|
|
||||||
-- Show that CREATE INDEX commands are included in the metadata snapshot
|
|
||||||
CREATE INDEX mx_index ON mx_test_table(col_2);
|
|
||||||
SELECT unnest(master_metadata_snapshot());
|
|
||||||
|
|
||||||
-- Show that schema changes are included in the metadata snapshot
|
|
||||||
CREATE SCHEMA mx_testing_schema;
|
|
||||||
ALTER TABLE mx_test_table SET SCHEMA mx_testing_schema;
|
|
||||||
SELECT unnest(master_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);
|
|
||||||
SELECT master_create_distributed_table('non_mx_test_table', 'col_1', 'append');
|
|
||||||
UPDATE pg_dist_partition SET repmodel='s' WHERE logicalrelid='non_mx_test_table'::regclass;
|
|
||||||
SELECT unnest(master_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;
|
|
||||||
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;
|
|
|
@ -0,0 +1,405 @@
|
||||||
|
--
|
||||||
|
-- MULTI_METADATA_SYNC
|
||||||
|
--
|
||||||
|
|
||||||
|
-- Tests for metadata snapshot functions, metadata syncing functions and propagation of
|
||||||
|
-- metadata changes to MX tables.
|
||||||
|
|
||||||
|
|
||||||
|
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1310000;
|
||||||
|
ALTER SEQUENCE pg_catalog.pg_dist_jobid_seq RESTART 1310000;
|
||||||
|
|
||||||
|
SELECT nextval('pg_catalog.pg_dist_shard_placement_placementid_seq') AS last_placement_id
|
||||||
|
\gset
|
||||||
|
ALTER SEQUENCE pg_catalog.pg_dist_shard_placement_placementid_seq RESTART 100000;
|
||||||
|
|
||||||
|
-- Create the necessary test utility function
|
||||||
|
CREATE FUNCTION master_metadata_snapshot()
|
||||||
|
RETURNS text[]
|
||||||
|
LANGUAGE C STRICT
|
||||||
|
AS 'citus';
|
||||||
|
|
||||||
|
COMMENT ON FUNCTION master_metadata_snapshot()
|
||||||
|
IS 'commands to create the metadata snapshot';
|
||||||
|
|
||||||
|
-- Show that none of the existing tables are qualified to be MX tables
|
||||||
|
SELECT * FROM pg_dist_partition WHERE partmethod='h' AND repmodel='s';
|
||||||
|
|
||||||
|
-- Show that, with no MX tables, metadata snapshot contains only the delete commands and
|
||||||
|
-- pg_dist_node entries
|
||||||
|
SELECT unnest(master_metadata_snapshot());
|
||||||
|
|
||||||
|
-- Create a test table with constraints and SERIAL
|
||||||
|
CREATE TABLE mx_test_table (col_1 int UNIQUE, col_2 text NOT NULL, col_3 SERIAL);
|
||||||
|
SELECT master_create_distributed_table('mx_test_table', 'col_1', 'hash');
|
||||||
|
SELECT master_create_worker_shards('mx_test_table', 8, 1);
|
||||||
|
|
||||||
|
-- Set the replication model of the test table to streaming replication so that it is
|
||||||
|
-- considered as an MX table
|
||||||
|
UPDATE pg_dist_partition SET repmodel='s' WHERE logicalrelid='mx_test_table'::regclass;
|
||||||
|
|
||||||
|
-- Show that the created MX table is included in the metadata snapshot
|
||||||
|
SELECT unnest(master_metadata_snapshot());
|
||||||
|
|
||||||
|
-- Show that CREATE INDEX commands are included in the metadata snapshot
|
||||||
|
CREATE INDEX mx_index ON mx_test_table(col_2);
|
||||||
|
SELECT unnest(master_metadata_snapshot());
|
||||||
|
|
||||||
|
-- Show that schema changes are included in the metadata snapshot
|
||||||
|
CREATE SCHEMA mx_testing_schema;
|
||||||
|
ALTER TABLE mx_test_table SET SCHEMA mx_testing_schema;
|
||||||
|
SELECT unnest(master_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);
|
||||||
|
SELECT master_create_distributed_table('non_mx_test_table', 'col_1', 'append');
|
||||||
|
UPDATE pg_dist_partition SET repmodel='s' WHERE logicalrelid='non_mx_test_table'::regclass;
|
||||||
|
SELECT unnest(master_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;
|
||||||
|
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
|
||||||
|
\c - - - :master_port
|
||||||
|
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');
|
||||||
|
|
||||||
|
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
|
||||||
|
DROP TABLE mx_testing_schema_2.fk_test_2;
|
||||||
|
DROP TABLE mx_testing_schema.fk_test_1;
|
||||||
|
|
||||||
|
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 the distributed table can be queried from the worker
|
||||||
|
\c - - - :master_port
|
||||||
|
SET citus.shard_replication_factor TO 1;
|
||||||
|
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||||
|
|
||||||
|
CREATE TABLE mx_query_test (a int, b text, c int);
|
||||||
|
SELECT create_distributed_table('mx_query_test', 'a');
|
||||||
|
|
||||||
|
SELECT repmodel FROM pg_dist_partition WHERE logicalrelid='mx_query_test'::regclass;
|
||||||
|
|
||||||
|
INSERT INTO mx_query_test VALUES (1, 'one', 1);
|
||||||
|
INSERT INTO mx_query_test VALUES (2, 'two', 4);
|
||||||
|
INSERT INTO mx_query_test VALUES (3, 'three', 9);
|
||||||
|
INSERT INTO mx_query_test VALUES (4, 'four', 16);
|
||||||
|
INSERT INTO mx_query_test VALUES (5, 'five', 24);
|
||||||
|
|
||||||
|
\c - - - :worker_1_port
|
||||||
|
SELECT * FROM mx_query_test ORDER BY a;
|
||||||
|
INSERT INTO mx_query_test VALUES (6, 'six', 36);
|
||||||
|
UPDATE mx_query_test SET c = 25 WHERE a = 5;
|
||||||
|
|
||||||
|
\c - - - :master_port
|
||||||
|
SELECT * FROM mx_query_test ORDER BY a;
|
||||||
|
|
||||||
|
\c - - - :master_port
|
||||||
|
DROP TABLE mx_query_test;
|
||||||
|
|
||||||
|
-- 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;
|
||||||
|
|
||||||
|
|
||||||
|
-- Test DDL propagation in MX tables
|
||||||
|
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||||
|
SET citus.shard_count = 5;
|
||||||
|
SET citus.multi_shard_commit_protocol TO '2pc';
|
||||||
|
CREATE SCHEMA mx_test_schema_1;
|
||||||
|
CREATE SCHEMA mx_test_schema_2;
|
||||||
|
|
||||||
|
-- Create MX tables
|
||||||
|
SET citus.shard_replication_factor TO 1;
|
||||||
|
CREATE TABLE mx_test_schema_1.mx_table_1 (col1 int UNIQUE, col2 text);
|
||||||
|
CREATE INDEX mx_index_1 ON mx_test_schema_1.mx_table_1 (col1);
|
||||||
|
|
||||||
|
CREATE TABLE mx_test_schema_2.mx_table_2 (col1 int, col2 text);
|
||||||
|
CREATE INDEX mx_index_2 ON mx_test_schema_2.mx_table_2 (col2);
|
||||||
|
ALTER TABLE mx_test_schema_2.mx_table_2 ADD CONSTRAINT mx_fk_constraint FOREIGN KEY(col1) REFERENCES mx_test_schema_1.mx_table_1(col1);
|
||||||
|
|
||||||
|
\d mx_test_schema_1.mx_table_1
|
||||||
|
\d mx_test_schema_2.mx_table_2
|
||||||
|
|
||||||
|
SELECT create_distributed_table('mx_test_schema_1.mx_table_1', 'col1');
|
||||||
|
SELECT create_distributed_table('mx_test_schema_2.mx_table_2', 'col1');
|
||||||
|
|
||||||
|
-- Check that created tables are marked as streaming replicated tables
|
||||||
|
SELECT
|
||||||
|
logicalrelid, repmodel
|
||||||
|
FROM
|
||||||
|
pg_dist_partition
|
||||||
|
WHERE
|
||||||
|
logicalrelid = 'mx_test_schema_1.mx_table_1'::regclass
|
||||||
|
OR logicalrelid = 'mx_test_schema_2.mx_table_2'::regclass
|
||||||
|
ORDER BY
|
||||||
|
logicalrelid;
|
||||||
|
|
||||||
|
-- See the shards and placements of the mx tables
|
||||||
|
SELECT
|
||||||
|
logicalrelid, shardid, nodename, nodeport
|
||||||
|
FROM
|
||||||
|
pg_dist_shard NATURAL JOIN pg_dist_shard_placement
|
||||||
|
WHERE
|
||||||
|
logicalrelid = 'mx_test_schema_1.mx_table_1'::regclass
|
||||||
|
OR logicalrelid = 'mx_test_schema_2.mx_table_2'::regclass
|
||||||
|
ORDER BY
|
||||||
|
logicalrelid, shardid;
|
||||||
|
|
||||||
|
-- Check that metadata of MX tables exist on the metadata worker
|
||||||
|
\c - - - :worker_1_port
|
||||||
|
|
||||||
|
-- Check that tables are created
|
||||||
|
\d mx_test_schema_1.mx_table_1
|
||||||
|
\d mx_test_schema_2.mx_table_2
|
||||||
|
|
||||||
|
-- Check that table metadata are created
|
||||||
|
SELECT
|
||||||
|
logicalrelid, repmodel
|
||||||
|
FROM
|
||||||
|
pg_dist_partition
|
||||||
|
WHERE
|
||||||
|
logicalrelid = 'mx_test_schema_1.mx_table_1'::regclass
|
||||||
|
OR logicalrelid = 'mx_test_schema_2.mx_table_2'::regclass;
|
||||||
|
|
||||||
|
-- Check that shard and placement data are created
|
||||||
|
SELECT
|
||||||
|
logicalrelid, shardid, nodename, nodeport
|
||||||
|
FROM
|
||||||
|
pg_dist_shard NATURAL JOIN pg_dist_shard_placement
|
||||||
|
WHERE
|
||||||
|
logicalrelid = 'mx_test_schema_1.mx_table_1'::regclass
|
||||||
|
OR logicalrelid = 'mx_test_schema_2.mx_table_2'::regclass
|
||||||
|
ORDER BY
|
||||||
|
logicalrelid, shardid;
|
||||||
|
|
||||||
|
-- Check that metadata of MX tables don't exist on the non-metadata worker
|
||||||
|
\c - - - :worker_2_port
|
||||||
|
|
||||||
|
\d mx_test_schema_1.mx_table_1
|
||||||
|
\d mx_test_schema_2.mx_table_2
|
||||||
|
|
||||||
|
SELECT * FROM pg_dist_partition;
|
||||||
|
SELECT * FROM pg_dist_shard;
|
||||||
|
SELECT * FROM pg_dist_shard_placement;
|
||||||
|
|
||||||
|
-- Check that CREATE INDEX statement is propagated
|
||||||
|
\c - - - :master_port
|
||||||
|
SET citus.multi_shard_commit_protocol TO '2pc';
|
||||||
|
CREATE INDEX mx_index_3 ON mx_test_schema_2.mx_table_2 USING hash (col1);
|
||||||
|
CREATE UNIQUE INDEX mx_index_4 ON mx_test_schema_2.mx_table_2(col1);
|
||||||
|
\c - - - :worker_1_port
|
||||||
|
\d mx_test_schema_2.mx_table_2
|
||||||
|
|
||||||
|
-- Check that DROP INDEX statement is propagated
|
||||||
|
\c - - - :master_port
|
||||||
|
SET citus.multi_shard_commit_protocol TO '2pc';
|
||||||
|
DROP INDEX mx_test_schema_2.mx_index_3;
|
||||||
|
\c - - - :worker_1_port
|
||||||
|
\d mx_test_schema_2.mx_table_2
|
||||||
|
|
||||||
|
-- Check that ALTER TABLE statements are propagated
|
||||||
|
\c - - - :master_port
|
||||||
|
SET citus.multi_shard_commit_protocol TO '2pc';
|
||||||
|
ALTER TABLE mx_test_schema_1.mx_table_1 ADD COLUMN col3 NUMERIC;
|
||||||
|
ALTER TABLE mx_test_schema_1.mx_table_1 ALTER COLUMN col3 SET DATA TYPE INT;
|
||||||
|
ALTER TABLE
|
||||||
|
mx_test_schema_1.mx_table_1
|
||||||
|
ADD CONSTRAINT
|
||||||
|
mx_fk_constraint
|
||||||
|
FOREIGN KEY
|
||||||
|
(col1)
|
||||||
|
REFERENCES
|
||||||
|
mx_test_schema_2.mx_table_2(col1);
|
||||||
|
\c - - - :worker_1_port
|
||||||
|
\d mx_test_schema_1.mx_table_1
|
||||||
|
|
||||||
|
-- Check that foreign key constraint with NOT VALID works as well
|
||||||
|
\c - - - :master_port
|
||||||
|
SET citus.multi_shard_commit_protocol TO '2pc';
|
||||||
|
ALTER TABLE mx_test_schema_1.mx_table_1 DROP CONSTRAINT mx_fk_constraint;
|
||||||
|
ALTER TABLE
|
||||||
|
mx_test_schema_1.mx_table_1
|
||||||
|
ADD CONSTRAINT
|
||||||
|
mx_fk_constraint_2
|
||||||
|
FOREIGN KEY
|
||||||
|
(col1)
|
||||||
|
REFERENCES
|
||||||
|
mx_test_schema_2.mx_table_2(col1)
|
||||||
|
NOT VALID;
|
||||||
|
\c - - - :worker_1_port
|
||||||
|
\d mx_test_schema_1.mx_table_1
|
||||||
|
|
||||||
|
-- Check that mark_tables_colocated call propagates the changes to the workers
|
||||||
|
\c - - - :master_port
|
||||||
|
SELECT nextval('pg_catalog.pg_dist_colocationid_seq') AS last_colocation_id \gset
|
||||||
|
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 10000;
|
||||||
|
SET citus.shard_count TO 7;
|
||||||
|
SET citus.shard_replication_factor TO 1;
|
||||||
|
|
||||||
|
CREATE TABLE mx_colocation_test_1 (a int);
|
||||||
|
SELECT create_distributed_table('mx_colocation_test_1', 'a');
|
||||||
|
|
||||||
|
CREATE TABLE mx_colocation_test_2 (a int);
|
||||||
|
SELECT create_distributed_table('mx_colocation_test_2', 'a');
|
||||||
|
|
||||||
|
-- Check the colocation IDs of the created tables
|
||||||
|
SELECT
|
||||||
|
logicalrelid, colocationid
|
||||||
|
FROM
|
||||||
|
pg_dist_partition
|
||||||
|
WHERE
|
||||||
|
logicalrelid = 'mx_colocation_test_1'::regclass
|
||||||
|
OR logicalrelid = 'mx_colocation_test_2'::regclass
|
||||||
|
ORDER BY logicalrelid;
|
||||||
|
|
||||||
|
-- Reset the colocation IDs of the test tables
|
||||||
|
DELETE FROM
|
||||||
|
pg_dist_colocation
|
||||||
|
WHERE EXISTS (
|
||||||
|
SELECT 1
|
||||||
|
FROM pg_dist_partition
|
||||||
|
WHERE
|
||||||
|
colocationid = pg_dist_partition.colocationid
|
||||||
|
AND pg_dist_partition.logicalrelid = 'mx_colocation_test_1'::regclass);
|
||||||
|
UPDATE
|
||||||
|
pg_dist_partition
|
||||||
|
SET
|
||||||
|
colocationid = 0
|
||||||
|
WHERE
|
||||||
|
logicalrelid = 'mx_colocation_test_1'::regclass
|
||||||
|
OR logicalrelid = 'mx_colocation_test_2'::regclass;
|
||||||
|
|
||||||
|
-- Mark tables colocated and see the changes on the master and the worker
|
||||||
|
SELECT mark_tables_colocated('mx_colocation_test_1', ARRAY['mx_colocation_test_2']);
|
||||||
|
SELECT
|
||||||
|
logicalrelid, colocationid
|
||||||
|
FROM
|
||||||
|
pg_dist_partition
|
||||||
|
WHERE
|
||||||
|
logicalrelid = 'mx_colocation_test_1'::regclass
|
||||||
|
OR logicalrelid = 'mx_colocation_test_2'::regclass;
|
||||||
|
\c - - - :worker_1_port
|
||||||
|
SELECT
|
||||||
|
logicalrelid, colocationid
|
||||||
|
FROM
|
||||||
|
pg_dist_partition
|
||||||
|
WHERE
|
||||||
|
logicalrelid = 'mx_colocation_test_1'::regclass
|
||||||
|
OR logicalrelid = 'mx_colocation_test_2'::regclass;
|
||||||
|
|
||||||
|
\c - - - :master_port
|
||||||
|
|
||||||
|
-- Check that DROP TABLE on MX tables works
|
||||||
|
DROP TABLE mx_colocation_test_1;
|
||||||
|
DROP TABLE mx_colocation_test_2;
|
||||||
|
\d mx_colocation_test_1
|
||||||
|
\d mx_colocation_test_2
|
||||||
|
|
||||||
|
\c - - - :worker_1_port
|
||||||
|
\d mx_colocation_test_1
|
||||||
|
\d mx_colocation_test_2
|
||||||
|
|
||||||
|
-- Check that dropped MX table can be recreated again
|
||||||
|
\c - - - :master_port
|
||||||
|
SET citus.shard_count TO 7;
|
||||||
|
SET citus.shard_replication_factor TO 1;
|
||||||
|
|
||||||
|
CREATE TABLE mx_temp_drop_test (a int);
|
||||||
|
SELECT create_distributed_table('mx_temp_drop_test', 'a');
|
||||||
|
SELECT logicalrelid, repmodel FROM pg_dist_partition WHERE logicalrelid = 'mx_temp_drop_test'::regclass;
|
||||||
|
|
||||||
|
DROP TABLE mx_temp_drop_test;
|
||||||
|
|
||||||
|
CREATE TABLE mx_temp_drop_test (a int);
|
||||||
|
SELECT create_distributed_table('mx_temp_drop_test', 'a');
|
||||||
|
SELECT logicalrelid, repmodel FROM pg_dist_partition WHERE logicalrelid = 'mx_temp_drop_test'::regclass;
|
||||||
|
|
||||||
|
DROP TABLE mx_temp_drop_test;
|
||||||
|
|
||||||
|
-- Cleanup
|
||||||
|
\c - - - :worker_1_port
|
||||||
|
DROP TABLE mx_test_schema_2.mx_table_2 CASCADE;
|
||||||
|
DROP TABLE mx_test_schema_1.mx_table_1 CASCADE;
|
||||||
|
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_test_schema_2.mx_table_2 CASCADE;
|
||||||
|
DROP TABLE mx_test_schema_1.mx_table_1 CASCADE;
|
||||||
|
DROP TABLE mx_testing_schema.mx_test_table;
|
||||||
|
|
||||||
|
RESET citus.shard_count;
|
||||||
|
RESET citus.shard_replication_factor;
|
||||||
|
RESET citus.multi_shard_commit_protocol;
|
||||||
|
|
||||||
|
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART :last_colocation_id;
|
||||||
|
ALTER SEQUENCE pg_catalog.pg_dist_shard_placement_placementid_seq RESTART :last_placement_id;
|
Loading…
Reference in New Issue