mirror of https://github.com/citusdata/citus.git
Merge pull request #5619 from citusdata/velioglu/table_wo_seq_prototype
Handle tables and sequences as objectspull/5671/head
commit
ed8e137467
|
@ -30,6 +30,7 @@
|
|||
#include "distributed/commands.h"
|
||||
#include "distributed/commands/sequence.h"
|
||||
#include "distributed/commands/utility_hook.h"
|
||||
#include "distributed/metadata/distobject.h"
|
||||
#include "distributed/foreign_key_relationship.h"
|
||||
#include "distributed/listutils.h"
|
||||
#include "distributed/local_executor.h"
|
||||
|
@ -90,7 +91,7 @@ static void TransferSequenceOwnership(Oid ownedSequenceId, Oid targetRelationId,
|
|||
char *columnName);
|
||||
static void InsertMetadataForCitusLocalTable(Oid citusLocalTableId, uint64 shardId,
|
||||
bool autoConverted);
|
||||
static void FinalizeCitusLocalTableCreation(Oid relationId, List *dependentSequenceList);
|
||||
static void FinalizeCitusLocalTableCreation(Oid relationId);
|
||||
|
||||
|
||||
PG_FUNCTION_INFO_V1(citus_add_local_table_to_metadata);
|
||||
|
@ -307,7 +308,13 @@ CreateCitusLocalTable(Oid relationId, bool cascadeViaForeignKeys, bool autoConve
|
|||
ObjectAddressSet(tableAddress, RelationRelationId, relationId);
|
||||
|
||||
/*
|
||||
* Ensure dependencies first as we will create shell table on the other nodes
|
||||
* Ensure that the sequences used in column defaults of the table
|
||||
* have proper types
|
||||
*/
|
||||
EnsureRelationHasCompatibleSequenceTypes(relationId);
|
||||
|
||||
/*
|
||||
* Ensure dependencies exist as we will create shell table on the other nodes
|
||||
* in the MX case.
|
||||
*/
|
||||
EnsureDependenciesExistOnAllNodes(&tableAddress);
|
||||
|
@ -354,18 +361,7 @@ CreateCitusLocalTable(Oid relationId, bool cascadeViaForeignKeys, bool autoConve
|
|||
|
||||
InsertMetadataForCitusLocalTable(shellRelationId, shardId, autoConverted);
|
||||
|
||||
/*
|
||||
* Ensure that the sequences used in column defaults of the table
|
||||
* have proper types
|
||||
*/
|
||||
List *attnumList = NIL;
|
||||
List *dependentSequenceList = NIL;
|
||||
GetDependentSequencesWithRelation(shellRelationId, &attnumList,
|
||||
&dependentSequenceList, 0);
|
||||
EnsureDistributedSequencesHaveOneType(shellRelationId, dependentSequenceList,
|
||||
attnumList);
|
||||
|
||||
FinalizeCitusLocalTableCreation(shellRelationId, dependentSequenceList);
|
||||
FinalizeCitusLocalTableCreation(shellRelationId);
|
||||
}
|
||||
|
||||
|
||||
|
@ -657,8 +653,10 @@ GetShellTableDDLEventsForCitusLocalTable(Oid relationId)
|
|||
*/
|
||||
IncludeSequenceDefaults includeSequenceDefaults = NEXTVAL_SEQUENCE_DEFAULTS;
|
||||
|
||||
bool creatingShellTableOnRemoteNode = false;
|
||||
List *tableDDLCommands = GetFullTableCreationCommands(relationId,
|
||||
includeSequenceDefaults);
|
||||
includeSequenceDefaults,
|
||||
creatingShellTableOnRemoteNode);
|
||||
|
||||
List *shellTableDDLEvents = NIL;
|
||||
TableDDLCommand *tableDDLCommand = NULL;
|
||||
|
@ -1227,7 +1225,7 @@ InsertMetadataForCitusLocalTable(Oid citusLocalTableId, uint64 shardId,
|
|||
* sequences dependent with the table.
|
||||
*/
|
||||
static void
|
||||
FinalizeCitusLocalTableCreation(Oid relationId, List *dependentSequenceList)
|
||||
FinalizeCitusLocalTableCreation(Oid relationId)
|
||||
{
|
||||
/*
|
||||
* If it is a foreign table, then skip creating citus truncate trigger
|
||||
|
@ -1240,16 +1238,7 @@ FinalizeCitusLocalTableCreation(Oid relationId, List *dependentSequenceList)
|
|||
|
||||
if (ShouldSyncTableMetadata(relationId))
|
||||
{
|
||||
if (ClusterHasKnownMetadataWorkers())
|
||||
{
|
||||
/*
|
||||
* Ensure sequence dependencies and mark them as distributed
|
||||
* before creating table metadata on workers
|
||||
*/
|
||||
MarkSequenceListDistributedAndPropagateWithDependencies(relationId,
|
||||
dependentSequenceList);
|
||||
}
|
||||
CreateTableMetadataOnWorkers(relationId);
|
||||
SyncCitusTableMetadata(relationId);
|
||||
}
|
||||
|
||||
/*
|
||||
|
|
|
@ -113,8 +113,9 @@ static void EnsureLocalTableEmptyIfNecessary(Oid relationId, char distributionMe
|
|||
static bool ShouldLocalTableBeEmpty(Oid relationId, char distributionMethod, bool
|
||||
viaDeprecatedAPI);
|
||||
static void EnsureCitusTableCanBeCreated(Oid relationOid);
|
||||
static void EnsureSequenceExistOnMetadataWorkersForRelation(Oid relationId,
|
||||
Oid sequenceOid);
|
||||
static void EnsureDistributedSequencesHaveOneType(Oid relationId,
|
||||
List *dependentSequenceList,
|
||||
List *attnumList);
|
||||
static List * GetFKeyCreationCommandsRelationInvolvedWithTableType(Oid relationId,
|
||||
int tableTypeFlag);
|
||||
static Oid DropFKeysAndUndistributeTable(Oid relationId);
|
||||
|
@ -432,6 +433,12 @@ CreateDistributedTable(Oid relationId, Var *distributionColumn, char distributio
|
|||
DropFKeysRelationInvolvedWithTableType(relationId, INCLUDE_LOCAL_TABLES);
|
||||
}
|
||||
|
||||
/*
|
||||
* Ensure that the sequences used in column defaults of the table
|
||||
* have proper types
|
||||
*/
|
||||
EnsureRelationHasCompatibleSequenceTypes(relationId);
|
||||
|
||||
/*
|
||||
* distributed tables might have dependencies on different objects, since we create
|
||||
* shards for a distributed table via multiple sessions these objects will be created
|
||||
|
@ -446,7 +453,6 @@ CreateDistributedTable(Oid relationId, Var *distributionColumn, char distributio
|
|||
colocateWithTableName,
|
||||
viaDeprecatedAPI);
|
||||
|
||||
|
||||
/*
|
||||
* Due to dropping columns, the parent's distribution key may not match the
|
||||
* partition's distribution key. The input distributionColumn belongs to
|
||||
|
@ -494,16 +500,6 @@ CreateDistributedTable(Oid relationId, Var *distributionColumn, char distributio
|
|||
InsertIntoPgDistPartition(relationId, distributionMethod, distributionColumn,
|
||||
colocationId, replicationModel, autoConverted);
|
||||
|
||||
/*
|
||||
* Ensure that the sequences used in column defaults of the table
|
||||
* have proper types
|
||||
*/
|
||||
List *attnumList = NIL;
|
||||
List *dependentSequenceList = NIL;
|
||||
GetDependentSequencesWithRelation(relationId, &attnumList, &dependentSequenceList, 0);
|
||||
EnsureDistributedSequencesHaveOneType(relationId, dependentSequenceList,
|
||||
attnumList);
|
||||
|
||||
/* foreign tables do not support TRUNCATE trigger */
|
||||
if (RegularTable(relationId))
|
||||
{
|
||||
|
@ -537,17 +533,7 @@ CreateDistributedTable(Oid relationId, Var *distributionColumn, char distributio
|
|||
|
||||
if (ShouldSyncTableMetadata(relationId))
|
||||
{
|
||||
if (ClusterHasKnownMetadataWorkers())
|
||||
{
|
||||
/*
|
||||
* Ensure both sequence and its' dependencies and mark them as distributed
|
||||
* before creating table metadata on workers
|
||||
*/
|
||||
MarkSequenceListDistributedAndPropagateWithDependencies(relationId,
|
||||
dependentSequenceList);
|
||||
}
|
||||
|
||||
CreateTableMetadataOnWorkers(relationId);
|
||||
SyncCitusTableMetadata(relationId);
|
||||
}
|
||||
|
||||
/*
|
||||
|
@ -605,11 +591,15 @@ CreateDistributedTable(Oid relationId, Var *distributionColumn, char distributio
|
|||
* If any other distributed table uses the input sequence, it checks whether
|
||||
* the types of the columns using the sequence match. If they don't, it errors out.
|
||||
* Otherwise, the condition is ensured.
|
||||
* Since the owner of the sequence may not distributed yet, it should be added
|
||||
* explicitly.
|
||||
*/
|
||||
void
|
||||
EnsureSequenceTypeSupported(Oid seqOid, Oid seqTypId)
|
||||
EnsureSequenceTypeSupported(Oid seqOid, Oid seqTypId, Oid ownerRelationId)
|
||||
{
|
||||
List *citusTableIdList = CitusTableTypeIdList(ANY_CITUS_TABLE_TYPE);
|
||||
citusTableIdList = list_append_unique_oid(citusTableIdList, ownerRelationId);
|
||||
|
||||
Oid citusTableId = InvalidOid;
|
||||
foreach_oid(citusTableId, citusTableIdList)
|
||||
{
|
||||
|
@ -676,59 +666,18 @@ AlterSequenceType(Oid seqOid, Oid typeOid)
|
|||
|
||||
|
||||
/*
|
||||
* MarkSequenceListDistributedAndPropagateWithDependencies ensures sequences and their
|
||||
* dependencies for the given sequence list exist on all nodes and marks them as distributed.
|
||||
* EnsureRelationHasCompatibleSequenceTypes ensures that sequences used for columns
|
||||
* of the table have compatible types both with the column type on that table and
|
||||
* all other distributed tables' columns they have used for
|
||||
*/
|
||||
void
|
||||
MarkSequenceListDistributedAndPropagateWithDependencies(Oid relationId,
|
||||
List *sequenceList)
|
||||
EnsureRelationHasCompatibleSequenceTypes(Oid relationId)
|
||||
{
|
||||
Oid sequenceOid = InvalidOid;
|
||||
foreach_oid(sequenceOid, sequenceList)
|
||||
{
|
||||
MarkSequenceDistributedAndPropagateWithDependencies(relationId, sequenceOid);
|
||||
}
|
||||
}
|
||||
List *attnumList = NIL;
|
||||
List *dependentSequenceList = NIL;
|
||||
|
||||
|
||||
/*
|
||||
* MarkSequenceDistributedAndPropagateWithDependencies ensures sequence and its'
|
||||
* dependencies for the given sequence exist on all nodes and marks them as distributed.
|
||||
*/
|
||||
void
|
||||
MarkSequenceDistributedAndPropagateWithDependencies(Oid relationId, Oid sequenceOid)
|
||||
{
|
||||
/* get sequence address */
|
||||
ObjectAddress sequenceAddress = { 0 };
|
||||
ObjectAddressSet(sequenceAddress, RelationRelationId, sequenceOid);
|
||||
EnsureDependenciesExistOnAllNodes(&sequenceAddress);
|
||||
EnsureSequenceExistOnMetadataWorkersForRelation(relationId, sequenceOid);
|
||||
MarkObjectDistributed(&sequenceAddress);
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* EnsureSequenceExistOnMetadataWorkersForRelation ensures sequence for the given relation
|
||||
* exist on each worker node with metadata.
|
||||
*/
|
||||
static void
|
||||
EnsureSequenceExistOnMetadataWorkersForRelation(Oid relationId, Oid sequenceOid)
|
||||
{
|
||||
Assert(ShouldSyncTableMetadata(relationId));
|
||||
|
||||
char *ownerName = TableOwner(relationId);
|
||||
List *sequenceDDLList = DDLCommandsForSequence(sequenceOid, ownerName);
|
||||
|
||||
/* prevent recursive propagation */
|
||||
SendCommandToWorkersWithMetadata(DISABLE_DDL_PROPAGATION);
|
||||
|
||||
const char *sequenceCommand = NULL;
|
||||
foreach_ptr(sequenceCommand, sequenceDDLList)
|
||||
{
|
||||
SendCommandToWorkersWithMetadata(sequenceCommand);
|
||||
}
|
||||
|
||||
SendCommandToWorkersWithMetadata(ENABLE_DDL_PROPAGATION);
|
||||
GetDependentSequencesWithRelation(relationId, &attnumList, &dependentSequenceList, 0);
|
||||
EnsureDistributedSequencesHaveOneType(relationId, dependentSequenceList, attnumList);
|
||||
}
|
||||
|
||||
|
||||
|
@ -737,7 +686,7 @@ EnsureSequenceExistOnMetadataWorkersForRelation(Oid relationId, Oid sequenceOid)
|
|||
* in which the sequence is used as default is supported for each sequence in input
|
||||
* dependentSequenceList, and then alters the sequence type if not the same with the column type.
|
||||
*/
|
||||
void
|
||||
static void
|
||||
EnsureDistributedSequencesHaveOneType(Oid relationId, List *dependentSequenceList,
|
||||
List *attnumList)
|
||||
{
|
||||
|
@ -753,7 +702,7 @@ EnsureDistributedSequencesHaveOneType(Oid relationId, List *dependentSequenceLis
|
|||
* that sequence is supported
|
||||
*/
|
||||
Oid seqTypId = GetAttributeTypeOid(relationId, attnum);
|
||||
EnsureSequenceTypeSupported(sequenceOid, seqTypId);
|
||||
EnsureSequenceTypeSupported(sequenceOid, seqTypId, relationId);
|
||||
|
||||
/*
|
||||
* Alter the sequence's data type in the coordinator if needed.
|
||||
|
|
|
@ -224,15 +224,47 @@ GetDependencyCreateDDLCommands(const ObjectAddress *dependency)
|
|||
{
|
||||
case OCLASS_CLASS:
|
||||
{
|
||||
char relKind = get_rel_relkind(dependency->objectId);
|
||||
|
||||
/*
|
||||
* types have an intermediate dependency on a relation (aka class), so we do
|
||||
* support classes when the relkind is composite
|
||||
*/
|
||||
if (get_rel_relkind(dependency->objectId) == RELKIND_COMPOSITE_TYPE)
|
||||
if (relKind == RELKIND_COMPOSITE_TYPE)
|
||||
{
|
||||
return NIL;
|
||||
}
|
||||
|
||||
if (relKind == RELKIND_RELATION || relKind == RELKIND_PARTITIONED_TABLE ||
|
||||
relKind == RELKIND_FOREIGN_TABLE)
|
||||
{
|
||||
Oid relationId = dependency->objectId;
|
||||
List *commandList = NIL;
|
||||
|
||||
if (IsCitusTable(relationId))
|
||||
{
|
||||
bool creatingShellTableOnRemoteNode = true;
|
||||
List *tableDDLCommands = GetFullTableCreationCommands(relationId,
|
||||
WORKER_NEXTVAL_SEQUENCE_DEFAULTS,
|
||||
creatingShellTableOnRemoteNode);
|
||||
TableDDLCommand *tableDDLCommand = NULL;
|
||||
foreach_ptr(tableDDLCommand, tableDDLCommands)
|
||||
{
|
||||
Assert(CitusIsA(tableDDLCommand, TableDDLCommand));
|
||||
commandList = lappend(commandList, GetTableDDLCommand(
|
||||
tableDDLCommand));
|
||||
}
|
||||
}
|
||||
|
||||
return commandList;
|
||||
}
|
||||
|
||||
if (relKind == RELKIND_SEQUENCE)
|
||||
{
|
||||
char *sequenceOwnerName = TableOwner(dependency->objectId);
|
||||
return DDLCommandsForSequence(dependency->objectId, sequenceOwnerName);
|
||||
}
|
||||
|
||||
/* if this relation is not supported, break to the error at the end */
|
||||
break;
|
||||
}
|
||||
|
@ -316,14 +348,15 @@ GetDependencyCreateDDLCommands(const ObjectAddress *dependency)
|
|||
|
||||
|
||||
/*
|
||||
* ReplicateAllDependenciesToNode replicate all previously marked objects to a worker
|
||||
* node. The function also sets clusterHasDistributedFunction if there are any
|
||||
* distributed functions.
|
||||
* ReplicateAllObjectsToNodeCommandList returns commands to replicate all
|
||||
* previously marked objects to a worker node. The function also sets
|
||||
* clusterHasDistributedFunction if there are any distributed functions.
|
||||
*/
|
||||
void
|
||||
ReplicateAllDependenciesToNode(const char *nodeName, int nodePort)
|
||||
List *
|
||||
ReplicateAllObjectsToNodeCommandList(const char *nodeName, int nodePort)
|
||||
{
|
||||
List *ddlCommands = NIL;
|
||||
/* since we are executing ddl commands disable propagation first, primarily for mx */
|
||||
List *ddlCommands = list_make1(DISABLE_DDL_PROPAGATION);
|
||||
|
||||
/*
|
||||
* collect all dependencies in creation order and get their ddl commands
|
||||
|
@ -360,21 +393,10 @@ ReplicateAllDependenciesToNode(const char *nodeName, int nodePort)
|
|||
ddlCommands = list_concat(ddlCommands,
|
||||
GetDependencyCreateDDLCommands(dependency));
|
||||
}
|
||||
if (list_length(ddlCommands) <= 0)
|
||||
{
|
||||
/* no commands to replicate dependencies to the new worker */
|
||||
return;
|
||||
}
|
||||
|
||||
/* since we are executing ddl commands lets disable propagation, primarily for mx */
|
||||
ddlCommands = list_concat(list_make1(DISABLE_DDL_PROPAGATION), ddlCommands);
|
||||
ddlCommands = lappend(ddlCommands, ENABLE_DDL_PROPAGATION);
|
||||
|
||||
/* send commands to new workers, the current user should a superuser */
|
||||
Assert(superuser());
|
||||
SendMetadataCommandListToWorkerInCoordinatedTransaction(nodeName,
|
||||
nodePort,
|
||||
CurrentUserName(),
|
||||
ddlCommands);
|
||||
return ddlCommands;
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -11,6 +11,7 @@
|
|||
|
||||
#include "postgres.h"
|
||||
|
||||
#include "access/xact.h"
|
||||
#include "catalog/dependency.h"
|
||||
#include "catalog/namespace.h"
|
||||
#include "commands/defrem.h"
|
||||
|
@ -23,6 +24,7 @@
|
|||
#include "distributed/metadata/distobject.h"
|
||||
#include "distributed/metadata_cache.h"
|
||||
#include "distributed/metadata_sync.h"
|
||||
#include "distributed/worker_create_or_replace.h"
|
||||
#include "nodes/parsenodes.h"
|
||||
#include "utils/builtins.h"
|
||||
#include "utils/lsyscache.h"
|
||||
|
@ -656,3 +658,77 @@ PostprocessAlterSequenceOwnerStmt(Node *node, const char *queryString)
|
|||
|
||||
return NIL;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* GenerateBackupNameForSequenceCollision generates a new sequence name for an existing
|
||||
* sequence. The name is generated in such a way that the new name doesn't overlap with
|
||||
* an existing relation by adding a suffix with incrementing number after the new name.
|
||||
*/
|
||||
char *
|
||||
GenerateBackupNameForSequenceCollision(const ObjectAddress *address)
|
||||
{
|
||||
char *newName = palloc0(NAMEDATALEN);
|
||||
char suffix[NAMEDATALEN] = { 0 };
|
||||
int count = 0;
|
||||
char *namespaceName = get_namespace_name(get_rel_namespace(address->objectId));
|
||||
Oid schemaId = get_namespace_oid(namespaceName, false);
|
||||
|
||||
char *baseName = get_rel_name(address->objectId);
|
||||
int baseLength = strlen(baseName);
|
||||
|
||||
while (true)
|
||||
{
|
||||
int suffixLength = SafeSnprintf(suffix, NAMEDATALEN - 1, "(citus_backup_%d)",
|
||||
count);
|
||||
|
||||
/* trim the base name at the end to leave space for the suffix and trailing \0 */
|
||||
baseLength = Min(baseLength, NAMEDATALEN - suffixLength - 1);
|
||||
|
||||
/* clear newName before copying the potentially trimmed baseName and suffix */
|
||||
memset(newName, 0, NAMEDATALEN);
|
||||
strncpy_s(newName, NAMEDATALEN, baseName, baseLength);
|
||||
strncpy_s(newName + baseLength, NAMEDATALEN - baseLength, suffix,
|
||||
suffixLength);
|
||||
|
||||
Oid newRelationId = get_relname_relid(newName, schemaId);
|
||||
if (newRelationId == InvalidOid)
|
||||
{
|
||||
return newName;
|
||||
}
|
||||
|
||||
count++;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* RenameExistingSequenceWithDifferentTypeIfExists renames the sequence's type if
|
||||
* that sequence exists and the desired sequence type is different than it's type.
|
||||
*/
|
||||
void
|
||||
RenameExistingSequenceWithDifferentTypeIfExists(RangeVar *sequence, Oid desiredSeqTypeId)
|
||||
{
|
||||
Oid sequenceOid;
|
||||
RangeVarGetAndCheckCreationNamespace(sequence, NoLock, &sequenceOid);
|
||||
|
||||
if (OidIsValid(sequenceOid))
|
||||
{
|
||||
Form_pg_sequence pgSequenceForm = pg_get_sequencedef(sequenceOid);
|
||||
if (pgSequenceForm->seqtypid != desiredSeqTypeId)
|
||||
{
|
||||
ObjectAddress sequenceAddress = { 0 };
|
||||
ObjectAddressSet(sequenceAddress, RelationRelationId, sequenceOid);
|
||||
|
||||
char *newName = GenerateBackupNameForCollision(&sequenceAddress);
|
||||
|
||||
RenameStmt *renameStmt = CreateRenameStatement(&sequenceAddress, newName);
|
||||
const char *sqlRenameStmt = DeparseTreeNode((Node *) renameStmt);
|
||||
ProcessUtilityParseTree((Node *) renameStmt, sqlRenameStmt,
|
||||
PROCESS_UTILITY_QUERY,
|
||||
NULL, None_Receiver, NULL);
|
||||
|
||||
CommandCounterIncrement();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1955,6 +1955,12 @@ PostprocessAlterTableStmt(AlterTableStmt *alterTableStatement)
|
|||
return;
|
||||
}
|
||||
|
||||
/*
|
||||
* Before ensuring each dependency exist, update dependent sequences
|
||||
* types if necessary.
|
||||
*/
|
||||
EnsureRelationHasCompatibleSequenceTypes(relationId);
|
||||
|
||||
/* changing a relation could introduce new dependencies */
|
||||
ObjectAddress tableAddress = { 0 };
|
||||
ObjectAddressSet(tableAddress, RelationRelationId, relationId);
|
||||
|
@ -2045,18 +2051,9 @@ PostprocessAlterTableStmt(AlterTableStmt *alterTableStatement)
|
|||
Oid seqOid = GetSequenceOid(relationId, attnum);
|
||||
if (seqOid != InvalidOid)
|
||||
{
|
||||
EnsureDistributedSequencesHaveOneType(relationId,
|
||||
list_make1_oid(
|
||||
seqOid),
|
||||
list_make1_int(
|
||||
attnum));
|
||||
|
||||
if (ShouldSyncTableMetadata(relationId) &&
|
||||
ClusterHasKnownMetadataWorkers())
|
||||
if (ShouldSyncTableMetadata(relationId))
|
||||
{
|
||||
needMetadataSyncForNewSequences = true;
|
||||
MarkSequenceDistributedAndPropagateWithDependencies(
|
||||
relationId, seqOid);
|
||||
alterTableDefaultNextvalCmd =
|
||||
GetAddColumnWithNextvalDefaultCmd(seqOid,
|
||||
relationId,
|
||||
|
@ -2088,16 +2085,9 @@ PostprocessAlterTableStmt(AlterTableStmt *alterTableStatement)
|
|||
Oid seqOid = GetSequenceOid(relationId, attnum);
|
||||
if (seqOid != InvalidOid)
|
||||
{
|
||||
EnsureDistributedSequencesHaveOneType(relationId,
|
||||
list_make1_oid(seqOid),
|
||||
list_make1_int(attnum));
|
||||
|
||||
if (ShouldSyncTableMetadata(relationId) &&
|
||||
ClusterHasKnownMetadataWorkers())
|
||||
if (ShouldSyncTableMetadata(relationId))
|
||||
{
|
||||
needMetadataSyncForNewSequences = true;
|
||||
MarkSequenceDistributedAndPropagateWithDependencies(relationId,
|
||||
seqOid);
|
||||
alterTableDefaultNextvalCmd = GetAlterColumnWithNextvalDefaultCmd(
|
||||
seqOid, relationId, command->name);
|
||||
}
|
||||
|
@ -2627,8 +2617,7 @@ ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement)
|
|||
* We currently don't support adding a serial column for an MX table
|
||||
* TODO: record the dependency in the workers
|
||||
*/
|
||||
if (ShouldSyncTableMetadata(relationId) &&
|
||||
ClusterHasKnownMetadataWorkers())
|
||||
if (ShouldSyncTableMetadata(relationId))
|
||||
{
|
||||
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
||||
errmsg(
|
||||
|
|
|
@ -37,6 +37,7 @@
|
|||
#include "distributed/metadata/dependency.h"
|
||||
#include "distributed/metadata/distobject.h"
|
||||
#include "distributed/metadata_cache.h"
|
||||
#include "distributed/metadata_sync.h"
|
||||
#include "distributed/version_compat.h"
|
||||
#include "miscadmin.h"
|
||||
#include "utils/fmgroids.h"
|
||||
|
@ -120,6 +121,7 @@ typedef struct ViewDependencyNode
|
|||
}ViewDependencyNode;
|
||||
|
||||
|
||||
static List * GetRelationSequenceDependencyList(Oid relationId);
|
||||
static List * GetRelationTriggerFunctionDependencyList(Oid relationId);
|
||||
static List * GetRelationStatsSchemaDependencyList(Oid relationId);
|
||||
static DependencyDefinition * CreateObjectAddressDependencyDef(Oid classId, Oid objectId);
|
||||
|
@ -671,11 +673,20 @@ SupportedDependencyByCitus(const ObjectAddress *address)
|
|||
|
||||
case OCLASS_CLASS:
|
||||
{
|
||||
char relKind = get_rel_relkind(address->objectId);
|
||||
|
||||
/*
|
||||
* composite types have a reference to a relation of composite type, we need
|
||||
* to follow those to get the dependencies of type fields.
|
||||
*
|
||||
* As we also handle tables as objects as well, follow dependencies
|
||||
* for tables.
|
||||
*/
|
||||
if (get_rel_relkind(address->objectId) == RELKIND_COMPOSITE_TYPE)
|
||||
if (relKind == RELKIND_COMPOSITE_TYPE ||
|
||||
relKind == RELKIND_RELATION ||
|
||||
relKind == RELKIND_PARTITIONED_TABLE ||
|
||||
relKind == RELKIND_FOREIGN_TABLE ||
|
||||
relKind == RELKIND_SEQUENCE)
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
@ -985,6 +996,15 @@ ExpandCitusSupportedTypes(ObjectAddressCollector *collector, ObjectAddress targe
|
|||
List *statisticsSchemaDependencyList =
|
||||
GetRelationStatsSchemaDependencyList(relationId);
|
||||
result = list_concat(result, statisticsSchemaDependencyList);
|
||||
|
||||
/*
|
||||
* Get the dependent sequences for tables (both as serial columns and
|
||||
* columns have nextval with existing sequences) and expand dependency list
|
||||
* with them.
|
||||
*/
|
||||
List *sequenceDependencyList = GetRelationSequenceDependencyList(relationId);
|
||||
|
||||
result = list_concat(result, sequenceDependencyList);
|
||||
}
|
||||
|
||||
default:
|
||||
|
@ -997,6 +1017,24 @@ ExpandCitusSupportedTypes(ObjectAddressCollector *collector, ObjectAddress targe
|
|||
}
|
||||
|
||||
|
||||
/*
|
||||
* GetRelationSequenceDependencyList returns the sequence dependency definition
|
||||
* list for the given relation.
|
||||
*/
|
||||
static List *
|
||||
GetRelationSequenceDependencyList(Oid relationId)
|
||||
{
|
||||
List *attnumList = NIL;
|
||||
List *dependentSequenceList = NIL;
|
||||
|
||||
GetDependentSequencesWithRelation(relationId, &attnumList, &dependentSequenceList, 0);
|
||||
List *sequenceDependencyDefList =
|
||||
CreateObjectAddressDependencyDefList(RelationRelationId, dependentSequenceList);
|
||||
|
||||
return sequenceDependencyDefList;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* GetRelationStatsSchemaDependencyList returns a list of DependencyDefinition
|
||||
* objects for the schemas that statistics' of the relation with relationId depends.
|
||||
|
|
|
@ -83,20 +83,15 @@
|
|||
char *EnableManualMetadataChangesForUser = "";
|
||||
|
||||
|
||||
static void EnsureSequentialModeMetadataOperations(void);
|
||||
static List * DistributedObjectMetadataSyncCommandList(void);
|
||||
static List * GetDistributedTableDDLEvents(Oid relationId);
|
||||
static void EnsureObjectMetadataIsSane(int distributionArgumentIndex,
|
||||
int colocationId);
|
||||
static char * LocalGroupIdUpdateCommand(int32 groupId);
|
||||
static List * SequenceDependencyCommandList(Oid relationId);
|
||||
static char * TruncateTriggerCreateCommand(Oid relationId);
|
||||
static char * SchemaOwnerName(Oid objectId);
|
||||
static bool HasMetadataWorkers(void);
|
||||
static List * DetachPartitionCommandList(void);
|
||||
static void CreateShellTableOnWorkers(Oid relationId);
|
||||
static void CreateTableMetadataOnWorkers(Oid relationId);
|
||||
static bool ShouldSyncTableMetadataInternal(bool hashDistributed,
|
||||
bool citusTableWithNoDistKey);
|
||||
static bool SyncMetadataSnapshotToNode(WorkerNode *workerNode, bool raiseOnError);
|
||||
static bool SyncNodeMetadataSnapshotToNode(WorkerNode *workerNode, bool raiseOnError);
|
||||
static void DropMetadataSnapshotOnNode(WorkerNode *workerNode);
|
||||
static char * CreateSequenceDependencyCommand(Oid relationId, Oid sequenceId,
|
||||
char *columnName);
|
||||
|
@ -111,6 +106,7 @@ static RoleSpec * GetRoleSpecObjectForGrantStmt(Oid roleOid);
|
|||
static List * GenerateGrantOnSchemaQueriesFromAclItem(Oid schemaOid,
|
||||
AclItem *aclItem);
|
||||
static void SetLocalEnableDependencyCreation(bool state);
|
||||
static void SetLocalReplicateReferenceTablesOnActivate(bool state);
|
||||
static char * GenerateSetRoleQuery(Oid roleOid);
|
||||
static void MetadataSyncSigTermHandler(SIGNAL_ARGS);
|
||||
static void MetadataSyncSigAlrmHandler(SIGNAL_ARGS);
|
||||
|
@ -155,7 +151,7 @@ static bool got_SIGALRM = false;
|
|||
|
||||
/*
|
||||
* start_metadata_sync_to_node function sets hasmetadata column of the given
|
||||
* node to true, and then synchronizes the metadata on the node.
|
||||
* node to true, and then activate node without replicating reference tables.
|
||||
*/
|
||||
Datum
|
||||
start_metadata_sync_to_node(PG_FUNCTION_ARGS)
|
||||
|
@ -165,20 +161,29 @@ start_metadata_sync_to_node(PG_FUNCTION_ARGS)
|
|||
text *nodeName = PG_GETARG_TEXT_P(0);
|
||||
int32 nodePort = PG_GETARG_INT32(1);
|
||||
|
||||
EnsureSuperUser();
|
||||
EnsureCoordinator();
|
||||
|
||||
char *nodeNameString = text_to_cstring(nodeName);
|
||||
|
||||
StartMetadataSyncToNode(nodeNameString, nodePort);
|
||||
bool prevReplicateRefTablesOnActivate = ReplicateReferenceTablesOnActivate;
|
||||
SetLocalReplicateReferenceTablesOnActivate(false);
|
||||
|
||||
ActivateNode(nodeNameString, nodePort);
|
||||
TransactionModifiedNodeMetadata = true;
|
||||
|
||||
SetLocalReplicateReferenceTablesOnActivate(prevReplicateRefTablesOnActivate);
|
||||
|
||||
PG_RETURN_VOID();
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* StartMetadataSyncToNode is the internal API for
|
||||
* SyncNodeMetadataToNode is the internal API for
|
||||
* start_metadata_sync_to_node().
|
||||
*/
|
||||
void
|
||||
StartMetadataSyncToNode(const char *nodeNameString, int32 nodePort)
|
||||
SyncNodeMetadataToNode(const char *nodeNameString, int32 nodePort)
|
||||
{
|
||||
char *escapedNodeName = quote_literal_cstr(nodeNameString);
|
||||
|
||||
|
@ -244,7 +249,29 @@ StartMetadataSyncToNode(const char *nodeNameString, int32 nodePort)
|
|||
|
||||
/* fail if metadata synchronization doesn't succeed */
|
||||
bool raiseInterrupts = true;
|
||||
SyncMetadataSnapshotToNode(workerNode, raiseInterrupts);
|
||||
SyncNodeMetadataSnapshotToNode(workerNode, raiseInterrupts);
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* SyncCitusTableMetadata syncs citus table metadata to worker nodes with metadata.
|
||||
* Our definition of metadata includes the shell table and its inter relations with
|
||||
* other shell tables, corresponding pg_dist_object, pg_dist_partiton, pg_dist_shard
|
||||
* and pg_dist_shard placement entries.
|
||||
*/
|
||||
void
|
||||
SyncCitusTableMetadata(Oid relationId)
|
||||
{
|
||||
CreateShellTableOnWorkers(relationId);
|
||||
CreateTableMetadataOnWorkers(relationId);
|
||||
CreateInterTableRelationshipOfRelationOnWorkers(relationId);
|
||||
|
||||
if (!IsTableOwnedByExtension(relationId))
|
||||
{
|
||||
ObjectAddress relationAddress = { 0 };
|
||||
ObjectAddressSet(relationAddress, RelationRelationId, relationId);
|
||||
MarkObjectDistributed(&relationAddress);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
@ -260,7 +287,7 @@ StartMetadataSyncToNode(const char *nodeNameString, int32 nodePort)
|
|||
* visible on all connections used by the transaction, meaning we can only use 1
|
||||
* connection per node.
|
||||
*/
|
||||
static void
|
||||
void
|
||||
EnsureSequentialModeMetadataOperations(void)
|
||||
{
|
||||
if (!IsTransactionBlock())
|
||||
|
@ -302,6 +329,7 @@ stop_metadata_sync_to_node(PG_FUNCTION_ARGS)
|
|||
{
|
||||
CheckCitusVersion(ERROR);
|
||||
EnsureCoordinator();
|
||||
EnsureSuperUser();
|
||||
|
||||
text *nodeName = PG_GETARG_TEXT_P(0);
|
||||
int32 nodePort = PG_GETARG_INT32(1);
|
||||
|
@ -350,6 +378,8 @@ stop_metadata_sync_to_node(PG_FUNCTION_ARGS)
|
|||
workerNode = SetWorkerColumn(workerNode, Anum_pg_dist_node_metadatasynced,
|
||||
BoolGetDatum(false));
|
||||
|
||||
TransactionModifiedNodeMetadata = true;
|
||||
|
||||
PG_RETURN_VOID();
|
||||
}
|
||||
|
||||
|
@ -443,25 +473,25 @@ ShouldSyncTableMetadataInternal(bool hashDistributed, bool citusTableWithNoDistK
|
|||
|
||||
|
||||
/*
|
||||
* SyncMetadataSnapshotToNode does the following:
|
||||
* SyncNodeMetadataSnapshotToNode does the following:
|
||||
* 1. Sets the localGroupId on the worker so the worker knows which tuple in
|
||||
* pg_dist_node represents itself.
|
||||
* 2. Recreates the distributed metadata on the given worker.
|
||||
* 2. Recreates the node metadata on the given worker.
|
||||
* If raiseOnError is true, it errors out if synchronization fails.
|
||||
*/
|
||||
static bool
|
||||
SyncMetadataSnapshotToNode(WorkerNode *workerNode, bool raiseOnError)
|
||||
SyncNodeMetadataSnapshotToNode(WorkerNode *workerNode, bool raiseOnError)
|
||||
{
|
||||
char *currentUser = CurrentUserName();
|
||||
|
||||
/* generate and add the local group id's update query */
|
||||
char *localGroupIdUpdateCommand = LocalGroupIdUpdateCommand(workerNode->groupId);
|
||||
|
||||
/* generate the queries which drop the metadata */
|
||||
List *dropMetadataCommandList = MetadataDropCommands();
|
||||
/* generate the queries which drop the node metadata */
|
||||
List *dropMetadataCommandList = NodeMetadataDropCommands();
|
||||
|
||||
/* generate the queries which create the metadata from scratch */
|
||||
List *createMetadataCommandList = MetadataCreateCommands();
|
||||
/* generate the queries which create the node metadata from scratch */
|
||||
List *createMetadataCommandList = NodeMetadataCreateCommands();
|
||||
|
||||
List *recreateMetadataSnapshotCommandList = list_make1(localGroupIdUpdateCommand);
|
||||
recreateMetadataSnapshotCommandList = list_concat(recreateMetadataSnapshotCommandList,
|
||||
|
@ -506,12 +536,28 @@ DropMetadataSnapshotOnNode(WorkerNode *workerNode)
|
|||
|
||||
char *userName = CurrentUserName();
|
||||
|
||||
/* generate the queries which drop the metadata */
|
||||
List *dropMetadataCommandList = MetadataDropCommands();
|
||||
|
||||
/*
|
||||
* Detach partitions, break dependencies between sequences and table then
|
||||
* remove shell tables first.
|
||||
*/
|
||||
List *dropMetadataCommandList = DetachPartitionCommandList();
|
||||
dropMetadataCommandList = lappend(dropMetadataCommandList,
|
||||
BREAK_CITUS_TABLE_SEQUENCE_DEPENDENCY_COMMAND);
|
||||
dropMetadataCommandList = lappend(dropMetadataCommandList,
|
||||
REMOVE_ALL_SHELL_TABLES_COMMAND);
|
||||
dropMetadataCommandList = list_concat(dropMetadataCommandList,
|
||||
NodeMetadataDropCommands());
|
||||
dropMetadataCommandList = lappend(dropMetadataCommandList,
|
||||
LocalGroupIdUpdateCommand(0));
|
||||
|
||||
/* remove all dist table and object/table related metadata afterwards */
|
||||
dropMetadataCommandList = lappend(dropMetadataCommandList, DELETE_ALL_PARTITIONS);
|
||||
dropMetadataCommandList = lappend(dropMetadataCommandList, DELETE_ALL_SHARDS);
|
||||
dropMetadataCommandList = lappend(dropMetadataCommandList, DELETE_ALL_PLACEMENTS);
|
||||
dropMetadataCommandList = lappend(dropMetadataCommandList,
|
||||
DELETE_ALL_DISTRIBUTED_OBJECTS);
|
||||
|
||||
Assert(superuser());
|
||||
SendOptionalMetadataCommandListToWorkerInCoordinatedTransaction(
|
||||
workerNode->workerName,
|
||||
workerNode->workerPort,
|
||||
|
@ -521,28 +567,19 @@ DropMetadataSnapshotOnNode(WorkerNode *workerNode)
|
|||
|
||||
|
||||
/*
|
||||
* MetadataCreateCommands returns list of queries that are
|
||||
* NodeMetadataCreateCommands returns list of queries that are
|
||||
* required to create the current metadata snapshot of the node that the
|
||||
* function is called. The metadata snapshot commands includes the
|
||||
* following queries:
|
||||
*
|
||||
* (i) Query that populates pg_dist_node table
|
||||
* (ii) Queries that create the clustered tables (including foreign keys,
|
||||
* partitioning hierarchy etc.)
|
||||
* (iii) Queries that populate pg_dist_partition table referenced by (ii)
|
||||
* (iv) Queries that populate pg_dist_shard table referenced by (iii)
|
||||
* (v) Queries that populate pg_dist_placement table referenced by (iv)
|
||||
* (vi) Queries that populate pg_dist_object table
|
||||
*/
|
||||
List *
|
||||
MetadataCreateCommands(void)
|
||||
NodeMetadataCreateCommands(void)
|
||||
{
|
||||
List *metadataSnapshotCommandList = NIL;
|
||||
List *distributedTableList = CitusTableList();
|
||||
List *propagatedTableList = NIL;
|
||||
bool includeNodesFromOtherClusters = true;
|
||||
List *workerNodeList = ReadDistNode(includeNodesFromOtherClusters);
|
||||
IncludeSequenceDefaults includeSequenceDefaults = WORKER_NEXTVAL_SEQUENCE_DEFAULTS;
|
||||
|
||||
/* make sure we have deterministic output for our tests */
|
||||
workerNodeList = SortList(workerNodeList, CompareWorkerNodes);
|
||||
|
@ -552,170 +589,6 @@ MetadataCreateCommands(void)
|
|||
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
|
||||
nodeListInsertCommand);
|
||||
|
||||
/* create the list of tables whose metadata will be created */
|
||||
CitusTableCacheEntry *cacheEntry = NULL;
|
||||
foreach_ptr(cacheEntry, distributedTableList)
|
||||
{
|
||||
if (ShouldSyncTableMetadata(cacheEntry->relationId))
|
||||
{
|
||||
propagatedTableList = lappend(propagatedTableList, cacheEntry);
|
||||
}
|
||||
}
|
||||
|
||||
/* create the tables, but not the metadata */
|
||||
foreach_ptr(cacheEntry, propagatedTableList)
|
||||
{
|
||||
Oid relationId = cacheEntry->relationId;
|
||||
ObjectAddress tableAddress = { 0 };
|
||||
|
||||
if (IsTableOwnedByExtension(relationId))
|
||||
{
|
||||
/* skip table creation when the Citus table is owned by an extension */
|
||||
continue;
|
||||
}
|
||||
|
||||
List *ddlCommandList = GetFullTableCreationCommands(relationId,
|
||||
includeSequenceDefaults);
|
||||
char *tableOwnerResetCommand = TableOwnerResetCommand(relationId);
|
||||
|
||||
/*
|
||||
* Tables might have dependencies on different objects, since we create shards for
|
||||
* table via multiple sessions these objects will be created via their own connection
|
||||
* and committed immediately so they become visible to all sessions creating shards.
|
||||
*/
|
||||
ObjectAddressSet(tableAddress, RelationRelationId, relationId);
|
||||
|
||||
/*
|
||||
* Set object propagation to off as we will mark objects distributed
|
||||
* at the end of this function.
|
||||
*/
|
||||
bool prevDependencyCreationValue = EnableDependencyCreation;
|
||||
SetLocalEnableDependencyCreation(false);
|
||||
|
||||
EnsureDependenciesExistOnAllNodes(&tableAddress);
|
||||
|
||||
/*
|
||||
* Ensure sequence dependencies and mark them as distributed
|
||||
*/
|
||||
List *attnumList = NIL;
|
||||
List *dependentSequenceList = NIL;
|
||||
GetDependentSequencesWithRelation(relationId, &attnumList,
|
||||
&dependentSequenceList, 0);
|
||||
|
||||
Oid sequenceOid = InvalidOid;
|
||||
foreach_oid(sequenceOid, dependentSequenceList)
|
||||
{
|
||||
ObjectAddress sequenceAddress = { 0 };
|
||||
ObjectAddressSet(sequenceAddress, RelationRelationId, sequenceOid);
|
||||
EnsureDependenciesExistOnAllNodes(&sequenceAddress);
|
||||
|
||||
/*
|
||||
* Sequences are not marked as distributed while creating table
|
||||
* if no metadata worker node exists. We are marking all sequences
|
||||
* distributed while syncing metadata in such case.
|
||||
*/
|
||||
MarkObjectDistributed(&sequenceAddress);
|
||||
}
|
||||
|
||||
SetLocalEnableDependencyCreation(prevDependencyCreationValue);
|
||||
|
||||
List *workerSequenceDDLCommands = SequenceDDLCommandsForTable(relationId);
|
||||
metadataSnapshotCommandList = list_concat(metadataSnapshotCommandList,
|
||||
workerSequenceDDLCommands);
|
||||
|
||||
/* ddlCommandList contains TableDDLCommand information, need to materialize */
|
||||
TableDDLCommand *tableDDLCommand = NULL;
|
||||
foreach_ptr(tableDDLCommand, ddlCommandList)
|
||||
{
|
||||
Assert(CitusIsA(tableDDLCommand, TableDDLCommand));
|
||||
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
|
||||
GetTableDDLCommand(tableDDLCommand));
|
||||
}
|
||||
|
||||
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
|
||||
tableOwnerResetCommand);
|
||||
|
||||
List *sequenceDependencyCommandList = SequenceDependencyCommandList(
|
||||
relationId);
|
||||
metadataSnapshotCommandList = list_concat(metadataSnapshotCommandList,
|
||||
sequenceDependencyCommandList);
|
||||
}
|
||||
|
||||
/* construct the foreign key constraints after all tables are created */
|
||||
foreach_ptr(cacheEntry, propagatedTableList)
|
||||
{
|
||||
Oid relationId = cacheEntry->relationId;
|
||||
|
||||
if (IsTableOwnedByExtension(relationId))
|
||||
{
|
||||
/* skip foreign key creation when the Citus table is owned by an extension */
|
||||
continue;
|
||||
}
|
||||
|
||||
List *foreignConstraintCommands =
|
||||
GetReferencingForeignConstaintCommands(relationId);
|
||||
|
||||
metadataSnapshotCommandList = list_concat(metadataSnapshotCommandList,
|
||||
foreignConstraintCommands);
|
||||
}
|
||||
|
||||
/* construct partitioning hierarchy after all tables are created */
|
||||
foreach_ptr(cacheEntry, propagatedTableList)
|
||||
{
|
||||
Oid relationId = cacheEntry->relationId;
|
||||
|
||||
if (IsTableOwnedByExtension(relationId))
|
||||
{
|
||||
/* skip partition creation when the Citus table is owned by an extension */
|
||||
continue;
|
||||
}
|
||||
|
||||
if (PartitionTable(relationId))
|
||||
{
|
||||
char *alterTableAttachPartitionCommands =
|
||||
GenerateAlterTableAttachPartitionCommand(relationId);
|
||||
|
||||
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
|
||||
alterTableAttachPartitionCommands);
|
||||
}
|
||||
}
|
||||
|
||||
/* after all tables are created, create the metadata */
|
||||
foreach_ptr(cacheEntry, propagatedTableList)
|
||||
{
|
||||
Oid relationId = cacheEntry->relationId;
|
||||
|
||||
/* add the table metadata command first*/
|
||||
char *metadataCommand = DistributionCreateCommand(cacheEntry);
|
||||
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
|
||||
metadataCommand);
|
||||
|
||||
if (!IsForeignTable(relationId))
|
||||
{
|
||||
/* add the truncate trigger command after the table became distributed */
|
||||
char *truncateTriggerCreateCommand =
|
||||
TruncateTriggerCreateCommand(cacheEntry->relationId);
|
||||
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
|
||||
truncateTriggerCreateCommand);
|
||||
}
|
||||
|
||||
/* add the pg_dist_shard{,placement} entries */
|
||||
List *shardIntervalList = LoadShardIntervalList(relationId);
|
||||
List *shardCreateCommandList = ShardListInsertCommand(shardIntervalList);
|
||||
|
||||
metadataSnapshotCommandList = list_concat(metadataSnapshotCommandList,
|
||||
shardCreateCommandList);
|
||||
}
|
||||
|
||||
/* As the last step, propagate the pg_dist_object entities */
|
||||
if (ShouldPropagate())
|
||||
{
|
||||
List *distributedObjectSyncCommandList =
|
||||
DistributedObjectMetadataSyncCommandList();
|
||||
metadataSnapshotCommandList = list_concat(metadataSnapshotCommandList,
|
||||
distributedObjectSyncCommandList);
|
||||
}
|
||||
|
||||
return metadataSnapshotCommandList;
|
||||
}
|
||||
|
||||
|
@ -724,7 +597,7 @@ MetadataCreateCommands(void)
|
|||
* DistributedObjectMetadataSyncCommandList returns the necessary commands to create
|
||||
* pg_dist_object entries on the new node.
|
||||
*/
|
||||
static List *
|
||||
List *
|
||||
DistributedObjectMetadataSyncCommandList(void)
|
||||
{
|
||||
HeapTuple pgDistObjectTup = NULL;
|
||||
|
@ -829,118 +702,42 @@ DistributedObjectMetadataSyncCommandList(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.
|
||||
* CitusTableMetadataCreateCommandList returns the set of commands necessary to
|
||||
* create the given distributed table metadata on a worker.
|
||||
*/
|
||||
static List *
|
||||
GetDistributedTableDDLEvents(Oid relationId)
|
||||
List *
|
||||
CitusTableMetadataCreateCommandList(Oid relationId)
|
||||
{
|
||||
CitusTableCacheEntry *cacheEntry = GetCitusTableCacheEntry(relationId);
|
||||
|
||||
List *commandList = NIL;
|
||||
IncludeSequenceDefaults includeSequenceDefaults = WORKER_NEXTVAL_SEQUENCE_DEFAULTS;
|
||||
|
||||
/* if the table is owned by an extension we only propagate pg_dist_* records */
|
||||
bool tableOwnedByExtension = IsTableOwnedByExtension(relationId);
|
||||
if (!tableOwnedByExtension)
|
||||
{
|
||||
/*
|
||||
* Commands to create the table, these commands are TableDDLCommands so lets
|
||||
* materialize to the non-sharded version
|
||||
*/
|
||||
List *tableDDLCommands = GetFullTableCreationCommands(relationId,
|
||||
includeSequenceDefaults);
|
||||
TableDDLCommand *tableDDLCommand = NULL;
|
||||
foreach_ptr(tableDDLCommand, tableDDLCommands)
|
||||
{
|
||||
Assert(CitusIsA(tableDDLCommand, TableDDLCommand));
|
||||
commandList = lappend(commandList, GetTableDDLCommand(tableDDLCommand));
|
||||
}
|
||||
|
||||
/* command to associate sequences with table */
|
||||
List *sequenceDependencyCommandList = SequenceDependencyCommandList(
|
||||
relationId);
|
||||
commandList = list_concat(commandList, sequenceDependencyCommandList);
|
||||
}
|
||||
|
||||
/* command to insert pg_dist_partition entry */
|
||||
char *metadataCommand = DistributionCreateCommand(cacheEntry);
|
||||
commandList = lappend(commandList, metadataCommand);
|
||||
|
||||
/* commands to create the truncate trigger of the table */
|
||||
if (!IsForeignTable(relationId))
|
||||
{
|
||||
char *truncateTriggerCreateCommand = TruncateTriggerCreateCommand(relationId);
|
||||
commandList = lappend(commandList, truncateTriggerCreateCommand);
|
||||
}
|
||||
|
||||
/* commands to insert pg_dist_shard & pg_dist_placement entries */
|
||||
List *shardIntervalList = LoadShardIntervalList(relationId);
|
||||
List *shardMetadataInsertCommandList = ShardListInsertCommand(shardIntervalList);
|
||||
commandList = list_concat(commandList, shardMetadataInsertCommandList);
|
||||
|
||||
if (!tableOwnedByExtension)
|
||||
{
|
||||
/* commands to create foreign key constraints */
|
||||
List *foreignConstraintCommands =
|
||||
GetReferencingForeignConstaintCommands(relationId);
|
||||
commandList = list_concat(commandList, foreignConstraintCommands);
|
||||
|
||||
/* commands to create partitioning hierarchy */
|
||||
if (PartitionTable(relationId))
|
||||
{
|
||||
char *alterTableAttachPartitionCommands =
|
||||
GenerateAlterTableAttachPartitionCommand(relationId);
|
||||
commandList = lappend(commandList, alterTableAttachPartitionCommands);
|
||||
}
|
||||
}
|
||||
|
||||
return commandList;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* MetadataDropCommands returns list of queries that are required to
|
||||
* drop all the metadata of the node that are related to clustered tables.
|
||||
* NodeMetadataDropCommands returns list of queries that are required to
|
||||
* drop all the metadata of the node that are not related to clustered tables.
|
||||
* The drop metadata snapshot commands includes the following queries:
|
||||
*
|
||||
* (i) Query to disable DDL propagation (necessary for (ii)
|
||||
* (ii) Queries that DETACH all partitions of distributed tables
|
||||
* (iii) Queries that delete all the rows from pg_dist_node table
|
||||
* (iv) Queries that drop the clustered tables and remove its references from
|
||||
* the pg_dist_partition. Note that distributed relation ids are gathered
|
||||
* from the worker itself to prevent dropping any non-distributed tables
|
||||
* with the same name.
|
||||
* (v) Queries that delete all the rows from pg_dist_shard table referenced by (iv)
|
||||
* (vi) Queries that delete all the rows from pg_dist_placement table
|
||||
* referenced by (v)
|
||||
* (vii) Queries that delete all the rows from pg_dist_object table
|
||||
* (i) Queries that delete all the rows from pg_dist_node table
|
||||
*/
|
||||
List *
|
||||
MetadataDropCommands(void)
|
||||
NodeMetadataDropCommands(void)
|
||||
{
|
||||
List *dropSnapshotCommandList = NIL;
|
||||
List *detachPartitionCommandList = DetachPartitionCommandList();
|
||||
|
||||
dropSnapshotCommandList = list_concat(dropSnapshotCommandList,
|
||||
detachPartitionCommandList);
|
||||
|
||||
/*
|
||||
* We are re-creating the metadata, so not lose track of the
|
||||
* sequences by preventing them dropped via DROP TABLE.
|
||||
*/
|
||||
dropSnapshotCommandList =
|
||||
lappend(dropSnapshotCommandList,
|
||||
BREAK_CITUS_TABLE_SEQUENCE_DEPENDENCY_COMMAND);
|
||||
|
||||
dropSnapshotCommandList = lappend(dropSnapshotCommandList,
|
||||
REMOVE_ALL_CITUS_TABLES_COMMAND);
|
||||
|
||||
dropSnapshotCommandList = lappend(dropSnapshotCommandList, DELETE_ALL_NODES);
|
||||
dropSnapshotCommandList = lappend(dropSnapshotCommandList,
|
||||
DELETE_ALL_DISTRIBUTED_OBJECTS);
|
||||
|
||||
return dropSnapshotCommandList;
|
||||
}
|
||||
|
@ -1530,7 +1327,7 @@ PlacementUpsertCommand(uint64 shardId, uint64 placementId, int shardState,
|
|||
* LocalGroupIdUpdateCommand creates the SQL command required to set the local group id
|
||||
* of a worker and returns the command in a string.
|
||||
*/
|
||||
static char *
|
||||
char *
|
||||
LocalGroupIdUpdateCommand(int32 groupId)
|
||||
{
|
||||
StringInfo updateCommand = makeStringInfo();
|
||||
|
@ -1542,36 +1339,6 @@ LocalGroupIdUpdateCommand(int32 groupId)
|
|||
}
|
||||
|
||||
|
||||
/*
|
||||
* SequenceDDLCommandsForTable returns a list of commands which create sequences (and
|
||||
* their schemas) to run on workers before creating the relation. The sequence creation
|
||||
* commands are wrapped with a `worker_apply_sequence_command` call, which sets the
|
||||
* sequence space uniquely for each worker. Notice that this function is relevant only
|
||||
* during metadata propagation to workers and adds nothing to the list of sequence
|
||||
* commands if none of the workers is marked as receiving metadata changes.
|
||||
*/
|
||||
List *
|
||||
SequenceDDLCommandsForTable(Oid relationId)
|
||||
{
|
||||
List *allSequencesDDLList = NIL;
|
||||
|
||||
List *attnumList = NIL;
|
||||
List *dependentSequenceList = NIL;
|
||||
GetDependentSequencesWithRelation(relationId, &attnumList, &dependentSequenceList, 0);
|
||||
|
||||
char *ownerName = TableOwner(relationId);
|
||||
|
||||
Oid sequenceOid = InvalidOid;
|
||||
foreach_oid(sequenceOid, dependentSequenceList)
|
||||
{
|
||||
List *sequenceDDLCommands = DDLCommandsForSequence(sequenceOid, ownerName);
|
||||
allSequencesDDLList = list_concat(allSequencesDDLList, sequenceDDLCommands);
|
||||
}
|
||||
|
||||
return allSequencesDDLList;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* DDLCommandsForSequence returns the DDL commands needs to be run to create the
|
||||
* sequence and alter the owner to the given owner name.
|
||||
|
@ -1785,7 +1552,7 @@ GetSequencesFromAttrDef(Oid attrdefOid)
|
|||
* necessary to ensure that the sequence is dropped when the table is
|
||||
* dropped.
|
||||
*/
|
||||
static List *
|
||||
List *
|
||||
SequenceDependencyCommandList(Oid relationId)
|
||||
{
|
||||
List *sequenceCommandList = NIL;
|
||||
|
@ -1815,7 +1582,8 @@ SequenceDependencyCommandList(Oid relationId)
|
|||
CreateSequenceDependencyCommand(relationId, sequenceId, columnName);
|
||||
|
||||
sequenceCommandList = lappend(sequenceCommandList,
|
||||
sequenceDependencyCommand);
|
||||
makeTableDDLCommandString(
|
||||
sequenceDependencyCommand));
|
||||
}
|
||||
|
||||
return sequenceCommandList;
|
||||
|
@ -2090,6 +1858,20 @@ SetLocalEnableDependencyCreation(bool state)
|
|||
}
|
||||
|
||||
|
||||
/*
|
||||
* SetLocalReplicateReferenceTablesOnActivate sets the
|
||||
* replicate_reference_tables_on_activate locally
|
||||
*/
|
||||
void
|
||||
SetLocalReplicateReferenceTablesOnActivate(bool state)
|
||||
{
|
||||
set_config_option("citus.replicate_reference_tables_on_activate",
|
||||
state == true ? "on" : "off",
|
||||
(superuser() ? PGC_SUSET : PGC_USERSET), PGC_S_SESSION,
|
||||
GUC_ACTION_LOCAL, true, 0, false);
|
||||
}
|
||||
|
||||
|
||||
static char *
|
||||
GenerateSetRoleQuery(Oid roleOid)
|
||||
{
|
||||
|
@ -2104,7 +1886,7 @@ GenerateSetRoleQuery(Oid roleOid)
|
|||
* TruncateTriggerCreateCommand creates a SQL query calling worker_create_truncate_trigger
|
||||
* function, which creates the truncate trigger on the worker.
|
||||
*/
|
||||
static char *
|
||||
TableDDLCommand *
|
||||
TruncateTriggerCreateCommand(Oid relationId)
|
||||
{
|
||||
StringInfo triggerCreateCommand = makeStringInfo();
|
||||
|
@ -2114,7 +1896,10 @@ TruncateTriggerCreateCommand(Oid relationId)
|
|||
"SELECT worker_create_truncate_trigger(%s)",
|
||||
quote_literal_cstr(tableName));
|
||||
|
||||
return triggerCreateCommand->data;
|
||||
TableDDLCommand *triggerDDLCommand = makeTableDDLCommandString(
|
||||
triggerCreateCommand->data);
|
||||
|
||||
return triggerDDLCommand;
|
||||
}
|
||||
|
||||
|
||||
|
@ -2168,16 +1953,101 @@ HasMetadataWorkers(void)
|
|||
|
||||
|
||||
/*
|
||||
* CreateTableMetadataOnWorkers creates the list of commands needed to create the
|
||||
* given distributed table and sends these commands to all metadata workers i.e. workers
|
||||
* with hasmetadata=true. Before sending the commands, in order to prevent recursive
|
||||
* propagation, DDL propagation on workers are disabled with a
|
||||
* `SET citus.enable_ddl_propagation TO off;` command.
|
||||
* CreateInterTableRelationshipOfRelationOnWorkers create inter table relationship
|
||||
* for the the given relation id on each worker node with metadata.
|
||||
*/
|
||||
void
|
||||
CreateInterTableRelationshipOfRelationOnWorkers(Oid relationId)
|
||||
{
|
||||
/* if the table is owned by an extension we don't create */
|
||||
bool tableOwnedByExtension = IsTableOwnedByExtension(relationId);
|
||||
if (tableOwnedByExtension)
|
||||
{
|
||||
return;
|
||||
}
|
||||
|
||||
List *commandList =
|
||||
InterTableRelationshipOfRelationCommandList(relationId);
|
||||
|
||||
/* prevent recursive propagation */
|
||||
SendCommandToWorkersWithMetadata(DISABLE_DDL_PROPAGATION);
|
||||
|
||||
const char *command = NULL;
|
||||
foreach_ptr(command, commandList)
|
||||
{
|
||||
SendCommandToWorkersWithMetadata(command);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* InterTableRelationshipOfRelationCommandList returns the command list to create
|
||||
* inter table relationship for the given relation.
|
||||
*/
|
||||
List *
|
||||
InterTableRelationshipOfRelationCommandList(Oid relationId)
|
||||
{
|
||||
/* commands to create foreign key constraints */
|
||||
List *commandList = GetReferencingForeignConstaintCommands(relationId);
|
||||
|
||||
/* commands to create partitioning hierarchy */
|
||||
if (PartitionTable(relationId))
|
||||
{
|
||||
char *alterTableAttachPartitionCommands =
|
||||
GenerateAlterTableAttachPartitionCommand(relationId);
|
||||
commandList = lappend(commandList, alterTableAttachPartitionCommands);
|
||||
}
|
||||
|
||||
return commandList;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* CreateShellTableOnWorkers creates the shell table on each worker node with metadata
|
||||
* including sequence dependency and truncate triggers.
|
||||
*/
|
||||
static void
|
||||
CreateShellTableOnWorkers(Oid relationId)
|
||||
{
|
||||
if (IsTableOwnedByExtension(relationId))
|
||||
{
|
||||
return;
|
||||
}
|
||||
|
||||
List *commandList = list_make1(DISABLE_DDL_PROPAGATION);
|
||||
|
||||
IncludeSequenceDefaults includeSequenceDefaults = WORKER_NEXTVAL_SEQUENCE_DEFAULTS;
|
||||
bool creatingShellTableOnRemoteNode = true;
|
||||
List *tableDDLCommands = GetFullTableCreationCommands(relationId,
|
||||
includeSequenceDefaults,
|
||||
creatingShellTableOnRemoteNode);
|
||||
|
||||
TableDDLCommand *tableDDLCommand = NULL;
|
||||
foreach_ptr(tableDDLCommand, tableDDLCommands)
|
||||
{
|
||||
Assert(CitusIsA(tableDDLCommand, TableDDLCommand));
|
||||
commandList = lappend(commandList, GetTableDDLCommand(tableDDLCommand));
|
||||
}
|
||||
|
||||
const char *command = NULL;
|
||||
foreach_ptr(command, commandList)
|
||||
{
|
||||
SendCommandToWorkersWithMetadata(command);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* CreateTableMetadataOnWorkers creates the list of commands needed to create the
|
||||
* metadata of the given distributed table and sends these commands to all metadata
|
||||
* workers i.e. workers with hasmetadata=true. Before sending the commands, in order
|
||||
* to prevent recursive propagation, DDL propagation on workers are disabled with a
|
||||
* `SET citus.enable_ddl_propagation TO off;` command.
|
||||
*/
|
||||
static void
|
||||
CreateTableMetadataOnWorkers(Oid relationId)
|
||||
{
|
||||
List *commandList = GetDistributedTableDDLEvents(relationId);
|
||||
List *commandList = CitusTableMetadataCreateCommandList(relationId);
|
||||
|
||||
/* prevent recursive propagation */
|
||||
SendCommandToWorkersWithMetadata(DISABLE_DDL_PROPAGATION);
|
||||
|
@ -2200,7 +2070,7 @@ CreateTableMetadataOnWorkers(Oid relationId)
|
|||
* an extra step, if there are no partitions to DETACH, this function simply returns
|
||||
* empty list to not disable/enable DDL propagation for nothing.
|
||||
*/
|
||||
static List *
|
||||
List *
|
||||
DetachPartitionCommandList(void)
|
||||
{
|
||||
List *detachPartitionCommandList = NIL;
|
||||
|
@ -2242,7 +2112,7 @@ DetachPartitionCommandList(void)
|
|||
|
||||
|
||||
/*
|
||||
* SyncMetadataToNodes tries recreating the metadata snapshot in the
|
||||
* SyncNodeMetadataToNodes tries recreating the metadata snapshot in the
|
||||
* metadata workers that are out of sync. Returns the result of
|
||||
* synchronization.
|
||||
*
|
||||
|
@ -2250,13 +2120,13 @@ DetachPartitionCommandList(void)
|
|||
* since updates on the pg_dist_node metadata must be rollbacked if anything
|
||||
* goes wrong.
|
||||
*/
|
||||
static MetadataSyncResult
|
||||
SyncMetadataToNodes(void)
|
||||
static NodeMetadataSyncResult
|
||||
SyncNodeMetadataToNodes(void)
|
||||
{
|
||||
MetadataSyncResult result = METADATA_SYNC_SUCCESS;
|
||||
NodeMetadataSyncResult result = NODE_METADATA_SYNC_SUCCESS;
|
||||
if (!IsCoordinator())
|
||||
{
|
||||
return METADATA_SYNC_SUCCESS;
|
||||
return NODE_METADATA_SYNC_SUCCESS;
|
||||
}
|
||||
|
||||
/*
|
||||
|
@ -2266,7 +2136,7 @@ SyncMetadataToNodes(void)
|
|||
*/
|
||||
if (!ConditionalLockRelationOid(DistNodeRelationId(), RowExclusiveLock))
|
||||
{
|
||||
return METADATA_SYNC_FAILED_LOCK;
|
||||
return NODE_METADATA_SYNC_FAILED_LOCK;
|
||||
}
|
||||
|
||||
List *syncedWorkerList = NIL;
|
||||
|
@ -2277,12 +2147,12 @@ SyncMetadataToNodes(void)
|
|||
if (workerNode->hasMetadata && !workerNode->metadataSynced)
|
||||
{
|
||||
bool raiseInterrupts = false;
|
||||
if (!SyncMetadataSnapshotToNode(workerNode, raiseInterrupts))
|
||||
if (!SyncNodeMetadataSnapshotToNode(workerNode, raiseInterrupts))
|
||||
{
|
||||
ereport(WARNING, (errmsg("failed to sync metadata to %s:%d",
|
||||
workerNode->workerName,
|
||||
workerNode->workerPort)));
|
||||
result = METADATA_SYNC_FAILED_SYNC;
|
||||
result = NODE_METADATA_SYNC_FAILED_SYNC;
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -2303,7 +2173,7 @@ SyncMetadataToNodes(void)
|
|||
if (!nodeUpdated->metadataSynced)
|
||||
{
|
||||
/* set the result to FAILED to trigger the sync again */
|
||||
result = METADATA_SYNC_FAILED_SYNC;
|
||||
result = NODE_METADATA_SYNC_FAILED_SYNC;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -2312,11 +2182,11 @@ SyncMetadataToNodes(void)
|
|||
|
||||
|
||||
/*
|
||||
* SyncMetadataToNodesMain is the main function for syncing metadata to
|
||||
* SyncNodeMetadataToNodesMain is the main function for syncing node metadata to
|
||||
* MX nodes. It retries until success and then exits.
|
||||
*/
|
||||
void
|
||||
SyncMetadataToNodesMain(Datum main_arg)
|
||||
SyncNodeMetadataToNodesMain(Datum main_arg)
|
||||
{
|
||||
Oid databaseOid = DatumGetObjectId(main_arg);
|
||||
|
||||
|
@ -2357,11 +2227,11 @@ SyncMetadataToNodesMain(Datum main_arg)
|
|||
{
|
||||
UseCoordinatedTransaction();
|
||||
|
||||
MetadataSyncResult result = SyncMetadataToNodes();
|
||||
syncedAllNodes = (result == METADATA_SYNC_SUCCESS);
|
||||
NodeMetadataSyncResult result = SyncNodeMetadataToNodes();
|
||||
syncedAllNodes = (result == NODE_METADATA_SYNC_SUCCESS);
|
||||
|
||||
/* we use LISTEN/NOTIFY to wait for metadata syncing in tests */
|
||||
if (result != METADATA_SYNC_FAILED_LOCK)
|
||||
if (result != NODE_METADATA_SYNC_FAILED_LOCK)
|
||||
{
|
||||
Async_Notify(METADATA_SYNC_CHANNEL, NULL);
|
||||
}
|
||||
|
@ -2445,11 +2315,11 @@ MetadataSyncSigAlrmHandler(SIGNAL_ARGS)
|
|||
|
||||
|
||||
/*
|
||||
* SpawnSyncMetadataToNodes starts a background worker which runs metadata
|
||||
* SpawnSyncNodeMetadataToNodes starts a background worker which runs node metadata
|
||||
* sync. On success it returns workers' handle. Otherwise it returns NULL.
|
||||
*/
|
||||
BackgroundWorkerHandle *
|
||||
SpawnSyncMetadataToNodes(Oid database, Oid extensionOwner)
|
||||
SpawnSyncNodeMetadataToNodes(Oid database, Oid extensionOwner)
|
||||
{
|
||||
BackgroundWorker worker;
|
||||
BackgroundWorkerHandle *handle = NULL;
|
||||
|
@ -2467,7 +2337,7 @@ SpawnSyncMetadataToNodes(Oid database, Oid extensionOwner)
|
|||
worker.bgw_restart_time = BGW_NEVER_RESTART;
|
||||
strcpy_s(worker.bgw_library_name, sizeof(worker.bgw_library_name), "citus");
|
||||
strcpy_s(worker.bgw_function_name, sizeof(worker.bgw_library_name),
|
||||
"SyncMetadataToNodesMain");
|
||||
"SyncNodeMetadataToNodesMain");
|
||||
worker.bgw_main_arg = ObjectIdGetDatum(MyDatabaseId);
|
||||
memcpy_s(worker.bgw_extra, sizeof(worker.bgw_extra), &extensionOwner,
|
||||
sizeof(Oid));
|
||||
|
|
|
@ -40,8 +40,10 @@
|
|||
#include "distributed/remote_commands.h"
|
||||
#include "distributed/resource_lock.h"
|
||||
#include "distributed/shardinterval_utils.h"
|
||||
#include "distributed/multi_partitioning_utils.h"
|
||||
#include "distributed/shared_connection_stats.h"
|
||||
#include "distributed/string_utils.h"
|
||||
#include "distributed/metadata/pg_dist_object.h"
|
||||
#include "distributed/transaction_recovery.h"
|
||||
#include "distributed/version_compat.h"
|
||||
#include "distributed/worker_manager.h"
|
||||
|
@ -90,7 +92,6 @@ typedef struct NodeMetadata
|
|||
} NodeMetadata;
|
||||
|
||||
/* local function forward declarations */
|
||||
static int ActivateNode(char *nodeName, int nodePort);
|
||||
static void RemoveNodeFromCluster(char *nodeName, int32 nodePort);
|
||||
static void ErrorIfNodeContainsNonRemovablePlacements(WorkerNode *workerNode);
|
||||
static bool PlacementHasActivePlacementOnAnotherGroup(GroupShardPlacement
|
||||
|
@ -105,9 +106,12 @@ static void InsertPlaceholderCoordinatorRecord(void);
|
|||
static void InsertNodeRow(int nodeid, char *nodename, int32 nodeport, NodeMetadata
|
||||
*nodeMetadata);
|
||||
static void DeleteNodeRow(char *nodename, int32 nodeport);
|
||||
static void SetUpDistributedTableDependencies(WorkerNode *workerNode);
|
||||
static void SyncDistributedObjectsToNode(WorkerNode *workerNode);
|
||||
static void UpdateLocalGroupIdOnNode(WorkerNode *workerNode);
|
||||
static void SyncPgDistTableMetadataToNode(WorkerNode *workerNode);
|
||||
static List * InterTableRelationshipCommandList();
|
||||
static WorkerNode * TupleToWorkerNode(TupleDesc tupleDescriptor, HeapTuple heapTuple);
|
||||
static void PropagateNodeWideObjects(WorkerNode *newWorkerNode);
|
||||
static List * PropagateNodeWideObjectsCommandList();
|
||||
static WorkerNode * ModifiableWorkerNode(const char *nodeName, int32 nodePort);
|
||||
static bool NodeIsLocal(WorkerNode *worker);
|
||||
static void SetLockTimeoutLocally(int32 lock_cooldown);
|
||||
|
@ -573,54 +577,118 @@ master_set_node_property(PG_FUNCTION_ARGS)
|
|||
|
||||
|
||||
/*
|
||||
* SetUpDistributedTableDependencies sets up up the following on a node if it's
|
||||
* a primary node that currently stores data:
|
||||
* - All dependencies (e.g., types, schemas)
|
||||
* - Reference tables, because they are needed to handle queries efficiently.
|
||||
* - Distributed functions
|
||||
* InterTableRelationshipCommandList returns the command list to
|
||||
* set up the multiple integrations including
|
||||
*
|
||||
* Note that we do not create the distributed dependencies on the coordinator
|
||||
* since all the dependencies should be present in the coordinator already.
|
||||
* (i) Foreign keys
|
||||
* (ii) Partionining hierarchy
|
||||
*
|
||||
* for each citus table.
|
||||
*/
|
||||
static void
|
||||
SetUpDistributedTableDependencies(WorkerNode *newWorkerNode)
|
||||
static List *
|
||||
InterTableRelationshipCommandList()
|
||||
{
|
||||
if (NodeIsPrimary(newWorkerNode))
|
||||
List *distributedTableList = CitusTableList();
|
||||
List *propagatedTableList = NIL;
|
||||
List *multipleTableIntegrationCommandList = NIL;
|
||||
|
||||
CitusTableCacheEntry *cacheEntry = NULL;
|
||||
foreach_ptr(cacheEntry, distributedTableList)
|
||||
{
|
||||
EnsureNoModificationsHaveBeenDone();
|
||||
|
||||
if (ShouldPropagate() && !NodeIsCoordinator(newWorkerNode))
|
||||
/*
|
||||
* Skip foreign key and partition creation when we shouldn't need to sync
|
||||
* tablem metadata or the Citus table is owned by an extension.
|
||||
*/
|
||||
if (ShouldSyncTableMetadata(cacheEntry->relationId) &&
|
||||
!IsTableOwnedByExtension(cacheEntry->relationId))
|
||||
{
|
||||
PropagateNodeWideObjects(newWorkerNode);
|
||||
ReplicateAllDependenciesToNode(newWorkerNode->workerName,
|
||||
newWorkerNode->workerPort);
|
||||
}
|
||||
else if (!NodeIsCoordinator(newWorkerNode))
|
||||
{
|
||||
ereport(WARNING, (errmsg("citus.enable_object_propagation is off, not "
|
||||
"creating distributed objects on worker"),
|
||||
errdetail("distributed objects are only kept in sync when "
|
||||
"citus.enable_object_propagation is set to on. "
|
||||
"Newly activated nodes will not get these "
|
||||
"objects created")));
|
||||
}
|
||||
|
||||
if (ReplicateReferenceTablesOnActivate)
|
||||
{
|
||||
ReplicateAllReferenceTablesToNode(newWorkerNode->workerName,
|
||||
newWorkerNode->workerPort);
|
||||
propagatedTableList = lappend(propagatedTableList, cacheEntry);
|
||||
}
|
||||
}
|
||||
|
||||
foreach_ptr(cacheEntry, propagatedTableList)
|
||||
{
|
||||
Oid relationId = cacheEntry->relationId;
|
||||
|
||||
List *commandListForRelation =
|
||||
InterTableRelationshipOfRelationCommandList(relationId);
|
||||
|
||||
multipleTableIntegrationCommandList = list_concat(
|
||||
multipleTableIntegrationCommandList,
|
||||
commandListForRelation);
|
||||
}
|
||||
|
||||
multipleTableIntegrationCommandList = lcons(DISABLE_DDL_PROPAGATION,
|
||||
multipleTableIntegrationCommandList);
|
||||
multipleTableIntegrationCommandList = lappend(multipleTableIntegrationCommandList,
|
||||
ENABLE_DDL_PROPAGATION);
|
||||
|
||||
return multipleTableIntegrationCommandList;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* PropagateNodeWideObjects is called during node activation to propagate any object that
|
||||
* should be propagated for every node. These are generally not linked to any distributed
|
||||
* object but change system wide behaviour.
|
||||
* PgDistTableMetadataSyncCommandList returns the command list to sync the pg_dist_*
|
||||
* (except pg_dist_node) metadata. We call them as table metadata.
|
||||
*/
|
||||
static void
|
||||
PropagateNodeWideObjects(WorkerNode *newWorkerNode)
|
||||
List *
|
||||
PgDistTableMetadataSyncCommandList(void)
|
||||
{
|
||||
List *distributedTableList = CitusTableList();
|
||||
List *propagatedTableList = NIL;
|
||||
List *metadataSnapshotCommandList = NIL;
|
||||
|
||||
/* create the list of tables whose metadata will be created */
|
||||
CitusTableCacheEntry *cacheEntry = NULL;
|
||||
foreach_ptr(cacheEntry, distributedTableList)
|
||||
{
|
||||
if (ShouldSyncTableMetadata(cacheEntry->relationId))
|
||||
{
|
||||
propagatedTableList = lappend(propagatedTableList, cacheEntry);
|
||||
}
|
||||
}
|
||||
|
||||
/* remove all dist table and object related metadata first */
|
||||
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
|
||||
DELETE_ALL_PARTITIONS);
|
||||
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList, DELETE_ALL_SHARDS);
|
||||
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
|
||||
DELETE_ALL_PLACEMENTS);
|
||||
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
|
||||
DELETE_ALL_DISTRIBUTED_OBJECTS);
|
||||
|
||||
/* create pg_dist_partition, pg_dist_shard and pg_dist_placement entries */
|
||||
foreach_ptr(cacheEntry, propagatedTableList)
|
||||
{
|
||||
List *tableMetadataCreateCommandList =
|
||||
CitusTableMetadataCreateCommandList(cacheEntry->relationId);
|
||||
|
||||
metadataSnapshotCommandList = list_concat(metadataSnapshotCommandList,
|
||||
tableMetadataCreateCommandList);
|
||||
}
|
||||
|
||||
/* As the last step, propagate the pg_dist_object entities */
|
||||
Assert(ShouldPropagate());
|
||||
List *distributedObjectSyncCommandList = DistributedObjectMetadataSyncCommandList();
|
||||
metadataSnapshotCommandList = list_concat(metadataSnapshotCommandList,
|
||||
distributedObjectSyncCommandList);
|
||||
|
||||
metadataSnapshotCommandList = lcons(DISABLE_DDL_PROPAGATION,
|
||||
metadataSnapshotCommandList);
|
||||
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
|
||||
ENABLE_DDL_PROPAGATION);
|
||||
|
||||
return metadataSnapshotCommandList;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* PropagateNodeWideObjectsCommandList is called during node activation to
|
||||
* propagate any object that should be propagated for every node. These are
|
||||
* generally not linked to any distributed object but change system wide behaviour.
|
||||
*/
|
||||
static List *
|
||||
PropagateNodeWideObjectsCommandList()
|
||||
{
|
||||
/* collect all commands */
|
||||
List *ddlCommands = NIL;
|
||||
|
@ -640,13 +708,142 @@ PropagateNodeWideObjects(WorkerNode *newWorkerNode)
|
|||
/* if there are command wrap them in enable_ddl_propagation off */
|
||||
ddlCommands = lcons(DISABLE_DDL_PROPAGATION, ddlCommands);
|
||||
ddlCommands = lappend(ddlCommands, ENABLE_DDL_PROPAGATION);
|
||||
}
|
||||
|
||||
return ddlCommands;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* SyncDistributedObjectsCommandList returns commands to sync object dependencies
|
||||
* to the given worker node. To be idempotent, it first drops the ones required to be
|
||||
* dropped.
|
||||
*
|
||||
* Object dependencies include:
|
||||
*
|
||||
* - All dependencies (e.g., types, schemas, sequences)
|
||||
* - All shell distributed tables
|
||||
* - Inter relation between those shell tables
|
||||
* - Node wide objects
|
||||
*
|
||||
* We also update the local group id here, as handling sequence dependencies
|
||||
* requires it.
|
||||
*/
|
||||
List *
|
||||
SyncDistributedObjectsCommandList(WorkerNode *workerNode)
|
||||
{
|
||||
List *commandList = NIL;
|
||||
|
||||
/*
|
||||
* Propagate node wide objects. It includes only roles for now.
|
||||
*/
|
||||
commandList = list_concat(commandList, PropagateNodeWideObjectsCommandList());
|
||||
|
||||
/*
|
||||
* Detach partitions, break dependencies between sequences and table then
|
||||
* remove shell tables first.
|
||||
*/
|
||||
commandList = list_concat(commandList, DetachPartitionCommandList());
|
||||
commandList = lappend(commandList, BREAK_CITUS_TABLE_SEQUENCE_DEPENDENCY_COMMAND);
|
||||
commandList = lappend(commandList, REMOVE_ALL_SHELL_TABLES_COMMAND);
|
||||
|
||||
/*
|
||||
* Replicate all objects of the pg_dist_object to the remote node.
|
||||
*/
|
||||
commandList = list_concat(commandList, ReplicateAllObjectsToNodeCommandList(
|
||||
workerNode->workerName, workerNode->workerPort));
|
||||
|
||||
/*
|
||||
* After creating each table, handle the inter table relationship between
|
||||
* those tables.
|
||||
*/
|
||||
commandList = list_concat(commandList, InterTableRelationshipCommandList());
|
||||
|
||||
return commandList;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* SyncDistributedObjectsToNode sync the distributed objects to the node. It includes
|
||||
* - All dependencies (e.g., types, schemas, sequences)
|
||||
* - All shell distributed table
|
||||
* - Inter relation between those shell tables
|
||||
*
|
||||
* Note that we do not create the distributed dependencies on the coordinator
|
||||
* since all the dependencies should be present in the coordinator already.
|
||||
*/
|
||||
static void
|
||||
SyncDistributedObjectsToNode(WorkerNode *workerNode)
|
||||
{
|
||||
if (NodeIsCoordinator(workerNode))
|
||||
{
|
||||
/* coordinator has all the objects */
|
||||
return;
|
||||
}
|
||||
|
||||
if (!NodeIsPrimary(workerNode))
|
||||
{
|
||||
/* secondary nodes gets the objects from their primaries via replication */
|
||||
return;
|
||||
}
|
||||
|
||||
EnsureNoModificationsHaveBeenDone();
|
||||
EnsureSequentialModeMetadataOperations();
|
||||
|
||||
Assert(ShouldPropagate());
|
||||
|
||||
List *commandList = SyncDistributedObjectsCommandList(workerNode);
|
||||
|
||||
/* send commands to new workers, the current user should be a superuser */
|
||||
Assert(superuser());
|
||||
SendMetadataCommandListToWorkerInCoordinatedTransaction(
|
||||
workerNode->workerName,
|
||||
workerNode->workerPort,
|
||||
CurrentUserName(),
|
||||
commandList);
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* UpdateLocalGroupIdOnNode updates local group id on node.
|
||||
*/
|
||||
static void
|
||||
UpdateLocalGroupIdOnNode(WorkerNode *workerNode)
|
||||
{
|
||||
if (NodeIsPrimary(workerNode) && !NodeIsCoordinator(workerNode))
|
||||
{
|
||||
List *commandList = list_make1(LocalGroupIdUpdateCommand(workerNode->groupId));
|
||||
|
||||
/* send commands to new workers, the current user should be a superuser */
|
||||
Assert(superuser());
|
||||
SendMetadataCommandListToWorkerInCoordinatedTransaction(newWorkerNode->workerName,
|
||||
newWorkerNode->workerPort,
|
||||
CurrentUserName(),
|
||||
ddlCommands);
|
||||
SendMetadataCommandListToWorkerInCoordinatedTransaction(
|
||||
workerNode->workerName,
|
||||
workerNode->workerPort,
|
||||
CurrentUserName(),
|
||||
commandList);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* SyncPgDistTableMetadataToNode syncs the pg_dist_partition, pg_dist_shard
|
||||
* pg_dist_placement and pg_dist_object metadata entries.
|
||||
*
|
||||
*/
|
||||
static void
|
||||
SyncPgDistTableMetadataToNode(WorkerNode *workerNode)
|
||||
{
|
||||
if (NodeIsPrimary(workerNode) && !NodeIsCoordinator(workerNode))
|
||||
{
|
||||
List *syncPgDistMetadataCommandList = PgDistTableMetadataSyncCommandList();
|
||||
|
||||
/* send commands to new workers, the current user should be a superuser */
|
||||
Assert(superuser());
|
||||
SendMetadataCommandListToWorkerInCoordinatedTransaction(
|
||||
workerNode->workerName,
|
||||
workerNode->workerPort,
|
||||
CurrentUserName(),
|
||||
syncPgDistMetadataCommandList);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -847,7 +1044,7 @@ PrimaryNodeForGroup(int32 groupId, bool *groupContainsNodes)
|
|||
* includes only replicating the reference tables and setting isactive column of the
|
||||
* given node.
|
||||
*/
|
||||
static int
|
||||
int
|
||||
ActivateNode(char *nodeName, int nodePort)
|
||||
{
|
||||
bool isActive = true;
|
||||
|
@ -897,10 +1094,11 @@ ActivateNode(char *nodeName, int nodePort)
|
|||
workerNode =
|
||||
SetWorkerColumnLocalOnly(workerNode, Anum_pg_dist_node_isactive,
|
||||
BoolGetDatum(isActive));
|
||||
bool syncMetadata =
|
||||
EnableMetadataSyncByDefault && NodeIsPrimary(workerNode);
|
||||
|
||||
if (syncMetadata)
|
||||
/* TODO: Once all tests will be enabled for MX, we can remove sync by default check */
|
||||
bool syncMetadata = EnableMetadataSyncByDefault && NodeIsPrimary(workerNode);
|
||||
|
||||
if (syncMetadata && EnableDependencyCreation)
|
||||
{
|
||||
/*
|
||||
* We are going to sync the metadata anyway in this transaction, so do
|
||||
|
@ -908,13 +1106,43 @@ ActivateNode(char *nodeName, int nodePort)
|
|||
*/
|
||||
SetWorkerColumn(workerNode, Anum_pg_dist_node_metadatasynced,
|
||||
BoolGetDatum(true));
|
||||
}
|
||||
|
||||
SetUpDistributedTableDependencies(workerNode);
|
||||
/*
|
||||
* Update local group id first, as object dependency logic requires to have
|
||||
* updated local group id.
|
||||
*/
|
||||
UpdateLocalGroupIdOnNode(workerNode);
|
||||
|
||||
if (syncMetadata)
|
||||
{
|
||||
StartMetadataSyncToNode(nodeName, nodePort);
|
||||
/*
|
||||
* Sync distributed objects first. We must sync distributed objects before
|
||||
* replicating reference tables to the remote node, as reference tables may
|
||||
* need such objects.
|
||||
*/
|
||||
SyncDistributedObjectsToNode(workerNode);
|
||||
|
||||
/*
|
||||
* We need to replicate reference tables before syncing node metadata, otherwise
|
||||
* reference table replication logic would try to get lock on the new node before
|
||||
* having the shard placement on it
|
||||
*/
|
||||
if (ReplicateReferenceTablesOnActivate)
|
||||
{
|
||||
ReplicateAllReferenceTablesToNode(workerNode);
|
||||
}
|
||||
|
||||
/*
|
||||
* Sync node metadata. We must sync node metadata before syncing table
|
||||
* related pg_dist_xxx metadata. Since table related metadata requires
|
||||
* to have right pg_dist_node entries.
|
||||
*/
|
||||
SyncNodeMetadataToNode(nodeName, nodePort);
|
||||
|
||||
/*
|
||||
* As the last step, sync the table related metadata to the remote node.
|
||||
* We must handle it as the last step because of limitations shared with
|
||||
* above comments.
|
||||
*/
|
||||
SyncPgDistTableMetadataToNode(workerNode);
|
||||
}
|
||||
|
||||
/* finally, let all other active metadata nodes to learn about this change */
|
||||
|
|
|
@ -471,6 +471,7 @@ ErrorIfCurrentUserCanNotDistributeObject(ObjectType type, ObjectAddress *addr,
|
|||
break;
|
||||
}
|
||||
|
||||
case OBJECT_FOREIGN_TABLE:
|
||||
case OBJECT_TABLE:
|
||||
{
|
||||
/* table distribution already does the ownership check, so we can stick to that over acl_check */
|
||||
|
|
|
@ -141,8 +141,10 @@ master_get_table_ddl_events(PG_FUNCTION_ARGS)
|
|||
functionContext->multi_call_memory_ctx);
|
||||
|
||||
/* allocate DDL statements, and then save position in DDL statements */
|
||||
bool creatingShellTableOnRemoteNode = false;
|
||||
List *tableDDLEventList = GetFullTableCreationCommands(relationId,
|
||||
includeSequenceDefaults);
|
||||
includeSequenceDefaults,
|
||||
creatingShellTableOnRemoteNode);
|
||||
tableDDLEventCell = list_head(tableDDLEventList);
|
||||
ListCellAndListWrapper *wrapper = palloc0(sizeof(ListCellAndListWrapper));
|
||||
wrapper->list = tableDDLEventList;
|
||||
|
@ -458,8 +460,9 @@ ResolveRelationId(text *relationName, bool missingOk)
|
|||
* constraint and trigger definitions.
|
||||
*/
|
||||
List *
|
||||
GetFullTableCreationCommands(Oid relationId, IncludeSequenceDefaults
|
||||
includeSequenceDefaults)
|
||||
GetFullTableCreationCommands(Oid relationId,
|
||||
IncludeSequenceDefaults includeSequenceDefaults,
|
||||
bool creatingShellTableOnRemoteNode)
|
||||
{
|
||||
List *tableDDLEventList = NIL;
|
||||
|
||||
|
@ -471,6 +474,25 @@ GetFullTableCreationCommands(Oid relationId, IncludeSequenceDefaults
|
|||
List *postLoadCreationCommandList =
|
||||
GetPostLoadTableCreationCommands(relationId, true, true);
|
||||
|
||||
if (creatingShellTableOnRemoteNode)
|
||||
{
|
||||
/*
|
||||
* While creating shell tables, we need to associate dependencies between
|
||||
* sequences and the relation. We also need to add truncate trigger for it
|
||||
* if it is not the foreign table.
|
||||
*/
|
||||
List *sequenceDependencyCommandList = SequenceDependencyCommandList(relationId);
|
||||
tableDDLEventList = list_concat(tableDDLEventList, sequenceDependencyCommandList);
|
||||
|
||||
if (!IsForeignTable(relationId))
|
||||
{
|
||||
TableDDLCommand *truncateTriggerCommand = TruncateTriggerCreateCommand(
|
||||
relationId);
|
||||
tableDDLEventList = lappend(tableDDLEventList,
|
||||
truncateTriggerCommand);
|
||||
}
|
||||
}
|
||||
|
||||
tableDDLEventList = list_concat(tableDDLEventList, postLoadCreationCommandList);
|
||||
|
||||
return tableDDLEventList;
|
||||
|
|
|
@ -325,8 +325,10 @@ CreateAppendDistributedShardPlacements(Oid relationId, int64 shardId,
|
|||
List *foreignConstraintCommandList =
|
||||
GetReferencingForeignConstaintCommands(relationId);
|
||||
IncludeSequenceDefaults includeSequenceDefaults = NO_SEQUENCE_DEFAULTS;
|
||||
bool creatingShellTableOnRemoteNode = false;
|
||||
List *ddlCommandList = GetFullTableCreationCommands(relationId,
|
||||
includeSequenceDefaults);
|
||||
includeSequenceDefaults,
|
||||
creatingShellTableOnRemoteNode);
|
||||
uint32 connectionFlag = FOR_DDL;
|
||||
char *relationOwner = TableOwner(relationId);
|
||||
|
||||
|
@ -438,8 +440,10 @@ CreateShardsOnWorkers(Oid distributedRelationId, List *shardPlacements,
|
|||
bool useExclusiveConnection, bool colocatedShard)
|
||||
{
|
||||
IncludeSequenceDefaults includeSequenceDefaults = NO_SEQUENCE_DEFAULTS;
|
||||
bool creatingShellTableOnRemoteNode = false;
|
||||
List *ddlCommandList = GetFullTableCreationCommands(distributedRelationId,
|
||||
includeSequenceDefaults);
|
||||
includeSequenceDefaults,
|
||||
creatingShellTableOnRemoteNode);
|
||||
List *foreignConstraintCommandList =
|
||||
GetReferencingForeignConstaintCommands(distributedRelationId);
|
||||
|
||||
|
|
|
@ -12,6 +12,7 @@
|
|||
#include "udfs/citus_internal_add_object_metadata/11.0-1.sql"
|
||||
#include "udfs/citus_run_local_command/11.0-1.sql"
|
||||
#include "udfs/worker_drop_sequence_dependency/11.0-1.sql"
|
||||
#include "udfs/worker_drop_shell_table/11.0-1.sql"
|
||||
|
||||
|
||||
DROP FUNCTION IF EXISTS pg_catalog.master_apply_delete_command(text);
|
||||
|
|
|
@ -47,6 +47,7 @@ DROP FUNCTION pg_catalog.citus_check_cluster_node_health ();
|
|||
DROP FUNCTION pg_catalog.citus_internal_add_object_metadata(text, text[], text[], integer, integer, boolean);
|
||||
DROP FUNCTION pg_catalog.citus_run_local_command(text);
|
||||
DROP FUNCTION pg_catalog.worker_drop_sequence_dependency(text);
|
||||
DROP FUNCTION pg_catalog.worker_drop_shell_table(table_name text);
|
||||
|
||||
CREATE OR REPLACE VIEW pg_catalog.citus_shards_on_worker AS
|
||||
SELECT n.nspname as "Schema",
|
||||
|
|
|
@ -0,0 +1,7 @@
|
|||
CREATE FUNCTION pg_catalog.worker_drop_shell_table(table_name text)
|
||||
RETURNS VOID
|
||||
LANGUAGE C STRICT
|
||||
AS 'MODULE_PATHNAME', $$worker_drop_shell_table$$;
|
||||
|
||||
COMMENT ON FUNCTION worker_drop_shell_table(table_name text)
|
||||
IS 'drop the distributed table only without the metadata';
|
|
@ -0,0 +1,7 @@
|
|||
CREATE FUNCTION pg_catalog.worker_drop_shell_table(table_name text)
|
||||
RETURNS VOID
|
||||
LANGUAGE C STRICT
|
||||
AS 'MODULE_PATHNAME', $$worker_drop_shell_table$$;
|
||||
|
||||
COMMENT ON FUNCTION worker_drop_shell_table(table_name text)
|
||||
IS 'drop the distributed table only without the metadata';
|
|
@ -20,6 +20,7 @@
|
|||
#include "distributed/maintenanced.h"
|
||||
#include "distributed/metadata_sync.h"
|
||||
#include "distributed/remote_commands.h"
|
||||
#include "distributed/worker_manager.h"
|
||||
#include "postmaster/postmaster.h"
|
||||
#include "miscadmin.h"
|
||||
#include "storage/latch.h"
|
||||
|
@ -28,45 +29,66 @@
|
|||
|
||||
|
||||
/* declarations for dynamic loading */
|
||||
PG_FUNCTION_INFO_V1(master_metadata_snapshot);
|
||||
PG_FUNCTION_INFO_V1(activate_node_snapshot);
|
||||
PG_FUNCTION_INFO_V1(wait_until_metadata_sync);
|
||||
PG_FUNCTION_INFO_V1(trigger_metadata_sync);
|
||||
PG_FUNCTION_INFO_V1(raise_error_in_metadata_sync);
|
||||
|
||||
|
||||
/*
|
||||
* master_metadata_snapshot prints all the queries that are required
|
||||
* to generate a metadata snapshot.
|
||||
* activate_node_snapshot prints all the queries that are required
|
||||
* to activate a node.
|
||||
*/
|
||||
Datum
|
||||
master_metadata_snapshot(PG_FUNCTION_ARGS)
|
||||
activate_node_snapshot(PG_FUNCTION_ARGS)
|
||||
{
|
||||
List *dropSnapshotCommands = MetadataDropCommands();
|
||||
List *createSnapshotCommands = MetadataCreateCommands();
|
||||
List *snapshotCommandList = NIL;
|
||||
int snapshotCommandIndex = 0;
|
||||
/*
|
||||
* Activate node commands are created using the given worker node,
|
||||
* so we are using first primary worker node just for test purposes.
|
||||
*/
|
||||
WorkerNode *dummyWorkerNode = GetFirstPrimaryWorkerNode();
|
||||
|
||||
List *updateLocalGroupCommand =
|
||||
list_make1(LocalGroupIdUpdateCommand(dummyWorkerNode->groupId));
|
||||
List *syncDistObjCommands = SyncDistributedObjectsCommandList(dummyWorkerNode);
|
||||
List *dropSnapshotCommands = NodeMetadataDropCommands();
|
||||
List *createSnapshotCommands = NodeMetadataCreateCommands();
|
||||
List *pgDistTableMetadataSyncCommands = PgDistTableMetadataSyncCommandList();
|
||||
|
||||
List *activateNodeCommandList = NIL;
|
||||
int activateNodeCommandIndex = 0;
|
||||
Oid ddlCommandTypeId = TEXTOID;
|
||||
|
||||
snapshotCommandList = list_concat(snapshotCommandList, dropSnapshotCommands);
|
||||
snapshotCommandList = list_concat(snapshotCommandList, createSnapshotCommands);
|
||||
activateNodeCommandList = list_concat(activateNodeCommandList,
|
||||
updateLocalGroupCommand);
|
||||
activateNodeCommandList = list_concat(activateNodeCommandList, syncDistObjCommands);
|
||||
activateNodeCommandList = list_concat(activateNodeCommandList, dropSnapshotCommands);
|
||||
activateNodeCommandList = list_concat(activateNodeCommandList,
|
||||
createSnapshotCommands);
|
||||
activateNodeCommandList = list_concat(activateNodeCommandList,
|
||||
pgDistTableMetadataSyncCommands);
|
||||
|
||||
int snapshotCommandCount = list_length(snapshotCommandList);
|
||||
Datum *snapshotCommandDatumArray = palloc0(snapshotCommandCount * sizeof(Datum));
|
||||
int activateNodeCommandCount = list_length(activateNodeCommandList);
|
||||
Datum *activateNodeCommandDatumArray = palloc0(activateNodeCommandCount *
|
||||
sizeof(Datum));
|
||||
|
||||
const char *metadataSnapshotCommand = NULL;
|
||||
foreach_ptr(metadataSnapshotCommand, snapshotCommandList)
|
||||
const char *activateNodeSnapshotCommand = NULL;
|
||||
foreach_ptr(activateNodeSnapshotCommand, activateNodeCommandList)
|
||||
{
|
||||
Datum metadataSnapshotCommandDatum = CStringGetTextDatum(metadataSnapshotCommand);
|
||||
Datum activateNodeSnapshotCommandDatum = CStringGetTextDatum(
|
||||
activateNodeSnapshotCommand);
|
||||
|
||||
snapshotCommandDatumArray[snapshotCommandIndex] = metadataSnapshotCommandDatum;
|
||||
snapshotCommandIndex++;
|
||||
activateNodeCommandDatumArray[activateNodeCommandIndex] =
|
||||
activateNodeSnapshotCommandDatum;
|
||||
activateNodeCommandIndex++;
|
||||
}
|
||||
|
||||
ArrayType *snapshotCommandArrayType = DatumArrayToArrayType(snapshotCommandDatumArray,
|
||||
snapshotCommandCount,
|
||||
ddlCommandTypeId);
|
||||
ArrayType *activateNodeCommandArrayType = DatumArrayToArrayType(
|
||||
activateNodeCommandDatumArray,
|
||||
activateNodeCommandCount,
|
||||
ddlCommandTypeId);
|
||||
|
||||
PG_RETURN_ARRAYTYPE_P(snapshotCommandArrayType);
|
||||
PG_RETURN_ARRAYTYPE_P(activateNodeCommandArrayType);
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -524,7 +524,7 @@ CitusMaintenanceDaemonMain(Datum main_arg)
|
|||
if (syncMetadata)
|
||||
{
|
||||
metadataSyncBgwHandle =
|
||||
SpawnSyncMetadataToNodes(MyDatabaseId, myDbData->userOid);
|
||||
SpawnSyncNodeMetadataToNodes(MyDatabaseId, myDbData->userOid);
|
||||
}
|
||||
|
||||
nextMetadataSyncTime =
|
||||
|
|
|
@ -537,7 +537,7 @@ ReferenceTableReplicationFactor(void)
|
|||
* reference table to prevent unnecessary data transfer.
|
||||
*/
|
||||
void
|
||||
ReplicateAllReferenceTablesToNode(char *nodeName, int nodePort)
|
||||
ReplicateAllReferenceTablesToNode(WorkerNode *workerNode)
|
||||
{
|
||||
List *referenceTableList = CitusTableTypeIdList(REFERENCE_TABLE);
|
||||
|
||||
|
@ -560,7 +560,9 @@ ReplicateAllReferenceTablesToNode(char *nodeName, int nodePort)
|
|||
List *shardPlacementList =
|
||||
ShardPlacementListIncludingOrphanedPlacements(shardInterval->shardId);
|
||||
ShardPlacement *targetPlacement =
|
||||
SearchShardPlacementInList(shardPlacementList, nodeName, nodePort);
|
||||
SearchShardPlacementInList(shardPlacementList,
|
||||
workerNode->workerName,
|
||||
workerNode->workerPort);
|
||||
if (targetPlacement != NULL &&
|
||||
targetPlacement->shardState == SHARD_STATE_ACTIVE)
|
||||
{
|
||||
|
@ -585,7 +587,9 @@ ReplicateAllReferenceTablesToNode(char *nodeName, int nodePort)
|
|||
|
||||
LockShardDistributionMetadata(shardId, ExclusiveLock);
|
||||
|
||||
ReplicateReferenceTableShardToNode(shardInterval, nodeName, nodePort);
|
||||
ReplicateReferenceTableShardToNode(shardInterval,
|
||||
workerNode->workerName,
|
||||
workerNode->workerPort);
|
||||
}
|
||||
|
||||
/* create foreign constraints between reference tables */
|
||||
|
@ -595,9 +599,11 @@ ReplicateAllReferenceTablesToNode(char *nodeName, int nodePort)
|
|||
|
||||
/* send commands to new workers, the current user should be a superuser */
|
||||
Assert(superuser());
|
||||
SendMetadataCommandListToWorkerInCoordinatedTransaction(nodeName, nodePort,
|
||||
CurrentUserName(),
|
||||
commandList);
|
||||
SendMetadataCommandListToWorkerInCoordinatedTransaction(
|
||||
workerNode->workerName,
|
||||
workerNode->workerPort,
|
||||
CurrentUserName(),
|
||||
commandList);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,8 +33,6 @@
|
|||
#include "distributed/worker_protocol.h"
|
||||
|
||||
static const char * CreateStmtByObjectAddress(const ObjectAddress *address);
|
||||
static RenameStmt * CreateRenameStatement(const ObjectAddress *address, char *newName);
|
||||
static char * GenerateBackupNameForCollision(const ObjectAddress *address);
|
||||
|
||||
PG_FUNCTION_INFO_V1(worker_create_or_replace_object);
|
||||
|
||||
|
@ -166,7 +164,7 @@ CreateStmtByObjectAddress(const ObjectAddress *address)
|
|||
* address. This name should be used when renaming an existing object before creating the
|
||||
* new object locally on the worker.
|
||||
*/
|
||||
static char *
|
||||
char *
|
||||
GenerateBackupNameForCollision(const ObjectAddress *address)
|
||||
{
|
||||
switch (getObjectClass(address))
|
||||
|
@ -186,13 +184,23 @@ GenerateBackupNameForCollision(const ObjectAddress *address)
|
|||
return GenerateBackupNameForTypeCollision(address);
|
||||
}
|
||||
|
||||
case OCLASS_CLASS:
|
||||
{
|
||||
char relKind = get_rel_relkind(address->objectId);
|
||||
if (relKind == RELKIND_SEQUENCE)
|
||||
{
|
||||
return GenerateBackupNameForSequenceCollision(address);
|
||||
}
|
||||
}
|
||||
|
||||
default:
|
||||
{
|
||||
ereport(ERROR, (errmsg("unsupported object to construct a rename statement"),
|
||||
errdetail(
|
||||
"unable to generate a backup name for the old type")));
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
ereport(ERROR, (errmsg("unsupported object to construct a rename statement"),
|
||||
errdetail("unable to generate a backup name for the old type")));
|
||||
}
|
||||
|
||||
|
||||
|
@ -243,6 +251,7 @@ CreateRenameTypeStmt(const ObjectAddress *address, char *newName)
|
|||
address->objectId));
|
||||
stmt->newname = newName;
|
||||
|
||||
|
||||
return stmt;
|
||||
}
|
||||
|
||||
|
@ -265,11 +274,43 @@ CreateRenameProcStmt(const ObjectAddress *address, char *newName)
|
|||
}
|
||||
|
||||
|
||||
/*
|
||||
* CreateRenameSequenceStmt creates a rename statement for a sequence based on its
|
||||
* ObjectAddress. The rename statement will rename the existing object on its address
|
||||
* to the value provided in newName.
|
||||
*/
|
||||
static RenameStmt *
|
||||
CreateRenameSequenceStmt(const ObjectAddress *address, char *newName)
|
||||
{
|
||||
RenameStmt *stmt = makeNode(RenameStmt);
|
||||
Oid seqOid = address->objectId;
|
||||
|
||||
HeapTuple seqClassTuple = SearchSysCache1(RELOID, seqOid);
|
||||
if (!HeapTupleIsValid(seqClassTuple))
|
||||
{
|
||||
ereport(ERROR, (errmsg("citus cache lookup error")));
|
||||
}
|
||||
Form_pg_class seqClassForm = (Form_pg_class) GETSTRUCT(seqClassTuple);
|
||||
|
||||
char *schemaName = get_namespace_name(seqClassForm->relnamespace);
|
||||
char *seqName = NameStr(seqClassForm->relname);
|
||||
List *name = list_make2(makeString(schemaName), makeString(seqName));
|
||||
ReleaseSysCache(seqClassTuple);
|
||||
|
||||
stmt->renameType = OBJECT_SEQUENCE;
|
||||
stmt->object = (Node *) name;
|
||||
stmt->relation = makeRangeVar(schemaName, seqName, -1);
|
||||
stmt->newname = newName;
|
||||
|
||||
return stmt;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* CreateRenameStatement creates a rename statement for an existing object to rename the
|
||||
* object to newName.
|
||||
*/
|
||||
static RenameStmt *
|
||||
RenameStmt *
|
||||
CreateRenameStatement(const ObjectAddress *address, char *newName)
|
||||
{
|
||||
switch (getObjectClass(address))
|
||||
|
@ -289,10 +330,21 @@ CreateRenameStatement(const ObjectAddress *address, char *newName)
|
|||
return CreateRenameTypeStmt(address, newName);
|
||||
}
|
||||
|
||||
case OCLASS_CLASS:
|
||||
{
|
||||
char relKind = get_rel_relkind(address->objectId);
|
||||
if (relKind == RELKIND_SEQUENCE)
|
||||
{
|
||||
return CreateRenameSequenceStmt(address, newName);
|
||||
}
|
||||
}
|
||||
|
||||
default:
|
||||
{
|
||||
ereport(ERROR, (errmsg("unsupported object to construct a rename statement"),
|
||||
errdetail("unable to generate a parsetree for the rename")));
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
ereport(ERROR, (errmsg("unsupported object to construct a rename statement"),
|
||||
errdetail("unable to generate a parsetree for the rename")));
|
||||
}
|
||||
|
|
|
@ -28,13 +28,16 @@
|
|||
#include "commands/extension.h"
|
||||
#include "commands/sequence.h"
|
||||
#include "distributed/citus_ruleutils.h"
|
||||
#include "distributed/commands.h"
|
||||
#include "distributed/commands/multi_copy.h"
|
||||
#include "distributed/commands/utility_hook.h"
|
||||
#include "distributed/connection_management.h"
|
||||
#include "distributed/coordinator_protocol.h"
|
||||
#include "distributed/deparser.h"
|
||||
#include "distributed/intermediate_results.h"
|
||||
#include "distributed/listutils.h"
|
||||
#include "distributed/metadata_cache.h"
|
||||
#include "distributed/metadata_sync.h"
|
||||
#include "distributed/multi_client_executor.h"
|
||||
#include "distributed/multi_logical_optimizer.h"
|
||||
#include "distributed/multi_partitioning_utils.h"
|
||||
|
@ -43,8 +46,10 @@
|
|||
#include "distributed/remote_commands.h"
|
||||
#include "distributed/resource_lock.h"
|
||||
|
||||
#include "distributed/worker_create_or_replace.h"
|
||||
#include "distributed/worker_protocol.h"
|
||||
#include "distributed/version_compat.h"
|
||||
#include "executor/spi.h"
|
||||
#include "nodes/makefuncs.h"
|
||||
#include "parser/parse_relation.h"
|
||||
#include "storage/lmgr.h"
|
||||
|
@ -461,19 +466,25 @@ worker_apply_sequence_command(PG_FUNCTION_ARGS)
|
|||
" SEQUENCE command string")));
|
||||
}
|
||||
|
||||
/*
|
||||
* If sequence with the same name exist for different type, it must have been
|
||||
* stayed on that node after a rollbacked create_distributed_table operation.
|
||||
* We must change it's name first to create the sequence with the correct type.
|
||||
*/
|
||||
CreateSeqStmt *createSequenceStatement = (CreateSeqStmt *) commandNode;
|
||||
RenameExistingSequenceWithDifferentTypeIfExists(createSequenceStatement->sequence,
|
||||
sequenceTypeId);
|
||||
|
||||
/* run the CREATE SEQUENCE command */
|
||||
ProcessUtilityParseTree(commandNode, commandString, PROCESS_UTILITY_QUERY, NULL,
|
||||
None_Receiver, NULL);
|
||||
CommandCounterIncrement();
|
||||
|
||||
CreateSeqStmt *createSequenceStatement = (CreateSeqStmt *) commandNode;
|
||||
|
||||
char *sequenceName = createSequenceStatement->sequence->relname;
|
||||
char *sequenceSchema = createSequenceStatement->sequence->schemaname;
|
||||
createSequenceStatement = (CreateSeqStmt *) commandNode;
|
||||
|
||||
Oid sequenceRelationId = RangeVarGetRelid(createSequenceStatement->sequence,
|
||||
AccessShareLock, false);
|
||||
char *sequenceName = createSequenceStatement->sequence->relname;
|
||||
char *sequenceSchema = createSequenceStatement->sequence->schemaname;
|
||||
|
||||
Assert(sequenceRelationId != InvalidOid);
|
||||
|
||||
AlterSequenceMinMax(sequenceRelationId, sequenceSchema, sequenceName, sequenceTypeId);
|
||||
|
|
|
@ -34,6 +34,7 @@
|
|||
#include "utils/fmgroids.h"
|
||||
|
||||
PG_FUNCTION_INFO_V1(worker_drop_distributed_table);
|
||||
PG_FUNCTION_INFO_V1(worker_drop_shell_table);
|
||||
PG_FUNCTION_INFO_V1(worker_drop_sequence_dependency);
|
||||
|
||||
|
||||
|
@ -104,6 +105,8 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS)
|
|||
UnmarkObjectDistributed(&ownedSequenceAddress);
|
||||
}
|
||||
|
||||
UnmarkObjectDistributed(&distributedTableObject);
|
||||
|
||||
if (!IsObjectAddressOwnedByExtension(&distributedTableObject, NULL))
|
||||
{
|
||||
/*
|
||||
|
@ -142,6 +145,82 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS)
|
|||
}
|
||||
|
||||
|
||||
/*
|
||||
* worker_drop_shell_table drops the shell table of with the given distributed
|
||||
* table without deleting related entries on pg_dist_placement, pg_dist_shard
|
||||
* and pg_dist_placement. We've separated that logic since we handle object
|
||||
* dependencies and table metadata separately while activating nodes.
|
||||
*/
|
||||
Datum
|
||||
worker_drop_shell_table(PG_FUNCTION_ARGS)
|
||||
{
|
||||
CheckCitusVersion(ERROR);
|
||||
|
||||
text *relationName = PG_GETARG_TEXT_P(0);
|
||||
Oid relationId = ResolveRelationId(relationName, true);
|
||||
|
||||
if (!OidIsValid(relationId))
|
||||
{
|
||||
ereport(NOTICE, (errmsg("relation %s does not exist, skipping",
|
||||
text_to_cstring(relationName))));
|
||||
PG_RETURN_VOID();
|
||||
}
|
||||
|
||||
EnsureTableOwner(relationId);
|
||||
|
||||
if (GetLocalGroupId() == COORDINATOR_GROUP_ID)
|
||||
{
|
||||
ereport(ERROR, (errmsg("worker_drop_shell_table is only allowed to run"
|
||||
" on worker nodes")));
|
||||
}
|
||||
|
||||
/* first check the relation type */
|
||||
Relation distributedRelation = relation_open(relationId, AccessShareLock);
|
||||
EnsureRelationKindSupported(relationId);
|
||||
|
||||
/* close the relation since we do not need anymore */
|
||||
relation_close(distributedRelation, AccessShareLock);
|
||||
|
||||
/* prepare distributedTableObject for dropping the table */
|
||||
ObjectAddress distributedTableObject = { InvalidOid, InvalidOid, 0 };
|
||||
distributedTableObject.classId = RelationRelationId;
|
||||
distributedTableObject.objectId = relationId;
|
||||
distributedTableObject.objectSubId = 0;
|
||||
|
||||
if (IsObjectAddressOwnedByExtension(&distributedTableObject, NULL))
|
||||
{
|
||||
PG_RETURN_VOID();
|
||||
}
|
||||
|
||||
/* Drop dependent sequences from pg_dist_object */
|
||||
#if PG_VERSION_NUM >= PG_VERSION_13
|
||||
List *ownedSequences = getOwnedSequences(relationId);
|
||||
#else
|
||||
List *ownedSequences = getOwnedSequences(relationId, InvalidAttrNumber);
|
||||
#endif
|
||||
|
||||
Oid ownedSequenceOid = InvalidOid;
|
||||
foreach_oid(ownedSequenceOid, ownedSequences)
|
||||
{
|
||||
ObjectAddress ownedSequenceAddress = { 0 };
|
||||
ObjectAddressSet(ownedSequenceAddress, RelationRelationId, ownedSequenceOid);
|
||||
UnmarkObjectDistributed(&ownedSequenceAddress);
|
||||
}
|
||||
|
||||
/*
|
||||
* If the table is owned by an extension, we cannot drop it, nor should we
|
||||
* until the user runs DROP EXTENSION. Therefore, we skip dropping the
|
||||
* table and only delete the metadata.
|
||||
*
|
||||
* We drop the table with cascade since other tables may be referring to it.
|
||||
*/
|
||||
performDeletion(&distributedTableObject, DROP_CASCADE,
|
||||
PERFORM_DELETION_INTERNAL);
|
||||
|
||||
PG_RETURN_VOID();
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* worker_drop_sequence_dependency is a UDF that removes the dependency
|
||||
* of all the sequences for the given table.
|
||||
|
|
|
@ -397,6 +397,9 @@ extern ObjectAddress AlterSequenceOwnerStmtObjectAddress(Node *node, bool missin
|
|||
extern ObjectAddress RenameSequenceStmtObjectAddress(Node *node, bool missing_ok);
|
||||
extern void ErrorIfUnsupportedSeqStmt(CreateSeqStmt *createSeqStmt);
|
||||
extern void ErrorIfDistributedAlterSeqOwnedBy(AlterSeqStmt *alterSeqStmt);
|
||||
extern char * GenerateBackupNameForSequenceCollision(const ObjectAddress *address);
|
||||
extern void RenameExistingSequenceWithDifferentTypeIfExists(RangeVar *sequence,
|
||||
Oid desiredSeqTypeId);
|
||||
|
||||
/* statistics.c - forward declarations */
|
||||
extern List * PreprocessCreateStatisticsStmt(Node *node, const char *queryString,
|
||||
|
|
|
@ -224,7 +224,8 @@ extern uint64 GetNextShardId(void);
|
|||
extern uint64 GetNextPlacementId(void);
|
||||
extern Oid ResolveRelationId(text *relationName, bool missingOk);
|
||||
extern List * GetFullTableCreationCommands(Oid relationId,
|
||||
IncludeSequenceDefaults includeSequenceDefaults);
|
||||
IncludeSequenceDefaults includeSequenceDefaults,
|
||||
bool creatingShellTableOnRemoteNode);
|
||||
extern List * GetPostLoadTableCreationCommands(Oid relationId, bool includeIndexes,
|
||||
bool includeReplicaIdentity);
|
||||
extern List * GetPreLoadTableCreationCommands(Oid relationId, IncludeSequenceDefaults
|
||||
|
|
|
@ -13,6 +13,7 @@
|
|||
#define METADATA_SYNC_H
|
||||
|
||||
|
||||
#include "distributed/coordinator_protocol.h"
|
||||
#include "distributed/metadata_cache.h"
|
||||
#include "nodes/pg_list.h"
|
||||
|
||||
|
@ -22,18 +23,23 @@ extern int MetadataSyncRetryInterval;
|
|||
|
||||
typedef enum
|
||||
{
|
||||
METADATA_SYNC_SUCCESS = 0,
|
||||
METADATA_SYNC_FAILED_LOCK = 1,
|
||||
METADATA_SYNC_FAILED_SYNC = 2
|
||||
} MetadataSyncResult;
|
||||
NODE_METADATA_SYNC_SUCCESS = 0,
|
||||
NODE_METADATA_SYNC_FAILED_LOCK = 1,
|
||||
NODE_METADATA_SYNC_FAILED_SYNC = 2
|
||||
} NodeMetadataSyncResult;
|
||||
|
||||
/* Functions declarations for metadata syncing */
|
||||
extern void StartMetadataSyncToNode(const char *nodeNameString, int32 nodePort);
|
||||
extern void SyncNodeMetadataToNode(const char *nodeNameString, int32 nodePort);
|
||||
extern void SyncCitusTableMetadata(Oid relationId);
|
||||
extern void EnsureSequentialModeMetadataOperations(void);
|
||||
extern bool ClusterHasKnownMetadataWorkers(void);
|
||||
extern char * LocalGroupIdUpdateCommand(int32 groupId);
|
||||
extern bool ShouldSyncTableMetadata(Oid relationId);
|
||||
extern bool ShouldSyncTableMetadataViaCatalog(Oid relationId);
|
||||
extern List * MetadataCreateCommands(void);
|
||||
extern List * MetadataDropCommands(void);
|
||||
extern List * NodeMetadataCreateCommands(void);
|
||||
extern List * DistributedObjectMetadataSyncCommandList(void);
|
||||
extern List * CitusTableMetadataCreateCommandList(Oid relationId);
|
||||
extern List * NodeMetadataDropCommands(void);
|
||||
extern char * MarkObjectsDistributedCreateCommand(List *addresses,
|
||||
List *distributionArgumentIndexes,
|
||||
List *colocationIds,
|
||||
|
@ -52,22 +58,30 @@ extern char * CreateSchemaDDLCommand(Oid schemaId);
|
|||
extern List * GrantOnSchemaDDLCommands(Oid schemaId);
|
||||
extern char * PlacementUpsertCommand(uint64 shardId, uint64 placementId, int shardState,
|
||||
uint64 shardLength, int32 groupId);
|
||||
extern void CreateTableMetadataOnWorkers(Oid relationId);
|
||||
extern BackgroundWorkerHandle * SpawnSyncMetadataToNodes(Oid database, Oid owner);
|
||||
extern void SyncMetadataToNodesMain(Datum main_arg);
|
||||
extern TableDDLCommand * TruncateTriggerCreateCommand(Oid relationId);
|
||||
extern void CreateInterTableRelationshipOfRelationOnWorkers(Oid relationId);
|
||||
extern List * InterTableRelationshipOfRelationCommandList(Oid relationId);
|
||||
extern List * DetachPartitionCommandList(void);
|
||||
extern BackgroundWorkerHandle * SpawnSyncNodeMetadataToNodes(Oid database, Oid owner);
|
||||
extern void SyncNodeMetadataToNodesMain(Datum main_arg);
|
||||
extern void SignalMetadataSyncDaemon(Oid database, int sig);
|
||||
extern bool ShouldInitiateMetadataSync(bool *lockFailure);
|
||||
extern List * SequenceDependencyCommandList(Oid relationId);
|
||||
|
||||
extern List * DDLCommandsForSequence(Oid sequenceOid, char *ownerName);
|
||||
extern List * SequenceDDLCommandsForTable(Oid relationId);
|
||||
extern List * GetSequencesFromAttrDef(Oid attrdefOid);
|
||||
extern void GetDependentSequencesWithRelation(Oid relationId, List **attnumList,
|
||||
List **dependentSequenceList, AttrNumber
|
||||
attnum);
|
||||
extern Oid GetAttributeTypeOid(Oid relationId, AttrNumber attnum);
|
||||
|
||||
#define DELETE_ALL_NODES "TRUNCATE pg_dist_node CASCADE"
|
||||
#define DELETE_ALL_DISTRIBUTED_OBJECTS "TRUNCATE citus.pg_dist_object"
|
||||
#define DELETE_ALL_NODES "DELETE FROM pg_dist_node"
|
||||
#define DELETE_ALL_PLACEMENTS "DELETE FROM pg_dist_placement"
|
||||
#define DELETE_ALL_SHARDS "DELETE FROM pg_dist_shard"
|
||||
#define DELETE_ALL_DISTRIBUTED_OBJECTS "DELETE FROM citus.pg_dist_object"
|
||||
#define DELETE_ALL_PARTITIONS "DELETE FROM pg_dist_partition"
|
||||
#define REMOVE_ALL_SHELL_TABLES_COMMAND \
|
||||
"SELECT worker_drop_shell_table(logicalrelid::regclass::text) FROM pg_dist_partition"
|
||||
#define REMOVE_ALL_CITUS_TABLES_COMMAND \
|
||||
"SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition"
|
||||
#define BREAK_CITUS_TABLE_SEQUENCE_DEPENDENCY_COMMAND \
|
||||
|
|
|
@ -249,7 +249,7 @@ extern void EnsureDependenciesExistOnAllNodes(const ObjectAddress *target);
|
|||
extern List * GetDistributableDependenciesForObject(const ObjectAddress *target);
|
||||
extern bool ShouldPropagate(void);
|
||||
extern bool ShouldPropagateObject(const ObjectAddress *address);
|
||||
extern void ReplicateAllDependenciesToNode(const char *nodeName, int nodePort);
|
||||
extern List * ReplicateAllObjectsToNodeCommandList(const char *nodeName, int nodePort);
|
||||
|
||||
/* Remaining metadata utility functions */
|
||||
extern char * TableOwner(Oid relationId);
|
||||
|
@ -286,13 +286,7 @@ extern bool GetNodeDiskSpaceStatsForConnection(MultiConnection *connection,
|
|||
uint64 *availableBytes,
|
||||
uint64 *totalBytes);
|
||||
extern void ExecuteQueryViaSPI(char *query, int SPIOK);
|
||||
extern void EnsureSequenceTypeSupported(Oid seqOid, Oid seqTypId);
|
||||
extern void EnsureSequenceTypeSupported(Oid seqOid, Oid seqTypId, Oid ownerRelationId);
|
||||
extern void AlterSequenceType(Oid seqOid, Oid typeOid);
|
||||
extern void MarkSequenceListDistributedAndPropagateWithDependencies(Oid relationId,
|
||||
List *sequenceList);
|
||||
extern void MarkSequenceDistributedAndPropagateWithDependencies(Oid relationId, Oid
|
||||
sequenceOid);
|
||||
extern void EnsureDistributedSequencesHaveOneType(Oid relationId,
|
||||
List *dependentSequenceList,
|
||||
List *attnumList);
|
||||
extern void EnsureRelationHasCompatibleSequenceTypes(Oid relationId);
|
||||
#endif /* METADATA_UTILITY_H */
|
||||
|
|
|
@ -25,6 +25,6 @@ extern void DeleteAllReplicatedTablePlacementsFromNodeGroup(int32 groupId,
|
|||
bool localOnly);
|
||||
extern int CompareOids(const void *leftElement, const void *rightElement);
|
||||
extern int ReferenceTableReplicationFactor(void);
|
||||
extern void ReplicateAllReferenceTablesToNode(char *nodeName, int nodePort);
|
||||
extern void ReplicateAllReferenceTablesToNode(WorkerNode *workerNode);
|
||||
|
||||
#endif /* REFERENCE_TABLE_UTILS_H_ */
|
||||
|
|
|
@ -14,8 +14,12 @@
|
|||
#ifndef WORKER_CREATE_OR_REPLACE_H
|
||||
#define WORKER_CREATE_OR_REPLACE_H
|
||||
|
||||
#include "catalog/objectaddress.h"
|
||||
|
||||
#define CREATE_OR_REPLACE_COMMAND "SELECT worker_create_or_replace_object(%s);"
|
||||
|
||||
extern char * WrapCreateOrReplace(const char *sql);
|
||||
extern char * GenerateBackupNameForCollision(const ObjectAddress *address);
|
||||
extern RenameStmt * CreateRenameStatement(const ObjectAddress *address, char *newName);
|
||||
|
||||
#endif /* WORKER_CREATE_OR_REPLACE_H */
|
||||
|
|
|
@ -63,6 +63,7 @@ extern char *WorkerListFileName;
|
|||
extern char *CurrentCluster;
|
||||
extern bool ReplicateReferenceTablesOnActivate;
|
||||
|
||||
extern int ActivateNode(char *nodeName, int nodePort);
|
||||
|
||||
/* Function declarations for finding worker nodes to place shards on */
|
||||
extern WorkerNode * WorkerGetRandomCandidateNode(List *currentNodeList);
|
||||
|
@ -103,6 +104,8 @@ extern WorkerNode * SetWorkerColumnLocalOnly(WorkerNode *workerNode, int columnI
|
|||
Datum value);
|
||||
extern uint32 CountPrimariesWithMetadata(void);
|
||||
extern WorkerNode * GetFirstPrimaryWorkerNode(void);
|
||||
extern List * SyncDistributedObjectsCommandList(WorkerNode *workerNode);
|
||||
extern List * PgDistTableMetadataSyncCommandList(void);
|
||||
|
||||
/* Function declarations for worker node utilities */
|
||||
extern int CompareWorkerNodes(const void *leftElement, const void *rightElement);
|
||||
|
|
|
@ -43,12 +43,12 @@ FOR EACH ROW EXECUTE FUNCTION dummy_function();
|
|||
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1508000, 'citus_local_tables_mx', 'CREATE TRIGGER dummy_function_trigger
|
||||
BEFORE UPDATE OF value ON citus_local_table
|
||||
FOR EACH ROW EXECUTE FUNCTION dummy_function();')
|
||||
-- Show that we can sync metadata successfully. That means, we create
|
||||
-- Show that we can activate node successfully. That means, we create
|
||||
-- the function that trigger needs in mx workers too.
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
start_metadata_sync_to_node
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
|
||||
1
|
||||
(1 row)
|
||||
|
||||
CREATE EXTENSION seg;
|
||||
|
|
|
@ -25,13 +25,6 @@ SELECT create_distributed_table('notices', 'id');
|
|||
(1 row)
|
||||
|
||||
INSERT INTO notices VALUES (1, 'hello world');
|
||||
-- Create the necessary test utility function
|
||||
CREATE OR REPLACE FUNCTION master_metadata_snapshot()
|
||||
RETURNS text[]
|
||||
LANGUAGE C STRICT
|
||||
AS 'citus';
|
||||
COMMENT ON FUNCTION master_metadata_snapshot()
|
||||
IS 'commands to create the metadata snapshot';
|
||||
CREATE FUNCTION notice(text)
|
||||
RETURNS void
|
||||
LANGUAGE plpgsql AS $$
|
||||
|
@ -595,13 +588,6 @@ SELECT create_distributed_function('eq_with_param_names(macaddr, macaddr)', dist
|
|||
|
||||
(1 row)
|
||||
|
||||
-- show that we are able to propagate objects with multiple item on address arrays
|
||||
SELECT * FROM (SELECT unnest(master_metadata_snapshot()) as metadata_command order by 1) as innerResult WHERE metadata_command like '%distributed_object_data%';
|
||||
metadata_command
|
||||
---------------------------------------------------------------------
|
||||
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('type', ARRAY['public.usage_access_type']::text[], ARRAY[]::text[], -1, 0, false), ('type', ARRAY['function_tests.dup_result']::text[], ARRAY[]::text[], -1, 0, false), ('function', ARRAY['public', 'usage_access_func']::text[], ARRAY['public.usage_access_type', 'integer[]']::text[], -1, 0, false), ('function', ARRAY['public', 'usage_access_func_third']::text[], ARRAY['integer', 'integer[]']::text[], 0, 50, false), ('function', ARRAY['function_tests', 'notice']::text[], ARRAY['pg_catalog.text']::text[], -1, 0, false), ('function', ARRAY['function_tests', 'dup']::text[], ARRAY['pg_catalog.macaddr']::text[], 0, 52, false), ('function', ARRAY['function_tests', 'eq_with_param_names']::text[], ARRAY['pg_catalog.macaddr', 'pg_catalog.macaddr']::text[], 0, 52, false), ('function', ARRAY['function_tests', 'eq_mi''xed_param_names']::text[], ARRAY['pg_catalog.macaddr', 'pg_catalog.macaddr']::text[], -1, 0, false), ('function', ARRAY['function_tests', 'agg_sfunc']::text[], ARRAY['integer', 'integer']::text[], -1, 0, false), ('function', ARRAY['function_tests', 'agg_invfunc']::text[], ARRAY['integer', 'integer']::text[], -1, 0, false), ('function', ARRAY['function_tests', 'agg_finalfunc']::text[], ARRAY['integer', 'integer']::text[], -1, 0, false), ('aggregate', ARRAY['function_tests', 'my_rank']::text[], ARRAY['pg_catalog."any"']::text[], -1, 0, false), ('function', ARRAY['function_tests', 'agg_names_sfunc']::text[], ARRAY['function_tests.dup_result', 'function_tests.dup_result', 'function_tests.dup_result']::text[], -1, 0, false), ('function', ARRAY['function_tests', 'agg_names_finalfunc']::text[], ARRAY['function_tests.dup_result']::text[], -1, 0, false), ('aggregate', ARRAY['function_tests', 'agg_names']::text[], ARRAY['function_tests.dup_result', 'function_tests.dup_result']::text[], -1, 0, false), ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('server', ARRAY['fake_fdw_server']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_testing_schema']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_testing_schema_2']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_test_schema_1']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_test_schema_2']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['schema_colocation']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['function_tests']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['function_tests2']::text[], ARRAY[]::text[], -1, 0, false), ('extension', ARRAY['plpgsql']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
|
||||
(1 row)
|
||||
|
||||
-- valid distribution with distribution_arg_index
|
||||
SELECT create_distributed_function('eq_with_param_names(macaddr, macaddr)','$1');
|
||||
create_distributed_function
|
||||
|
|
|
@ -42,7 +42,7 @@ SELECT citus.mitmproxy('conn.onQuery(query="^UPDATE pg_dist_local_group SET grou
|
|||
|
||||
(1 row)
|
||||
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_proxy_port);
|
||||
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
|
||||
ERROR: canceling statement due to user request
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^UPDATE pg_dist_local_group SET groupid").kill()');
|
||||
mitmproxy
|
||||
|
@ -50,33 +50,33 @@ SELECT citus.mitmproxy('conn.onQuery(query="^UPDATE pg_dist_local_group SET grou
|
|||
|
||||
(1 row)
|
||||
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_proxy_port);
|
||||
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
|
||||
ERROR: server closed the connection unexpectedly
|
||||
This probably means the server terminated abnormally
|
||||
before or while processing the request.
|
||||
CONTEXT: while executing command on localhost:xxxxx
|
||||
-- Failure to drop all tables in pg_dist_partition
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT worker_drop_distributed_table").cancel(' || :pid || ')');
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^DELETE FROM pg_dist_partition").cancel(' || :pid || ')');
|
||||
mitmproxy
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_proxy_port);
|
||||
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
|
||||
ERROR: canceling statement due to user request
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT worker_drop_distributed_table").kill()');
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^DELETE FROM pg_dist_partition").kill()');
|
||||
mitmproxy
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_proxy_port);
|
||||
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
|
||||
ERROR: server closed the connection unexpectedly
|
||||
This probably means the server terminated abnormally
|
||||
before or while processing the request.
|
||||
CONTEXT: while executing command on localhost:xxxxx
|
||||
-- Failure to truncate pg_dist_node in the worker
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^TRUNCATE pg_dist_node CASCADE").cancel(' || :pid || ')');
|
||||
-- Failure to delete pg_dist_node entries from the worker
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^DELETE FROM pg_dist_node").cancel(' || :pid || ')');
|
||||
mitmproxy
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -84,7 +84,7 @@ SELECT citus.mitmproxy('conn.onQuery(query="^TRUNCATE pg_dist_node CASCADE").can
|
|||
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_proxy_port);
|
||||
ERROR: canceling statement due to user request
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^TRUNCATE pg_dist_node CASCADE").kill()');
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^DELETE FROM pg_dist_node").kill()');
|
||||
mitmproxy
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -122,17 +122,17 @@ SELECT hasmetadata FROM pg_dist_node WHERE nodeport=:worker_2_proxy_port;
|
|||
f
|
||||
(1 row)
|
||||
|
||||
-- Verify we can sync metadata after unsuccessful attempts
|
||||
-- Verify we can activate node after unsuccessful attempts
|
||||
SELECT citus.mitmproxy('conn.allow()');
|
||||
mitmproxy
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_proxy_port);
|
||||
start_metadata_sync_to_node
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_2_proxy_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
|
||||
1
|
||||
(1 row)
|
||||
|
||||
SELECT hasmetadata FROM pg_dist_node WHERE nodeport=:worker_2_proxy_port;
|
||||
|
@ -224,8 +224,8 @@ CONTEXT: while executing command on localhost:xxxxx
|
|||
|
||||
(1 row)
|
||||
|
||||
-- Failure to drop all tables in pg_dist_partition
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT worker_drop_distributed_table").cancel(' || :pid || ')');
|
||||
-- Failure to delete pg_dist_node entries from the worker
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^DELETE FROM pg_dist_node").cancel(' || :pid || ')');
|
||||
mitmproxy
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -234,60 +234,7 @@ SELECT citus.mitmproxy('conn.onQuery(query="^SELECT worker_drop_distributed_tabl
|
|||
SELECT stop_metadata_sync_to_node('localhost', :worker_2_proxy_port);
|
||||
NOTICE: dropping metadata on the node (localhost,9060)
|
||||
ERROR: canceling statement due to user request
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT worker_drop_distributed_table").kill()');
|
||||
mitmproxy
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_2_proxy_port);
|
||||
NOTICE: dropping metadata on the node (localhost,9060)
|
||||
WARNING: server closed the connection unexpectedly
|
||||
This probably means the server terminated abnormally
|
||||
before or while processing the request.
|
||||
CONTEXT: while executing command on localhost:xxxxx
|
||||
WARNING: server closed the connection unexpectedly
|
||||
This probably means the server terminated abnormally
|
||||
before or while processing the request.
|
||||
connection not open
|
||||
CONTEXT: while executing command on localhost:xxxxx
|
||||
WARNING: server closed the connection unexpectedly
|
||||
This probably means the server terminated abnormally
|
||||
before or while processing the request.
|
||||
connection not open
|
||||
connection not open
|
||||
CONTEXT: while executing command on localhost:xxxxx
|
||||
WARNING: server closed the connection unexpectedly
|
||||
This probably means the server terminated abnormally
|
||||
before or while processing the request.
|
||||
connection not open
|
||||
connection not open
|
||||
connection not open
|
||||
CONTEXT: while executing command on localhost:xxxxx
|
||||
WARNING: server closed the connection unexpectedly
|
||||
This probably means the server terminated abnormally
|
||||
before or while processing the request.
|
||||
connection not open
|
||||
connection not open
|
||||
connection not open
|
||||
connection not open
|
||||
CONTEXT: while executing command on localhost:xxxxx
|
||||
stop_metadata_sync_to_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
-- Failure to truncate pg_dist_node in the worker
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^TRUNCATE pg_dist_node CASCADE").cancel(' || :pid || ')');
|
||||
mitmproxy
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_2_proxy_port);
|
||||
NOTICE: dropping metadata on the node (localhost,9060)
|
||||
ERROR: canceling statement due to user request
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^TRUNCATE pg_dist_node CASCADE").kill()');
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^DELETE FROM pg_dist_node").kill()');
|
||||
mitmproxy
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -344,7 +291,7 @@ SELECT hasmetadata FROM pg_dist_node WHERE nodeport=:worker_2_proxy_port;
|
|||
f
|
||||
(1 row)
|
||||
|
||||
-- Verify we can drop metadata after unsuccessful attempts
|
||||
-- Verify we can stop metadata sync after unsuccessful attempts
|
||||
SELECT citus.mitmproxy('conn.allow()');
|
||||
mitmproxy
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -68,7 +68,7 @@ step s3-commit:
|
|||
COMMIT;
|
||||
|
||||
|
||||
starting permutation: s1-create-table s1-begin s1-insert s1-verify-current-xact-is-on-worker s1-commit
|
||||
starting permutation: s1-create-table s1-begin s1-insert s1-verify-current-xact-is-on-worker s1-drop-table s1-commit
|
||||
step s1-create-table:
|
||||
-- some tests also use distributed table
|
||||
CREATE TABLE distributed_transaction_id_table(some_value int, other_value int);
|
||||
|
@ -104,6 +104,9 @@ nodeport|xact_exists
|
|||
57638|t
|
||||
(2 rows)
|
||||
|
||||
step s1-drop-table:
|
||||
DROP TABLE distributed_transaction_id_table;
|
||||
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
|
|
|
@ -118,7 +118,8 @@ pg_identify_object_as_address
|
|||
(database,{regression},{})
|
||||
(role,{postgres},{})
|
||||
(schema,{public},{})
|
||||
(3 rows)
|
||||
(table,"{public,t1}",{})
|
||||
(4 rows)
|
||||
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -310,7 +311,8 @@ pg_identify_object_as_address
|
|||
(database,{regression},{})
|
||||
(role,{postgres},{})
|
||||
(schema,{public},{})
|
||||
(3 rows)
|
||||
(table,"{public,t1}",{})
|
||||
(4 rows)
|
||||
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -502,7 +504,8 @@ pg_identify_object_as_address
|
|||
(database,{regression},{})
|
||||
(role,{postgres},{})
|
||||
(schema,{public},{})
|
||||
(3 rows)
|
||||
(table,"{public,t1}",{})
|
||||
(4 rows)
|
||||
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -690,7 +693,8 @@ pg_identify_object_as_address
|
|||
(role,{postgres},{})
|
||||
(schema,{myschema},{})
|
||||
(schema,{public},{})
|
||||
(4 rows)
|
||||
(table,"{myschema,t1}",{})
|
||||
(5 rows)
|
||||
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -884,7 +888,8 @@ pg_identify_object_as_address
|
|||
(role,{postgres},{})
|
||||
(schema,{myschema},{})
|
||||
(schema,{public},{})
|
||||
(4 rows)
|
||||
(table,"{myschema,t1}",{})
|
||||
(5 rows)
|
||||
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -1078,7 +1083,8 @@ pg_identify_object_as_address
|
|||
(role,{postgres},{})
|
||||
(schema,{myschema},{})
|
||||
(schema,{public},{})
|
||||
(4 rows)
|
||||
(table,"{myschema,t1}",{})
|
||||
(5 rows)
|
||||
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -1272,7 +1278,8 @@ pg_identify_object_as_address
|
|||
(role,{postgres},{})
|
||||
(schema,{myschema},{})
|
||||
(schema,{public},{})
|
||||
(4 rows)
|
||||
(table,"{myschema,t1}",{})
|
||||
(5 rows)
|
||||
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -1480,7 +1487,9 @@ pg_identify_object_as_address
|
|||
(role,{postgres},{})
|
||||
(schema,{myschema},{})
|
||||
(schema,{public},{})
|
||||
(4 rows)
|
||||
(table,"{myschema,t1}",{})
|
||||
(table,"{myschema,t2}",{})
|
||||
(6 rows)
|
||||
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -1697,7 +1706,9 @@ pg_identify_object_as_address
|
|||
(schema,{myschema},{})
|
||||
(schema,{myschema2},{})
|
||||
(schema,{public},{})
|
||||
(5 rows)
|
||||
(table,"{myschema,t1}",{})
|
||||
(table,"{myschema2,t2}",{})
|
||||
(7 rows)
|
||||
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -2251,8 +2262,9 @@ pg_identify_object_as_address
|
|||
(role,{postgres},{})
|
||||
(schema,{myschema},{})
|
||||
(schema,{public},{})
|
||||
(table,"{myschema,t1}",{})
|
||||
(type,{myschema.tt1},{})
|
||||
(5 rows)
|
||||
(6 rows)
|
||||
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -122,142 +122,3 @@ restore_isolation_tester_func
|
|||
|
||||
(1 row)
|
||||
|
||||
|
||||
starting permutation: increase-retry-interval reload-conf s2-start-session-level-connection s2-begin-on-worker s2-truncate-on-worker s3-invalidate-metadata s3-resync s3-wait s1-count-daemons s1-cancel-metadata-sync s1-count-daemons reset-retry-interval reload-conf s2-commit-on-worker s2-stop-connection s3-resync s3-wait
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
step increase-retry-interval:
|
||||
ALTER SYSTEM SET citus.metadata_sync_retry_interval TO 20000;
|
||||
|
||||
step reload-conf:
|
||||
SELECT pg_reload_conf();
|
||||
|
||||
pg_reload_conf
|
||||
---------------------------------------------------------------------
|
||||
t
|
||||
(1 row)
|
||||
|
||||
step s2-start-session-level-connection:
|
||||
SELECT start_session_level_connection_to_node('localhost', 57638);
|
||||
|
||||
start_session_level_connection_to_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
step s2-begin-on-worker:
|
||||
SELECT run_commands_on_session_level_connection_to_node('BEGIN');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
step s2-truncate-on-worker:
|
||||
SELECT run_commands_on_session_level_connection_to_node('TRUNCATE t2');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
step s3-invalidate-metadata:
|
||||
update pg_dist_node SET metadatasynced = false;
|
||||
|
||||
step s3-resync:
|
||||
SELECT trigger_metadata_sync();
|
||||
|
||||
trigger_metadata_sync
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
step s3-wait:
|
||||
SELECT pg_sleep(2);
|
||||
|
||||
pg_sleep
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
step s1-count-daemons:
|
||||
SELECT count(*) FROM pg_stat_activity WHERE application_name LIKE 'Citus Met%';
|
||||
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
step s1-cancel-metadata-sync:
|
||||
SELECT pg_cancel_backend(pid) FROM pg_stat_activity WHERE application_name LIKE 'Citus Met%';
|
||||
SELECT pg_sleep(2);
|
||||
|
||||
pg_cancel_backend
|
||||
---------------------------------------------------------------------
|
||||
t
|
||||
(1 row)
|
||||
|
||||
pg_sleep
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
step s1-count-daemons:
|
||||
SELECT count(*) FROM pg_stat_activity WHERE application_name LIKE 'Citus Met%';
|
||||
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
(1 row)
|
||||
|
||||
step reset-retry-interval:
|
||||
ALTER SYSTEM RESET citus.metadata_sync_retry_interval;
|
||||
|
||||
step reload-conf:
|
||||
SELECT pg_reload_conf();
|
||||
|
||||
pg_reload_conf
|
||||
---------------------------------------------------------------------
|
||||
t
|
||||
(1 row)
|
||||
|
||||
step s2-commit-on-worker:
|
||||
SELECT run_commands_on_session_level_connection_to_node('COMMIT');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
step s2-stop-connection:
|
||||
SELECT stop_session_level_connection_to_node();
|
||||
|
||||
stop_session_level_connection_to_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
step s3-resync:
|
||||
SELECT trigger_metadata_sync();
|
||||
|
||||
trigger_metadata_sync
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
step s3-wait:
|
||||
SELECT pg_sleep(2);
|
||||
|
||||
pg_sleep
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
restore_isolation_tester_func
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
|
|
|
@ -90,10 +90,7 @@ query |query_hostname |query_hostport|distribute
|
|||
|
||||
update ref_table set a = a + 1;
|
||||
|coordinator_host| 57636|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression
|
||||
|
||||
update ref_table set a = a + 1;
|
||||
|localhost | 57636|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression
|
||||
(2 rows)
|
||||
(1 row)
|
||||
|
||||
step s2-view-worker:
|
||||
SELECT query, query_hostname, query_hostport, distributed_query_host_name,
|
||||
|
@ -106,11 +103,9 @@ step s2-view-worker:
|
|||
|
||||
query |query_hostname |query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname
|
||||
---------------------------------------------------------------------
|
||||
UPDATE public.ref_table_1500775 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57638|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression
|
||||
UPDATE public.ref_table_1500775 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57637|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression
|
||||
UPDATE public.ref_table_1500775 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|coordinator_host| 57636|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression
|
||||
UPDATE public.ref_table_1500775 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57636|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression
|
||||
(4 rows)
|
||||
UPDATE public.ref_table_1500767 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57638|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression
|
||||
UPDATE public.ref_table_1500767 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57637|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression
|
||||
(2 rows)
|
||||
|
||||
step s2-end:
|
||||
END;
|
||||
|
@ -146,12 +141,12 @@ step s2-active-transactions:
|
|||
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
2
|
||||
1
|
||||
(1 row)
|
||||
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
6
|
||||
3
|
||||
(1 row)
|
||||
|
||||
step s1-end:
|
||||
|
|
|
@ -158,7 +158,7 @@ nodeid|groupid|nodename |nodeport
|
|||
|
||||
master_run_on_worker
|
||||
---------------------------------------------------------------------
|
||||
(localhost,57638,t,"[{""f1"": 25, ""f2"": 25, ""f3"": ""localhost"", ""f4"": 58637}, {""f1"": 26, ""f2"": 26, ""f3"": ""localhost"", ""f4"": 57638}]")
|
||||
(localhost,57638,t,"[{""f1"": 10, ""f2"": 10, ""f3"": ""localhost"", ""f4"": 57638}]")
|
||||
(1 row)
|
||||
|
||||
nodeid|nodename|nodeport
|
||||
|
|
|
@ -52,17 +52,9 @@ SELECT create_distributed_table('collections_list', 'key');
|
|||
CREATE TABLE collections_list_0
|
||||
PARTITION OF collections_list (key, collection_id)
|
||||
FOR VALUES IN ( 0 );
|
||||
DEBUG: relation "collections_list_key_seq" already exists, skipping
|
||||
DETAIL: from localhost:xxxxx
|
||||
DEBUG: relation "collections_list_key_seq" already exists, skipping
|
||||
DETAIL: from localhost:xxxxx
|
||||
CREATE TABLE collections_list_1
|
||||
PARTITION OF collections_list (key, collection_id)
|
||||
FOR VALUES IN ( 1 );
|
||||
DEBUG: relation "collections_list_key_seq" already exists, skipping
|
||||
DETAIL: from localhost:xxxxx
|
||||
DEBUG: relation "collections_list_key_seq" already exists, skipping
|
||||
DETAIL: from localhost:xxxxx
|
||||
-- connection worker and get ready for the tests
|
||||
\c - - - :worker_1_port
|
||||
SET search_path TO local_shard_copy;
|
||||
|
|
|
@ -28,16 +28,16 @@ SELECT create_distributed_table('t1', 'c');
|
|||
(1 row)
|
||||
|
||||
ALTER TABLE t1 DROP COLUMN b;
|
||||
SELECT start_metadata_sync_to_node('localhost',:worker_1_port);
|
||||
start_metadata_sync_to_node
|
||||
SELECT 1 FROM citus_activate_node('localhost',:worker_1_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
|
||||
1
|
||||
(1 row)
|
||||
|
||||
SELECT start_metadata_sync_to_node('localhost',:worker_2_port);
|
||||
start_metadata_sync_to_node
|
||||
SELECT 1 FROM citus_activate_node('localhost',:worker_2_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
|
||||
1
|
||||
(1 row)
|
||||
|
||||
\c - - - :worker_1_port
|
||||
|
|
|
@ -110,6 +110,8 @@ SELECT count(*) FROM history;
|
|||
-- test we can replicate MX tables
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
-- metadata sync will succeed even if we have rep > 1 tables
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles_single_shard'::regclass::oid, 0);
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
start_metadata_sync_to_node
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -1,20 +1,15 @@
|
|||
SET citus.next_shard_id TO 1220000;
|
||||
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 1390000;
|
||||
ALTER SEQUENCE pg_catalog.pg_dist_groupid_seq RESTART 1;
|
||||
SET citus.enable_object_propagation TO off; -- prevent object propagation on add node during setup
|
||||
-- Tests functions related to cluster membership
|
||||
-- add the nodes to the cluster
|
||||
SELECT 1 FROM master_add_node('localhost', :worker_1_port);
|
||||
WARNING: citus.enable_object_propagation is off, not creating distributed objects on worker
|
||||
DETAIL: distributed objects are only kept in sync when citus.enable_object_propagation is set to on. Newly activated nodes will not get these objects created
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||
WARNING: citus.enable_object_propagation is off, not creating distributed objects on worker
|
||||
DETAIL: distributed objects are only kept in sync when citus.enable_object_propagation is set to on. Newly activated nodes will not get these objects created
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -59,8 +54,6 @@ SELECT master_get_active_worker_nodes();
|
|||
|
||||
-- try to disable a node with no placements see that node is s=removed
|
||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||
WARNING: citus.enable_object_propagation is off, not creating distributed objects on worker
|
||||
DETAIL: distributed objects are only kept in sync when citus.enable_object_propagation is set to on. Newly activated nodes will not get these objects created
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -109,12 +102,10 @@ SELECT * FROM rebalance_table_shards();
|
|||
-- insert stuff into pg_dist_colocation
|
||||
TRUNCATE pg_dist_colocation;
|
||||
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 1390000;
|
||||
SELECT * FROM citus_activate_node('localhost', :worker_2_port);
|
||||
WARNING: citus.enable_object_propagation is off, not creating distributed objects on worker
|
||||
DETAIL: distributed objects are only kept in sync when citus.enable_object_propagation is set to on. Newly activated nodes will not get these objects created
|
||||
citus_activate_node
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_2_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
3
|
||||
1
|
||||
(1 row)
|
||||
|
||||
CREATE TABLE cluster_management_test (col_1 text, col_2 int);
|
||||
|
@ -183,8 +174,6 @@ SELECT citus_disable_node('localhost.noexist', 2345);
|
|||
ERROR: node at "localhost.noexist:2345" does not exist
|
||||
-- drop the table without leaving a shard placement behind (messes up other tests)
|
||||
SELECT master_activate_node('localhost', :worker_2_port);
|
||||
WARNING: citus.enable_object_propagation is off, not creating distributed objects on worker
|
||||
DETAIL: distributed objects are only kept in sync when citus.enable_object_propagation is set to on. Newly activated nodes will not get these objects created
|
||||
master_activate_node
|
||||
---------------------------------------------------------------------
|
||||
3
|
||||
|
@ -192,7 +181,6 @@ DETAIL: distributed objects are only kept in sync when citus.enable_object_prop
|
|||
|
||||
DROP TABLE test_reference_table, cluster_management_test;
|
||||
-- create users like this so results of community and enterprise are same
|
||||
SET citus.enable_object_propagation TO ON;
|
||||
SET client_min_messages TO ERROR;
|
||||
CREATE USER non_super_user;
|
||||
CREATE USER node_metadata_user;
|
||||
|
@ -204,7 +192,6 @@ SELECT 1 FROM run_command_on_workers('CREATE USER node_metadata_user');
|
|||
(2 rows)
|
||||
|
||||
RESET client_min_messages;
|
||||
SET citus.enable_object_propagation TO OFF;
|
||||
GRANT EXECUTE ON FUNCTION master_activate_node(text,int) TO node_metadata_user;
|
||||
GRANT EXECUTE ON FUNCTION master_add_inactive_node(text,int,int,noderole,name) TO node_metadata_user;
|
||||
GRANT EXECUTE ON FUNCTION master_add_node(text,int,int,noderole,name) TO node_metadata_user;
|
||||
|
@ -269,7 +256,6 @@ SELECT master_update_node(nodeid, 'localhost', :worker_2_port + 3) FROM pg_dist_
|
|||
ERROR: permission denied for function master_update_node
|
||||
-- try to manipulate node metadata via privileged user
|
||||
SET ROLE node_metadata_user;
|
||||
SET citus.enable_object_propagation TO off; -- prevent master activate node to actually connect for this test
|
||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||
ERROR: operation is not allowed
|
||||
HINT: Run the command with a superuser.
|
||||
|
@ -307,7 +293,6 @@ SELECT nodename, nodeport, noderole FROM pg_dist_node ORDER BY nodeport;
|
|||
ABORT;
|
||||
\c - postgres - :master_port
|
||||
SET citus.next_shard_id TO 1220000;
|
||||
SET citus.enable_object_propagation TO off; -- prevent object propagation on add node during setup
|
||||
SET citus.shard_count TO 16;
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
SELECT master_get_active_worker_nodes();
|
||||
|
@ -318,8 +303,6 @@ SELECT master_get_active_worker_nodes();
|
|||
|
||||
-- restore the node for next tests
|
||||
SELECT * FROM master_add_node('localhost', :worker_2_port);
|
||||
WARNING: citus.enable_object_propagation is off, not creating distributed objects on worker
|
||||
DETAIL: distributed objects are only kept in sync when citus.enable_object_propagation is set to on. Newly activated nodes will not get these objects created
|
||||
master_add_node
|
||||
---------------------------------------------------------------------
|
||||
7
|
||||
|
@ -496,8 +479,6 @@ SELECT * INTO old_placements FROM pg_dist_placement WHERE groupid = :worker_2_gr
|
|||
DELETE FROM pg_dist_placement WHERE groupid = :worker_2_group;
|
||||
SELECT master_add_node('localhost', :worker_2_port) AS new_node \gset
|
||||
WARNING: could not find any shard placements for shardId 1220001
|
||||
WARNING: citus.enable_object_propagation is off, not creating distributed objects on worker
|
||||
DETAIL: distributed objects are only kept in sync when citus.enable_object_propagation is set to on. Newly activated nodes will not get these objects created
|
||||
WARNING: could not find any shard placements for shardId 1220001
|
||||
WARNING: could not find any shard placements for shardId 1220003
|
||||
WARNING: could not find any shard placements for shardId 1220005
|
||||
|
@ -555,8 +536,6 @@ SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
|||
(1 row)
|
||||
|
||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||
WARNING: citus.enable_object_propagation is off, not creating distributed objects on worker
|
||||
DETAIL: distributed objects are only kept in sync when citus.enable_object_propagation is set to on. Newly activated nodes will not get these objects created
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -583,7 +562,6 @@ SELECT nodename, nodeport FROM pg_dist_node WHERE nodename='localhost' AND nodep
|
|||
(0 rows)
|
||||
|
||||
\c - - - :master_port
|
||||
SET citus.enable_object_propagation TO off; -- prevent object propagation on add node during setup
|
||||
-- check that added nodes are not propagated to nodes without metadata
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
NOTICE: dropping metadata on the node (localhost,57637)
|
||||
|
@ -593,8 +571,6 @@ NOTICE: dropping metadata on the node (localhost,57637)
|
|||
(1 row)
|
||||
|
||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||
WARNING: citus.enable_object_propagation is off, not creating distributed objects on worker
|
||||
DETAIL: distributed objects are only kept in sync when citus.enable_object_propagation is set to on. Newly activated nodes will not get these objects created
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -607,7 +583,6 @@ SELECT nodename, nodeport FROM pg_dist_node WHERE nodename='localhost' AND nodep
|
|||
(0 rows)
|
||||
|
||||
\c - - - :master_port
|
||||
SET citus.enable_object_propagation TO off; -- prevent object propagation on add node during setup
|
||||
-- check that removing two nodes in the same transaction works
|
||||
SELECT
|
||||
master_remove_node('localhost', :worker_1_port),
|
||||
|
@ -627,10 +602,6 @@ SELECT count(1) FROM pg_dist_node;
|
|||
SELECT
|
||||
master_add_node('localhost', :worker_1_port),
|
||||
master_add_node('localhost', :worker_2_port);
|
||||
WARNING: citus.enable_object_propagation is off, not creating distributed objects on worker
|
||||
DETAIL: distributed objects are only kept in sync when citus.enable_object_propagation is set to on. Newly activated nodes will not get these objects created
|
||||
WARNING: citus.enable_object_propagation is off, not creating distributed objects on worker
|
||||
DETAIL: distributed objects are only kept in sync when citus.enable_object_propagation is set to on. Newly activated nodes will not get these objects created
|
||||
master_add_node | master_add_node
|
||||
---------------------------------------------------------------------
|
||||
11 | 12
|
||||
|
@ -652,8 +623,6 @@ SELECT master_remove_node('localhost', :worker_2_port);
|
|||
(1 row)
|
||||
|
||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||
WARNING: citus.enable_object_propagation is off, not creating distributed objects on worker
|
||||
DETAIL: distributed objects are only kept in sync when citus.enable_object_propagation is set to on. Newly activated nodes will not get these objects created
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -679,8 +648,6 @@ SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
|||
|
||||
BEGIN;
|
||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||
WARNING: citus.enable_object_propagation is off, not creating distributed objects on worker
|
||||
DETAIL: distributed objects are only kept in sync when citus.enable_object_propagation is set to on. Newly activated nodes will not get these objects created
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -693,8 +660,6 @@ SELECT master_remove_node('localhost', :worker_2_port);
|
|||
(1 row)
|
||||
|
||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||
WARNING: citus.enable_object_propagation is off, not creating distributed objects on worker
|
||||
DETAIL: distributed objects are only kept in sync when citus.enable_object_propagation is set to on. Newly activated nodes will not get these objects created
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -715,7 +680,6 @@ SELECT nodename, nodeport FROM pg_dist_node WHERE nodename='localhost' AND nodep
|
|||
(1 row)
|
||||
|
||||
\c - - - :master_port
|
||||
SET citus.enable_object_propagation TO off; -- prevent object propagation on add node during setup
|
||||
SELECT master_remove_node(nodename, nodeport) FROM pg_dist_node;
|
||||
master_remove_node
|
||||
---------------------------------------------------------------------
|
||||
|
@ -724,16 +688,12 @@ SELECT master_remove_node(nodename, nodeport) FROM pg_dist_node;
|
|||
(2 rows)
|
||||
|
||||
SELECT 1 FROM master_add_node('localhost', :worker_1_port);
|
||||
WARNING: citus.enable_object_propagation is off, not creating distributed objects on worker
|
||||
DETAIL: distributed objects are only kept in sync when citus.enable_object_propagation is set to on. Newly activated nodes will not get these objects created
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||
WARNING: citus.enable_object_propagation is off, not creating distributed objects on worker
|
||||
DETAIL: distributed objects are only kept in sync when citus.enable_object_propagation is set to on. Newly activated nodes will not get these objects created
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -749,8 +709,6 @@ SELECT master_remove_node('localhost', :worker_2_port);
|
|||
|
||||
BEGIN;
|
||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||
WARNING: citus.enable_object_propagation is off, not creating distributed objects on worker
|
||||
DETAIL: distributed objects are only kept in sync when citus.enable_object_propagation is set to on. Newly activated nodes will not get these objects created
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -793,7 +751,6 @@ DELETE FROM pg_dist_shard;
|
|||
DELETE FROM pg_dist_placement;
|
||||
DELETE FROM pg_dist_node;
|
||||
\c - - - :master_port
|
||||
SET citus.enable_object_propagation TO off; -- prevent object propagation on add node during setup
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
NOTICE: dropping metadata on the node (localhost,57637)
|
||||
stop_metadata_sync_to_node
|
||||
|
|
|
@ -1,5 +1,7 @@
|
|||
SET citus.next_shard_id TO 1300000;
|
||||
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 4;
|
||||
-- Delete orphaned entries from pg_dist_colocation
|
||||
DELETE FROM pg_dist_colocation where colocationid = 5 or colocationid = 6;
|
||||
-- ===================================================================
|
||||
-- create test utility function
|
||||
-- ===================================================================
|
||||
|
@ -968,31 +970,22 @@ SELECT update_distributed_table_colocation('table1_group_none', colocate_with =>
|
|||
|
||||
(1 row)
|
||||
|
||||
-- sync metadata to get rid of inconsistencies in pg_dist tables
|
||||
select stop_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
NOTICE: dropping metadata on the node (localhost,57637)
|
||||
stop_metadata_sync_to_node
|
||||
-- activate nodes to get rid of inconsistencies in pg_dist tables
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table1_group1'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table2_group1'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table3_group2'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table4_group2'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table5_groupX'::regclass::oid, 0);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
|
||||
1
|
||||
(1 row)
|
||||
|
||||
select stop_metadata_sync_to_node('localhost', :worker_2_port);
|
||||
NOTICE: dropping metadata on the node (localhost,57638)
|
||||
stop_metadata_sync_to_node
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_2_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
select start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
start_metadata_sync_to_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
select start_metadata_sync_to_node('localhost', :worker_2_port);
|
||||
start_metadata_sync_to_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
1
|
||||
(1 row)
|
||||
|
||||
-- move a table with a colocation id which is already not in pg_dist_colocation
|
||||
|
|
|
@ -492,6 +492,7 @@ SELECT * FROM multi_extension.print_extension_changes();
|
|||
-- Test downgrade to 9.4-1 from 9.5-1
|
||||
ALTER EXTENSION citus UPDATE TO '9.5-1';
|
||||
BEGIN;
|
||||
SET citus.enable_object_propagation TO on;
|
||||
SELECT master_add_node('localhost', :master_port, groupId=>0);
|
||||
NOTICE: localhost:xxxxx is the coordinator and already contains metadata, skipping syncing the metadata
|
||||
master_add_node
|
||||
|
@ -507,6 +508,7 @@ NOTICE: create_citus_local_table is deprecated in favour of citus_add_local_tab
|
|||
|
||||
(1 row)
|
||||
|
||||
RESET citus.enable_object_propagation;
|
||||
-- downgrade from 9.5-1 to 9.4-1 should fail as we have a citus local table
|
||||
ALTER EXTENSION citus UPDATE TO '9.4-1';
|
||||
ERROR: citus local tables are introduced in Citus 9.5
|
||||
|
@ -1010,7 +1012,8 @@ SELECT * FROM multi_extension.print_extension_changes();
|
|||
| function citus_shards_on_worker() SETOF record
|
||||
| function create_distributed_function(regprocedure,text,text,boolean) void
|
||||
| function worker_drop_sequence_dependency(text) void
|
||||
(14 rows)
|
||||
| function worker_drop_shell_table(text) void
|
||||
(15 rows)
|
||||
|
||||
DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff;
|
||||
-- show running version
|
||||
|
@ -1260,10 +1263,7 @@ CREATE EXTENSION citus;
|
|||
\c - - - :worker_1_port
|
||||
CREATE EXTENSION citus;
|
||||
\c - - - :master_port
|
||||
SET citus.enable_object_propagation TO off; -- prevent distributed transactions during add node
|
||||
SELECT FROM master_add_node('localhost', :worker_1_port);
|
||||
WARNING: citus.enable_object_propagation is off, not creating distributed objects on worker
|
||||
DETAIL: distributed objects are only kept in sync when citus.enable_object_propagation is set to on. Newly activated nodes will not get these objects created
|
||||
--
|
||||
(1 row)
|
||||
|
||||
|
|
|
@ -89,10 +89,10 @@ SELECT tablename, indexname FROM pg_indexes WHERE schemaname = 'fix_idx_names' A
|
|||
|
||||
\c - - - :master_port
|
||||
-- this should work properly
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
start_metadata_sync_to_node
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
|
||||
1
|
||||
(1 row)
|
||||
|
||||
\c - - - :worker_1_port
|
||||
|
@ -670,14 +670,18 @@ NOTICE: issuing ALTER TABLE fix_idx_names.p2 OWNER TO postgres
|
|||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||
NOTICE: issuing ALTER TABLE fix_idx_names.p2 OWNER TO postgres
|
||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||
NOTICE: issuing SELECT citus_internal_add_partition_metadata ('fix_idx_names.p2'::regclass, 'h', 'dist_col', 1370000, 's')
|
||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||
NOTICE: issuing SELECT citus_internal_add_partition_metadata ('fix_idx_names.p2'::regclass, 'h', 'dist_col', 1370000, 's')
|
||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||
NOTICE: issuing SELECT worker_create_truncate_trigger('fix_idx_names.p2')
|
||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||
NOTICE: issuing SELECT worker_create_truncate_trigger('fix_idx_names.p2')
|
||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||
NOTICE: issuing SET citus.enable_ddl_propagation TO 'off'
|
||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||
NOTICE: issuing SET citus.enable_ddl_propagation TO 'off'
|
||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||
NOTICE: issuing SELECT citus_internal_add_partition_metadata ('fix_idx_names.p2'::regclass, 'h', 'dist_col', 1370000, 's')
|
||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||
NOTICE: issuing SELECT citus_internal_add_partition_metadata ('fix_idx_names.p2'::regclass, 'h', 'dist_col', 1370000, 's')
|
||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||
NOTICE: issuing WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('fix_idx_names.p2'::regclass, 915002, 't'::"char", '-2147483648', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||
NOTICE: issuing WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('fix_idx_names.p2'::regclass, 915002, 't'::"char", '-2147483648', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||
|
@ -690,6 +694,14 @@ NOTICE: issuing SET citus.enable_ddl_propagation TO 'off'
|
|||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||
NOTICE: issuing SET citus.enable_ddl_propagation TO 'off'
|
||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||
NOTICE: issuing WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['fix_idx_names', 'p2']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
|
||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||
NOTICE: issuing WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['fix_idx_names', 'p2']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
|
||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||
NOTICE: issuing SET citus.enable_ddl_propagation TO 'off'
|
||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||
NOTICE: issuing SET citus.enable_ddl_propagation TO 'off'
|
||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||
NOTICE: issuing SET search_path TO fix_idx_names,public;
|
||||
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
|
||||
NOTICE: issuing SET search_path TO fix_idx_names,public;
|
||||
|
|
|
@ -26,30 +26,67 @@ ALTER SEQUENCE pg_catalog.pg_dist_placement_placementid_seq RESTART 100000;
|
|||
SELECT nextval('pg_catalog.pg_dist_groupid_seq') AS last_group_id \gset
|
||||
SELECT nextval('pg_catalog.pg_dist_node_nodeid_seq') AS last_node_id \gset
|
||||
-- Create the necessary test utility function
|
||||
CREATE FUNCTION master_metadata_snapshot()
|
||||
CREATE FUNCTION activate_node_snapshot()
|
||||
RETURNS text[]
|
||||
LANGUAGE C STRICT
|
||||
AS 'citus';
|
||||
COMMENT ON FUNCTION master_metadata_snapshot()
|
||||
IS 'commands to create the metadata snapshot';
|
||||
COMMENT ON FUNCTION activate_node_snapshot()
|
||||
IS 'commands to activate node 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';
|
||||
logicalrelid | partmethod | partkey | colocationid | repmodel | autoconverted
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
-- Show that, with no MX tables, metadata snapshot contains only the delete commands,
|
||||
-- pg_dist_node entries and reference tables
|
||||
SELECT unnest(master_metadata_snapshot()) order by 1;
|
||||
unnest
|
||||
-- Since password_encryption default has been changed to sha from md5 with PG14
|
||||
-- we are updating it manually just for consistent test results between PG versions.
|
||||
ALTER SYSTEM SET password_encryption TO md5;
|
||||
SELECT pg_reload_conf();
|
||||
pg_reload_conf
|
||||
---------------------------------------------------------------------
|
||||
t
|
||||
(1 row)
|
||||
|
||||
SELECT pg_sleep(0.1);
|
||||
pg_sleep
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
ALTER ROLE CURRENT_USER WITH PASSWORD 'dummypassword';
|
||||
-- Show that, with no MX tables, activate node snapshot contains only the delete commands,
|
||||
-- pg_dist_node entries, pg_dist_object entries and roles.
|
||||
SELECT unnest(activate_node_snapshot()) order by 1;
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION postgres
|
||||
DELETE FROM citus.pg_dist_object
|
||||
DELETE FROM pg_dist_node
|
||||
DELETE FROM pg_dist_partition
|
||||
DELETE FROM pg_dist_placement
|
||||
DELETE FROM pg_dist_shard
|
||||
GRANT CREATE ON SCHEMA public TO PUBLIC;
|
||||
GRANT CREATE ON SCHEMA public TO postgres;
|
||||
GRANT USAGE ON SCHEMA public TO PUBLIC;
|
||||
GRANT USAGE ON SCHEMA public TO postgres;
|
||||
INSERT INTO pg_dist_node (nodeid, groupid, nodename, nodeport, noderack, hasmetadata, metadatasynced, isactive, noderole, nodecluster, shouldhaveshards) VALUES (1, 1, 'localhost', 57637, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default', TRUE),(2, 2, 'localhost', 57638, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default', TRUE)
|
||||
RESET ROLE
|
||||
RESET ROLE
|
||||
SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''')
|
||||
SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition
|
||||
SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition
|
||||
TRUNCATE citus.pg_dist_object
|
||||
TRUNCATE pg_dist_node CASCADE
|
||||
SELECT worker_create_or_alter_role('postgres', null, 'ALTER ROLE postgres SUPERUSER CREATEDB CREATEROLE INHERIT LOGIN REPLICATION BYPASSRLS CONNECTION LIMIT 0 PASSWORD ''md5c53670dddfc3bb4b5675c7872bc2249a'' VALID UNTIL ''2052-05-05 00:00:00-07''')
|
||||
SELECT worker_drop_shell_table(logicalrelid::regclass::text) FROM pg_dist_partition
|
||||
SET ROLE postgres
|
||||
SET ROLE postgres
|
||||
SET citus.enable_ddl_propagation TO 'off'
|
||||
SET citus.enable_ddl_propagation TO 'off'
|
||||
SET citus.enable_ddl_propagation TO 'off'
|
||||
SET citus.enable_ddl_propagation TO 'on'
|
||||
SET citus.enable_ddl_propagation TO 'on'
|
||||
SET citus.enable_ddl_propagation TO 'on'
|
||||
UPDATE pg_dist_local_group SET groupid = 1
|
||||
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
|
||||
(6 rows)
|
||||
(27 rows)
|
||||
|
||||
-- this function is dropped in Citus10, added here for tests
|
||||
CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass,
|
||||
|
@ -62,111 +99,164 @@ COMMENT ON FUNCTION pg_catalog.master_create_distributed_table(table_name regcla
|
|||
distribution_column text,
|
||||
distribution_method citus.distribution_type)
|
||||
IS 'define the table distribution functions';
|
||||
-- this function is dropped in Citus10, added here for tests
|
||||
CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name text, shard_count integer,
|
||||
replication_factor integer DEFAULT 2)
|
||||
RETURNS void
|
||||
AS 'citus', $$master_create_worker_shards$$
|
||||
LANGUAGE C STRICT;
|
||||
-- Create a test table with constraints and SERIAL and default from user defined sequence
|
||||
CREATE SEQUENCE user_defined_seq;
|
||||
CREATE TABLE mx_test_table (col_1 int UNIQUE, col_2 text NOT NULL, col_3 BIGSERIAL, col_4 BIGINT DEFAULT nextval('user_defined_seq'));
|
||||
SELECT master_create_distributed_table('mx_test_table', 'col_1', 'hash');
|
||||
master_create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT master_create_worker_shards('mx_test_table', 8, 1);
|
||||
master_create_worker_shards
|
||||
set citus.shard_count to 8;
|
||||
set citus.shard_replication_factor to 1;
|
||||
SELECT create_distributed_table('mx_test_table', 'col_1');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
reset citus.shard_count;
|
||||
reset citus.shard_replication_factor;
|
||||
-- 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()) order by 1;
|
||||
unnest
|
||||
-- Show that the created MX table is and its sequences are included in the activate node snapshot
|
||||
SELECT unnest(activate_node_snapshot()) order by 1;
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
ALTER SEQUENCE public.mx_test_table_col_3_seq OWNER TO postgres
|
||||
ALTER SEQUENCE public.user_defined_seq OWNER TO postgres
|
||||
ALTER TABLE public.mx_test_table ADD CONSTRAINT mx_test_table_col_1_key UNIQUE (col_1)
|
||||
ALTER TABLE public.mx_test_table OWNER TO postgres
|
||||
ALTER TABLE public.mx_test_table OWNER TO postgres
|
||||
CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION postgres
|
||||
CREATE TABLE public.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('public.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass))
|
||||
DELETE FROM citus.pg_dist_object
|
||||
DELETE FROM pg_dist_node
|
||||
DELETE FROM pg_dist_partition
|
||||
DELETE FROM pg_dist_placement
|
||||
DELETE FROM pg_dist_shard
|
||||
GRANT CREATE ON SCHEMA public TO PUBLIC;
|
||||
GRANT CREATE ON SCHEMA public TO postgres;
|
||||
GRANT USAGE ON SCHEMA public TO PUBLIC;
|
||||
GRANT USAGE ON SCHEMA public TO postgres;
|
||||
INSERT INTO pg_dist_node (nodeid, groupid, nodename, nodeport, noderack, hasmetadata, metadatasynced, isactive, noderole, nodecluster, shouldhaveshards) VALUES (1, 1, 'localhost', 57637, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default', TRUE),(2, 2, 'localhost', 57638, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default', TRUE)
|
||||
SELECT citus_internal_add_partition_metadata ('public.mx_test_table'::regclass, 'h', 'col_1', 0, 's')
|
||||
RESET ROLE
|
||||
RESET ROLE
|
||||
SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''')
|
||||
SELECT citus_internal_add_partition_metadata ('public.mx_test_table'::regclass, 'h', 'col_1', 2, 's')
|
||||
SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition
|
||||
SELECT pg_catalog.worker_record_sequence_dependency('public.mx_test_table_col_3_seq'::regclass,'public.mx_test_table'::regclass,'col_3')
|
||||
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint')
|
||||
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.user_defined_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint')
|
||||
SELECT worker_create_or_alter_role('postgres', null, 'ALTER ROLE postgres SUPERUSER CREATEDB CREATEROLE INHERIT LOGIN REPLICATION BYPASSRLS CONNECTION LIMIT 0 PASSWORD ''md5c53670dddfc3bb4b5675c7872bc2249a'' VALID UNTIL ''2052-05-05 00:00:00-07''')
|
||||
SELECT worker_create_truncate_trigger('public.mx_test_table')
|
||||
SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition
|
||||
TRUNCATE citus.pg_dist_object
|
||||
TRUNCATE pg_dist_node CASCADE
|
||||
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['public', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
|
||||
SELECT worker_drop_shell_table(logicalrelid::regclass::text) FROM pg_dist_partition
|
||||
SET ROLE postgres
|
||||
SET ROLE postgres
|
||||
SET citus.enable_ddl_propagation TO 'off'
|
||||
SET citus.enable_ddl_propagation TO 'off'
|
||||
SET citus.enable_ddl_propagation TO 'off'
|
||||
SET citus.enable_ddl_propagation TO 'on'
|
||||
SET citus.enable_ddl_propagation TO 'on'
|
||||
SET citus.enable_ddl_propagation TO 'on'
|
||||
UPDATE pg_dist_local_group SET groupid = 1
|
||||
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['public', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0, false), ('table', ARRAY['public', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
|
||||
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310000, 1, 0, 1, 100000), (1310001, 1, 0, 2, 100001), (1310002, 1, 0, 1, 100002), (1310003, 1, 0, 2, 100003), (1310004, 1, 0, 1, 100004), (1310005, 1, 0, 2, 100005), (1310006, 1, 0, 1, 100006), (1310007, 1, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||
(19 rows)
|
||||
(39 rows)
|
||||
|
||||
-- Show that CREATE INDEX commands are included in the metadata snapshot
|
||||
-- Show that CREATE INDEX commands are included in the activate node snapshot
|
||||
CREATE INDEX mx_index ON mx_test_table(col_2);
|
||||
SELECT unnest(master_metadata_snapshot()) order by 1;
|
||||
unnest
|
||||
SELECT unnest(activate_node_snapshot()) order by 1;
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
ALTER SEQUENCE public.mx_test_table_col_3_seq OWNER TO postgres
|
||||
ALTER SEQUENCE public.user_defined_seq OWNER TO postgres
|
||||
ALTER TABLE public.mx_test_table ADD CONSTRAINT mx_test_table_col_1_key UNIQUE (col_1)
|
||||
ALTER TABLE public.mx_test_table OWNER TO postgres
|
||||
ALTER TABLE public.mx_test_table OWNER TO postgres
|
||||
CREATE INDEX mx_index ON public.mx_test_table USING btree (col_2)
|
||||
CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION postgres
|
||||
CREATE TABLE public.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('public.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass))
|
||||
DELETE FROM citus.pg_dist_object
|
||||
DELETE FROM pg_dist_node
|
||||
DELETE FROM pg_dist_partition
|
||||
DELETE FROM pg_dist_placement
|
||||
DELETE FROM pg_dist_shard
|
||||
GRANT CREATE ON SCHEMA public TO PUBLIC;
|
||||
GRANT CREATE ON SCHEMA public TO postgres;
|
||||
GRANT USAGE ON SCHEMA public TO PUBLIC;
|
||||
GRANT USAGE ON SCHEMA public TO postgres;
|
||||
INSERT INTO pg_dist_node (nodeid, groupid, nodename, nodeport, noderack, hasmetadata, metadatasynced, isactive, noderole, nodecluster, shouldhaveshards) VALUES (1, 1, 'localhost', 57637, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default', TRUE),(2, 2, 'localhost', 57638, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default', TRUE)
|
||||
SELECT citus_internal_add_partition_metadata ('public.mx_test_table'::regclass, 'h', 'col_1', 0, 's')
|
||||
RESET ROLE
|
||||
RESET ROLE
|
||||
SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''')
|
||||
SELECT citus_internal_add_partition_metadata ('public.mx_test_table'::regclass, 'h', 'col_1', 2, 's')
|
||||
SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition
|
||||
SELECT pg_catalog.worker_record_sequence_dependency('public.mx_test_table_col_3_seq'::regclass,'public.mx_test_table'::regclass,'col_3')
|
||||
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint')
|
||||
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.user_defined_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint')
|
||||
SELECT worker_create_or_alter_role('postgres', null, 'ALTER ROLE postgres SUPERUSER CREATEDB CREATEROLE INHERIT LOGIN REPLICATION BYPASSRLS CONNECTION LIMIT 0 PASSWORD ''md5c53670dddfc3bb4b5675c7872bc2249a'' VALID UNTIL ''2052-05-05 00:00:00-07''')
|
||||
SELECT worker_create_truncate_trigger('public.mx_test_table')
|
||||
SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition
|
||||
TRUNCATE citus.pg_dist_object
|
||||
TRUNCATE pg_dist_node CASCADE
|
||||
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['public', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
|
||||
SELECT worker_drop_shell_table(logicalrelid::regclass::text) FROM pg_dist_partition
|
||||
SET ROLE postgres
|
||||
SET ROLE postgres
|
||||
SET citus.enable_ddl_propagation TO 'off'
|
||||
SET citus.enable_ddl_propagation TO 'off'
|
||||
SET citus.enable_ddl_propagation TO 'off'
|
||||
SET citus.enable_ddl_propagation TO 'on'
|
||||
SET citus.enable_ddl_propagation TO 'on'
|
||||
SET citus.enable_ddl_propagation TO 'on'
|
||||
UPDATE pg_dist_local_group SET groupid = 1
|
||||
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['public', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0, false), ('table', ARRAY['public', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
|
||||
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310000, 1, 0, 1, 100000), (1310001, 1, 0, 2, 100001), (1310002, 1, 0, 1, 100002), (1310003, 1, 0, 2, 100003), (1310004, 1, 0, 1, 100004), (1310005, 1, 0, 2, 100005), (1310006, 1, 0, 1, 100006), (1310007, 1, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||
(20 rows)
|
||||
(40 rows)
|
||||
|
||||
-- Show that schema changes are included in the metadata snapshot
|
||||
-- Show that schema changes are included in the activate node snapshot
|
||||
CREATE SCHEMA mx_testing_schema;
|
||||
ALTER TABLE mx_test_table SET SCHEMA mx_testing_schema;
|
||||
SELECT unnest(master_metadata_snapshot()) order by 1;
|
||||
unnest
|
||||
SELECT unnest(activate_node_snapshot()) order by 1;
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
ALTER SEQUENCE mx_testing_schema.mx_test_table_col_3_seq OWNER TO postgres
|
||||
ALTER SEQUENCE public.user_defined_seq OWNER TO postgres
|
||||
ALTER TABLE mx_testing_schema.mx_test_table ADD CONSTRAINT mx_test_table_col_1_key UNIQUE (col_1)
|
||||
ALTER TABLE mx_testing_schema.mx_test_table OWNER TO postgres
|
||||
ALTER TABLE mx_testing_schema.mx_test_table OWNER TO postgres
|
||||
CREATE INDEX mx_index ON mx_testing_schema.mx_test_table USING btree (col_2)
|
||||
CREATE SCHEMA IF NOT EXISTS mx_testing_schema AUTHORIZATION postgres
|
||||
CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION postgres
|
||||
CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass))
|
||||
DELETE FROM citus.pg_dist_object
|
||||
DELETE FROM pg_dist_node
|
||||
DELETE FROM pg_dist_partition
|
||||
DELETE FROM pg_dist_placement
|
||||
DELETE FROM pg_dist_shard
|
||||
GRANT CREATE ON SCHEMA public TO PUBLIC;
|
||||
GRANT CREATE ON SCHEMA public TO postgres;
|
||||
GRANT USAGE ON SCHEMA public TO PUBLIC;
|
||||
GRANT USAGE ON SCHEMA public TO postgres;
|
||||
INSERT INTO pg_dist_node (nodeid, groupid, nodename, nodeport, noderack, hasmetadata, metadatasynced, isactive, noderole, nodecluster, shouldhaveshards) VALUES (1, 1, 'localhost', 57637, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default', TRUE),(2, 2, 'localhost', 57638, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default', TRUE)
|
||||
SELECT citus_internal_add_partition_metadata ('mx_testing_schema.mx_test_table'::regclass, 'h', 'col_1', 0, 's')
|
||||
RESET ROLE
|
||||
RESET ROLE
|
||||
SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''')
|
||||
SELECT citus_internal_add_partition_metadata ('mx_testing_schema.mx_test_table'::regclass, 'h', 'col_1', 2, 's')
|
||||
SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition
|
||||
SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3')
|
||||
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint')
|
||||
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.user_defined_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint')
|
||||
SELECT worker_create_or_alter_role('postgres', null, 'ALTER ROLE postgres SUPERUSER CREATEDB CREATEROLE INHERIT LOGIN REPLICATION BYPASSRLS CONNECTION LIMIT 0 PASSWORD ''md5c53670dddfc3bb4b5675c7872bc2249a'' VALID UNTIL ''2052-05-05 00:00:00-07''')
|
||||
SELECT worker_create_truncate_trigger('mx_testing_schema.mx_test_table')
|
||||
SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition
|
||||
TRUNCATE citus.pg_dist_object
|
||||
TRUNCATE pg_dist_node CASCADE
|
||||
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['mx_testing_schema', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_testing_schema']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
|
||||
SELECT worker_drop_shell_table(logicalrelid::regclass::text) FROM pg_dist_partition
|
||||
SET ROLE postgres
|
||||
SET ROLE postgres
|
||||
SET citus.enable_ddl_propagation TO 'off'
|
||||
SET citus.enable_ddl_propagation TO 'off'
|
||||
SET citus.enable_ddl_propagation TO 'off'
|
||||
SET citus.enable_ddl_propagation TO 'on'
|
||||
SET citus.enable_ddl_propagation TO 'on'
|
||||
SET citus.enable_ddl_propagation TO 'on'
|
||||
UPDATE pg_dist_local_group SET groupid = 1
|
||||
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['mx_testing_schema', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0, false), ('table', ARRAY['mx_testing_schema', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_testing_schema']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
|
||||
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310000, 1, 0, 1, 100000), (1310001, 1, 0, 2, 100001), (1310002, 1, 0, 1, 100002), (1310003, 1, 0, 2, 100003), (1310004, 1, 0, 1, 100004), (1310005, 1, 0, 2, 100005), (1310006, 1, 0, 1, 100006), (1310007, 1, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||
(20 rows)
|
||||
(41 rows)
|
||||
|
||||
-- Show that append distributed tables are not included in the metadata snapshot
|
||||
-- Show that append distributed tables are not included in the activate node 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');
|
||||
master_create_distributed_table
|
||||
|
@ -175,59 +265,101 @@ SELECT master_create_distributed_table('non_mx_test_table', 'col_1', 'append');
|
|||
(1 row)
|
||||
|
||||
UPDATE pg_dist_partition SET repmodel='s' WHERE logicalrelid='non_mx_test_table'::regclass;
|
||||
SELECT unnest(master_metadata_snapshot()) order by 1;
|
||||
unnest
|
||||
SELECT unnest(activate_node_snapshot()) order by 1;
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
ALTER SEQUENCE mx_testing_schema.mx_test_table_col_3_seq OWNER TO postgres
|
||||
ALTER SEQUENCE public.user_defined_seq OWNER TO postgres
|
||||
ALTER TABLE mx_testing_schema.mx_test_table ADD CONSTRAINT mx_test_table_col_1_key UNIQUE (col_1)
|
||||
ALTER TABLE mx_testing_schema.mx_test_table OWNER TO postgres
|
||||
ALTER TABLE mx_testing_schema.mx_test_table OWNER TO postgres
|
||||
CREATE INDEX mx_index ON mx_testing_schema.mx_test_table USING btree (col_2)
|
||||
CREATE SCHEMA IF NOT EXISTS mx_testing_schema AUTHORIZATION postgres
|
||||
CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION postgres
|
||||
CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass))
|
||||
DELETE FROM citus.pg_dist_object
|
||||
DELETE FROM pg_dist_node
|
||||
DELETE FROM pg_dist_partition
|
||||
DELETE FROM pg_dist_placement
|
||||
DELETE FROM pg_dist_shard
|
||||
GRANT CREATE ON SCHEMA public TO PUBLIC;
|
||||
GRANT CREATE ON SCHEMA public TO postgres;
|
||||
GRANT USAGE ON SCHEMA public TO PUBLIC;
|
||||
GRANT USAGE ON SCHEMA public TO postgres;
|
||||
INSERT INTO pg_dist_node (nodeid, groupid, nodename, nodeport, noderack, hasmetadata, metadatasynced, isactive, noderole, nodecluster, shouldhaveshards) VALUES (1, 1, 'localhost', 57637, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default', TRUE),(2, 2, 'localhost', 57638, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default', TRUE)
|
||||
SELECT citus_internal_add_partition_metadata ('mx_testing_schema.mx_test_table'::regclass, 'h', 'col_1', 0, 's')
|
||||
RESET ROLE
|
||||
RESET ROLE
|
||||
SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''')
|
||||
SELECT citus_internal_add_partition_metadata ('mx_testing_schema.mx_test_table'::regclass, 'h', 'col_1', 2, 's')
|
||||
SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition
|
||||
SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3')
|
||||
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint')
|
||||
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.user_defined_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint')
|
||||
SELECT worker_create_or_alter_role('postgres', null, 'ALTER ROLE postgres SUPERUSER CREATEDB CREATEROLE INHERIT LOGIN REPLICATION BYPASSRLS CONNECTION LIMIT 0 PASSWORD ''md5c53670dddfc3bb4b5675c7872bc2249a'' VALID UNTIL ''2052-05-05 00:00:00-07''')
|
||||
SELECT worker_create_truncate_trigger('mx_testing_schema.mx_test_table')
|
||||
SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition
|
||||
TRUNCATE citus.pg_dist_object
|
||||
TRUNCATE pg_dist_node CASCADE
|
||||
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['mx_testing_schema', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_testing_schema']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
|
||||
SELECT worker_drop_shell_table(logicalrelid::regclass::text) FROM pg_dist_partition
|
||||
SET ROLE postgres
|
||||
SET ROLE postgres
|
||||
SET citus.enable_ddl_propagation TO 'off'
|
||||
SET citus.enable_ddl_propagation TO 'off'
|
||||
SET citus.enable_ddl_propagation TO 'off'
|
||||
SET citus.enable_ddl_propagation TO 'on'
|
||||
SET citus.enable_ddl_propagation TO 'on'
|
||||
SET citus.enable_ddl_propagation TO 'on'
|
||||
UPDATE pg_dist_local_group SET groupid = 1
|
||||
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['mx_testing_schema', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0, false), ('table', ARRAY['mx_testing_schema', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_testing_schema']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
|
||||
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310000, 1, 0, 1, 100000), (1310001, 1, 0, 2, 100001), (1310002, 1, 0, 1, 100002), (1310003, 1, 0, 2, 100003), (1310004, 1, 0, 1, 100004), (1310005, 1, 0, 2, 100005), (1310006, 1, 0, 1, 100006), (1310007, 1, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||
(20 rows)
|
||||
(41 rows)
|
||||
|
||||
-- Show that range distributed tables are not included in the metadata snapshot
|
||||
-- Show that range distributed tables are not included in the activate node snapshot
|
||||
UPDATE pg_dist_partition SET partmethod='r' WHERE logicalrelid='non_mx_test_table'::regclass;
|
||||
SELECT unnest(master_metadata_snapshot()) order by 1;
|
||||
unnest
|
||||
SELECT unnest(activate_node_snapshot()) order by 1;
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
ALTER SEQUENCE mx_testing_schema.mx_test_table_col_3_seq OWNER TO postgres
|
||||
ALTER SEQUENCE public.user_defined_seq OWNER TO postgres
|
||||
ALTER TABLE mx_testing_schema.mx_test_table ADD CONSTRAINT mx_test_table_col_1_key UNIQUE (col_1)
|
||||
ALTER TABLE mx_testing_schema.mx_test_table OWNER TO postgres
|
||||
ALTER TABLE mx_testing_schema.mx_test_table OWNER TO postgres
|
||||
CREATE INDEX mx_index ON mx_testing_schema.mx_test_table USING btree (col_2)
|
||||
CREATE SCHEMA IF NOT EXISTS mx_testing_schema AUTHORIZATION postgres
|
||||
CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION postgres
|
||||
CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass))
|
||||
DELETE FROM citus.pg_dist_object
|
||||
DELETE FROM pg_dist_node
|
||||
DELETE FROM pg_dist_partition
|
||||
DELETE FROM pg_dist_placement
|
||||
DELETE FROM pg_dist_shard
|
||||
GRANT CREATE ON SCHEMA public TO PUBLIC;
|
||||
GRANT CREATE ON SCHEMA public TO postgres;
|
||||
GRANT USAGE ON SCHEMA public TO PUBLIC;
|
||||
GRANT USAGE ON SCHEMA public TO postgres;
|
||||
INSERT INTO pg_dist_node (nodeid, groupid, nodename, nodeport, noderack, hasmetadata, metadatasynced, isactive, noderole, nodecluster, shouldhaveshards) VALUES (1, 1, 'localhost', 57637, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default', TRUE),(2, 2, 'localhost', 57638, 'default', FALSE, FALSE, TRUE, 'primary'::noderole, 'default', TRUE)
|
||||
SELECT citus_internal_add_partition_metadata ('mx_testing_schema.mx_test_table'::regclass, 'h', 'col_1', 0, 's')
|
||||
RESET ROLE
|
||||
RESET ROLE
|
||||
SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''')
|
||||
SELECT citus_internal_add_partition_metadata ('mx_testing_schema.mx_test_table'::regclass, 'h', 'col_1', 2, 's')
|
||||
SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition
|
||||
SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3')
|
||||
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint')
|
||||
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.user_defined_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint')
|
||||
SELECT worker_create_or_alter_role('postgres', null, 'ALTER ROLE postgres SUPERUSER CREATEDB CREATEROLE INHERIT LOGIN REPLICATION BYPASSRLS CONNECTION LIMIT 0 PASSWORD ''md5c53670dddfc3bb4b5675c7872bc2249a'' VALID UNTIL ''2052-05-05 00:00:00-07''')
|
||||
SELECT worker_create_truncate_trigger('mx_testing_schema.mx_test_table')
|
||||
SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition
|
||||
TRUNCATE citus.pg_dist_object
|
||||
TRUNCATE pg_dist_node CASCADE
|
||||
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['mx_testing_schema', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_testing_schema']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
|
||||
SELECT worker_drop_shell_table(logicalrelid::regclass::text) FROM pg_dist_partition
|
||||
SET ROLE postgres
|
||||
SET ROLE postgres
|
||||
SET citus.enable_ddl_propagation TO 'off'
|
||||
SET citus.enable_ddl_propagation TO 'off'
|
||||
SET citus.enable_ddl_propagation TO 'off'
|
||||
SET citus.enable_ddl_propagation TO 'on'
|
||||
SET citus.enable_ddl_propagation TO 'on'
|
||||
SET citus.enable_ddl_propagation TO 'on'
|
||||
UPDATE pg_dist_local_group SET groupid = 1
|
||||
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['mx_testing_schema', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0, false), ('table', ARRAY['mx_testing_schema', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_testing_schema']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
|
||||
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310000, 1, 0, 1, 100000), (1310001, 1, 0, 2, 100001), (1310002, 1, 0, 1, 100002), (1310003, 1, 0, 2, 100003), (1310004, 1, 0, 1, 100004), (1310005, 1, 0, 2, 100005), (1310006, 1, 0, 1, 100006), (1310007, 1, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||
(20 rows)
|
||||
(41 rows)
|
||||
|
||||
-- Test start_metadata_sync_to_node UDF
|
||||
-- Test start_metadata_sync_to_node and citus_activate_node UDFs
|
||||
-- Ensure that hasmetadata=false for all nodes
|
||||
SELECT count(*) FROM pg_dist_node WHERE hasmetadata=true;
|
||||
count
|
||||
|
@ -235,7 +367,7 @@ SELECT count(*) FROM pg_dist_node WHERE hasmetadata=true;
|
|||
0
|
||||
(1 row)
|
||||
|
||||
-- Ensure it works when run on a secondary node
|
||||
-- Show that metadata can not be synced on secondary node
|
||||
SELECT groupid AS worker_1_group FROM pg_dist_node WHERE nodeport = :worker_1_port \gset
|
||||
SELECT master_add_node('localhost', 8888, groupid => :worker_1_group, noderole => 'secondary');
|
||||
master_add_node
|
||||
|
@ -252,7 +384,7 @@ SELECT start_metadata_sync_to_node('localhost', 8888);
|
|||
SELECT hasmetadata FROM pg_dist_node WHERE nodeport = 8888;
|
||||
hasmetadata
|
||||
---------------------------------------------------------------------
|
||||
t
|
||||
f
|
||||
(1 row)
|
||||
|
||||
SELECT stop_metadata_sync_to_node('localhost', 8888);
|
||||
|
@ -275,11 +407,12 @@ SELECT master_add_secondary_node('localhost', 8889, 'localhost', :worker_1_port,
|
|||
5
|
||||
(1 row)
|
||||
|
||||
-- Run start_metadata_sync_to_node and check that it marked hasmetadata for that worker
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
start_metadata_sync_to_node
|
||||
\c - - - :master_port
|
||||
-- Run start_metadata_sync_to_node and citus_activate_node and check that it marked hasmetadata for that worker
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
|
||||
1
|
||||
(1 row)
|
||||
|
||||
SELECT nodeid, hasmetadata FROM pg_dist_node WHERE nodename='localhost' AND nodeport=:worker_1_port;
|
||||
|
@ -306,9 +439,9 @@ SELECT * FROM pg_dist_node ORDER BY nodeid;
|
|||
(4 rows)
|
||||
|
||||
SELECT * FROM pg_dist_partition WHERE logicalrelid::text LIKE 'mx_testing_schema%' ORDER BY logicalrelid;
|
||||
logicalrelid | partmethod | partkey | colocationid | repmodel | autoconverted
|
||||
logicalrelid | partmethod | partkey | colocationid | repmodel | autoconverted
|
||||
---------------------------------------------------------------------
|
||||
mx_testing_schema.mx_test_table | h | {VAR :varno 1 :varattno 1 :vartype 23 :vartypmod -1 :varcollid 0 :varlevelsup 0 :varnoold 1 :varoattno 1 :location -1} | 0 | s | f
|
||||
mx_testing_schema.mx_test_table | h | {VAR :varno 1 :varattno 1 :vartype 23 :vartypmod -1 :varcollid 0 :varlevelsup 0 :varnoold 1 :varoattno 1 :location -1} | 2 | s | f
|
||||
(1 row)
|
||||
|
||||
SELECT * FROM pg_dist_shard WHERE logicalrelid::text LIKE 'mx_testing_schema%' ORDER BY shardid;
|
||||
|
@ -373,7 +506,7 @@ SELECT count(*) FROM pg_trigger WHERE tgrelid='mx_testing_schema.mx_test_table':
|
|||
1
|
||||
(1 row)
|
||||
|
||||
-- Make sure that start_metadata_sync_to_node considers foreign key constraints
|
||||
-- Make sure that citus_activate_node considers foreign key constraints
|
||||
\c - - - :master_port
|
||||
-- Since we're superuser, we can set the replication model to 'streaming' to
|
||||
-- create some MX tables
|
||||
|
@ -394,10 +527,10 @@ SELECT create_distributed_table('mx_testing_schema_2.fk_test_2', 'col1');
|
|||
|
||||
(1 row)
|
||||
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
start_metadata_sync_to_node
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
|
||||
1
|
||||
(1 row)
|
||||
|
||||
-- Check that foreign key metadata exists on the worker
|
||||
|
@ -412,18 +545,18 @@ SELECT "Constraint", "Definition" FROM table_fkeys WHERE relid='mx_testing_schem
|
|||
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
|
||||
-- Check that repeated calls to citus_activate_node has no side effects
|
||||
\c - - - :master_port
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
start_metadata_sync_to_node
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
|
||||
1
|
||||
(1 row)
|
||||
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
start_metadata_sync_to_node
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
|
||||
1
|
||||
(1 row)
|
||||
|
||||
\c - - - :worker_1_port
|
||||
|
@ -443,9 +576,9 @@ SELECT * FROM pg_dist_node ORDER BY nodeid;
|
|||
(4 rows)
|
||||
|
||||
SELECT * FROM pg_dist_partition WHERE logicalrelid::text LIKE 'mx_testing_schema%' ORDER BY logicalrelid;
|
||||
logicalrelid | partmethod | partkey | colocationid | repmodel | autoconverted
|
||||
logicalrelid | partmethod | partkey | colocationid | repmodel | autoconverted
|
||||
---------------------------------------------------------------------
|
||||
mx_testing_schema.mx_test_table | h | {VAR :varno 1 :varattno 1 :vartype 23 :vartypmod -1 :varcollid 0 :varlevelsup 0 :varnoold 1 :varoattno 1 :location -1} | 0 | s | f
|
||||
mx_testing_schema.mx_test_table | h | {VAR :varno 1 :varattno 1 :vartype 23 :vartypmod -1 :varcollid 0 :varlevelsup 0 :varnoold 1 :varoattno 1 :location -1} | 2 | s | f
|
||||
(1 row)
|
||||
|
||||
SELECT * FROM pg_dist_shard WHERE logicalrelid::text LIKE 'mx_testing_schema%' ORDER BY shardid;
|
||||
|
@ -503,13 +636,13 @@ SELECT count(*) FROM pg_trigger WHERE tgrelid='mx_testing_schema.mx_test_table':
|
|||
1
|
||||
(1 row)
|
||||
|
||||
-- Make sure that start_metadata_sync_to_node can be called inside a transaction and rollbacked
|
||||
-- Make sure that citus_activate_node can be called inside a transaction and rollbacked
|
||||
\c - - - :master_port
|
||||
BEGIN;
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_port);
|
||||
start_metadata_sync_to_node
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_2_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
|
||||
1
|
||||
(1 row)
|
||||
|
||||
ROLLBACK;
|
||||
|
@ -522,10 +655,10 @@ 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);
|
||||
start_metadata_sync_to_node
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
|
||||
1
|
||||
(1 row)
|
||||
|
||||
CREATE TABLE mx_query_test (a int, b text, c int);
|
||||
|
@ -996,10 +1129,10 @@ SELECT create_distributed_table('mx_table_with_small_sequence', 'a');
|
|||
|
||||
(1 row)
|
||||
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
start_metadata_sync_to_node
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
|
||||
1
|
||||
(1 row)
|
||||
|
||||
DROP TABLE mx_table_with_small_sequence;
|
||||
|
@ -1105,10 +1238,10 @@ SELECT nextval('mx_table_with_sequence_c_seq');
|
|||
|
||||
-- Check that adding a new metadata node sets the sequence space correctly
|
||||
\c - - - :master_port
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_port);
|
||||
start_metadata_sync_to_node
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_2_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
|
||||
1
|
||||
(1 row)
|
||||
|
||||
\c - - - :worker_2_port
|
||||
|
@ -1224,7 +1357,7 @@ DROP TABLE mx_table_with_small_sequence, mx_table_with_sequence;
|
|||
-- Check that MX sequences play well with non-super users
|
||||
\c - - - :master_port
|
||||
-- Remove a node so that shards and sequences won't be created on table creation. Therefore,
|
||||
-- we can test that start_metadata_sync_to_node can actually create the sequence with proper
|
||||
-- we can test that citus_activate_node can actually create the sequence with proper
|
||||
-- owner
|
||||
CREATE TABLE pg_dist_placement_temp AS SELECT * FROM pg_dist_placement;
|
||||
CREATE TABLE pg_dist_partition_temp AS SELECT * FROM pg_dist_partition;
|
||||
|
@ -1268,12 +1401,6 @@ SELECT master_add_node('localhost', :worker_2_port);
|
|||
6
|
||||
(1 row)
|
||||
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_port);
|
||||
start_metadata_sync_to_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
\c - mx_user - :worker_1_port
|
||||
SELECT nextval('mx_table_b_seq');
|
||||
nextval
|
||||
|
@ -1674,12 +1801,6 @@ SELECT master_add_node('localhost', :worker_2_port);
|
|||
7
|
||||
(1 row)
|
||||
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_port);
|
||||
start_metadata_sync_to_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
CREATE SEQUENCE mx_test_sequence_0;
|
||||
CREATE SEQUENCE mx_test_sequence_1;
|
||||
-- test create_distributed_table
|
||||
|
@ -1697,8 +1818,8 @@ ERROR: cannot execute ALTER TABLE command involving partition column
|
|||
ALTER TABLE test_table ADD COLUMN id2 int DEFAULT nextval('mx_test_sequence_1');
|
||||
ALTER TABLE test_table ALTER COLUMN id2 DROP DEFAULT;
|
||||
ALTER TABLE test_table ALTER COLUMN id2 SET DEFAULT nextval('mx_test_sequence_1');
|
||||
SELECT unnest(master_metadata_snapshot()) order by 1;
|
||||
unnest
|
||||
SELECT unnest(activate_node_snapshot()) order by 1;
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
ALTER SEQUENCE mx_testing_schema.mx_test_table_col_3_seq OWNER TO postgres
|
||||
ALTER SEQUENCE public.mx_test_sequence_0 OWNER TO postgres
|
||||
|
@ -1707,33 +1828,44 @@ SELECT unnest(master_metadata_snapshot()) order by 1;
|
|||
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
|
||||
ALTER TABLE mx_test_schema_1.mx_table_1 ADD CONSTRAINT mx_table_1_col1_key UNIQUE (col1)
|
||||
ALTER TABLE mx_test_schema_1.mx_table_1 OWNER TO postgres
|
||||
ALTER TABLE mx_test_schema_1.mx_table_1 OWNER TO postgres
|
||||
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)
|
||||
ALTER TABLE mx_test_schema_2.mx_table_2 ADD CONSTRAINT mx_table_2_col1_key UNIQUE (col1)
|
||||
ALTER TABLE mx_test_schema_2.mx_table_2 OWNER TO postgres
|
||||
ALTER TABLE mx_test_schema_2.mx_table_2 OWNER TO postgres
|
||||
ALTER TABLE mx_testing_schema.mx_test_table ADD CONSTRAINT mx_test_table_col_1_key UNIQUE (col_1)
|
||||
ALTER TABLE mx_testing_schema.mx_test_table OWNER TO postgres
|
||||
ALTER TABLE mx_testing_schema.mx_test_table OWNER TO postgres
|
||||
ALTER TABLE public.dist_table_1 OWNER TO postgres
|
||||
ALTER TABLE public.dist_table_1 OWNER TO postgres
|
||||
ALTER TABLE public.mx_ref OWNER TO postgres
|
||||
ALTER TABLE public.mx_ref OWNER TO postgres
|
||||
ALTER TABLE public.test_table OWNER TO postgres
|
||||
ALTER TABLE public.test_table OWNER TO postgres
|
||||
CREATE INDEX mx_index ON mx_testing_schema.mx_test_table USING btree (col_2)
|
||||
CREATE INDEX mx_index_1 ON mx_test_schema_1.mx_table_1 USING btree (col1)
|
||||
CREATE INDEX mx_index_2 ON mx_test_schema_2.mx_table_2 USING btree (col2)
|
||||
CREATE SCHEMA IF NOT EXISTS mx_test_schema_1 AUTHORIZATION postgres
|
||||
CREATE SCHEMA IF NOT EXISTS mx_test_schema_2 AUTHORIZATION postgres
|
||||
CREATE SCHEMA IF NOT EXISTS mx_testing_schema AUTHORIZATION postgres
|
||||
CREATE SCHEMA IF NOT EXISTS mx_testing_schema_2 AUTHORIZATION postgres
|
||||
CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION postgres
|
||||
CREATE TABLE mx_test_schema_1.mx_table_1 (col1 integer, col2 text, col3 integer)
|
||||
CREATE TABLE mx_test_schema_2.mx_table_2 (col1 integer, col2 text)
|
||||
CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass))
|
||||
CREATE TABLE public.dist_table_1 (a integer)
|
||||
CREATE TABLE public.mx_ref (col_1 integer, col_2 text)
|
||||
CREATE TABLE public.test_table (id integer DEFAULT worker_nextval('public.mx_test_sequence_0'::regclass), id2 integer DEFAULT worker_nextval('public.mx_test_sequence_1'::regclass))
|
||||
DELETE FROM citus.pg_dist_object
|
||||
DELETE FROM pg_dist_node
|
||||
DELETE FROM pg_dist_partition
|
||||
DELETE FROM pg_dist_placement
|
||||
DELETE FROM pg_dist_shard
|
||||
GRANT CREATE ON SCHEMA public TO PUBLIC;
|
||||
GRANT CREATE ON SCHEMA public TO postgres;
|
||||
GRANT USAGE ON SCHEMA public TO PUBLIC;
|
||||
GRANT USAGE ON SCHEMA public TO postgres;
|
||||
INSERT INTO pg_dist_node (nodeid, groupid, nodename, nodeport, noderack, hasmetadata, metadatasynced, isactive, noderole, nodecluster, shouldhaveshards) VALUES (4, 1, 'localhost', 8888, 'default', FALSE, FALSE, TRUE, 'secondary'::noderole, 'default', TRUE),(5, 1, 'localhost', 8889, 'default', FALSE, FALSE, TRUE, 'secondary'::noderole, 'second-cluster', TRUE),(1, 1, 'localhost', 57637, 'default', TRUE, TRUE, TRUE, 'primary'::noderole, 'default', TRUE),(7, 5, 'localhost', 57638, 'default', TRUE, TRUE, TRUE, 'primary'::noderole, 'default', TRUE)
|
||||
SELECT citus_internal_add_partition_metadata ('mx_test_schema_1.mx_table_1'::regclass, 'h', 'col1', 3, 's')
|
||||
SELECT citus_internal_add_partition_metadata ('mx_test_schema_2.mx_table_2'::regclass, 'h', 'col1', 3, 's')
|
||||
SELECT citus_internal_add_partition_metadata ('mx_testing_schema.mx_test_table'::regclass, 'h', 'col_1', 0, 's')
|
||||
RESET ROLE
|
||||
RESET ROLE
|
||||
SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''')
|
||||
SELECT citus_internal_add_partition_metadata ('mx_test_schema_1.mx_table_1'::regclass, 'h', 'col1', 4, 's')
|
||||
SELECT citus_internal_add_partition_metadata ('mx_test_schema_2.mx_table_2'::regclass, 'h', 'col1', 4, 's')
|
||||
SELECT citus_internal_add_partition_metadata ('mx_testing_schema.mx_test_table'::regclass, 'h', 'col_1', 2, 's')
|
||||
SELECT citus_internal_add_partition_metadata ('public.dist_table_1'::regclass, 'h', 'a', 10005, 's')
|
||||
SELECT citus_internal_add_partition_metadata ('public.mx_ref'::regclass, 'n', NULL, 10003, 't')
|
||||
SELECT citus_internal_add_partition_metadata ('public.test_table'::regclass, 'h', 'id', 10005, 's')
|
||||
|
@ -1743,16 +1875,24 @@ SELECT unnest(master_metadata_snapshot()) order by 1;
|
|||
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.mx_test_sequence_0 AS integer INCREMENT BY 1 MINVALUE 1 MAXVALUE 2147483647 START WITH 1 CACHE 1 NO CYCLE','integer')
|
||||
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.mx_test_sequence_1 AS integer INCREMENT BY 1 MINVALUE 1 MAXVALUE 2147483647 START WITH 1 CACHE 1 NO CYCLE','integer')
|
||||
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.user_defined_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint')
|
||||
SELECT worker_create_or_alter_role('postgres', null, 'ALTER ROLE postgres SUPERUSER CREATEDB CREATEROLE INHERIT LOGIN REPLICATION BYPASSRLS CONNECTION LIMIT 0 PASSWORD ''md5c53670dddfc3bb4b5675c7872bc2249a'' VALID UNTIL ''2052-05-05 00:00:00-07''')
|
||||
SELECT worker_create_truncate_trigger('mx_test_schema_1.mx_table_1')
|
||||
SELECT worker_create_truncate_trigger('mx_test_schema_2.mx_table_2')
|
||||
SELECT worker_create_truncate_trigger('mx_testing_schema.mx_test_table')
|
||||
SELECT worker_create_truncate_trigger('public.dist_table_1')
|
||||
SELECT worker_create_truncate_trigger('public.mx_ref')
|
||||
SELECT worker_create_truncate_trigger('public.test_table')
|
||||
SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition
|
||||
TRUNCATE citus.pg_dist_object
|
||||
TRUNCATE pg_dist_node CASCADE
|
||||
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['mx_testing_schema', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['public', 'mx_test_sequence_0']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['public', 'mx_test_sequence_1']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_testing_schema']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_testing_schema_2']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_test_schema_1']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_test_schema_2']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
|
||||
SELECT worker_drop_shell_table(logicalrelid::regclass::text) FROM pg_dist_partition
|
||||
SET ROLE postgres
|
||||
SET ROLE postgres
|
||||
SET citus.enable_ddl_propagation TO 'off'
|
||||
SET citus.enable_ddl_propagation TO 'off'
|
||||
SET citus.enable_ddl_propagation TO 'off'
|
||||
SET citus.enable_ddl_propagation TO 'on'
|
||||
SET citus.enable_ddl_propagation TO 'on'
|
||||
SET citus.enable_ddl_propagation TO 'on'
|
||||
UPDATE pg_dist_local_group SET groupid = 1
|
||||
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['mx_testing_schema', 'mx_test_table_col_3_seq']::text[], ARRAY[]::text[], -1, 0, false), ('table', ARRAY['mx_testing_schema', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false), ('table', ARRAY['mx_test_schema_1', 'mx_table_1']::text[], ARRAY[]::text[], -1, 0, false), ('table', ARRAY['mx_test_schema_2', 'mx_table_2']::text[], ARRAY[]::text[], -1, 0, false), ('table', ARRAY['public', 'mx_ref']::text[], ARRAY[]::text[], -1, 0, false), ('table', ARRAY['public', 'dist_table_1']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['public', 'mx_test_sequence_0']::text[], ARRAY[]::text[], -1, 0, false), ('sequence', ARRAY['public', 'mx_test_sequence_1']::text[], ARRAY[]::text[], -1, 0, false), ('table', ARRAY['public', 'test_table']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_testing_schema']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_testing_schema_2']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_test_schema_1']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_test_schema_2']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
|
||||
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310000, 1, 0, 1, 100000), (1310001, 1, 0, 5, 100001), (1310002, 1, 0, 1, 100002), (1310003, 1, 0, 5, 100003), (1310004, 1, 0, 1, 100004), (1310005, 1, 0, 5, 100005), (1310006, 1, 0, 1, 100006), (1310007, 1, 0, 5, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310020, 1, 0, 1, 100020), (1310021, 1, 0, 5, 100021), (1310022, 1, 0, 1, 100022), (1310023, 1, 0, 5, 100023), (1310024, 1, 0, 1, 100024)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS (VALUES (1310025, 1, 0, 1, 100025), (1310026, 1, 0, 5, 100026), (1310027, 1, 0, 1, 100027), (1310028, 1, 0, 5, 100028), (1310029, 1, 0, 1, 100029)) SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
|
||||
|
@ -1765,7 +1905,7 @@ SELECT unnest(master_metadata_snapshot()) order by 1;
|
|||
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.dist_table_1'::regclass, 1310074, 't'::"char", '-2147483648', '-1073741825'), ('public.dist_table_1'::regclass, 1310075, 't'::"char", '-1073741824', '-1'), ('public.dist_table_1'::regclass, 1310076, 't'::"char", '0', '1073741823'), ('public.dist_table_1'::regclass, 1310077, 't'::"char", '1073741824', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_ref'::regclass, 1310073, 't'::"char", NULL, NULL)) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.test_table'::regclass, 1310083, 't'::"char", '-2147483648', '-1073741825'), ('public.test_table'::regclass, 1310084, 't'::"char", '-1073741824', '-1'), ('public.test_table'::regclass, 1310085, 't'::"char", '0', '1073741823'), ('public.test_table'::regclass, 1310086, 't'::"char", '1073741824', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
|
||||
(65 rows)
|
||||
(84 rows)
|
||||
|
||||
-- shouldn't work since test_table is MX
|
||||
ALTER TABLE test_table ADD COLUMN id3 bigserial;
|
||||
|
@ -1784,13 +1924,16 @@ ALTER TABLE test_table ADD COLUMN id4 bigserial CHECK (id4 > 0);
|
|||
ERROR: cannot execute ADD COLUMN commands involving serial pseudotypes when metadata is synchronized to workers
|
||||
\c - - - :worker_1_port
|
||||
\ds
|
||||
List of relations
|
||||
Schema | Name | Type | Owner
|
||||
List of relations
|
||||
Schema | Name | Type | Owner
|
||||
---------------------------------------------------------------------
|
||||
public | mx_test_sequence_0 | sequence | postgres
|
||||
public | mx_test_sequence_1 | sequence | postgres
|
||||
public | user_defined_seq | sequence | postgres
|
||||
(3 rows)
|
||||
public | mx_test_sequence_0 | sequence | postgres
|
||||
public | mx_test_sequence_1 | sequence | postgres
|
||||
public | mx_test_table_col_3_seq | sequence | postgres
|
||||
public | sequence_rollback | sequence | postgres
|
||||
public | sequence_rollback(citus_backup_0) | sequence | postgres
|
||||
public | user_defined_seq | sequence | postgres
|
||||
(6 rows)
|
||||
|
||||
\c - - - :master_port
|
||||
CREATE SEQUENCE local_sequence;
|
||||
|
@ -1802,11 +1945,14 @@ DETAIL: drop cascades to default value for column id2 of table test_table
|
|||
drop cascades to default value for column id of table test_table
|
||||
\c - - - :worker_1_port
|
||||
\ds
|
||||
List of relations
|
||||
Schema | Name | Type | Owner
|
||||
List of relations
|
||||
Schema | Name | Type | Owner
|
||||
---------------------------------------------------------------------
|
||||
public | user_defined_seq | sequence | postgres
|
||||
(1 row)
|
||||
public | mx_test_table_col_3_seq | sequence | postgres
|
||||
public | sequence_rollback | sequence | postgres
|
||||
public | sequence_rollback(citus_backup_0) | sequence | postgres
|
||||
public | user_defined_seq | sequence | postgres
|
||||
(4 rows)
|
||||
|
||||
\c - - - :master_port
|
||||
DROP TABLE test_table CASCADE;
|
||||
|
@ -1917,36 +2063,43 @@ SELECT run_command_on_workers('GRANT USAGE ON SCHEMA mx_test_schema_2 TO non_sup
|
|||
(2 rows)
|
||||
|
||||
SET ROLE non_super_metadata_user;
|
||||
-- user must be super user stop/start metadata
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
ERROR: operation is not allowed
|
||||
HINT: Run the command with a superuser.
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
ERROR: operation is not allowed
|
||||
HINT: Run the command with a superuser.
|
||||
RESET ROLE;
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
NOTICE: dropping metadata on the node (localhost,57637)
|
||||
stop_metadata_sync_to_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
(1 row)
|
||||
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
start_metadata_sync_to_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
(1 row)
|
||||
|
||||
RESET ROLE;
|
||||
RESET citus.shard_count;
|
||||
RESET citus.shard_replication_factor;
|
||||
ALTER SEQUENCE pg_catalog.pg_dist_groupid_seq RESTART :last_group_id;
|
||||
ALTER SEQUENCE pg_catalog.pg_dist_node_nodeid_seq RESTART :last_node_id;
|
||||
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART :last_colocation_id;
|
||||
ALTER SEQUENCE pg_catalog.pg_dist_placement_placementid_seq RESTART :last_placement_id;
|
||||
-- Turn metadata sync back on at the end
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
start_metadata_sync_to_node
|
||||
-- Activate them at the end
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
|
||||
1
|
||||
(1 row)
|
||||
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_port);
|
||||
start_metadata_sync_to_node
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_2_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
|
||||
1
|
||||
(1 row)
|
||||
|
||||
|
|
|
@ -49,11 +49,17 @@ SET citus.shard_count TO 8;
|
|||
SET citus.next_shard_id TO 7000000;
|
||||
SET citus.next_placement_id TO 7000000;
|
||||
SET client_min_messages TO WARNING;
|
||||
-- test that coordinator pg_dist_node entry is synced to the workers
|
||||
SELECT wait_until_metadata_sync(30000);
|
||||
wait_until_metadata_sync
|
||||
-- test that metadata is synced to the workers
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_2_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
SELECT verify_metadata('localhost', :worker_1_port),
|
||||
|
@ -335,11 +341,17 @@ SELECT master_remove_node('localhost', :master_port);
|
|||
|
||||
(1 row)
|
||||
|
||||
-- test that coordinator pg_dist_node entry was removed from the workers
|
||||
SELECT wait_until_metadata_sync(30000);
|
||||
wait_until_metadata_sync
|
||||
-- test that metadata is synced to the workers
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_2_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
SELECT verify_metadata('localhost', :worker_1_port),
|
||||
|
|
|
@ -4301,10 +4301,10 @@ WHERE schemaname = 'partitioning_schema' AND tablename ilike '%part_table_with_%
|
|||
(2 rows)
|
||||
|
||||
-- should work properly - no names clashes
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
start_metadata_sync_to_node
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
|
||||
1
|
||||
(1 row)
|
||||
|
||||
\c - - - :worker_1_port
|
||||
|
|
|
@ -130,72 +130,6 @@ ALTER SEQUENCE seq_0 AS bigint;
|
|||
ERROR: Altering a distributed sequence is currently not supported.
|
||||
ALTER SEQUENCE seq_0_local_table AS bigint;
|
||||
ERROR: Altering a distributed sequence is currently not supported.
|
||||
-- we can change other things like increment
|
||||
-- if metadata is not synced to workers
|
||||
BEGIN;
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
NOTICE: dropping metadata on the node (localhost,57637)
|
||||
stop_metadata_sync_to_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_2_port);
|
||||
NOTICE: dropping metadata on the node (localhost,57638)
|
||||
stop_metadata_sync_to_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
CREATE SEQUENCE seq_13;
|
||||
CREATE SEQUENCE seq_13_local_table;
|
||||
CREATE TABLE seq_test_13 (x int, y int);
|
||||
CREATE TABLE seq_test_13_local_table (x int, y int);
|
||||
SELECT create_distributed_table('seq_test_13','x');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT citus_add_local_table_to_metadata('seq_test_13_local_table');
|
||||
citus_add_local_table_to_metadata
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
ALTER TABLE seq_test_13 ADD COLUMN z int DEFAULT nextval('seq_13');
|
||||
ALTER TABLE seq_test_13_local_table ADD COLUMN z int DEFAULT nextval('seq_13_local_table');
|
||||
ALTER SEQUENCE seq_13 INCREMENT BY 2;
|
||||
ALTER SEQUENCE seq_13_local_table INCREMENT BY 2;
|
||||
\d seq_13
|
||||
Sequence "sequence_default.seq_13"
|
||||
Type | Start | Minimum | Maximum | Increment | Cycles? | Cache
|
||||
---------------------------------------------------------------------
|
||||
integer | 1 | 1 | 2147483647 | 2 | no | 1
|
||||
|
||||
\d seq_13_local_table
|
||||
Sequence "sequence_default.seq_13_local_table"
|
||||
Type | Start | Minimum | Maximum | Increment | Cycles? | Cache
|
||||
---------------------------------------------------------------------
|
||||
integer | 1 | 1 | 2147483647 | 2 | no | 1
|
||||
|
||||
-- check that we can add serial pseudo-type columns
|
||||
-- when metadata is not synced to workers
|
||||
TRUNCATE seq_test_0;
|
||||
ALTER TABLE seq_test_0 ADD COLUMN w00 smallserial;
|
||||
ALTER TABLE seq_test_0 ADD COLUMN w01 serial2;
|
||||
ALTER TABLE seq_test_0 ADD COLUMN w10 serial;
|
||||
ALTER TABLE seq_test_0 ADD COLUMN w11 serial4;
|
||||
ALTER TABLE seq_test_0 ADD COLUMN w20 bigserial;
|
||||
ALTER TABLE seq_test_0 ADD COLUMN w21 serial8;
|
||||
TRUNCATE seq_test_0_local_table;
|
||||
ALTER TABLE seq_test_0_local_table ADD COLUMN w00 smallserial;
|
||||
ALTER TABLE seq_test_0_local_table ADD COLUMN w01 serial2;
|
||||
ALTER TABLE seq_test_0_local_table ADD COLUMN w10 serial;
|
||||
ALTER TABLE seq_test_0_local_table ADD COLUMN w11 serial4;
|
||||
ALTER TABLE seq_test_0_local_table ADD COLUMN w20 bigserial;
|
||||
ALTER TABLE seq_test_0_local_table ADD COLUMN w21 serial8;
|
||||
ROLLBACK;
|
||||
-- check alter column type precaution
|
||||
ALTER TABLE seq_test_0 ALTER COLUMN z TYPE bigint;
|
||||
ERROR: cannot execute ALTER COLUMN TYPE .. command because the column involves a default coming from a sequence
|
||||
|
@ -216,12 +150,6 @@ SELECT create_distributed_table('seq_test_4','x');
|
|||
|
||||
CREATE SEQUENCE seq_4;
|
||||
ALTER TABLE seq_test_4 ADD COLUMN a bigint DEFAULT nextval('seq_4');
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
start_metadata_sync_to_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
DROP SEQUENCE seq_4 CASCADE;
|
||||
NOTICE: drop cascades to default value for column a of table seq_test_4
|
||||
TRUNCATE seq_test_4;
|
||||
|
@ -770,10 +698,10 @@ SELECT create_reference_table('seq_test_10');
|
|||
INSERT INTO seq_test_10 VALUES (0);
|
||||
CREATE TABLE seq_test_11 (col0 int, col1 bigint DEFAULT nextval('seq_11'::text));
|
||||
-- works but doesn't create seq_11 in the workers
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
start_metadata_sync_to_node
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
|
||||
1
|
||||
(1 row)
|
||||
|
||||
-- works because there is no dependency created between seq_11 and seq_test_10
|
||||
|
@ -812,10 +740,10 @@ SELECT create_distributed_table('seq_test_12', 'col0');
|
|||
|
||||
(1 row)
|
||||
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
start_metadata_sync_to_node
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
|
||||
1
|
||||
(1 row)
|
||||
|
||||
INSERT INTO seq_test_12 VALUES ('hello0') RETURNING *;
|
||||
|
@ -918,10 +846,10 @@ ERROR: nextval: reached maximum value of sequence "seq_14" (32767)
|
|||
\c - - - :master_port
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
SET search_path = sequence_default, public;
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
start_metadata_sync_to_node
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
|
||||
1
|
||||
(1 row)
|
||||
|
||||
SELECT undistribute_table('seq_test_12');
|
||||
|
@ -981,8 +909,92 @@ SELECT nextval('seq_13');
|
|||
ERROR: nextval: reached maximum value of sequence "seq_13" (2147483647)
|
||||
SELECT nextval('seq_14');
|
||||
ERROR: nextval: reached maximum value of sequence "seq_14" (32767)
|
||||
\c - - - :master_port
|
||||
-- Show that sequence and its dependency schema will be propagated if a distributed
|
||||
-- table with default column is added
|
||||
CREATE SCHEMA test_schema_for_sequence_default_propagation;
|
||||
CREATE SEQUENCE test_schema_for_sequence_default_propagation.seq_10;
|
||||
-- Both should return 0 rows
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation.seq_10'::regclass);
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation'::regnamespace);
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
-- Create distributed table with default column to propagate dependencies
|
||||
CREATE TABLE test_seq_dist(a int, x BIGINT DEFAULT nextval('test_schema_for_sequence_default_propagation.seq_10'));
|
||||
SELECT create_distributed_table('test_seq_dist', 'a');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
-- Both sequence and dependency schema should be distributed
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation.seq_10'::regclass);
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(sequence,"{test_schema_for_sequence_default_propagation,seq_10}",{})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation'::regnamespace);
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(schema,{test_schema_for_sequence_default_propagation},{})
|
||||
(1 row)
|
||||
|
||||
-- Show that sequence can stay on the worker node if the transaction is
|
||||
-- rollbacked after distributing the table
|
||||
BEGIN;
|
||||
CREATE SEQUENCE sequence_rollback;
|
||||
CREATE TABLE sequence_rollback_table(id int, val_1 int default nextval('sequence_rollback'));
|
||||
SELECT create_distributed_table('sequence_rollback_table', 'id');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
ROLLBACK;
|
||||
-- Show that there is a sequence on the worker with the sequence type int
|
||||
\c - - - :worker_1_port
|
||||
SELECT seqtypid::regtype, seqmax, seqmin FROM pg_sequence WHERE seqrelid::regclass::text = 'sequence_rollback';
|
||||
seqtypid | seqmax | seqmin
|
||||
---------------------------------------------------------------------
|
||||
integer | 2147483647 | 1
|
||||
(1 row)
|
||||
|
||||
\c - - - :master_port
|
||||
-- Show that we can create a sequence with the same name and different data type
|
||||
BEGIN;
|
||||
CREATE SEQUENCE sequence_rollback;
|
||||
CREATE TABLE sequence_rollback_table(id int, val_1 bigint default nextval('sequence_rollback'));
|
||||
SELECT create_distributed_table('sequence_rollback_table', 'id');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
ROLLBACK;
|
||||
-- Show that existing sequence has been renamed and a new sequence with the same name
|
||||
-- created for another type
|
||||
\c - - - :worker_1_port
|
||||
SELECT seqrelid::regclass, seqtypid::regtype, seqmax, seqmin FROM pg_sequence WHERE seqrelid::regclass::text like '%sequence_rollback%' ORDER BY 1,2;
|
||||
seqrelid | seqtypid | seqmax | seqmin
|
||||
---------------------------------------------------------------------
|
||||
"sequence_rollback(citus_backup_0)" | integer | 2147483647 | 1
|
||||
sequence_rollback | bigint | 562949953421313 | 281474976710657
|
||||
(2 rows)
|
||||
|
||||
\c - - - :master_port
|
||||
-- clean up
|
||||
DROP SCHEMA test_schema_for_sequence_default_propagation CASCADE;
|
||||
NOTICE: drop cascades to 2 other objects
|
||||
DETAIL: drop cascades to sequence test_schema_for_sequence_default_propagation.seq_10
|
||||
drop cascades to default value for column x of table test_seq_dist
|
||||
DROP TABLE test_seq_dist;
|
||||
DROP TABLE sequence_default.seq_test_7_par;
|
||||
SET client_min_messages TO error; -- suppress cascading objects dropping
|
||||
DROP SCHEMA sequence_default CASCADE;
|
||||
|
|
|
@ -262,11 +262,6 @@ SELECT hasmetadata FROM pg_dist_node WHERE nodeport=:worker_2_port;
|
|||
(1 row)
|
||||
|
||||
\c - - - :worker_2_port
|
||||
SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition WHERE logicalrelid::text LIKE 'mx\_%table%';
|
||||
worker_drop_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
SELECT count(*) FROM pg_dist_partition WHERE logicalrelid::text LIKE 'mx\_%table%';
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -416,22 +416,34 @@ SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_
|
|||
localhost | 57638 | t | (extension,{ltree},{})
|
||||
(2 rows)
|
||||
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
NOTICE: dropping metadata on the node (localhost,57637)
|
||||
stop_metadata_sync_to_node
|
||||
-- Show that dropping a distributed table drops the pg_dist_object entry on worker
|
||||
CREATE TABLE extension_schema.table_to_check_object(id int);
|
||||
SELECT create_distributed_table('extension_schema.table_to_check_object', 'id');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_2_port);
|
||||
NOTICE: dropping metadata on the node (localhost,57638)
|
||||
stop_metadata_sync_to_node
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';
|
||||
obj_identifier
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(table,"{extension_schema,table_to_check_object}",{})
|
||||
(1 row)
|
||||
|
||||
-- Show that we don't have any object metadata after stopping syncing
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (table,"{extension_schema,table_to_check_object}",{})
|
||||
localhost | 57638 | t | (table,"{extension_schema,table_to_check_object}",{})
|
||||
(2 rows)
|
||||
|
||||
DROP TABLE extension_schema.table_to_check_object;
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';
|
||||
obj_identifier
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t |
|
||||
|
@ -441,15 +453,3 @@ SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(clas
|
|||
-- Revert the settings for following tests
|
||||
RESET citus.enable_ddl_propagation;
|
||||
RESET citus.shard_replication_factor;
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
start_metadata_sync_to_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_port);
|
||||
start_metadata_sync_to_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
|
|
|
@ -140,10 +140,10 @@ INSERT INTO sensors_news VALUES (DEFAULT, DEFAULT, '2021-01-01') RETURNING *;
|
|||
(1 row)
|
||||
|
||||
\c - - - :master_port
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
start_metadata_sync_to_node
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
|
||||
1
|
||||
(1 row)
|
||||
|
||||
\c - - - :worker_1_port
|
||||
|
|
|
@ -251,24 +251,6 @@ SELECT count(*) > 0 FROM pg_dist_node;
|
|||
f
|
||||
(1 row)
|
||||
|
||||
SELECT count(*) > 0 FROM pg_dist_shard;
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
f
|
||||
(1 row)
|
||||
|
||||
SELECT count(*) > 0 FROM pg_class WHERE relname LIKE 'distributed_table__' AND relnamespace IN (SELECT oid FROM pg_namespace WHERE nspname = 'start_stop_metadata_sync');
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
f
|
||||
(1 row)
|
||||
|
||||
SELECT count(*) > 0 FROM pg_class WHERE relname LIKE 'reference_table__' AND relnamespace IN (SELECT oid FROM pg_namespace WHERE nspname = 'start_stop_metadata_sync');
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
f
|
||||
(1 row)
|
||||
|
||||
\c - - - :master_port
|
||||
SET search_path TO "start_stop_metadata_sync";
|
||||
SELECT * FROM distributed_table_1;
|
||||
|
@ -291,24 +273,6 @@ SELECT count(*) > 0 FROM pg_dist_node;
|
|||
t
|
||||
(1 row)
|
||||
|
||||
SELECT count(*) > 0 FROM pg_dist_shard;
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
t
|
||||
(1 row)
|
||||
|
||||
SELECT count(*) > 0 FROM pg_class WHERE relname LIKE 'distributed_table__' AND relnamespace IN (SELECT oid FROM pg_namespace WHERE nspname = 'start_stop_metadata_sync');
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
t
|
||||
(1 row)
|
||||
|
||||
SELECT count(*) > 0 FROM pg_class WHERE relname LIKE 'reference_table__' AND relnamespace IN (SELECT oid FROM pg_namespace WHERE nspname = 'start_stop_metadata_sync');
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
t
|
||||
(1 row)
|
||||
|
||||
\c - - - :master_port
|
||||
-- test synchronization for pg_dist_node flags
|
||||
SELECT citus_set_node_property('localhost', :worker_2_port, 'shouldhaveshards', false);
|
||||
|
@ -460,24 +424,6 @@ SELECT count(*) > 0 FROM pg_dist_node;
|
|||
f
|
||||
(1 row)
|
||||
|
||||
SELECT count(*) > 0 FROM pg_dist_shard;
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
f
|
||||
(1 row)
|
||||
|
||||
SELECT count(*) > 0 FROM pg_class WHERE relname LIKE 'distributed_table__' AND relnamespace IN (SELECT oid FROM pg_namespace WHERE nspname = 'start_stop_metadata_sync');
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
f
|
||||
(1 row)
|
||||
|
||||
SELECT count(*) > 0 FROM pg_class WHERE relname LIKE 'reference_table__' AND relnamespace IN (SELECT oid FROM pg_namespace WHERE nspname = 'start_stop_metadata_sync');
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
f
|
||||
(1 row)
|
||||
|
||||
\c - - - :master_port
|
||||
SET search_path TO "start_stop_metadata_sync";
|
||||
-- start metadata sync sets the multi-shard modify mode to sequential
|
||||
|
@ -522,9 +468,7 @@ BEGIN;
|
|||
(1 row)
|
||||
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
ERROR: cannot execute metadata syncing operation because there was a parallel operation on a distributed table in the transaction
|
||||
DETAIL: When modifying metadata, Citus needs to perform all operations over a single connection per node to ensure consistency.
|
||||
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
|
||||
ERROR: cannot open new connections after the first modification command within a transaction
|
||||
ROLLBACK;
|
||||
-- this is safe because start_metadata_sync_to_node already switches to
|
||||
-- sequential execution
|
||||
|
@ -561,9 +505,7 @@ BEGIN;
|
|||
(1 row)
|
||||
|
||||
ROLLBACK;
|
||||
-- multi-shard commands are allowed with start_metadata_sync
|
||||
-- as long as the start_metadata_sync_to_node executed
|
||||
-- when it is OK to switch to sequential execution
|
||||
-- multi-shard commands are not allowed with start_metadata_sync
|
||||
BEGIN;
|
||||
-- sync at the start of the tx
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
|
@ -597,15 +539,9 @@ BEGIN;
|
|||
|
||||
-- sync at the end of the tx
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
start_metadata_sync_to_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
ERROR: cannot open new connections after the first modification command within a transaction
|
||||
ROLLBACK;
|
||||
-- multi-shard commands are allowed with start_metadata_sync
|
||||
-- as long as the start_metadata_sync_to_node executed
|
||||
-- when it is OK to switch to sequential execution
|
||||
-- multi-shard commands are not allowed with start_metadata_sync
|
||||
BEGIN;
|
||||
-- sync at the start of the tx
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
|
@ -640,11 +576,7 @@ BEGIN;
|
|||
|
||||
-- sync at the end of the tx
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
start_metadata_sync_to_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
ERROR: cannot open new connections after the first modification command within a transaction
|
||||
ROLLBACK;
|
||||
-- cleanup
|
||||
\c - - - :master_port
|
||||
|
|
|
@ -5,6 +5,12 @@ SELECT pg_reload_conf();
|
|||
t
|
||||
(1 row)
|
||||
|
||||
SELECT pg_sleep(0.1);
|
||||
pg_sleep
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SET client_min_messages TO ERROR;
|
||||
SELECT start_metadata_sync_to_node(nodename, nodeport) FROM pg_dist_node WHERE isactive = 't' and noderole = 'primary';
|
||||
start_metadata_sync_to_node
|
||||
|
|
|
@ -5,6 +5,12 @@ SELECT pg_reload_conf();
|
|||
t
|
||||
(1 row)
|
||||
|
||||
SELECT pg_sleep(0.1);
|
||||
pg_sleep
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SET client_min_messages TO ERROR;
|
||||
SELECT start_metadata_sync_to_node(nodename, nodeport) FROM pg_dist_node WHERE isactive = 't' and noderole = 'primary';
|
||||
start_metadata_sync_to_node
|
||||
|
|
|
@ -5,6 +5,12 @@ SELECT pg_reload_conf();
|
|||
t
|
||||
(1 row)
|
||||
|
||||
SELECT pg_sleep(0.1);
|
||||
pg_sleep
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SET client_min_messages TO ERROR;
|
||||
SELECT start_metadata_sync_to_node(nodename, nodeport) FROM pg_dist_node WHERE isactive = 't' and noderole = 'primary';
|
||||
start_metadata_sync_to_node
|
||||
|
|
|
@ -207,6 +207,7 @@ ORDER BY 1;
|
|||
function worker_create_truncate_trigger(regclass)
|
||||
function worker_drop_distributed_table(text)
|
||||
function worker_drop_sequence_dependency(text)
|
||||
function worker_drop_shell_table(text)
|
||||
function worker_fetch_foreign_file(text,text,bigint,text[],integer[])
|
||||
function worker_fetch_partition_file(bigint,integer,integer,integer,text,integer)
|
||||
function worker_fix_partition_shard_index_names(regclass,text,text)
|
||||
|
@ -266,5 +267,5 @@ ORDER BY 1;
|
|||
view citus_worker_stat_activity
|
||||
view pg_dist_shard_placement
|
||||
view time_partitions
|
||||
(250 rows)
|
||||
(251 rows)
|
||||
|
||||
|
|
|
@ -637,3 +637,23 @@ CREATE TABLE referenced_table(i int UNIQUE);
|
|||
SELECT create_distributed_table('referenced_table', 'i');
|
||||
ALTER TABLE test_table_1 ADD COLUMN test_col int REFERENCES referenced_table(i);
|
||||
DROP TABLE referenced_table, test_table_1;
|
||||
|
||||
-- Check sequence propagate its own dependencies while adding a column
|
||||
CREATE TABLE table_without_sequence(a int);
|
||||
SELECT create_distributed_table('table_without_sequence', 'a');
|
||||
|
||||
CREATE SCHEMA test_schema_for_sequence_propagation;
|
||||
CREATE SEQUENCE test_schema_for_sequence_propagation.seq_10;
|
||||
|
||||
-- Both should have zero rows
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation.seq_10'::regclass);
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation'::regnamespace);
|
||||
|
||||
ALTER TABLE table_without_sequence ADD COLUMN x BIGINT DEFAULT nextval('test_schema_for_sequence_propagation.seq_10');
|
||||
|
||||
-- Should be distributed along with the sequence
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation.seq_10'::regclass);
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation'::regnamespace);
|
||||
|
||||
DROP SCHEMA test_schema_for_sequence_propagation CASCADE;
|
||||
DROP TABLE table_without_sequence;
|
||||
|
|
|
@ -497,7 +497,17 @@ SELECT shardid, nodename, nodeport
|
|||
WHERE logicalrelid = 'numbers_append'::regclass order by placementid;
|
||||
|
||||
-- add the node back
|
||||
SET client_min_messages TO ERROR;
|
||||
-- before adding the node, add pg_dist_object entry for tables created with
|
||||
-- master_create_distributed_table as we don't have the entry for them.
|
||||
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'objects'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'customer_with_special_\\_character'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, '1_customer'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'packed_numbers_hash'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'super_packed_numbers_hash'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table_to_distribute'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'second_dustbunnies'::regclass::oid, 0);
|
||||
|
||||
SELECT 1 FROM master_activate_node('localhost', :worker_1_port);
|
||||
RESET client_min_messages;
|
||||
RESET citus.shard_replication_factor;
|
||||
|
|
|
@ -92,8 +92,8 @@ test: isolation_replicated_dist_on_mx
|
|||
|
||||
# MXless tests
|
||||
test: isolation_check_mx
|
||||
test: isolation_turn_mx_off
|
||||
test: isolation_replicate_reference_tables_to_coordinator
|
||||
test: isolation_turn_mx_off
|
||||
test: isolation_reference_copy_vs_all
|
||||
test: isolation_ref2ref_foreign_keys
|
||||
test: isolation_multiuser_locking
|
||||
|
|
|
@ -1259,3 +1259,43 @@ ALTER TABLE test_table_1 ADD COLUMN test_col int REFERENCES referenced_table(i);
|
|||
ERROR: cannot create foreign key constraint
|
||||
DETAIL: Foreign keys are supported in two cases, either in between two colocated tables including partition column in the same ordinal in the both tables or from distributed to reference tables
|
||||
DROP TABLE referenced_table, test_table_1;
|
||||
-- Check sequence propagate its own dependencies while adding a column
|
||||
CREATE TABLE table_without_sequence(a int);
|
||||
SELECT create_distributed_table('table_without_sequence', 'a');
|
||||
create_distributed_table
|
||||
--------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
CREATE SCHEMA test_schema_for_sequence_propagation;
|
||||
CREATE SEQUENCE test_schema_for_sequence_propagation.seq_10;
|
||||
-- Both should have zero rows
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation.seq_10'::regclass);
|
||||
pg_identify_object_as_address
|
||||
-------------------------------
|
||||
(0 rows)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation'::regnamespace);
|
||||
pg_identify_object_as_address
|
||||
-------------------------------
|
||||
(0 rows)
|
||||
|
||||
ALTER TABLE table_without_sequence ADD COLUMN x BIGINT DEFAULT nextval('test_schema_for_sequence_propagation.seq_10');
|
||||
-- Should be distributed along with the sequence
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation.seq_10'::regclass);
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------
|
||||
(sequence,"{test_schema_for_sequence_propagation,seq_10}",{})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation'::regnamespace);
|
||||
pg_identify_object_as_address
|
||||
----------------------------------------------------
|
||||
(schema,{test_schema_for_sequence_propagation},{})
|
||||
(1 row)
|
||||
|
||||
DROP SCHEMA test_schema_for_sequence_propagation CASCADE;
|
||||
NOTICE: drop cascades to 2 other objects
|
||||
DETAIL: drop cascades to sequence test_schema_for_sequence_propagation.seq_10
|
||||
drop cascades to default value for column x of table table_without_sequence
|
||||
DROP TABLE table_without_sequence;
|
||||
|
|
|
@ -625,7 +625,15 @@ SELECT shardid, nodename, nodeport
|
|||
(6 rows)
|
||||
|
||||
-- add the node back
|
||||
SET client_min_messages TO ERROR;
|
||||
-- before adding the node, add pg_dist_object entry for tables created with
|
||||
-- master_create_distributed_table as we don't have the entry for them.
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'objects'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'customer_with_special_\\_character'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, '1_customer'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'packed_numbers_hash'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'super_packed_numbers_hash'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table_to_distribute'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'second_dustbunnies'::regclass::oid, 0);
|
||||
SELECT 1 FROM master_activate_node('localhost', :worker_1_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -64,6 +64,11 @@ step "s1-get-all-transactions"
|
|||
SELECT initiator_node_identifier, transaction_number, transaction_stamp FROM get_current_transaction_id() ORDER BY 1,2,3;
|
||||
}
|
||||
|
||||
step "s1-drop-table"
|
||||
{
|
||||
DROP TABLE distributed_transaction_id_table;
|
||||
}
|
||||
|
||||
session "s2"
|
||||
|
||||
step "s2-begin"
|
||||
|
@ -119,7 +124,7 @@ permutation "s1-begin" "s1-assign-transaction-id" "s1-get-all-transactions" "s2-
|
|||
|
||||
// now show that distributed transaction id on the coordinator
|
||||
// is the same with the one on the worker
|
||||
permutation "s1-create-table" "s1-begin" "s1-insert" "s1-verify-current-xact-is-on-worker" "s1-commit"
|
||||
permutation "s1-create-table" "s1-begin" "s1-insert" "s1-verify-current-xact-is-on-worker" "s1-drop-table" "s1-commit"
|
||||
|
||||
// we would initially forget the distributed transaction ID on pg_dist_partition invalidations
|
||||
permutation "s1-begin" "s1-assign-transaction-id" "s1-has-transaction-number" "s2-vacuum" "s1-has-transaction-number" "s1-commit"
|
||||
|
|
|
@ -31,16 +31,6 @@ teardown
|
|||
|
||||
session "s1"
|
||||
|
||||
step "increase-retry-interval"
|
||||
{
|
||||
ALTER SYSTEM SET citus.metadata_sync_retry_interval TO 20000;
|
||||
}
|
||||
|
||||
step "reset-retry-interval"
|
||||
{
|
||||
ALTER SYSTEM RESET citus.metadata_sync_retry_interval;
|
||||
}
|
||||
|
||||
step "enable-deadlock-detection"
|
||||
{
|
||||
ALTER SYSTEM SET citus.distributed_deadlock_detection_factor TO 3;
|
||||
|
@ -76,17 +66,6 @@ step "s1-commit"
|
|||
COMMIT;
|
||||
}
|
||||
|
||||
step "s1-count-daemons"
|
||||
{
|
||||
SELECT count(*) FROM pg_stat_activity WHERE application_name LIKE 'Citus Met%';
|
||||
}
|
||||
|
||||
step "s1-cancel-metadata-sync"
|
||||
{
|
||||
SELECT pg_cancel_backend(pid) FROM pg_stat_activity WHERE application_name LIKE 'Citus Met%';
|
||||
SELECT pg_sleep(2);
|
||||
}
|
||||
|
||||
session "s2"
|
||||
|
||||
step "s2-start-session-level-connection"
|
||||
|
@ -147,7 +126,3 @@ step "s3-wait"
|
|||
// themselves involved in a distributed deadlock.
|
||||
// See https://github.com/citusdata/citus/issues/4393 for more details.
|
||||
permutation "enable-deadlock-detection" "reload-conf" "s2-start-session-level-connection" "s1-begin" "s1-update-1" "s2-begin-on-worker" "s2-update-2-on-worker" "s2-truncate-on-worker" "s3-invalidate-metadata" "s3-resync" "s3-wait" "s2-update-1-on-worker" "s1-update-2" "s1-commit" "s2-commit-on-worker" "disable-deadlock-detection" "reload-conf" "s2-stop-connection"
|
||||
|
||||
// Test that when metadata sync is waiting for locks, cancelling it terminates it.
|
||||
// This is important in cases where the metadata sync daemon itself is involved in a deadlock.
|
||||
permutation "increase-retry-interval" "reload-conf" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-truncate-on-worker" "s3-invalidate-metadata" "s3-resync" "s3-wait" "s1-count-daemons" "s1-cancel-metadata-sync" "s1-count-daemons" "reset-retry-interval" "reload-conf" "s2-commit-on-worker" "s2-stop-connection" "s3-resync" "s3-wait"
|
||||
|
|
|
@ -3,7 +3,7 @@ setup
|
|||
SELECT citus_internal.replace_isolation_tester_func();
|
||||
SELECT citus_internal.refresh_isolation_tester_prepared_statement();
|
||||
|
||||
SELECT master_add_node('localhost', 57636);
|
||||
SELECT master_add_node('localhost', 57636, groupid => 0);
|
||||
|
||||
CREATE TABLE ref_table(a int primary key);
|
||||
SELECT create_reference_table('ref_table');
|
||||
|
|
|
@ -135,6 +135,8 @@ permutation "s1-begin" "s1-update-node-1" "s2-begin" "s2-update-node-1" "s1-comm
|
|||
// master_update_node should block start_metadata_sync_to_node. Note that we
|
||||
// cannot run start_metadata_sync_to_node in a transaction, so we're not
|
||||
// testing the reverse order here.
|
||||
// Having different result on coordinator and worker is expected for now since
|
||||
// we run test after disabling mx.
|
||||
permutation "s1-begin" "s1-update-node-1" "s2-start-metadata-sync-node-2" "s1-commit" "s2-verify-metadata"
|
||||
|
||||
// make sure we have entries in prepared statement cache
|
||||
|
|
|
@ -33,9 +33,9 @@ CREATE TRIGGER dummy_function_trigger
|
|||
BEFORE UPDATE OF value ON citus_local_table
|
||||
FOR EACH ROW EXECUTE FUNCTION dummy_function();
|
||||
|
||||
-- Show that we can sync metadata successfully. That means, we create
|
||||
-- Show that we can activate node successfully. That means, we create
|
||||
-- the function that trigger needs in mx workers too.
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
|
||||
CREATE EXTENSION seg;
|
||||
ALTER TRIGGER dummy_function_trigger ON citus_local_table DEPENDS ON EXTENSION seg;
|
||||
|
|
|
@ -19,15 +19,6 @@ CREATE TABLE notices (
|
|||
SELECT create_distributed_table('notices', 'id');
|
||||
INSERT INTO notices VALUES (1, 'hello world');
|
||||
|
||||
-- Create the necessary test utility function
|
||||
CREATE OR REPLACE FUNCTION master_metadata_snapshot()
|
||||
RETURNS text[]
|
||||
LANGUAGE C STRICT
|
||||
AS 'citus';
|
||||
|
||||
COMMENT ON FUNCTION master_metadata_snapshot()
|
||||
IS 'commands to create the metadata snapshot';
|
||||
|
||||
CREATE FUNCTION notice(text)
|
||||
RETURNS void
|
||||
LANGUAGE plpgsql AS $$
|
||||
|
@ -358,9 +349,6 @@ SELECT run_command_on_workers($$SELECT count(*) FROM pg_proc WHERE proname='eq_w
|
|||
-- valid distribution with distribution_arg_name -- case insensitive
|
||||
SELECT create_distributed_function('eq_with_param_names(macaddr, macaddr)', distribution_arg_name:='VaL1');
|
||||
|
||||
-- show that we are able to propagate objects with multiple item on address arrays
|
||||
SELECT * FROM (SELECT unnest(master_metadata_snapshot()) as metadata_command order by 1) as innerResult WHERE metadata_command like '%distributed_object_data%';
|
||||
|
||||
-- valid distribution with distribution_arg_index
|
||||
SELECT create_distributed_function('eq_with_param_names(macaddr, macaddr)','$1');
|
||||
|
||||
|
|
|
@ -20,20 +20,20 @@ SELECT hasmetadata FROM pg_dist_node WHERE nodeport=:worker_2_proxy_port;
|
|||
|
||||
-- Failure to set groupid in the worker
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^UPDATE pg_dist_local_group SET groupid").cancel(' || :pid || ')');
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_proxy_port);
|
||||
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^UPDATE pg_dist_local_group SET groupid").kill()');
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_proxy_port);
|
||||
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
|
||||
|
||||
-- Failure to drop all tables in pg_dist_partition
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT worker_drop_distributed_table").cancel(' || :pid || ')');
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_proxy_port);
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT worker_drop_distributed_table").kill()');
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_proxy_port);
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^DELETE FROM pg_dist_partition").cancel(' || :pid || ')');
|
||||
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^DELETE FROM pg_dist_partition").kill()');
|
||||
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
|
||||
|
||||
-- Failure to truncate pg_dist_node in the worker
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^TRUNCATE pg_dist_node CASCADE").cancel(' || :pid || ')');
|
||||
-- Failure to delete pg_dist_node entries from the worker
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^DELETE FROM pg_dist_node").cancel(' || :pid || ')');
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_proxy_port);
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^TRUNCATE pg_dist_node CASCADE").kill()');
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^DELETE FROM pg_dist_node").kill()');
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_proxy_port);
|
||||
|
||||
-- Failure to populate pg_dist_node in the worker
|
||||
|
@ -45,9 +45,9 @@ SELECT start_metadata_sync_to_node('localhost', :worker_2_proxy_port);
|
|||
-- Verify that coordinator knows worker does not have valid metadata
|
||||
SELECT hasmetadata FROM pg_dist_node WHERE nodeport=:worker_2_proxy_port;
|
||||
|
||||
-- Verify we can sync metadata after unsuccessful attempts
|
||||
-- Verify we can activate node after unsuccessful attempts
|
||||
SELECT citus.mitmproxy('conn.allow()');
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_proxy_port);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_2_proxy_port);
|
||||
SELECT hasmetadata FROM pg_dist_node WHERE nodeport=:worker_2_proxy_port;
|
||||
|
||||
-- Check failures on DDL command propagation
|
||||
|
@ -70,16 +70,10 @@ SELECT stop_metadata_sync_to_node('localhost', :worker_2_proxy_port);
|
|||
SELECT citus.mitmproxy('conn.onQuery(query="^UPDATE pg_dist_local_group SET groupid").kill()');
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_2_proxy_port);
|
||||
|
||||
-- Failure to drop all tables in pg_dist_partition
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT worker_drop_distributed_table").cancel(' || :pid || ')');
|
||||
-- Failure to delete pg_dist_node entries from the worker
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^DELETE FROM pg_dist_node").cancel(' || :pid || ')');
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_2_proxy_port);
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT worker_drop_distributed_table").kill()');
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_2_proxy_port);
|
||||
|
||||
-- Failure to truncate pg_dist_node in the worker
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^TRUNCATE pg_dist_node CASCADE").cancel(' || :pid || ')');
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_2_proxy_port);
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^TRUNCATE pg_dist_node CASCADE").kill()');
|
||||
SELECT citus.mitmproxy('conn.onQuery(query="^DELETE FROM pg_dist_node").kill()');
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_2_proxy_port);
|
||||
|
||||
\c - - - :worker_2_port
|
||||
|
@ -87,7 +81,7 @@ SELECT count(*) FROM pg_dist_node;
|
|||
|
||||
\c - - - :master_port
|
||||
SELECT hasmetadata FROM pg_dist_node WHERE nodeport=:worker_2_proxy_port;
|
||||
-- Verify we can drop metadata after unsuccessful attempts
|
||||
-- Verify we can stop metadata sync after unsuccessful attempts
|
||||
SELECT citus.mitmproxy('conn.allow()');
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_2_proxy_port);
|
||||
|
||||
|
|
|
@ -15,8 +15,9 @@ SET citus.shard_replication_factor TO 1;
|
|||
CREATE TABLE t1 (a int, b int, c int UNIQUE);
|
||||
SELECT create_distributed_table('t1', 'c');
|
||||
ALTER TABLE t1 DROP COLUMN b;
|
||||
SELECT start_metadata_sync_to_node('localhost',:worker_1_port);
|
||||
SELECT start_metadata_sync_to_node('localhost',:worker_2_port);
|
||||
|
||||
SELECT 1 FROM citus_activate_node('localhost',:worker_1_port);
|
||||
SELECT 1 FROM citus_activate_node('localhost',:worker_2_port);
|
||||
|
||||
\c - - - :worker_1_port
|
||||
SET search_path TO local_shard_execution_dropped_column;
|
||||
|
|
|
@ -93,6 +93,8 @@ SELECT count(*) FROM history;
|
|||
SET citus.shard_replication_factor TO 1;
|
||||
|
||||
-- metadata sync will succeed even if we have rep > 1 tables
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles_single_shard'::regclass::oid, 0);
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
|
||||
CREATE TABLE mx_table(a int);
|
||||
|
|
|
@ -1,7 +1,6 @@
|
|||
SET citus.next_shard_id TO 1220000;
|
||||
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 1390000;
|
||||
ALTER SEQUENCE pg_catalog.pg_dist_groupid_seq RESTART 1;
|
||||
SET citus.enable_object_propagation TO off; -- prevent object propagation on add node during setup
|
||||
|
||||
-- Tests functions related to cluster membership
|
||||
|
||||
|
@ -49,7 +48,7 @@ SELECT * FROM rebalance_table_shards();
|
|||
TRUNCATE pg_dist_colocation;
|
||||
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 1390000;
|
||||
|
||||
SELECT * FROM citus_activate_node('localhost', :worker_2_port);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_2_port);
|
||||
|
||||
CREATE TABLE cluster_management_test (col_1 text, col_2 int);
|
||||
SELECT create_distributed_table('cluster_management_test', 'col_1', 'hash');
|
||||
|
@ -85,13 +84,11 @@ SELECT master_activate_node('localhost', :worker_2_port);
|
|||
DROP TABLE test_reference_table, cluster_management_test;
|
||||
|
||||
-- create users like this so results of community and enterprise are same
|
||||
SET citus.enable_object_propagation TO ON;
|
||||
SET client_min_messages TO ERROR;
|
||||
CREATE USER non_super_user;
|
||||
CREATE USER node_metadata_user;
|
||||
SELECT 1 FROM run_command_on_workers('CREATE USER node_metadata_user');
|
||||
RESET client_min_messages;
|
||||
SET citus.enable_object_propagation TO OFF;
|
||||
GRANT EXECUTE ON FUNCTION master_activate_node(text,int) TO node_metadata_user;
|
||||
GRANT EXECUTE ON FUNCTION master_add_inactive_node(text,int,int,noderole,name) TO node_metadata_user;
|
||||
GRANT EXECUTE ON FUNCTION master_add_node(text,int,int,noderole,name) TO node_metadata_user;
|
||||
|
@ -125,7 +122,6 @@ SELECT master_update_node(nodeid, 'localhost', :worker_2_port + 3) FROM pg_dist_
|
|||
|
||||
-- try to manipulate node metadata via privileged user
|
||||
SET ROLE node_metadata_user;
|
||||
SET citus.enable_object_propagation TO off; -- prevent master activate node to actually connect for this test
|
||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||
BEGIN;
|
||||
SELECT 1 FROM master_add_inactive_node('localhost', :worker_2_port);
|
||||
|
@ -137,7 +133,6 @@ ABORT;
|
|||
|
||||
\c - postgres - :master_port
|
||||
SET citus.next_shard_id TO 1220000;
|
||||
SET citus.enable_object_propagation TO off; -- prevent object propagation on add node during setup
|
||||
SET citus.shard_count TO 16;
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
SELECT master_get_active_worker_nodes();
|
||||
|
@ -222,7 +217,6 @@ SELECT master_remove_node('localhost', :worker_2_port);
|
|||
\c - - - :worker_1_port
|
||||
SELECT nodename, nodeport FROM pg_dist_node WHERE nodename='localhost' AND nodeport=:worker_2_port;
|
||||
\c - - - :master_port
|
||||
SET citus.enable_object_propagation TO off; -- prevent object propagation on add node during setup
|
||||
|
||||
-- check that added nodes are not propagated to nodes without metadata
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
|
@ -230,7 +224,6 @@ SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
|||
\c - - - :worker_1_port
|
||||
SELECT nodename, nodeport FROM pg_dist_node WHERE nodename='localhost' AND nodeport=:worker_2_port;
|
||||
\c - - - :master_port
|
||||
SET citus.enable_object_propagation TO off; -- prevent object propagation on add node during setup
|
||||
|
||||
-- check that removing two nodes in the same transaction works
|
||||
SELECT
|
||||
|
@ -266,8 +259,6 @@ SELECT nodename, nodeport FROM pg_dist_node WHERE nodename='localhost' AND nodep
|
|||
SELECT nodename, nodeport FROM pg_dist_node WHERE nodename='localhost' AND nodeport=:worker_2_port;
|
||||
\c - - - :master_port
|
||||
|
||||
SET citus.enable_object_propagation TO off; -- prevent object propagation on add node during setup
|
||||
|
||||
SELECT master_remove_node(nodename, nodeport) FROM pg_dist_node;
|
||||
SELECT 1 FROM master_add_node('localhost', :worker_1_port);
|
||||
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
|
||||
|
@ -303,7 +294,6 @@ DELETE FROM pg_dist_shard;
|
|||
DELETE FROM pg_dist_placement;
|
||||
DELETE FROM pg_dist_node;
|
||||
\c - - - :master_port
|
||||
SET citus.enable_object_propagation TO off; -- prevent object propagation on add node during setup
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_2_port);
|
||||
|
||||
|
|
|
@ -2,6 +2,9 @@
|
|||
SET citus.next_shard_id TO 1300000;
|
||||
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 4;
|
||||
|
||||
-- Delete orphaned entries from pg_dist_colocation
|
||||
DELETE FROM pg_dist_colocation where colocationid = 5 or colocationid = 6;
|
||||
|
||||
-- ===================================================================
|
||||
-- create test utility function
|
||||
-- ===================================================================
|
||||
|
@ -95,6 +98,7 @@ SELECT master_create_worker_shards('table5_groupX', 4, 2);
|
|||
CREATE TABLE table6_append ( id int );
|
||||
SELECT master_create_distributed_table('table6_append', 'id', 'append');
|
||||
SELECT master_create_empty_shard('table6_append');
|
||||
|
||||
SELECT master_create_empty_shard('table6_append');
|
||||
|
||||
|
||||
|
@ -105,7 +109,6 @@ SELECT colocation_test_colocate_tables('table1_group1', 'table2_group1');
|
|||
SELECT get_table_colocation_id('table1_group1');
|
||||
SELECT get_table_colocation_id('table5_groupX');
|
||||
SELECT get_table_colocation_id('table6_append');
|
||||
|
||||
-- check self table co-location
|
||||
SELECT tables_colocated('table1_group1', 'table1_group1');
|
||||
SELECT tables_colocated('table5_groupX', 'table5_groupX');
|
||||
|
@ -120,7 +123,6 @@ SELECT tables_colocated('table1_group1', 'table3_group2');
|
|||
-- check table co-location with invalid co-location group
|
||||
SELECT tables_colocated('table1_group1', 'table5_groupX');
|
||||
SELECT tables_colocated('table1_group1', 'table6_append');
|
||||
|
||||
-- check self shard co-location
|
||||
SELECT shards_colocated(1300000, 1300000);
|
||||
SELECT shards_colocated(1300016, 1300016);
|
||||
|
@ -155,7 +157,6 @@ SELECT find_shard_interval_index(1300001);
|
|||
SELECT find_shard_interval_index(1300002);
|
||||
SELECT find_shard_interval_index(1300003);
|
||||
SELECT find_shard_interval_index(1300016);
|
||||
|
||||
-- check external colocation API
|
||||
|
||||
SELECT count(*) FROM pg_dist_partition WHERE colocationid IN (4, 5);
|
||||
|
@ -409,11 +410,14 @@ SELECT update_distributed_table_colocation('table1_group_none', colocate_with =>
|
|||
SELECT update_distributed_table_colocation('table1_group_none', colocate_with => 'table2_groupE');
|
||||
SELECT update_distributed_table_colocation('table1_group_none', colocate_with => 'table3_groupE');
|
||||
|
||||
-- sync metadata to get rid of inconsistencies in pg_dist tables
|
||||
select stop_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
select stop_metadata_sync_to_node('localhost', :worker_2_port);
|
||||
select start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
select start_metadata_sync_to_node('localhost', :worker_2_port);
|
||||
-- activate nodes to get rid of inconsistencies in pg_dist tables
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table1_group1'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table2_group1'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table3_group2'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table4_group2'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table5_groupX'::regclass::oid, 0);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_2_port);
|
||||
|
||||
-- move a table with a colocation id which is already not in pg_dist_colocation
|
||||
SELECT update_distributed_table_colocation('table1_group_none', colocate_with => 'table2_group_none');
|
||||
|
|
|
@ -212,9 +212,11 @@ SELECT * FROM multi_extension.print_extension_changes();
|
|||
ALTER EXTENSION citus UPDATE TO '9.5-1';
|
||||
|
||||
BEGIN;
|
||||
SET citus.enable_object_propagation TO on;
|
||||
SELECT master_add_node('localhost', :master_port, groupId=>0);
|
||||
CREATE TABLE citus_local_table (a int);
|
||||
SELECT create_citus_local_table('citus_local_table');
|
||||
RESET citus.enable_object_propagation;
|
||||
|
||||
-- downgrade from 9.5-1 to 9.4-1 should fail as we have a citus local table
|
||||
ALTER EXTENSION citus UPDATE TO '9.4-1';
|
||||
|
@ -660,7 +662,6 @@ CREATE EXTENSION citus;
|
|||
CREATE EXTENSION citus;
|
||||
\c - - - :master_port
|
||||
|
||||
SET citus.enable_object_propagation TO off; -- prevent distributed transactions during add node
|
||||
SELECT FROM master_add_node('localhost', :worker_1_port);
|
||||
|
||||
\c - - - :worker_1_port
|
||||
|
|
|
@ -54,7 +54,7 @@ SELECT tablename, indexname FROM pg_indexes WHERE schemaname = 'fix_idx_names' A
|
|||
|
||||
\c - - - :master_port
|
||||
-- this should work properly
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
|
||||
\c - - - :worker_1_port
|
||||
-- we have no clashes
|
||||
|
|
|
@ -20,20 +20,27 @@ SELECT nextval('pg_catalog.pg_dist_groupid_seq') AS last_group_id \gset
|
|||
SELECT nextval('pg_catalog.pg_dist_node_nodeid_seq') AS last_node_id \gset
|
||||
|
||||
-- Create the necessary test utility function
|
||||
CREATE FUNCTION master_metadata_snapshot()
|
||||
CREATE FUNCTION activate_node_snapshot()
|
||||
RETURNS text[]
|
||||
LANGUAGE C STRICT
|
||||
AS 'citus';
|
||||
|
||||
COMMENT ON FUNCTION master_metadata_snapshot()
|
||||
IS 'commands to create the metadata snapshot';
|
||||
COMMENT ON FUNCTION activate_node_snapshot()
|
||||
IS 'commands to activate node 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,
|
||||
-- pg_dist_node entries and reference tables
|
||||
SELECT unnest(master_metadata_snapshot()) order by 1;
|
||||
-- Since password_encryption default has been changed to sha from md5 with PG14
|
||||
-- we are updating it manually just for consistent test results between PG versions.
|
||||
ALTER SYSTEM SET password_encryption TO md5;
|
||||
SELECT pg_reload_conf();
|
||||
SELECT pg_sleep(0.1);
|
||||
ALTER ROLE CURRENT_USER WITH PASSWORD 'dummypassword';
|
||||
|
||||
-- Show that, with no MX tables, activate node snapshot contains only the delete commands,
|
||||
-- pg_dist_node entries, pg_dist_object entries and roles.
|
||||
SELECT unnest(activate_node_snapshot()) order by 1;
|
||||
|
||||
-- this function is dropped in Citus10, added here for tests
|
||||
CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass,
|
||||
|
@ -47,52 +54,48 @@ COMMENT ON FUNCTION pg_catalog.master_create_distributed_table(table_name regcla
|
|||
distribution_method citus.distribution_type)
|
||||
IS 'define the table distribution functions';
|
||||
|
||||
-- this function is dropped in Citus10, added here for tests
|
||||
CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name text, shard_count integer,
|
||||
replication_factor integer DEFAULT 2)
|
||||
RETURNS void
|
||||
AS 'citus', $$master_create_worker_shards$$
|
||||
LANGUAGE C STRICT;
|
||||
|
||||
-- Create a test table with constraints and SERIAL and default from user defined sequence
|
||||
CREATE SEQUENCE user_defined_seq;
|
||||
CREATE TABLE mx_test_table (col_1 int UNIQUE, col_2 text NOT NULL, col_3 BIGSERIAL, col_4 BIGINT DEFAULT nextval('user_defined_seq'));
|
||||
SELECT master_create_distributed_table('mx_test_table', 'col_1', 'hash');
|
||||
SELECT master_create_worker_shards('mx_test_table', 8, 1);
|
||||
set citus.shard_count to 8;
|
||||
set citus.shard_replication_factor to 1;
|
||||
SELECT create_distributed_table('mx_test_table', 'col_1');
|
||||
reset citus.shard_count;
|
||||
reset citus.shard_replication_factor;
|
||||
|
||||
-- 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()) order by 1;
|
||||
-- Show that the created MX table is and its sequences are included in the activate node snapshot
|
||||
SELECT unnest(activate_node_snapshot()) order by 1;
|
||||
|
||||
-- Show that CREATE INDEX commands are included in the metadata snapshot
|
||||
-- Show that CREATE INDEX commands are included in the activate node snapshot
|
||||
CREATE INDEX mx_index ON mx_test_table(col_2);
|
||||
SELECT unnest(master_metadata_snapshot()) order by 1;
|
||||
SELECT unnest(activate_node_snapshot()) order by 1;
|
||||
|
||||
-- Show that schema changes are included in the metadata snapshot
|
||||
-- Show that schema changes are included in the activate node snapshot
|
||||
CREATE SCHEMA mx_testing_schema;
|
||||
ALTER TABLE mx_test_table SET SCHEMA mx_testing_schema;
|
||||
SELECT unnest(master_metadata_snapshot()) order by 1;
|
||||
SELECT unnest(activate_node_snapshot()) order by 1;
|
||||
|
||||
-- Show that append distributed tables are not included in the metadata snapshot
|
||||
-- Show that append distributed tables are not included in the activate node 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()) order by 1;
|
||||
SELECT unnest(activate_node_snapshot()) order by 1;
|
||||
|
||||
-- Show that range distributed tables are not included in the metadata snapshot
|
||||
-- Show that range distributed tables are not included in the activate node snapshot
|
||||
UPDATE pg_dist_partition SET partmethod='r' WHERE logicalrelid='non_mx_test_table'::regclass;
|
||||
SELECT unnest(master_metadata_snapshot()) order by 1;
|
||||
SELECT unnest(activate_node_snapshot()) order by 1;
|
||||
|
||||
|
||||
-- Test start_metadata_sync_to_node UDF
|
||||
-- Test start_metadata_sync_to_node and citus_activate_node UDFs
|
||||
|
||||
-- Ensure that hasmetadata=false for all nodes
|
||||
SELECT count(*) FROM pg_dist_node WHERE hasmetadata=true;
|
||||
|
||||
-- Ensure it works when run on a secondary node
|
||||
-- Show that metadata can not be synced on secondary node
|
||||
SELECT groupid AS worker_1_group FROM pg_dist_node WHERE nodeport = :worker_1_port \gset
|
||||
SELECT master_add_node('localhost', 8888, groupid => :worker_1_group, noderole => 'secondary');
|
||||
SELECT start_metadata_sync_to_node('localhost', 8888);
|
||||
|
@ -103,8 +106,10 @@ SELECT hasmetadata FROM pg_dist_node WHERE nodeport = 8888;
|
|||
-- Add a node to another cluster to make sure it's also synced
|
||||
SELECT master_add_secondary_node('localhost', 8889, 'localhost', :worker_1_port, nodecluster => 'second-cluster');
|
||||
|
||||
-- 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);
|
||||
\c - - - :master_port
|
||||
-- Run start_metadata_sync_to_node and citus_activate_node and check that it marked hasmetadata for that worker
|
||||
SELECT 1 FROM citus_activate_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
|
||||
|
@ -126,7 +131,7 @@ 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
|
||||
-- Make sure that citus_activate_node considers foreign key constraints
|
||||
\c - - - :master_port
|
||||
|
||||
-- Since we're superuser, we can set the replication model to 'streaming' to
|
||||
|
@ -142,7 +147,7 @@ CREATE TABLE mx_testing_schema_2.fk_test_2 (col1 int, col2 int, col3 text,
|
|||
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);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
|
||||
-- Check that foreign key metadata exists on the worker
|
||||
\c - - - :worker_1_port
|
||||
|
@ -154,10 +159,10 @@ 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
|
||||
-- Check that repeated calls to citus_activate_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);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
\c - - - :worker_1_port
|
||||
SELECT * FROM pg_dist_local_group;
|
||||
SELECT * FROM pg_dist_node ORDER BY nodeid;
|
||||
|
@ -171,10 +176,10 @@ SELECT "Column", "Type", "Definition" FROM index_attrs WHERE
|
|||
relid = 'mx_testing_schema.mx_index'::regclass;
|
||||
SELECT count(*) FROM pg_trigger WHERE tgrelid='mx_testing_schema.mx_test_table'::regclass;
|
||||
|
||||
-- Make sure that start_metadata_sync_to_node can be called inside a transaction and rollbacked
|
||||
-- Make sure that citus_activate_node can be called inside a transaction and rollbacked
|
||||
\c - - - :master_port
|
||||
BEGIN;
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_port);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_2_port);
|
||||
ROLLBACK;
|
||||
|
||||
SELECT hasmetadata FROM pg_dist_node WHERE nodeport=:worker_2_port;
|
||||
|
@ -182,7 +187,7 @@ 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);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
|
||||
CREATE TABLE mx_query_test (a int, b text, c int);
|
||||
SELECT create_distributed_table('mx_query_test', 'a');
|
||||
|
@ -441,7 +446,7 @@ SELECT stop_metadata_sync_to_node('localhost', :worker_2_port);
|
|||
-- sync table with serial column after create_distributed_table
|
||||
CREATE TABLE mx_table_with_small_sequence(a int, b SERIAL, c SMALLSERIAL);
|
||||
SELECT create_distributed_table('mx_table_with_small_sequence', 'a');
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
DROP TABLE mx_table_with_small_sequence;
|
||||
|
||||
-- Show that create_distributed_table works with a serial column
|
||||
|
@ -486,7 +491,7 @@ SELECT nextval('mx_table_with_sequence_c_seq');
|
|||
|
||||
-- Check that adding a new metadata node sets the sequence space correctly
|
||||
\c - - - :master_port
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_port);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_2_port);
|
||||
|
||||
\c - - - :worker_2_port
|
||||
SELECT groupid FROM pg_dist_local_group;
|
||||
|
@ -531,7 +536,7 @@ DROP TABLE mx_table_with_small_sequence, mx_table_with_sequence;
|
|||
\c - - - :master_port
|
||||
|
||||
-- Remove a node so that shards and sequences won't be created on table creation. Therefore,
|
||||
-- we can test that start_metadata_sync_to_node can actually create the sequence with proper
|
||||
-- we can test that citus_activate_node can actually create the sequence with proper
|
||||
-- owner
|
||||
CREATE TABLE pg_dist_placement_temp AS SELECT * FROM pg_dist_placement;
|
||||
CREATE TABLE pg_dist_partition_temp AS SELECT * FROM pg_dist_partition;
|
||||
|
@ -557,7 +562,6 @@ SELECT create_distributed_table('mx_table', 'a');
|
|||
|
||||
\c - postgres - :master_port
|
||||
SELECT master_add_node('localhost', :worker_2_port);
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_port);
|
||||
|
||||
\c - mx_user - :worker_1_port
|
||||
SELECT nextval('mx_table_b_seq');
|
||||
|
@ -771,7 +775,6 @@ SELECT pg_reload_conf();
|
|||
UPDATE pg_dist_node SET metadatasynced=true WHERE nodeport=:worker_1_port;
|
||||
|
||||
SELECT master_add_node('localhost', :worker_2_port);
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_port);
|
||||
|
||||
CREATE SEQUENCE mx_test_sequence_0;
|
||||
CREATE SEQUENCE mx_test_sequence_1;
|
||||
|
@ -788,7 +791,7 @@ ALTER TABLE test_table ADD COLUMN id2 int DEFAULT nextval('mx_test_sequence_1');
|
|||
ALTER TABLE test_table ALTER COLUMN id2 DROP DEFAULT;
|
||||
ALTER TABLE test_table ALTER COLUMN id2 SET DEFAULT nextval('mx_test_sequence_1');
|
||||
|
||||
SELECT unnest(master_metadata_snapshot()) order by 1;
|
||||
SELECT unnest(activate_node_snapshot()) order by 1;
|
||||
|
||||
-- shouldn't work since test_table is MX
|
||||
ALTER TABLE test_table ADD COLUMN id3 bigserial;
|
||||
|
@ -852,11 +855,15 @@ SELECT run_command_on_workers('GRANT USAGE ON SCHEMA mx_test_schema_2 TO non_sup
|
|||
|
||||
SET ROLE non_super_metadata_user;
|
||||
|
||||
-- user must be super user stop/start metadata
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
|
||||
RESET ROLE;
|
||||
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
|
||||
RESET citus.shard_count;
|
||||
RESET citus.shard_replication_factor;
|
||||
|
||||
|
@ -865,6 +872,6 @@ ALTER SEQUENCE pg_catalog.pg_dist_node_nodeid_seq RESTART :last_node_id;
|
|||
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART :last_colocation_id;
|
||||
ALTER SEQUENCE pg_catalog.pg_dist_placement_placementid_seq RESTART :last_placement_id;
|
||||
|
||||
-- Turn metadata sync back on at the end
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_port);
|
||||
-- Activate them at the end
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_2_port);
|
||||
|
|
|
@ -31,8 +31,9 @@ SET citus.next_shard_id TO 7000000;
|
|||
SET citus.next_placement_id TO 7000000;
|
||||
SET client_min_messages TO WARNING;
|
||||
|
||||
-- test that coordinator pg_dist_node entry is synced to the workers
|
||||
SELECT wait_until_metadata_sync(30000);
|
||||
-- test that metadata is synced to the workers
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_2_port);
|
||||
|
||||
SELECT verify_metadata('localhost', :worker_1_port),
|
||||
verify_metadata('localhost', :worker_2_port);
|
||||
|
@ -174,8 +175,10 @@ SET client_min_messages TO ERROR;
|
|||
DROP TABLE distributed_table, referece_table, local_fkey_table;
|
||||
SELECT master_remove_node('localhost', :master_port);
|
||||
|
||||
-- test that coordinator pg_dist_node entry was removed from the workers
|
||||
SELECT wait_until_metadata_sync(30000);
|
||||
-- test that metadata is synced to the workers
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_2_port);
|
||||
|
||||
SELECT verify_metadata('localhost', :worker_1_port),
|
||||
verify_metadata('localhost', :worker_2_port);
|
||||
|
||||
|
|
|
@ -1992,7 +1992,7 @@ SELECT tablename, indexname FROM pg_indexes
|
|||
WHERE schemaname = 'partitioning_schema' AND tablename ilike '%part_table_with_%' ORDER BY 1, 2;
|
||||
|
||||
-- should work properly - no names clashes
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
|
||||
\c - - - :worker_1_port
|
||||
-- check that indexes are named properly
|
||||
|
|
|
@ -51,46 +51,6 @@ SELECT * FROM seq_test_0_local_table ORDER BY 1, 2 LIMIT 5;
|
|||
ALTER SEQUENCE seq_0 AS bigint;
|
||||
ALTER SEQUENCE seq_0_local_table AS bigint;
|
||||
|
||||
-- we can change other things like increment
|
||||
-- if metadata is not synced to workers
|
||||
BEGIN;
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_2_port);
|
||||
CREATE SEQUENCE seq_13;
|
||||
CREATE SEQUENCE seq_13_local_table;
|
||||
CREATE TABLE seq_test_13 (x int, y int);
|
||||
CREATE TABLE seq_test_13_local_table (x int, y int);
|
||||
SELECT create_distributed_table('seq_test_13','x');
|
||||
SELECT citus_add_local_table_to_metadata('seq_test_13_local_table');
|
||||
ALTER TABLE seq_test_13 ADD COLUMN z int DEFAULT nextval('seq_13');
|
||||
ALTER TABLE seq_test_13_local_table ADD COLUMN z int DEFAULT nextval('seq_13_local_table');
|
||||
|
||||
ALTER SEQUENCE seq_13 INCREMENT BY 2;
|
||||
ALTER SEQUENCE seq_13_local_table INCREMENT BY 2;
|
||||
\d seq_13
|
||||
\d seq_13_local_table
|
||||
|
||||
|
||||
-- check that we can add serial pseudo-type columns
|
||||
-- when metadata is not synced to workers
|
||||
TRUNCATE seq_test_0;
|
||||
ALTER TABLE seq_test_0 ADD COLUMN w00 smallserial;
|
||||
ALTER TABLE seq_test_0 ADD COLUMN w01 serial2;
|
||||
ALTER TABLE seq_test_0 ADD COLUMN w10 serial;
|
||||
ALTER TABLE seq_test_0 ADD COLUMN w11 serial4;
|
||||
ALTER TABLE seq_test_0 ADD COLUMN w20 bigserial;
|
||||
ALTER TABLE seq_test_0 ADD COLUMN w21 serial8;
|
||||
|
||||
TRUNCATE seq_test_0_local_table;
|
||||
ALTER TABLE seq_test_0_local_table ADD COLUMN w00 smallserial;
|
||||
ALTER TABLE seq_test_0_local_table ADD COLUMN w01 serial2;
|
||||
ALTER TABLE seq_test_0_local_table ADD COLUMN w10 serial;
|
||||
ALTER TABLE seq_test_0_local_table ADD COLUMN w11 serial4;
|
||||
ALTER TABLE seq_test_0_local_table ADD COLUMN w20 bigserial;
|
||||
ALTER TABLE seq_test_0_local_table ADD COLUMN w21 serial8;
|
||||
|
||||
ROLLBACK;
|
||||
|
||||
-- check alter column type precaution
|
||||
ALTER TABLE seq_test_0 ALTER COLUMN z TYPE bigint;
|
||||
ALTER TABLE seq_test_0 ALTER COLUMN z TYPE smallint;
|
||||
|
@ -106,7 +66,6 @@ CREATE TABLE seq_test_4 (x int, y int);
|
|||
SELECT create_distributed_table('seq_test_4','x');
|
||||
CREATE SEQUENCE seq_4;
|
||||
ALTER TABLE seq_test_4 ADD COLUMN a bigint DEFAULT nextval('seq_4');
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
DROP SEQUENCE seq_4 CASCADE;
|
||||
TRUNCATE seq_test_4;
|
||||
CREATE SEQUENCE seq_4;
|
||||
|
@ -381,7 +340,7 @@ SELECT create_reference_table('seq_test_10');
|
|||
INSERT INTO seq_test_10 VALUES (0);
|
||||
CREATE TABLE seq_test_11 (col0 int, col1 bigint DEFAULT nextval('seq_11'::text));
|
||||
-- works but doesn't create seq_11 in the workers
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
-- works because there is no dependency created between seq_11 and seq_test_10
|
||||
SELECT create_distributed_table('seq_test_11', 'col1');
|
||||
-- insertion from workers fails
|
||||
|
@ -403,7 +362,7 @@ CREATE TABLE seq_test_12(col0 text, col1 smallint DEFAULT nextval('seq_12'),
|
|||
col2 int DEFAULT nextval('seq_13'),
|
||||
col3 bigint DEFAULT nextval('seq_14'));
|
||||
SELECT create_distributed_table('seq_test_12', 'col0');
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
INSERT INTO seq_test_12 VALUES ('hello0') RETURNING *;
|
||||
|
||||
\c - - - :worker_1_port
|
||||
|
@ -458,7 +417,7 @@ SELECT nextval('seq_14');
|
|||
\c - - - :master_port
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
SET search_path = sequence_default, public;
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
SELECT undistribute_table('seq_test_12');
|
||||
SELECT create_distributed_table('seq_test_12', 'col0');
|
||||
INSERT INTO seq_test_12 VALUES ('hello2') RETURNING *;
|
||||
|
@ -478,7 +437,53 @@ SELECT nextval('seq_14');
|
|||
|
||||
\c - - - :master_port
|
||||
|
||||
-- Show that sequence and its dependency schema will be propagated if a distributed
|
||||
-- table with default column is added
|
||||
CREATE SCHEMA test_schema_for_sequence_default_propagation;
|
||||
CREATE SEQUENCE test_schema_for_sequence_default_propagation.seq_10;
|
||||
|
||||
-- Both should return 0 rows
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation.seq_10'::regclass);
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation'::regnamespace);
|
||||
|
||||
-- Create distributed table with default column to propagate dependencies
|
||||
CREATE TABLE test_seq_dist(a int, x BIGINT DEFAULT nextval('test_schema_for_sequence_default_propagation.seq_10'));
|
||||
SELECT create_distributed_table('test_seq_dist', 'a');
|
||||
|
||||
-- Both sequence and dependency schema should be distributed
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation.seq_10'::regclass);
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation'::regnamespace);
|
||||
|
||||
-- Show that sequence can stay on the worker node if the transaction is
|
||||
-- rollbacked after distributing the table
|
||||
BEGIN;
|
||||
CREATE SEQUENCE sequence_rollback;
|
||||
CREATE TABLE sequence_rollback_table(id int, val_1 int default nextval('sequence_rollback'));
|
||||
SELECT create_distributed_table('sequence_rollback_table', 'id');
|
||||
ROLLBACK;
|
||||
|
||||
-- Show that there is a sequence on the worker with the sequence type int
|
||||
\c - - - :worker_1_port
|
||||
SELECT seqtypid::regtype, seqmax, seqmin FROM pg_sequence WHERE seqrelid::regclass::text = 'sequence_rollback';
|
||||
|
||||
\c - - - :master_port
|
||||
-- Show that we can create a sequence with the same name and different data type
|
||||
BEGIN;
|
||||
CREATE SEQUENCE sequence_rollback;
|
||||
CREATE TABLE sequence_rollback_table(id int, val_1 bigint default nextval('sequence_rollback'));
|
||||
SELECT create_distributed_table('sequence_rollback_table', 'id');
|
||||
ROLLBACK;
|
||||
|
||||
-- Show that existing sequence has been renamed and a new sequence with the same name
|
||||
-- created for another type
|
||||
\c - - - :worker_1_port
|
||||
SELECT seqrelid::regclass, seqtypid::regtype, seqmax, seqmin FROM pg_sequence WHERE seqrelid::regclass::text like '%sequence_rollback%' ORDER BY 1,2;
|
||||
|
||||
\c - - - :master_port
|
||||
|
||||
-- clean up
|
||||
DROP SCHEMA test_schema_for_sequence_default_propagation CASCADE;
|
||||
DROP TABLE test_seq_dist;
|
||||
DROP TABLE sequence_default.seq_test_7_par;
|
||||
SET client_min_messages TO error; -- suppress cascading objects dropping
|
||||
DROP SCHEMA sequence_default CASCADE;
|
||||
|
|
|
@ -164,7 +164,6 @@ SELECT hasmetadata FROM pg_dist_node WHERE nodeport=:worker_2_port;
|
|||
SELECT stop_metadata_sync_to_node('localhost', :worker_2_port);
|
||||
SELECT hasmetadata FROM pg_dist_node WHERE nodeport=:worker_2_port;
|
||||
\c - - - :worker_2_port
|
||||
SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition WHERE logicalrelid::text LIKE 'mx\_%table%';
|
||||
SELECT count(*) FROM pg_dist_partition WHERE logicalrelid::text LIKE 'mx\_%table%';
|
||||
SELECT count(*) FROM pg_dist_node;
|
||||
\c - - - :worker_1_port
|
||||
|
|
|
@ -177,14 +177,18 @@ SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_
|
|||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{ltree}%';
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{ltree}%';$$) ORDER BY 1,2;
|
||||
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_2_port);
|
||||
-- Show that dropping a distributed table drops the pg_dist_object entry on worker
|
||||
CREATE TABLE extension_schema.table_to_check_object(id int);
|
||||
SELECT create_distributed_table('extension_schema.table_to_check_object', 'id');
|
||||
|
||||
-- Show that we don't have any object metadata after stopping syncing
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object;$$) ORDER BY 1,2;
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';$$) ORDER BY 1,2;
|
||||
|
||||
DROP TABLE extension_schema.table_to_check_object;
|
||||
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';$$) ORDER BY 1,2;
|
||||
|
||||
-- Revert the settings for following tests
|
||||
RESET citus.enable_ddl_propagation;
|
||||
RESET citus.shard_replication_factor;
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_port);
|
||||
|
|
|
@ -55,7 +55,7 @@ INSERT INTO sensors VALUES (DEFAULT, DEFAULT, '2010-01-01') RETURNING *;
|
|||
INSERT INTO sensors_news VALUES (DEFAULT, DEFAULT, '2021-01-01') RETURNING *;
|
||||
|
||||
\c - - - :master_port
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
|
||||
|
||||
\c - - - :worker_1_port
|
||||
|
|
|
@ -131,9 +131,6 @@ SELECT count(*) > 0 FROM pg_class WHERE relname LIKE 'reference_table__' AND rel
|
|||
SET search_path TO "start_stop_metadata_sync";
|
||||
|
||||
SELECT count(*) > 0 FROM pg_dist_node;
|
||||
SELECT count(*) > 0 FROM pg_dist_shard;
|
||||
SELECT count(*) > 0 FROM pg_class WHERE relname LIKE 'distributed_table__' AND relnamespace IN (SELECT oid FROM pg_namespace WHERE nspname = 'start_stop_metadata_sync');
|
||||
SELECT count(*) > 0 FROM pg_class WHERE relname LIKE 'reference_table__' AND relnamespace IN (SELECT oid FROM pg_namespace WHERE nspname = 'start_stop_metadata_sync');
|
||||
|
||||
\c - - - :master_port
|
||||
SET search_path TO "start_stop_metadata_sync";
|
||||
|
@ -144,9 +141,6 @@ COMMIT;
|
|||
|
||||
\c - - - :worker_1_port
|
||||
SELECT count(*) > 0 FROM pg_dist_node;
|
||||
SELECT count(*) > 0 FROM pg_dist_shard;
|
||||
SELECT count(*) > 0 FROM pg_class WHERE relname LIKE 'distributed_table__' AND relnamespace IN (SELECT oid FROM pg_namespace WHERE nspname = 'start_stop_metadata_sync');
|
||||
SELECT count(*) > 0 FROM pg_class WHERE relname LIKE 'reference_table__' AND relnamespace IN (SELECT oid FROM pg_namespace WHERE nspname = 'start_stop_metadata_sync');
|
||||
|
||||
\c - - - :master_port
|
||||
-- test synchronization for pg_dist_node flags
|
||||
|
@ -209,9 +203,6 @@ COMMIT;
|
|||
|
||||
\c - - - :worker_1_port
|
||||
SELECT count(*) > 0 FROM pg_dist_node;
|
||||
SELECT count(*) > 0 FROM pg_dist_shard;
|
||||
SELECT count(*) > 0 FROM pg_class WHERE relname LIKE 'distributed_table__' AND relnamespace IN (SELECT oid FROM pg_namespace WHERE nspname = 'start_stop_metadata_sync');
|
||||
SELECT count(*) > 0 FROM pg_class WHERE relname LIKE 'reference_table__' AND relnamespace IN (SELECT oid FROM pg_namespace WHERE nspname = 'start_stop_metadata_sync');
|
||||
|
||||
\c - - - :master_port
|
||||
SET search_path TO "start_stop_metadata_sync";
|
||||
|
@ -253,9 +244,7 @@ BEGIN;
|
|||
SELECT create_distributed_table('test_table_rep', 'a');
|
||||
ROLLBACK;
|
||||
|
||||
-- multi-shard commands are allowed with start_metadata_sync
|
||||
-- as long as the start_metadata_sync_to_node executed
|
||||
-- when it is OK to switch to sequential execution
|
||||
-- multi-shard commands are not allowed with start_metadata_sync
|
||||
BEGIN;
|
||||
-- sync at the start of the tx
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
|
@ -271,9 +260,7 @@ BEGIN;
|
|||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
ROLLBACK;
|
||||
|
||||
-- multi-shard commands are allowed with start_metadata_sync
|
||||
-- as long as the start_metadata_sync_to_node executed
|
||||
-- when it is OK to switch to sequential execution
|
||||
-- multi-shard commands are not allowed with start_metadata_sync
|
||||
BEGIN;
|
||||
-- sync at the start of the tx
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
|
|
|
@ -1,5 +1,6 @@
|
|||
ALTER SYSTEM SET citus.enable_metadata_sync_by_default TO ON;
|
||||
SELECT pg_reload_conf();
|
||||
SELECT pg_sleep(0.1);
|
||||
|
||||
SET client_min_messages TO ERROR;
|
||||
SELECT start_metadata_sync_to_node(nodename, nodeport) FROM pg_dist_node WHERE isactive = 't' and noderole = 'primary';
|
||||
|
|
Loading…
Reference in New Issue