diff --git a/src/backend/distributed/commands/citus_add_local_table_to_metadata.c b/src/backend/distributed/commands/citus_add_local_table_to_metadata.c index 93f1e7d28ec..d67ee1b65ef 100644 --- a/src/backend/distributed/commands/citus_add_local_table_to_metadata.c +++ b/src/backend/distributed/commands/citus_add_local_table_to_metadata.c @@ -1454,8 +1454,9 @@ InsertMetadataForCitusLocalTable(Oid citusLocalTableId, uint64 shardId, text *shardMinValue = NULL; text *shardMaxValue = NULL; + bool needsSeparateNode = false; InsertShardRow(citusLocalTableId, shardId, shardStorageType, - shardMinValue, shardMaxValue); + shardMinValue, shardMaxValue, needsSeparateNode); List *nodeList = list_make1(CoordinatorNodeIfAddedAsWorkerOrError()); diff --git a/src/backend/distributed/commands/create_distributed_table.c b/src/backend/distributed/commands/create_distributed_table.c index 5ec6d6dd7bb..2e6dcdfad3c 100644 --- a/src/backend/distributed/commands/create_distributed_table.c +++ b/src/backend/distributed/commands/create_distributed_table.c @@ -135,6 +135,7 @@ static char DecideDistTableReplicationModel(char distributionMethod, static List * HashSplitPointsForShardList(List *shardList); static List * HashSplitPointsForShardCount(int shardCount); static List * WorkerNodesForShardList(List *shardList); +static List * NeedsSeparateNodeForShardList(List *shardList); static List * RoundRobinWorkerNodeList(List *workerNodeList, int listLength); static CitusTableParams DecideCitusTableParams(CitusTableType tableType, DistributedTableParams * @@ -572,16 +573,10 @@ CreateDistributedTableConcurrently(Oid relationId, char *distributionColumnName, colocatedTableId = ColocatedTableId(colocationId); } - List *workerNodeList = DistributedTablePlacementNodeList(NoLock); - if (workerNodeList == NIL) - { - ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE), - errmsg("no worker nodes are available for placing shards"), - errhint("Add more worker nodes."))); - } - List *workersForPlacementList; List *shardSplitPointsList; + List *needsSeparateNodeForPlacementList; + if (colocatedTableId != InvalidOid) { @@ -596,6 +591,12 @@ CreateDistributedTableConcurrently(Oid relationId, char *distributionColumnName, * Find the node IDs of the shard placements. */ workersForPlacementList = WorkerNodesForShardList(colocatedShardList); + + /* + * Inherit needsseparatenode from the colocated shards. + */ + needsSeparateNodeForPlacementList = + NeedsSeparateNodeForShardList(colocatedShardList); } else { @@ -607,7 +608,21 @@ CreateDistributedTableConcurrently(Oid relationId, char *distributionColumnName, /* * Place shards in a round-robin fashion across all data nodes. */ + List *workerNodeList = NewDistributedTablePlacementNodeList(NoLock); + if (workerNodeList == NIL) + { + ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("no worker nodes are available for placing shards"), + errhint("Add more worker nodes."))); + } + workersForPlacementList = RoundRobinWorkerNodeList(workerNodeList, shardCount); + + /* + * For a new colocation group, needsseparatenode is set to false for + * all shards. + */ + needsSeparateNodeForPlacementList = GenerateListFromIntElement(false, shardCount); } /* @@ -646,6 +661,7 @@ CreateDistributedTableConcurrently(Oid relationId, char *distributionColumnName, shardToSplit->shardId, shardSplitPointsList, workersForPlacementList, + needsSeparateNodeForPlacementList, distributionColumnOverrides, sourceColocatedShardIntervalList, colocationId @@ -898,6 +914,26 @@ WorkerNodesForShardList(List *shardList) } +/* + * NeedsSeparateNodeForShardList returns a list of node booleans reflecting whether + * each shard in the given list needs a separate node. + */ +static List * +NeedsSeparateNodeForShardList(List *shardList) +{ + List *needsSeparateNodeList = NIL; + + ShardInterval *shardInterval = NULL; + foreach_ptr(shardInterval, shardList) + { + needsSeparateNodeList = lappend_int(needsSeparateNodeList, + shardInterval->needsSeparateNode); + } + + return needsSeparateNodeList; +} + + /* * RoundRobinWorkerNodeList round robins over the workers in the worker node list * and adds node ids to a list of length listLength. diff --git a/src/backend/distributed/metadata/metadata_cache.c b/src/backend/distributed/metadata/metadata_cache.c index 402dedb8a91..9fb8a6467bb 100644 --- a/src/backend/distributed/metadata/metadata_cache.c +++ b/src/backend/distributed/metadata/metadata_cache.c @@ -5379,6 +5379,8 @@ DeformedDistShardTupleToShardInterval(Datum *datumArray, bool *isNullArray, char storageType = DatumGetChar(datumArray[Anum_pg_dist_shard_shardstorage - 1]); Datum minValueTextDatum = datumArray[Anum_pg_dist_shard_shardminvalue - 1]; Datum maxValueTextDatum = datumArray[Anum_pg_dist_shard_shardmaxvalue - 1]; + bool needsSeparateNode = DatumGetBool( + datumArray[Anum_pg_dist_shard_needsseparatenode - 1]); bool minValueNull = isNullArray[Anum_pg_dist_shard_shardminvalue - 1]; bool maxValueNull = isNullArray[Anum_pg_dist_shard_shardmaxvalue - 1]; @@ -5415,6 +5417,7 @@ DeformedDistShardTupleToShardInterval(Datum *datumArray, bool *isNullArray, shardInterval->minValue = minValue; shardInterval->maxValue = maxValue; shardInterval->shardId = shardId; + shardInterval->needsSeparateNode = needsSeparateNode; return shardInterval; } diff --git a/src/backend/distributed/metadata/metadata_sync.c b/src/backend/distributed/metadata/metadata_sync.c index 14f5b462404..29c7474f580 100644 --- a/src/backend/distributed/metadata/metadata_sync.c +++ b/src/backend/distributed/metadata/metadata_sync.c @@ -185,6 +185,7 @@ PG_FUNCTION_INFO_V1(citus_internal_add_tenant_schema); PG_FUNCTION_INFO_V1(citus_internal_delete_tenant_schema); PG_FUNCTION_INFO_V1(citus_internal_update_none_dist_table_metadata); PG_FUNCTION_INFO_V1(citus_internal_database_command); +PG_FUNCTION_INFO_V1(citus_internal_shard_property_set); static bool got_SIGTERM = false; @@ -1267,7 +1268,7 @@ ShardListInsertCommand(List *shardIntervalList) StringInfo insertShardCommand = makeStringInfo(); appendStringInfo(insertShardCommand, "WITH shard_data(relationname, shardid, storagetype, " - "shardminvalue, shardmaxvalue) AS (VALUES "); + "shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES "); foreach_ptr(shardInterval, shardIntervalList) { @@ -1299,12 +1300,13 @@ ShardListInsertCommand(List *shardIntervalList) } appendStringInfo(insertShardCommand, - "(%s::regclass, %ld, '%c'::\"char\", %s, %s)", + "(%s::regclass, %ld, '%c'::\"char\", %s, %s, %s)", quote_literal_cstr(qualifiedRelationName), shardId, shardInterval->storageType, minHashToken->data, - maxHashToken->data); + maxHashToken->data, + shardInterval->needsSeparateNode ? "true" : "false"); if (llast(shardIntervalList) != shardInterval) { @@ -1316,7 +1318,7 @@ ShardListInsertCommand(List *shardIntervalList) appendStringInfo(insertShardCommand, "SELECT citus_internal.add_shard_metadata(relationname, shardid, " - "storagetype, shardminvalue, shardmaxvalue) " + "storagetype, shardminvalue, shardmaxvalue, needsseparatenode) " "FROM shard_data;"); /* @@ -3359,6 +3361,9 @@ citus_internal_add_shard_metadata(PG_FUNCTION_ARGS) shardMaxValue = PG_GETARG_TEXT_P(4); } + PG_ENSURE_ARGNOTNULL(5, "needs separate node"); + bool needsSeparateNode = PG_GETARG_BOOL(5); + /* only owner of the table (or superuser) is allowed to add the Citus metadata */ EnsureTableOwner(relationId); @@ -3379,7 +3384,8 @@ citus_internal_add_shard_metadata(PG_FUNCTION_ARGS) shardMaxValue); } - InsertShardRow(relationId, shardId, storageType, shardMinValue, shardMaxValue); + InsertShardRow(relationId, shardId, storageType, shardMinValue, shardMaxValue, + needsSeparateNode); PG_RETURN_VOID(); } @@ -4067,6 +4073,45 @@ citus_internal_database_command(PG_FUNCTION_ARGS) } +/* + * citus_internal_shard_property_set is an internal UDF to + * set shard properties for all the shards within the shard group + * that given shard belongs to. + */ +Datum +citus_internal_shard_property_set(PG_FUNCTION_ARGS) +{ + CheckCitusVersion(ERROR); + + PG_ENSURE_ARGNOTNULL(0, "shard_id"); + uint64 shardId = PG_GETARG_INT64(0); + + /* only owner of the table (or superuser) is allowed to modify the Citus metadata */ + Oid distributedRelationId = RelationIdForShard(shardId); + EnsureTableOwner(distributedRelationId); + + /* we want to serialize all the metadata changes to this table */ + LockRelationOid(distributedRelationId, ShareUpdateExclusiveLock); + + if (!ShouldSkipMetadataChecks()) + { + EnsureCitusInitiatedOperation(); + } + + bool *needsSeparateNodePtr = NULL; + + if (!PG_ARGISNULL(1)) + { + needsSeparateNodePtr = palloc(sizeof(bool)); + *needsSeparateNodePtr = PG_GETARG_BOOL(1); + } + + ShardgroupSetProperty(shardId, needsSeparateNodePtr); + + PG_RETURN_VOID(); +} + + /* * SyncNewColocationGroup synchronizes a new pg_dist_colocation entry to a worker. */ @@ -4266,6 +4311,24 @@ UpdateNoneDistTableMetadataCommand(Oid relationId, char replicationModel, } +/* + * ShardgroupSetPropertyCommand returns a command to call + * citus_internal.shard_property_set(). + */ +char * +ShardgroupSetPropertyCommand(uint64 shardId, bool *needsSeparateNodePtr) +{ + char *needsSeparateNodeStr = !needsSeparateNodePtr ? "null" : + (*needsSeparateNodePtr ? "true" : "false"); + StringInfo command = makeStringInfo(); + appendStringInfo(command, + "SELECT citus_internal.shard_property_set(%lu, %s)", + shardId, needsSeparateNodeStr); + + return command->data; +} + + /* * AddPlacementMetadataCommand returns a command to call * citus_internal_add_placement_metadata(). diff --git a/src/backend/distributed/metadata/metadata_utility.c b/src/backend/distributed/metadata/metadata_utility.c index 15e167008dd..993fb18dfa8 100644 --- a/src/backend/distributed/metadata/metadata_utility.c +++ b/src/backend/distributed/metadata/metadata_utility.c @@ -48,6 +48,7 @@ #include "pg_version_constants.h" +#include "distributed/argutils.h" #include "distributed/background_jobs.h" #include "distributed/citus_nodes.h" #include "distributed/citus_safe_lib.h" @@ -75,6 +76,7 @@ #include "distributed/remote_commands.h" #include "distributed/resource_lock.h" #include "distributed/shard_rebalancer.h" +#include "distributed/shard_transfer.h" #include "distributed/tuplestore.h" #include "distributed/utils/array_type.h" #include "distributed/version_compat.h" @@ -117,6 +119,7 @@ static void AppendShardIdNameValues(StringInfo selectQuery, ShardInterval *shard static HeapTuple CreateDiskSpaceTuple(TupleDesc tupleDesc, uint64 availableBytes, uint64 totalBytes); static bool GetLocalDiskSpaceStats(uint64 *availableBytes, uint64 *totalBytes); +static void ShardgroupSetPropertyGlobally(uint64 shardId, bool *needsSeparateNodePtr); static BackgroundTask * DeformBackgroundTaskHeapTuple(TupleDesc tupleDescriptor, HeapTuple taskTuple); @@ -139,6 +142,7 @@ PG_FUNCTION_INFO_V1(citus_table_size); PG_FUNCTION_INFO_V1(citus_total_relation_size); PG_FUNCTION_INFO_V1(citus_relation_size); PG_FUNCTION_INFO_V1(citus_shard_sizes); +PG_FUNCTION_INFO_V1(citus_shard_property_set); /* @@ -361,6 +365,169 @@ citus_relation_size(PG_FUNCTION_ARGS) } +/* + * citus_shard_property_set allows setting shard properties for all + * the shards within the shard group that given shard belongs to. + */ +Datum +citus_shard_property_set(PG_FUNCTION_ARGS) +{ + CheckCitusVersion(ERROR); + EnsureCoordinator(); + + PG_ENSURE_ARGNOTNULL(0, "shard_id"); + uint64 shardId = PG_GETARG_INT64(0); + + /* RelationIdForShard() first checks whether the shard id is valid */ + Oid distributedRelationId = RelationIdForShard(shardId); + + List *colocatedTableList = ColocatedTableList(distributedRelationId); + colocatedTableList = SortList(colocatedTableList, CompareOids); + EnsureTableListOwner(colocatedTableList); + + AcquirePlacementColocationLock(distributedRelationId, ExclusiveLock, + "set a property for a shard of"); + + Oid colocatedTableId = InvalidOid; + foreach_oid(colocatedTableId, colocatedTableList) + { + /* + * Prevent relations from being dropped while we are setting the + * property. + */ + LockRelationOid(colocatedTableId, AccessShareLock); + } + + bool *needsSeparateNodePtr = NULL; + + if (!PG_ARGISNULL(1)) + { + if (!IsCitusTableType(distributedRelationId, HASH_DISTRIBUTED) && + !IsCitusTableType(distributedRelationId, SINGLE_SHARD_DISTRIBUTED)) + { + ereport(ERROR, (errmsg("setting anti-affinity property is only " + "supported for hash distributed tables"))); + } + + needsSeparateNodePtr = palloc(sizeof(bool)); + *needsSeparateNodePtr = PG_GETARG_BOOL(1); + } + + ShardgroupSetPropertyGlobally(shardId, needsSeparateNodePtr); + + PG_RETURN_VOID(); +} + + +/* + * ShardgroupSetPropertyGlobally calls ShardgroupSetProperty + * on all nodes. + */ +static void +ShardgroupSetPropertyGlobally(uint64 shardId, bool *needsSeparateNodePtr) +{ + ShardgroupSetProperty(shardId, needsSeparateNodePtr); + + char *metadataCommand = + ShardgroupSetPropertyCommand(shardId, needsSeparateNodePtr); + SendCommandToWorkersWithMetadata(metadataCommand); +} + + +/* + * ShardgroupSetProperty sets shard properties for all the shards within + * the shard group that given shard belongs to. + */ +void +ShardgroupSetProperty(uint64 shardId, bool *needsSeparateNodePtr) +{ + ShardInterval *shardInterval = LoadShardInterval(shardId); + List *colocatedShardIntervalList = ColocatedShardIntervalList(shardInterval); + + int nShardInterval = list_length(colocatedShardIntervalList); + Datum *shardIdDatumArray = (Datum *) palloc(nShardInterval * sizeof(Datum)); + + int shardIndex = 0; + ShardInterval *colocatedShardInterval = NULL; + foreach_ptr(colocatedShardInterval, colocatedShardIntervalList) + { + shardIdDatumArray[shardIndex] = UInt64GetDatum(colocatedShardInterval->shardId); + shardIndex++; + } + + ArrayType *shardIdArrayDatum = DatumArrayToArrayType(shardIdDatumArray, + nShardInterval, INT8OID); + + Relation pgDistShard = table_open(DistShardRelationId(), RowExclusiveLock); + + ScanKeyData scanKey[1]; + int scanKeyCount = 1; + ScanKeyInit(&scanKey[0], Anum_pg_dist_shard_shardid, + BTEqualStrategyNumber, F_INT8EQ, PointerGetDatum(shardIdArrayDatum)); + scanKey[0].sk_flags |= SK_SEARCHARRAY; + + bool indexOK = true; + Oid indexId = DistShardShardidIndexId(); + SysScanDesc scanDescriptor = systable_beginscan(pgDistShard, + indexId, indexOK, NULL, + scanKeyCount, scanKey); + + Datum values[Natts_pg_dist_shard]; + bool isnull[Natts_pg_dist_shard]; + bool replace[Natts_pg_dist_shard]; + + memset(values, 0, sizeof(values)); + memset(isnull, false, sizeof(isnull)); + memset(replace, false, sizeof(replace)); + + if (needsSeparateNodePtr) + { + values[Anum_pg_dist_shard_needsseparatenode - 1] = BoolGetDatum( + *needsSeparateNodePtr); + isnull[Anum_pg_dist_shard_needsseparatenode - 1] = false; + replace[Anum_pg_dist_shard_needsseparatenode - 1] = true; + } + + bool updatedAny = false; + + CatalogIndexState indexState = CatalogOpenIndexes(pgDistShard); + + HeapTuple heapTuple = NULL; + while (HeapTupleIsValid(heapTuple = systable_getnext(scanDescriptor))) + { + TupleDesc tupleDescriptor = RelationGetDescr(pgDistShard); + heapTuple = heap_modify_tuple(heapTuple, tupleDescriptor, values, isnull, + replace); + + CatalogTupleUpdateWithInfo(pgDistShard, &heapTuple->t_self, heapTuple, + indexState); + + updatedAny = true; + } + + if (!updatedAny) + { + ereport(ERROR, (errmsg("could not find valid entry for shard " + UINT64_FORMAT, + shardId))); + } + + CatalogCloseIndexes(indexState); + + /* + * We don't need to send invalidations for all the shards as + * CitusInvalidateRelcacheByShardId() will send the invalidation based on + * id of the belonging distributed table, not just for the input shard. + */ + CitusInvalidateRelcacheByShardId(shardInterval->shardId); + + CommandCounterIncrement(); + + systable_endscan(scanDescriptor); + table_close(pgDistShard, NoLock); +} + + /* * SendShardStatisticsQueriesInParallel generates query lists for obtaining shard * statistics and then sends the commands in parallel by opening connections @@ -1350,6 +1517,7 @@ CopyShardInterval(ShardInterval *srcInterval) destInterval->maxValueExists = srcInterval->maxValueExists; destInterval->shardId = srcInterval->shardId; destInterval->shardIndex = srcInterval->shardIndex; + destInterval->needsSeparateNode = srcInterval->needsSeparateNode; destInterval->minValue = 0; if (destInterval->minValueExists) @@ -1397,6 +1565,92 @@ ShardLength(uint64 shardId) } +/* + * NodeGroupGetSeparatedShardgroupPlacement returns the shard group placement + * that given node group is used to separate from others. Returns NULL if this + * node is not used to separate a shard group placement. + */ +ShardgroupPlacement * +NodeGroupGetSeparatedShardgroupPlacement(int32 groupId) +{ + ShardgroupPlacement *nodeShardgroupPlacement = NULL; + bool shardgroupPlacementNeedsSeparateNode = false; + + bool indexOK = true; + ScanKeyData scanKey[1]; + + Relation pgDistPlacement = table_open(DistPlacementRelationId(), + AccessShareLock); + + ScanKeyInit(&scanKey[0], Anum_pg_dist_placement_groupid, + BTEqualStrategyNumber, F_INT4EQ, Int32GetDatum(groupId)); + SysScanDesc scanDescriptor = systable_beginscan(pgDistPlacement, + DistPlacementGroupidIndexId(), + indexOK, + NULL, lengthof(scanKey), scanKey); + + HeapTuple heapTuple = NULL; + while (HeapTupleIsValid(heapTuple = systable_getnext(scanDescriptor))) + { + TupleDesc tupleDescriptor = RelationGetDescr(pgDistPlacement); + + GroupShardPlacement *placement = + TupleToGroupShardPlacement(tupleDescriptor, heapTuple); + + ShardInterval *shardInterval = LoadShardInterval(placement->shardId); + Oid citusTableId = shardInterval->relationId; + if (!IsCitusTableType(citusTableId, DISTRIBUTED_TABLE)) + { + continue; + } + + ShardgroupPlacement *shardgroupPlacement = + GetShardgroupPlacementForPlacement(placement->shardId, + placement->placementId); + + if (nodeShardgroupPlacement && + !ShardgroupPlacementsSame(shardgroupPlacement, + nodeShardgroupPlacement)) + { + /* + * If we have more than one shardgroup placement on the node, + * then this means that the node is not actually used to separate + * a shardgroup placement. + */ + nodeShardgroupPlacement = NULL; + shardgroupPlacementNeedsSeparateNode = false; + break; + } + + nodeShardgroupPlacement = shardgroupPlacement; + shardgroupPlacementNeedsSeparateNode = shardInterval->needsSeparateNode; + } + + systable_endscan(scanDescriptor); + table_close(pgDistPlacement, NoLock); + + if (!shardgroupPlacementNeedsSeparateNode) + { + return NULL; + } + + return nodeShardgroupPlacement; +} + + +/* + * ShardgroupPlacementsSame returns true if two shardgroup placements are the same. + */ +bool +ShardgroupPlacementsSame(const ShardgroupPlacement *leftGroup, + const ShardgroupPlacement *rightGroup) +{ + return leftGroup->colocatationId == rightGroup->colocatationId && + leftGroup->shardIntervalIndex == rightGroup->shardIntervalIndex && + leftGroup->nodeGroupId == rightGroup->nodeGroupId; +} + + /* * NodeGroupHasShardPlacements returns whether any active shards are placed on the group */ @@ -1429,6 +1683,70 @@ NodeGroupHasShardPlacements(int32 groupId) } +/* + * NodeGroupHasDistributedTableShardPlacements returns whether any active + * distributed table shards are placed on the group + */ +bool +NodeGroupHasDistributedTableShardPlacements(int32 groupId) +{ + bool nodeGroupHasDistributedTableShardPlacements = false; + + Relation pgPlacement = table_open(DistPlacementRelationId(), AccessShareLock); + + ScanKeyData scanKey[1]; + ScanKeyInit(&scanKey[0], Anum_pg_dist_placement_groupid, + BTEqualStrategyNumber, F_INT4EQ, Int32GetDatum(groupId)); + + bool indexOK = true; + SysScanDesc scanDescriptor = systable_beginscan(pgPlacement, + DistPlacementGroupidIndexId(), + indexOK, + NULL, lengthof(scanKey), scanKey); + + HeapTuple heapTuple = NULL; + while (HeapTupleIsValid(heapTuple = systable_getnext(scanDescriptor))) + { + TupleDesc tupleDescriptor = RelationGetDescr(pgPlacement); + + GroupShardPlacement *placement = + TupleToGroupShardPlacement(tupleDescriptor, heapTuple); + + ShardInterval *shardInterval = LoadShardInterval(placement->shardId); + Oid citusTableId = shardInterval->relationId; + if (IsCitusTableType(citusTableId, DISTRIBUTED_TABLE)) + { + nodeGroupHasDistributedTableShardPlacements = true; + break; + } + } + + systable_endscan(scanDescriptor); + table_close(pgPlacement, NoLock); + + return nodeGroupHasDistributedTableShardPlacements; +} + + +/* + * GetShardgroupPlacementForPlacement returns ShardgroupPlacement that placement + * with given shardId & placementId belongs to. + */ +ShardgroupPlacement * +GetShardgroupPlacementForPlacement(uint64 shardId, uint64 placementId) +{ + ShardPlacement *shardPlacement = LoadShardPlacement(shardId, placementId); + ShardInterval *shardInterval = LoadShardInterval(shardId); + + ShardgroupPlacement *placementGroup = palloc(sizeof(ShardgroupPlacement)); + placementGroup->colocatationId = shardPlacement->colocationGroupId; + placementGroup->shardIntervalIndex = shardInterval->shardIndex; + placementGroup->nodeGroupId = shardPlacement->groupId; + + return placementGroup; +} + + /* * IsActiveShardPlacement checks if the shard placement is labelled as * active, and that it is placed in an active worker. @@ -1803,7 +2121,8 @@ IsDummyPlacement(ShardPlacement *taskPlacement) */ void InsertShardRow(Oid relationId, uint64 shardId, char storageType, - text *shardMinValue, text *shardMaxValue) + text *shardMinValue, text *shardMaxValue, + bool needsSeparateNode) { Datum values[Natts_pg_dist_shard]; bool isNulls[Natts_pg_dist_shard]; @@ -1815,6 +2134,7 @@ InsertShardRow(Oid relationId, uint64 shardId, char storageType, values[Anum_pg_dist_shard_logicalrelid - 1] = ObjectIdGetDatum(relationId); values[Anum_pg_dist_shard_shardid - 1] = Int64GetDatum(shardId); values[Anum_pg_dist_shard_shardstorage - 1] = CharGetDatum(storageType); + values[Anum_pg_dist_shard_needsseparatenode - 1] = BoolGetDatum(needsSeparateNode); /* dropped shardalias column must also be set; it is still part of the tuple */ isNulls[Anum_pg_dist_shard_shardalias_DROPPED - 1] = true; diff --git a/src/backend/distributed/metadata/node_metadata.c b/src/backend/distributed/metadata/node_metadata.c index 3fa2549e783..f71ef86d07f 100644 --- a/src/backend/distributed/metadata/node_metadata.c +++ b/src/backend/distributed/metadata/node_metadata.c @@ -62,8 +62,6 @@ #include "distributed/worker_manager.h" #include "distributed/worker_transaction.h" -#define INVALID_GROUP_ID -1 - /* default group size */ int GroupSize = 1; diff --git a/src/backend/distributed/operations/citus_split_shard_by_split_points.c b/src/backend/distributed/operations/citus_split_shard_by_split_points.c index 076e58d4c02..87257558b4e 100644 --- a/src/backend/distributed/operations/citus_split_shard_by_split_points.c +++ b/src/backend/distributed/operations/citus_split_shard_by_split_points.c @@ -20,6 +20,7 @@ #include "distributed/colocation_utils.h" #include "distributed/connection_management.h" #include "distributed/coordinator_protocol.h" +#include "distributed/listutils.h" #include "distributed/metadata_cache.h" #include "distributed/remote_commands.h" #include "distributed/shard_split.h" @@ -55,6 +56,10 @@ citus_split_shard_by_split_points(PG_FUNCTION_ARGS) Oid shardTransferModeOid = PG_GETARG_OID(3); SplitMode shardSplitMode = LookupSplitMode(shardTransferModeOid); + /* we don't inherit needsseparatenode for new shards */ + List *needsSeparateNodeForPlacementList = + GenerateListFromIntElement(false, list_length(nodeIdsForPlacementList)); + DistributionColumnMap *distributionColumnOverrides = NULL; List *sourceColocatedShardIntervalList = NIL; SplitShard( @@ -63,6 +68,7 @@ citus_split_shard_by_split_points(PG_FUNCTION_ARGS) shardIdToSplit, shardSplitPointsList, nodeIdsForPlacementList, + needsSeparateNodeForPlacementList, distributionColumnOverrides, sourceColocatedShardIntervalList, INVALID_COLOCATION_ID); diff --git a/src/backend/distributed/operations/create_shards.c b/src/backend/distributed/operations/create_shards.c index 96254705122..2f9644580d6 100644 --- a/src/backend/distributed/operations/create_shards.c +++ b/src/backend/distributed/operations/create_shards.c @@ -147,15 +147,16 @@ CreateShardsWithRoundRobinPolicy(Oid distributedTableId, int32 shardCount, LockRelationOid(DistNodeRelationId(), RowShareLock); /* load and sort the worker node list for deterministic placement */ - List *workerNodeList = DistributedTablePlacementNodeList(NoLock); + List *workerNodeList = NewDistributedTablePlacementNodeList(NoLock); workerNodeList = SortList(workerNodeList, CompareWorkerNodes); int32 workerNodeCount = list_length(workerNodeList); if (replicationFactor > workerNodeCount) { ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE), - errmsg("replication_factor (%d) exceeds number of worker nodes " - "(%d)", replicationFactor, workerNodeCount), + errmsg("replication_factor (%d) exceeds number of " + "available worker nodes (%d)", + replicationFactor, workerNodeCount), errhint("Add more worker nodes or try again with a lower " "replication factor."))); } @@ -184,8 +185,9 @@ CreateShardsWithRoundRobinPolicy(Oid distributedTableId, int32 shardCount, text *minHashTokenText = IntegerToText(shardMinHashToken); text *maxHashTokenText = IntegerToText(shardMaxHashToken); + bool needsSeparateNode = false; InsertShardRow(distributedTableId, *shardIdPtr, shardStorageType, - minHashTokenText, maxHashTokenText); + minHashTokenText, maxHashTokenText, needsSeparateNode); InsertShardPlacementRows(distributedTableId, *shardIdPtr, @@ -282,8 +284,10 @@ CreateColocatedShards(Oid targetRelationId, Oid sourceRelationId, bool List *sourceShardPlacementList = ShardPlacementListSortedByWorker( sourceShardId); + /* inherit from the colocated shard */ + bool needsSeparateNode = sourceShardInterval->needsSeparateNode; InsertShardRow(targetRelationId, *newShardIdPtr, targetShardStorageType, - shardMinValueText, shardMaxValueText); + shardMinValueText, shardMaxValueText, needsSeparateNode); ShardPlacement *sourcePlacement = NULL; foreach_ptr(sourcePlacement, sourceShardPlacementList) @@ -365,8 +369,9 @@ CreateReferenceTableShard(Oid distributedTableId) /* get the next shard id */ uint64 shardId = GetNextShardId(); + bool needsSeparateNode = false; InsertShardRow(distributedTableId, shardId, shardStorageType, shardMinValue, - shardMaxValue); + shardMaxValue, needsSeparateNode); InsertShardPlacementRows(distributedTableId, shardId, @@ -411,7 +416,7 @@ CreateSingleShardTableShardWithRoundRobinPolicy(Oid relationId, uint32 colocatio * Also take a RowShareLock on pg_dist_node to disallow concurrent * node list changes that require an exclusive lock. */ - List *workerNodeList = DistributedTablePlacementNodeList(RowShareLock); + List *workerNodeList = NewDistributedTablePlacementNodeList(RowShareLock); workerNodeList = SortList(workerNodeList, CompareWorkerNodes); int roundRobinNodeIdx = @@ -421,8 +426,9 @@ CreateSingleShardTableShardWithRoundRobinPolicy(Oid relationId, uint32 colocatio text *minHashTokenText = NULL; text *maxHashTokenText = NULL; uint64 shardId = GetNextShardId(); + bool needsSeparateNode = false; InsertShardRow(relationId, shardId, shardStorageType, - minHashTokenText, maxHashTokenText); + minHashTokenText, maxHashTokenText, needsSeparateNode); int replicationFactor = 1; InsertShardPlacementRows(relationId, @@ -454,17 +460,17 @@ CreateSingleShardTableShardWithRoundRobinPolicy(Oid relationId, uint32 colocatio * group" should be placed on. * * This is determined by modulo of the colocation id by the length of the - * list returned by DistributedTablePlacementNodeList(). + * list returned by NewDistributedTablePlacementNodeList(). */ int EmptySingleShardTableColocationDecideNodeId(uint32 colocationId) { - List *workerNodeList = DistributedTablePlacementNodeList(RowShareLock); + List *workerNodeList = NewDistributedTablePlacementNodeList(RowShareLock); int32 workerNodeCount = list_length(workerNodeList); if (workerNodeCount == 0) { ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE), - errmsg("couldn't find any worker nodes"), + errmsg("couldn't find any available worker nodes"), errhint("Add more worker nodes"))); } diff --git a/src/backend/distributed/operations/isolate_shards.c b/src/backend/distributed/operations/isolate_shards.c index 502b00f5bb9..fe4cf8e8fbc 100644 --- a/src/backend/distributed/operations/isolate_shards.c +++ b/src/backend/distributed/operations/isolate_shards.c @@ -168,6 +168,10 @@ isolate_tenant_to_new_shard(PG_FUNCTION_ARGS) nodeIdsForPlacementList = lappend_int(nodeIdsForPlacementList, sourceNodeId); } + /* we don't inherit needsseparatenode for new shards */ + List *needsSeparateNodeForPlacementList = + GenerateListFromIntElement(false, list_length(nodeIdsForPlacementList)); + DistributionColumnMap *distributionColumnOverrides = NULL; List *sourceColocatedShardIntervalList = NIL; SplitMode splitMode = LookupSplitMode(shardTransferModeOid); @@ -176,6 +180,7 @@ isolate_tenant_to_new_shard(PG_FUNCTION_ARGS) sourceShard->shardId, shardSplitPointsList, nodeIdsForPlacementList, + needsSeparateNodeForPlacementList, distributionColumnOverrides, sourceColocatedShardIntervalList, INVALID_COLOCATION_ID); diff --git a/src/backend/distributed/operations/rebalancer_placement_separation.c b/src/backend/distributed/operations/rebalancer_placement_separation.c new file mode 100644 index 00000000000..c18c85041a3 --- /dev/null +++ b/src/backend/distributed/operations/rebalancer_placement_separation.c @@ -0,0 +1,433 @@ +/*------------------------------------------------------------------------- + * + * rebalancer_placement_separation.c + * Routines to determine which worker node should be used to separate + * a colocated set of shard placements that need separate nodes. + * + * Copyright (c) Citus Data, Inc. + * + *------------------------------------------------------------------------- + */ + +#include "postgres.h" + +#include "nodes/pg_list.h" +#include "utils/hsearch.h" +#include "utils/lsyscache.h" + +#include "distributed/colocation_utils.h" +#include "distributed/hash_helpers.h" +#include "distributed/listutils.h" +#include "distributed/metadata_cache.h" +#include "distributed/metadata_utility.h" +#include "distributed/multi_physical_planner.h" +#include "distributed/rebalancer_placement_separation.h" +#include "distributed/shard_rebalancer.h" + + +typedef struct RebalancerPlacementSeparationContext +{ + /* + * Hash table where each entry is of the form NodeToPlacementGroupHashEntry, + * meaning that each entry maps the node with nodeGroupId to + * a NodeToPlacementGroupHashEntry. + */ + HTAB *nodePlacementGroupHash; +} RebalancerPlacementSeparationContext; + + +/* + * Entry of the hash table that maps each primary worker node to a shard + * placement group that is determined to be separated from other shards in + * the cluster via that node. + */ +typedef struct NodeToPlacementGroupHashEntry +{ + /* hash key -- group id of the node */ + int32 nodeGroupId; + + /* + * Whether given node is allowed to have any shards. + * + * Inherited from WorkerNode->shouldHaveShards. + */ + bool shouldHaveShards; + + /* + * Whether given node has some shard placements that cannot be moved away. + * + * For the nodes that this rebalancer-run is not allowed to move the + * placements away from, InitRebalancerPlacementSeparationContext() sets + * this to true if the node has some shard placements already. And if the + * node has a single shard placement group that needs a separate node, it + * also sets assignedPlacementGroup. + * + * We do so to prevent TryAssignPlacementGroupsToNodeGroups() making + * incorrect assignments later on. + * + * See InitRebalancerPlacementSeparationContext() for more details. + */ + bool hasPlacementsThatCannotBeMovedAway; + + /* + * Shardgroup placement that is assigned to this node to be separated + * from others in the cluster. + * + * NULL if no shardgroup placement is not assigned yet. + */ + ShardgroupPlacement *assignedPlacementGroup; +} NodeToPlacementGroupHashEntry; + +/* + * Routines to prepare RebalancerPlacementSeparationContext. + */ +static void InitRebalancerPlacementSeparationContext( + RebalancerPlacementSeparationContext *context, + List *activeWorkerNodeList, + List *rebalancePlacementList); +static void TryAssignPlacementGroupsToNodeGroups( + RebalancerPlacementSeparationContext *context, + List *activeWorkerNodeList, + List *rebalancePlacementList, + FmgrInfo shardAllowedOnNodeUDF); +static bool TryAssignPlacementGroupToNodeGroup( + RebalancerPlacementSeparationContext *context, + int32 candidateNodeGroupId, + ShardPlacement *shardPlacement, + FmgrInfo shardAllowedOnNodeUDF); + + +/* other helpers */ +static List * LoadAllShardgroupPlacements(void); +static HTAB * ShardPlacementListToShardgroupPlacementSet(List *shardPlacementList); + + +/* + * PrepareRebalancerPlacementSeparationContext creates RebalancerPlacementSeparationContext + * that keeps track of which worker nodes are used to separate which shardgroup placements + * that need separate nodes. + */ +RebalancerPlacementSeparationContext * +PrepareRebalancerPlacementSeparationContext(List *activeWorkerNodeList, + List *rebalancePlacementList, + FmgrInfo shardAllowedOnNodeUDF) +{ + HTAB *nodePlacementGroupHash = + CreateSimpleHashWithNameAndSize(int32, NodeToPlacementGroupHashEntry, + "NodeToPlacementGroupHash", + list_length(activeWorkerNodeList)); + + RebalancerPlacementSeparationContext *context = + palloc0(sizeof(RebalancerPlacementSeparationContext)); + context->nodePlacementGroupHash = nodePlacementGroupHash; + + activeWorkerNodeList = SortList(activeWorkerNodeList, CompareWorkerNodes); + rebalancePlacementList = SortList(rebalancePlacementList, CompareShardPlacements); + + InitRebalancerPlacementSeparationContext(context, activeWorkerNodeList, + rebalancePlacementList); + + TryAssignPlacementGroupsToNodeGroups(context, + activeWorkerNodeList, + rebalancePlacementList, + shardAllowedOnNodeUDF); + + return context; +} + + +/* + * InitRebalancerPlacementSeparationContext initializes given + * RebalancerPlacementSeparationContext by using given list + * of worker nodes and the worker node that is being drained, + * if specified. + */ +static void +InitRebalancerPlacementSeparationContext(RebalancerPlacementSeparationContext *context, + List *activeWorkerNodeList, + List *rebalancePlacementList) +{ + HTAB *nodePlacementGroupHash = context->nodePlacementGroupHash; + + WorkerNode *workerNode = NULL; + foreach_ptr(workerNode, activeWorkerNodeList) + { + NodeToPlacementGroupHashEntry *nodePlacementGroupHashEntry = + hash_search(nodePlacementGroupHash, &workerNode->groupId, HASH_ENTER, + NULL); + + nodePlacementGroupHashEntry->shouldHaveShards = workerNode->shouldHaveShards; + nodePlacementGroupHashEntry->hasPlacementsThatCannotBeMovedAway = false; + nodePlacementGroupHashEntry->assignedPlacementGroup = NULL; + + if (!nodePlacementGroupHashEntry->shouldHaveShards) + { + continue; + } + + nodePlacementGroupHashEntry->assignedPlacementGroup = + NodeGroupGetSeparatedShardgroupPlacement( + nodePlacementGroupHashEntry->nodeGroupId); + } + + HTAB *balancingShardgroupPlacementsSet = + ShardPlacementListToShardgroupPlacementSet(rebalancePlacementList); + + /* iterate over all shardgroups to find nodes that have shardgroups not balancing */ + List *allShardgroupPlacements = LoadAllShardgroupPlacements(); + ShardgroupPlacement *shardgroupPlacement = NULL; + foreach_ptr(shardgroupPlacement, allShardgroupPlacements) + { + bool found = false; + hash_search(balancingShardgroupPlacementsSet, shardgroupPlacement, HASH_FIND, + &found); + if (found) + { + /* we are balancing this shardgroup placement, skip */ + continue; + } + + /* we have a shardgroupPlacement we are not balancing, marking node as such */ + NodeToPlacementGroupHashEntry *nodePlacementGroupHashEntry = + hash_search(nodePlacementGroupHash, &shardgroupPlacement->nodeGroupId, + HASH_ENTER, NULL); + + nodePlacementGroupHashEntry->hasPlacementsThatCannotBeMovedAway = true; + } +} + + +/* + * TryAssignPlacementGroupsToNodeGroups tries to assign placements that need + * separate nodes within given placement list to individual worker nodes. + */ +static void +TryAssignPlacementGroupsToNodeGroups(RebalancerPlacementSeparationContext *context, + List *activeWorkerNodeList, + List *rebalancePlacementList, + FmgrInfo shardAllowedOnNodeUDF) +{ + List *unassignedPlacementList = NIL; + + /* + * Assign as much as possible shardgroup placements to worker nodes where + * they are stored already. + */ + ShardPlacement *shardPlacement = NULL; + foreach_ptr(shardPlacement, rebalancePlacementList) + { + ShardInterval *shardInterval = LoadShardInterval(shardPlacement->shardId); + if (!shardInterval->needsSeparateNode) + { + continue; + } + + int32 currentNodeGroupId = shardPlacement->groupId; + if (!TryAssignPlacementGroupToNodeGroup(context, + currentNodeGroupId, + shardPlacement, + shardAllowedOnNodeUDF)) + { + unassignedPlacementList = + lappend(unassignedPlacementList, shardPlacement); + } + } + + bool emitWarning = false; + + /* + * For the shardgroup placements that could not be assigned to their + * current node, assign them to any other node. + */ + ShardPlacement *unassignedShardPlacement = NULL; + foreach_ptr(unassignedShardPlacement, unassignedPlacementList) + { + bool separated = false; + + WorkerNode *activeWorkerNode = NULL; + foreach_ptr(activeWorkerNode, activeWorkerNodeList) + { + if (TryAssignPlacementGroupToNodeGroup(context, + activeWorkerNode->groupId, + unassignedShardPlacement, + shardAllowedOnNodeUDF)) + { + separated = true; + break; + } + } + + if (!separated) + { + emitWarning = true; + } + } + + if (emitWarning) + { + ereport(WARNING, (errmsg("could not separate all shard placements " + "that need a separate node"))); + } +} + + +/* + * TryAssignPlacementGroupToNodeGroup is an helper to + * TryAssignPlacementGroupsToNodeGroups that tries to assign given + * shard placement to given node and returns true if it succeeds. + */ +static bool +TryAssignPlacementGroupToNodeGroup(RebalancerPlacementSeparationContext *context, + int32 candidateNodeGroupId, + ShardPlacement *shardPlacement, + FmgrInfo shardAllowedOnNodeUDF) +{ + HTAB *nodePlacementGroupHash = context->nodePlacementGroupHash; + + bool found = false; + NodeToPlacementGroupHashEntry *nodePlacementGroupHashEntry = + hash_search(nodePlacementGroupHash, &candidateNodeGroupId, HASH_FIND, &found); + + if (!found) + { + ereport(ERROR, (errmsg("no such node is found"))); + } + + ShardgroupPlacement *shardgroupPlacement = + GetShardgroupPlacementForPlacement(shardPlacement->shardId, + shardPlacement->placementId); + + if (nodePlacementGroupHashEntry->assignedPlacementGroup) + { + return ShardgroupPlacementsSame(shardgroupPlacement, + nodePlacementGroupHashEntry-> + assignedPlacementGroup); + } + + if (nodePlacementGroupHashEntry->hasPlacementsThatCannotBeMovedAway) + { + return false; + } + + if (!nodePlacementGroupHashEntry->shouldHaveShards) + { + return false; + } + + WorkerNode *workerNode = PrimaryNodeForGroup(candidateNodeGroupId, NULL); + Datum allowed = FunctionCall2(&shardAllowedOnNodeUDF, shardPlacement->shardId, + workerNode->nodeId); + if (!DatumGetBool(allowed)) + { + return false; + } + + nodePlacementGroupHashEntry->assignedPlacementGroup = shardgroupPlacement; + + return true; +} + + +/* + * RebalancerPlacementSeparationContextPlacementIsAllowedOnWorker returns true + * if shard placement with given shardId & placementId is allowed to be stored + * on given worker node. + */ +bool +RebalancerPlacementSeparationContextPlacementIsAllowedOnWorker( + RebalancerPlacementSeparationContext *context, + uint64 shardId, + uint64 placementId, + WorkerNode *workerNode) +{ + HTAB *nodePlacementGroupHash = context->nodePlacementGroupHash; + + bool found = false; + NodeToPlacementGroupHashEntry *nodePlacementGroupHashEntry = + hash_search(nodePlacementGroupHash, &(workerNode->groupId), HASH_FIND, &found); + + if (!found) + { + ereport(ERROR, (errmsg("no such node is found"))); + } + + ShardInterval *shardInterval = LoadShardInterval(shardId); + if (!shardInterval->needsSeparateNode) + { + /* + * It doesn't need a separate node, but is the node used to separate + * a shardgroup placement? If so, we cannot store it on this node. + */ + return nodePlacementGroupHashEntry->shouldHaveShards && + nodePlacementGroupHashEntry->assignedPlacementGroup == NULL; + } + + /* + * Given shard placement needs a separate node. + * Check if given worker node is the one that is assigned to separate it. + */ + if (nodePlacementGroupHashEntry->assignedPlacementGroup == NULL) + { + /* the node is not supposed to separate a placement group */ + return false; + } + + ShardgroupPlacement *placementGroup = + GetShardgroupPlacementForPlacement(shardId, placementId); + return ShardgroupPlacementsSame(nodePlacementGroupHashEntry->assignedPlacementGroup, + placementGroup); +} + + +/* + * LoadAllShardgroupPlacements loads all ShardgroupPlacements that belong + * to distributed tables in the cluster. + */ +static List * +LoadAllShardgroupPlacements(void) +{ + List *shardgroupPlacementList = NIL; + + List *relationIdList = NonColocatedDistRelationIdList(); + Oid relationId = InvalidOid; + foreach_oid(relationId, relationIdList) + { + ArrayType *excludedShardArray = construct_empty_array(INT4OID); + List *shardPlacementList = FullShardPlacementList(relationId, excludedShardArray); + + ShardPlacement *shardPlacement = NULL; + foreach_ptr(shardPlacement, shardPlacementList) + { + ShardgroupPlacement *shardgroupPlacement = + GetShardgroupPlacementForPlacement(shardPlacement->shardId, + shardPlacement->placementId); + shardgroupPlacementList = lappend(shardgroupPlacementList, + shardgroupPlacement); + } + } + + return shardgroupPlacementList; +} + + +/* + * ShardPlacementListToShardgroupPlacementSet returns a hash set that contains + * all ShardgroupPlacements that are represented by given list of ShardPlacements. + */ +static HTAB * +ShardPlacementListToShardgroupPlacementSet(List *shardPlacementList) +{ + HTAB *shardgroupPlacementSet = CreateSimpleHashSet(ShardgroupPlacement); + + ShardPlacement *shardPlacement = NULL; + foreach_ptr(shardPlacement, shardPlacementList) + { + ShardgroupPlacement *findShardgroupPlacement = + GetShardgroupPlacementForPlacement(shardPlacement->shardId, + shardPlacement->placementId); + + hash_search(shardgroupPlacementSet, findShardgroupPlacement, HASH_ENTER, NULL); + } + + return shardgroupPlacementSet; +} diff --git a/src/backend/distributed/operations/shard_rebalancer.c b/src/backend/distributed/operations/shard_rebalancer.c index d1868d3c48f..18bd0846206 100644 --- a/src/backend/distributed/operations/shard_rebalancer.c +++ b/src/backend/distributed/operations/shard_rebalancer.c @@ -59,6 +59,7 @@ #include "distributed/multi_progress.h" #include "distributed/multi_server_executor.h" #include "distributed/pg_dist_rebalance_strategy.h" +#include "distributed/rebalancer_placement_separation.h" #include "distributed/reference_table_utils.h" #include "distributed/remote_commands.h" #include "distributed/resource_lock.h" @@ -147,6 +148,8 @@ typedef struct RebalanceContext FmgrInfo shardCostUDF; FmgrInfo nodeCapacityUDF; FmgrInfo shardAllowedOnNodeUDF; + + RebalancerPlacementSeparationContext *shardgroupPlacementSeparationContext; } RebalanceContext; /* WorkerHashKey contains hostname and port to be used as a key in a hash */ @@ -255,7 +258,8 @@ static bool FindAndMoveShardCost(float4 utilizationLowerBound, float4 utilizationUpperBound, float4 improvementThreshold, RebalanceState *state); -static NodeFillState * FindAllowedTargetFillState(RebalanceState *state, uint64 shardId); +static NodeFillState * FindAllowedTargetFillState(RebalanceState *state, uint64 shardId, + uint64 placementId); static void MoveShardCost(NodeFillState *sourceFillState, NodeFillState *targetFillState, ShardCost *shardCost, RebalanceState *state); static int CompareNodeFillStateAsc(const void *void1, const void *void2); @@ -264,10 +268,10 @@ static int CompareShardCostAsc(const void *void1, const void *void2); static int CompareShardCostDesc(const void *void1, const void *void2); static int CompareDisallowedPlacementAsc(const void *void1, const void *void2); static int CompareDisallowedPlacementDesc(const void *void1, const void *void2); -static bool ShardAllowedOnNode(uint64 shardId, WorkerNode *workerNode, void *context); +static bool ShardAllowedOnNode(uint64 shardId, uint64 placementId, WorkerNode *workerNode, + void *context); static float4 NodeCapacity(WorkerNode *workerNode, void *context); -static ShardCost GetShardCost(uint64 shardId, void *context); -static List * NonColocatedDistRelationIdList(void); +static ShardCost GetShardCost(uint64 shardId, uint64 placementId, void *context); static void RebalanceTableShards(RebalanceOptions *options, Oid shardReplicationModeOid); static int64 RebalanceTableShardsBackground(RebalanceOptions *options, Oid shardReplicationModeOid); @@ -435,7 +439,7 @@ BigIntArrayDatumContains(Datum *array, int arrayLength, uint64 toFind) * FullShardPlacementList returns a List containing all the shard placements of * a specific table (excluding the excludedShardArray) */ -static List * +List * FullShardPlacementList(Oid relationId, ArrayType *excludedShardArray) { List *shardPlacementList = NIL; @@ -467,6 +471,9 @@ FullShardPlacementList(Oid relationId, ArrayType *excludedShardArray) ShardPlacement *placement = CitusMakeNode(ShardPlacement); placement->shardId = groupPlacement->shardId; placement->shardLength = groupPlacement->shardLength; + placement->groupId = groupPlacement->groupId; + placement->colocationGroupId = citusTableCacheEntry->colocationId; + placement->partitionMethod = citusTableCacheEntry->partitionMethod; placement->nodeId = worker->nodeId; placement->nodeName = pstrdup(worker->workerName); placement->nodePort = worker->workerPort; @@ -590,6 +597,12 @@ GetRebalanceSteps(RebalanceOptions *options) options->threshold = options->rebalanceStrategy->minimumThreshold; } + context.shardgroupPlacementSeparationContext = + PrepareRebalancerPlacementSeparationContext( + activeWorkerList, + FlattenNestedList(activeShardPlacementListList), + context.shardAllowedOnNodeUDF); + return RebalancePlacementUpdates(activeWorkerList, activeShardPlacementListList, options->threshold, @@ -604,7 +617,8 @@ GetRebalanceSteps(RebalanceOptions *options) * ShardAllowedOnNode determines if shard is allowed on a specific worker node. */ static bool -ShardAllowedOnNode(uint64 shardId, WorkerNode *workerNode, void *voidContext) +ShardAllowedOnNode(uint64 shardId, uint64 placementId, WorkerNode *workerNode, + void *voidContext) { if (!workerNode->shouldHaveShards) { @@ -612,6 +626,14 @@ ShardAllowedOnNode(uint64 shardId, WorkerNode *workerNode, void *voidContext) } RebalanceContext *context = voidContext; + + if (!RebalancerPlacementSeparationContextPlacementIsAllowedOnWorker( + context->shardgroupPlacementSeparationContext, + shardId, placementId, workerNode)) + { + return false; + } + Datum allowed = FunctionCall2(&context->shardAllowedOnNodeUDF, shardId, workerNode->nodeId); return DatumGetBool(allowed); @@ -645,10 +667,11 @@ NodeCapacity(WorkerNode *workerNode, void *voidContext) * to be. */ static ShardCost -GetShardCost(uint64 shardId, void *voidContext) +GetShardCost(uint64 shardId, uint64 placementId, void *voidContext) { ShardCost shardCost = { 0 }; shardCost.shardId = shardId; + shardCost.placementId = placementId; RebalanceContext *context = voidContext; Datum shardCostDatum = FunctionCall1(&context->shardCostUDF, UInt64GetDatum(shardId)); shardCost.cost = DatumGetFloat4(shardCostDatum); @@ -794,9 +817,9 @@ AcquirePlacementColocationLock(Oid relationId, int lockMode, ereport(ERROR, (errmsg("could not acquire the lock required to %s %s", operationName, generate_qualified_relation_name(relationId)), - errdetail("It means that either a concurrent shard move " - "or colocated distributed table creation is " - "happening."), + errdetail("It means that either a concurrent shard move, " + "colocated distributed table creation or " + "shard property change is happening."), errhint("Make sure that the concurrent operation has " "finished and re-run the command"))); } @@ -1828,7 +1851,7 @@ AddToWorkerShardIdSet(HTAB *shardsByWorker, char *workerName, int workerPort, * NonColocatedDistRelationIdList returns a list of distributed table oids, one * for each existing colocation group. */ -static List * +List * NonColocatedDistRelationIdList(void) { List *relationIdList = NIL; @@ -2562,7 +2585,8 @@ InitRebalanceState(List *workerNodeList, List *shardPlacementList, Assert(fillState != NULL); - *shardCost = functions->shardCost(placement->shardId, functions->context); + *shardCost = functions->shardCost(placement->shardId, placement->placementId, + functions->context); fillState->totalCost += shardCost->cost; fillState->utilization = CalculateUtilization(fillState->totalCost, @@ -2574,8 +2598,8 @@ InitRebalanceState(List *workerNodeList, List *shardPlacementList, state->totalCost += shardCost->cost; - if (!functions->shardAllowedOnNode(placement->shardId, fillState->node, - functions->context)) + if (!functions->shardAllowedOnNode(placement->shardId, placement->placementId, + fillState->node, functions->context)) { DisallowedPlacement *disallowed = palloc0(sizeof(DisallowedPlacement)); disallowed->shardCost = shardCost; @@ -2735,7 +2759,8 @@ MoveShardsAwayFromDisallowedNodes(RebalanceState *state) foreach_ptr(disallowedPlacement, state->disallowedPlacementList) { NodeFillState *targetFillState = FindAllowedTargetFillState( - state, disallowedPlacement->shardCost->shardId); + state, disallowedPlacement->shardCost->shardId, + disallowedPlacement->shardCost->placementId); if (targetFillState == NULL) { ereport(WARNING, (errmsg( @@ -2784,7 +2809,7 @@ CompareDisallowedPlacementDesc(const void *a, const void *b) * where the shard can be moved to. */ static NodeFillState * -FindAllowedTargetFillState(RebalanceState *state, uint64 shardId) +FindAllowedTargetFillState(RebalanceState *state, uint64 shardId, uint64 placementId) { NodeFillState *targetFillState = NULL; foreach_ptr(targetFillState, state->fillStateListAsc) @@ -2795,6 +2820,7 @@ FindAllowedTargetFillState(RebalanceState *state, uint64 shardId) targetFillState->node); if (!hasShard && state->functions->shardAllowedOnNode( shardId, + placementId, targetFillState->node, state->functions->context)) { @@ -2969,6 +2995,7 @@ FindAndMoveShardCost(float4 utilizationLowerBound, /* Skip shards that already are not allowed on the node */ if (!state->functions->shardAllowedOnNode(shardCost->shardId, + shardCost->placementId, targetFillState->node, state->functions->context)) { @@ -3165,7 +3192,7 @@ ReplicationPlacementUpdates(List *workerNodeList, List *activeShardPlacementList { WorkerNode *workerNode = list_nth(workerNodeList, workerNodeIndex); - if (!NodeCanHaveDistTablePlacements(workerNode)) + if (!NodeCanBeUsedForNonSeparatedPlacements(workerNode)) { /* never replicate placements to nodes that should not have placements */ continue; diff --git a/src/backend/distributed/operations/shard_split.c b/src/backend/distributed/operations/shard_split.c index ac7ed6bf365..b4f0b60c995 100644 --- a/src/backend/distributed/operations/shard_split.c +++ b/src/backend/distributed/operations/shard_split.c @@ -94,22 +94,26 @@ static void CreateAuxiliaryStructuresForShardGroup(List *shardGroupSplitInterval static void CreateReplicaIdentitiesForDummyShards(HTAB *mapOfPlacementToDummyShardList); static void CreateObjectOnPlacement(List *objectCreationCommandList, WorkerNode *workerNode); -static List * CreateSplitIntervalsForShardGroup(List *sourceColocatedShardList, - List *splitPointsForShard); +static List * CreateSplitIntervalsForShardGroup(List *sourceColocatedShardList, + List *splitPointsForShard, + List *needsSeparateNodeForShardList); static void CreateSplitIntervalsForShard(ShardInterval *sourceShard, List *splitPointsForShard, + List *needsSeparateNodeForShardList, List **shardSplitChildrenIntervalList); static void BlockingShardSplit(SplitOperation splitOperation, uint64 splitWorkflowId, List *sourceColocatedShardIntervalList, List *shardSplitPointsList, List *workersForPlacementList, + List *needsSeparateNodeForShardList, DistributionColumnMap *distributionColumnOverrides); static void NonBlockingShardSplit(SplitOperation splitOperation, uint64 splitWorkflowId, List *sourceColocatedShardIntervalList, List *shardSplitPointsList, List *workersForPlacementList, + List *needsSeparateNodeForShardList, DistributionColumnMap *distributionColumnOverrides, uint32 targetColocationId); static void DoSplitCopy(WorkerNode *sourceShardNode, @@ -422,18 +426,19 @@ GetWorkerNodesFromWorkerIds(List *nodeIdsForPlacementList) /* * SplitShard API to split a given shard (or shard group) based on specified split points * to a set of destination nodes. - * 'splitMode' : Mode of split operation. - * 'splitOperation' : Customer operation that triggered split. - * 'shardInterval' : Source shard interval to be split. - * 'shardSplitPointsList' : Split Points list for the source 'shardInterval'. - * 'nodeIdsForPlacementList' : Placement list corresponding to split children. - * 'distributionColumnOverrides': Maps relation IDs to distribution columns. - * If not specified, the distribution column is read - * from the metadata. - * 'colocatedShardIntervalList' : Shard interval list for colocation group. (only used for - * create_distributed_table_concurrently). - * 'targetColocationId' : Specifies the colocation ID (only used for - * create_distributed_table_concurrently). + * 'splitMode' : Mode of split operation. + * 'splitOperation' : Customer operation that triggered split. + * 'shardInterval' : Source shard interval to be split. + * 'shardSplitPointsList' : Split Points list for the source 'shardInterval'. + * 'nodeIdsForPlacementList' : Placement list corresponding to split children. + * 'needsSeparateNodeForShardList' : Whether each split children needs a separate node. + * 'distributionColumnOverrides' : Maps relation IDs to distribution columns. + * If not specified, the distribution column is read + * from the metadata. + * 'colocatedShardIntervalList' : Shard interval list for colocation group. (only used for + * create_distributed_table_concurrently). + * 'targetColocationId' : Specifies the colocation ID (only used for + * create_distributed_table_concurrently). */ void SplitShard(SplitMode splitMode, @@ -441,6 +446,7 @@ SplitShard(SplitMode splitMode, uint64 shardIdToSplit, List *shardSplitPointsList, List *nodeIdsForPlacementList, + List *needsSeparateNodeForShardList, DistributionColumnMap *distributionColumnOverrides, List *colocatedShardIntervalList, uint32 targetColocationId) @@ -514,6 +520,7 @@ SplitShard(SplitMode splitMode, sourceColocatedShardIntervalList, shardSplitPointsList, workersForPlacementList, + needsSeparateNodeForShardList, distributionColumnOverrides); } else @@ -526,6 +533,7 @@ SplitShard(SplitMode splitMode, sourceColocatedShardIntervalList, shardSplitPointsList, workersForPlacementList, + needsSeparateNodeForShardList, distributionColumnOverrides, targetColocationId); @@ -547,6 +555,7 @@ SplitShard(SplitMode splitMode, * sourceColocatedShardIntervalList : Source shard group to be split. * shardSplitPointsList : Split Points list for the source 'shardInterval'. * workersForPlacementList : Placement list corresponding to split children. + * needsSeparateNodeForShardList : Whether each split children needs a separate node. */ static void BlockingShardSplit(SplitOperation splitOperation, @@ -554,6 +563,7 @@ BlockingShardSplit(SplitOperation splitOperation, List *sourceColocatedShardIntervalList, List *shardSplitPointsList, List *workersForPlacementList, + List *needsSeparateNodeForShardList, DistributionColumnMap *distributionColumnOverrides) { const char *operationName = SplitOperationAPIName[splitOperation]; @@ -563,7 +573,8 @@ BlockingShardSplit(SplitOperation splitOperation, /* First create shard interval metadata for split children */ List *shardGroupSplitIntervalListList = CreateSplitIntervalsForShardGroup( sourceColocatedShardIntervalList, - shardSplitPointsList); + shardSplitPointsList, + needsSeparateNodeForShardList); /* Only single placement allowed (already validated RelationReplicationFactor = 1) */ ShardInterval *firstShard = linitial(sourceColocatedShardIntervalList); @@ -1021,10 +1032,12 @@ CreateObjectOnPlacement(List *objectCreationCommandList, * [ S1_1(-2147483648, 0), S1_2(1, 2147483647) ], // Split Interval List for S1. * [ S2_1(-2147483648, 0), S2_2(1, 2147483647) ] // Split Interval List for S2. * ] + * 'needsSeparateNodeForShardList': Whether each split children needs a separate node */ static List * CreateSplitIntervalsForShardGroup(List *sourceColocatedShardIntervalList, - List *splitPointsForShard) + List *splitPointsForShard, + List *needsSeparateNodeForShardList) { List *shardGroupSplitIntervalListList = NIL; @@ -1033,6 +1046,7 @@ CreateSplitIntervalsForShardGroup(List *sourceColocatedShardIntervalList, { List *shardSplitIntervalList = NIL; CreateSplitIntervalsForShard(shardToSplitInterval, splitPointsForShard, + needsSeparateNodeForShardList, &shardSplitIntervalList); shardGroupSplitIntervalListList = lappend(shardGroupSplitIntervalListList, @@ -1051,6 +1065,7 @@ CreateSplitIntervalsForShardGroup(List *sourceColocatedShardIntervalList, static void CreateSplitIntervalsForShard(ShardInterval *sourceShard, List *splitPointsForShard, + List *needsSeparateNodeForShardList, List **shardSplitChildrenIntervalList) { /* For 'N' split points, we will have N+1 shard intervals created. */ @@ -1075,7 +1090,8 @@ CreateSplitIntervalsForShard(ShardInterval *sourceShard, ShardInterval *splitChildShardInterval = CopyShardInterval(sourceShard); splitChildShardInterval->shardIndex = -1; splitChildShardInterval->shardId = GetNextShardIdForSplitChild(); - + splitChildShardInterval->needsSeparateNode = + list_nth_int(needsSeparateNodeForShardList, index); splitChildShardInterval->minValueExists = true; splitChildShardInterval->minValue = currentSplitChildMinValue; splitChildShardInterval->maxValueExists = true; @@ -1177,7 +1193,8 @@ InsertSplitChildrenShardMetadata(List *shardGroupSplitIntervalListList, shardInterval->shardId, shardInterval->storageType, IntegerToText(DatumGetInt32(shardInterval->minValue)), - IntegerToText(DatumGetInt32(shardInterval->maxValue))); + IntegerToText(DatumGetInt32(shardInterval->maxValue)), + shardInterval->needsSeparateNode); InsertShardPlacementRow( shardInterval->shardId, @@ -1373,6 +1390,7 @@ AcquireNonblockingSplitLock(Oid relationId) * sourceColocatedShardIntervalList : Source shard group to be split. * shardSplitPointsList : Split Points list for the source 'shardInterval'. * workersForPlacementList : Placement list corresponding to split children. + * needsSeparateNodeForShardList : Whether each split children needs a separate node. * distributionColumnList : Maps relation IDs to distribution columns. * If not specified, the distribution column is read * from the metadata. @@ -1385,6 +1403,7 @@ NonBlockingShardSplit(SplitOperation splitOperation, List *sourceColocatedShardIntervalList, List *shardSplitPointsList, List *workersForPlacementList, + List *needsSeparateNodeForShardList, DistributionColumnMap *distributionColumnOverrides, uint32 targetColocationId) { @@ -1398,7 +1417,8 @@ NonBlockingShardSplit(SplitOperation splitOperation, /* First create shard interval metadata for split children */ List *shardGroupSplitIntervalListList = CreateSplitIntervalsForShardGroup( sourceColocatedShardIntervalList, - shardSplitPointsList); + shardSplitPointsList, + needsSeparateNodeForShardList); ShardInterval *firstShard = linitial(sourceColocatedShardIntervalList); diff --git a/src/backend/distributed/operations/shard_transfer.c b/src/backend/distributed/operations/shard_transfer.c index 6796346c53d..f1f45d88245 100644 --- a/src/backend/distributed/operations/shard_transfer.c +++ b/src/backend/distributed/operations/shard_transfer.c @@ -121,7 +121,6 @@ static void EnsureShardCanBeCopied(int64 shardId, const char *sourceNodeName, int32 sourceNodePort, const char *targetNodeName, int32 targetNodePort); static List * RecreateTableDDLCommandList(Oid relationId); -static void EnsureTableListOwner(List *tableIdList); static void ErrorIfReplicatingDistributedTableWithFKeys(List *tableIdList); static void DropShardPlacementsFromMetadata(List *shardList, @@ -153,7 +152,6 @@ static bool TransferAlreadyCompleted(List *colocatedShardList, char *sourceNodeName, uint32 sourceNodePort, char *targetNodeName, uint32 targetNodePort, ShardTransferType transferType); -static void LockColocatedRelationsForMove(List *colocatedTableList); static void ErrorIfForeignTableForShardTransfer(List *colocatedTableList, ShardTransferType transferType); static List * RecreateShardDDLCommandList(ShardInterval *shardInterval, @@ -679,7 +677,7 @@ IsShardListOnNode(List *colocatedShardList, char *targetNodeName, uint32 targetN * LockColocatedRelationsForMove takes a list of relations, locks all of them * using ShareUpdateExclusiveLock */ -static void +void LockColocatedRelationsForMove(List *colocatedTableList) { Oid colocatedTableId = InvalidOid; @@ -1292,7 +1290,7 @@ LookupShardTransferMode(Oid shardReplicationModeOid) * EnsureTableListOwner ensures current user owns given tables. Superusers * are regarded as owners. */ -static void +void EnsureTableListOwner(List *tableIdList) { Oid tableId = InvalidOid; diff --git a/src/backend/distributed/operations/stage_protocol.c b/src/backend/distributed/operations/stage_protocol.c index 5770d648e21..c0225455308 100644 --- a/src/backend/distributed/operations/stage_protocol.c +++ b/src/backend/distributed/operations/stage_protocol.c @@ -166,7 +166,14 @@ master_create_empty_shard(PG_FUNCTION_ARGS) uint64 shardId = GetNextShardId(); /* if enough live groups, add an extra candidate node as backup */ - List *workerNodeList = DistributedTablePlacementNodeList(NoLock); + List *workerNodeList = NewDistributedTablePlacementNodeList(NoLock); + + if (workerNodeList == NIL) + { + ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("no worker nodes are available for placing shards"), + errhint("Add more worker nodes."))); + } if (list_length(workerNodeList) > ShardReplicationFactor) { @@ -193,7 +200,9 @@ master_create_empty_shard(PG_FUNCTION_ARGS) candidateNodeIndex++; } - InsertShardRow(relationId, shardId, storageType, nullMinValue, nullMaxValue); + bool needsSeparateNode = false; + InsertShardRow(relationId, shardId, storageType, nullMinValue, nullMaxValue, + needsSeparateNode); CreateAppendDistributedShardPlacements(relationId, shardId, candidateNodeList, ShardReplicationFactor); diff --git a/src/backend/distributed/operations/worker_node_manager.c b/src/backend/distributed/operations/worker_node_manager.c index ba622e4d7ab..31053aec581 100644 --- a/src/backend/distributed/operations/worker_node_manager.c +++ b/src/backend/distributed/operations/worker_node_manager.c @@ -273,30 +273,36 @@ ErrorIfCoordinatorNotAddedAsWorkerNode() /* - * DistributedTablePlacementNodeList returns a list of all active, primary + * NewDistributedTablePlacementNodeList returns a list of all active, primary * worker nodes that can store new data, i.e shouldstoreshards is 'true' + * and that is not used to isolate a shardgroup placement. */ List * -DistributedTablePlacementNodeList(LOCKMODE lockMode) +NewDistributedTablePlacementNodeList(LOCKMODE lockMode) { EnsureModificationsCanRun(); - return FilterActiveNodeListFunc(lockMode, NodeCanHaveDistTablePlacements); + return FilterActiveNodeListFunc(lockMode, NodeCanBeUsedForNonSeparatedPlacements); } /* - * NodeCanHaveDistTablePlacements returns true if the given node can have - * shards of a distributed table. + * NodeCanBeUsedForNonSeparatedPlacements returns true if given node can be + * used to store shard placements that don't need separate nodes. */ bool -NodeCanHaveDistTablePlacements(WorkerNode *node) +NodeCanBeUsedForNonSeparatedPlacements(WorkerNode *node) { if (!NodeIsPrimary(node)) { return false; } - return node->shouldHaveShards; + if (!node->shouldHaveShards) + { + return false; + } + + return NodeGroupGetSeparatedShardgroupPlacement(node->groupId) == NULL; } diff --git a/src/backend/distributed/planner/multi_physical_planner.c b/src/backend/distributed/planner/multi_physical_planner.c index fb7f844c7b4..0c12bc022b9 100644 --- a/src/backend/distributed/planner/multi_physical_planner.c +++ b/src/backend/distributed/planner/multi_physical_planner.c @@ -4522,6 +4522,8 @@ GenerateSyntheticShardIntervalArray(int partitionCount) shardInterval->shardId = INVALID_SHARD_ID; shardInterval->valueTypeId = INT4OID; + shardInterval->needsSeparateNode = false; + shardIntervalArray[shardIndex] = shardInterval; } diff --git a/src/backend/distributed/sql/citus--12.1-1--12.2-1.sql b/src/backend/distributed/sql/citus--12.1-1--12.2-1.sql index 68823b3be3a..a78b84fba7e 100644 --- a/src/backend/distributed/sql/citus--12.1-1--12.2-1.sql +++ b/src/backend/distributed/sql/citus--12.1-1--12.2-1.sql @@ -30,7 +30,6 @@ REVOKE ALL ON FUNCTION citus_internal.start_management_transaction FROM PUBLIC; #include "udfs/citus_internal_add_object_metadata/12.2-1.sql" #include "udfs/citus_internal_add_partition_metadata/12.2-1.sql" #include "udfs/citus_internal_add_placement_metadata/12.2-1.sql" -#include "udfs/citus_internal_add_shard_metadata/12.2-1.sql" #include "udfs/citus_internal_add_tenant_schema/12.2-1.sql" #include "udfs/citus_internal_adjust_local_clock_to_remote/12.2-1.sql" #include "udfs/citus_internal_delete_colocation_metadata/12.2-1.sql" @@ -52,3 +51,14 @@ DROP VIEW IF EXISTS pg_catalog.citus_lock_waits; #include "udfs/citus_internal_update_placement_metadata/12.2-1.sql" #include "udfs/citus_internal_update_relation_colocation/12.2-1.sql" #include "udfs/repl_origin_helper/12.2-1.sql" + +ALTER TABLE pg_dist_shard ADD COLUMN needsseparatenode boolean NOT NULL DEFAULT false; + +DROP FUNCTION pg_catalog.citus_internal_add_shard_metadata(regclass, bigint, "char", text, text); +#include "udfs/citus_internal_add_shard_metadata/12.2-1.sql" + +#include "udfs/citus_internal_shard_property_set/12.2-1.sql" +#include "udfs/citus_shard_property_set/12.2-1.sql" + +DROP VIEW citus_shards; +#include "udfs/citus_shards/12.2-1.sql" diff --git a/src/backend/distributed/sql/downgrades/citus--12.2-1--12.1-1.sql b/src/backend/distributed/sql/downgrades/citus--12.2-1--12.1-1.sql index 5b2828cfe2c..62cc006b6d8 100644 --- a/src/backend/distributed/sql/downgrades/citus--12.2-1--12.1-1.sql +++ b/src/backend/distributed/sql/downgrades/citus--12.2-1--12.1-1.sql @@ -27,7 +27,6 @@ DROP FUNCTION citus_internal.add_colocation_metadata(int, int, int, regtype, oid DROP FUNCTION citus_internal.add_object_metadata(text, text[], text[], integer, integer, boolean); DROP FUNCTION citus_internal.add_partition_metadata(regclass, "char", text, integer, "char"); DROP FUNCTION citus_internal.add_placement_metadata(bigint, bigint, integer, bigint); -DROP FUNCTION citus_internal.add_shard_metadata(regclass, bigint, "char", text, text); DROP FUNCTION citus_internal.add_tenant_schema(oid, integer); DROP FUNCTION citus_internal.adjust_local_clock_to_remote(pg_catalog.cluster_clock); DROP FUNCTION citus_internal.delete_colocation_metadata(int); @@ -51,3 +50,15 @@ DROP FUNCTION citus_internal.update_relation_colocation(oid, int); DROP FUNCTION citus_internal.start_replication_origin_tracking(); DROP FUNCTION citus_internal.stop_replication_origin_tracking(); DROP FUNCTION citus_internal.is_replication_origin_tracking_active(); + +DROP VIEW pg_catalog.citus_shards; +#include "../udfs/citus_shards/12.0-1.sql" + +DROP FUNCTION citus_internal.add_shard_metadata(regclass, bigint, "char", text, text, boolean); +DROP FUNCTION pg_catalog.citus_internal_add_shard_metadata(regclass, bigint, "char", text, text, boolean); +#include "../udfs/citus_internal_add_shard_metadata/10.2-1.sql" + +DROP FUNCTION pg_catalog.citus_shard_property_set(shard_id bigint, anti_affinity boolean); +DROP FUNCTION citus_internal.shard_property_set(shard_id bigint, needs_separate_node boolean); + +ALTER TABLE pg_dist_shard DROP COLUMN needsseparatenode; diff --git a/src/backend/distributed/sql/udfs/citus_internal_add_shard_metadata/12.2-1.sql b/src/backend/distributed/sql/udfs/citus_internal_add_shard_metadata/12.2-1.sql index 82c29f054dc..8e25c53b9a0 100644 --- a/src/backend/distributed/sql/udfs/citus_internal_add_shard_metadata/12.2-1.sql +++ b/src/backend/distributed/sql/udfs/citus_internal_add_shard_metadata/12.2-1.sql @@ -1,21 +1,23 @@ CREATE OR REPLACE FUNCTION citus_internal.add_shard_metadata( relation_id regclass, shard_id bigint, storage_type "char", shard_min_value text, - shard_max_value text + shard_max_value text, + needs_separate_node boolean default false ) RETURNS void LANGUAGE C AS 'MODULE_PATHNAME', $$citus_internal_add_shard_metadata$$; -COMMENT ON FUNCTION citus_internal.add_shard_metadata(regclass, bigint, "char", text, text) IS +COMMENT ON FUNCTION citus_internal.add_shard_metadata(regclass, bigint, "char", text, text, boolean) IS 'Inserts into pg_dist_shard with user checks'; CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_add_shard_metadata( relation_id regclass, shard_id bigint, storage_type "char", shard_min_value text, - shard_max_value text + shard_max_value text, + needs_separate_node boolean default false ) RETURNS void LANGUAGE C AS 'MODULE_PATHNAME'; -COMMENT ON FUNCTION pg_catalog.citus_internal_add_shard_metadata(regclass, bigint, "char", text, text) IS +COMMENT ON FUNCTION pg_catalog.citus_internal_add_shard_metadata(regclass, bigint, "char", text, text, boolean) IS 'Inserts into pg_dist_shard with user checks'; diff --git a/src/backend/distributed/sql/udfs/citus_internal_add_shard_metadata/latest.sql b/src/backend/distributed/sql/udfs/citus_internal_add_shard_metadata/latest.sql index 82c29f054dc..8e25c53b9a0 100644 --- a/src/backend/distributed/sql/udfs/citus_internal_add_shard_metadata/latest.sql +++ b/src/backend/distributed/sql/udfs/citus_internal_add_shard_metadata/latest.sql @@ -1,21 +1,23 @@ CREATE OR REPLACE FUNCTION citus_internal.add_shard_metadata( relation_id regclass, shard_id bigint, storage_type "char", shard_min_value text, - shard_max_value text + shard_max_value text, + needs_separate_node boolean default false ) RETURNS void LANGUAGE C AS 'MODULE_PATHNAME', $$citus_internal_add_shard_metadata$$; -COMMENT ON FUNCTION citus_internal.add_shard_metadata(regclass, bigint, "char", text, text) IS +COMMENT ON FUNCTION citus_internal.add_shard_metadata(regclass, bigint, "char", text, text, boolean) IS 'Inserts into pg_dist_shard with user checks'; CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_add_shard_metadata( relation_id regclass, shard_id bigint, storage_type "char", shard_min_value text, - shard_max_value text + shard_max_value text, + needs_separate_node boolean default false ) RETURNS void LANGUAGE C AS 'MODULE_PATHNAME'; -COMMENT ON FUNCTION pg_catalog.citus_internal_add_shard_metadata(regclass, bigint, "char", text, text) IS +COMMENT ON FUNCTION pg_catalog.citus_internal_add_shard_metadata(regclass, bigint, "char", text, text, boolean) IS 'Inserts into pg_dist_shard with user checks'; diff --git a/src/backend/distributed/sql/udfs/citus_internal_shard_property_set/12.2-1.sql b/src/backend/distributed/sql/udfs/citus_internal_shard_property_set/12.2-1.sql new file mode 100644 index 00000000000..337c676e6a3 --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_internal_shard_property_set/12.2-1.sql @@ -0,0 +1,6 @@ +CREATE OR REPLACE FUNCTION citus_internal.shard_property_set( + shard_id bigint, + needs_separate_node boolean) + RETURNS void + LANGUAGE C VOLATILE + AS 'MODULE_PATHNAME', $$citus_internal_shard_property_set$$; diff --git a/src/backend/distributed/sql/udfs/citus_internal_shard_property_set/latest.sql b/src/backend/distributed/sql/udfs/citus_internal_shard_property_set/latest.sql new file mode 100644 index 00000000000..337c676e6a3 --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_internal_shard_property_set/latest.sql @@ -0,0 +1,6 @@ +CREATE OR REPLACE FUNCTION citus_internal.shard_property_set( + shard_id bigint, + needs_separate_node boolean) + RETURNS void + LANGUAGE C VOLATILE + AS 'MODULE_PATHNAME', $$citus_internal_shard_property_set$$; diff --git a/src/backend/distributed/sql/udfs/citus_shard_property_set/12.2-1.sql b/src/backend/distributed/sql/udfs/citus_shard_property_set/12.2-1.sql new file mode 100644 index 00000000000..9f41bd8cbd8 --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_shard_property_set/12.2-1.sql @@ -0,0 +1,6 @@ +CREATE OR REPLACE FUNCTION pg_catalog.citus_shard_property_set(shard_id bigint, anti_affinity boolean default null) + RETURNS void + LANGUAGE C VOLATILE + AS 'MODULE_PATHNAME', $$citus_shard_property_set$$; +COMMENT ON FUNCTION pg_catalog.citus_shard_property_set(bigint, boolean) IS + 'Allows setting shard properties for all the shards within the shard group that given shard belongs to.'; diff --git a/src/backend/distributed/sql/udfs/citus_shard_property_set/latest.sql b/src/backend/distributed/sql/udfs/citus_shard_property_set/latest.sql new file mode 100644 index 00000000000..9f41bd8cbd8 --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_shard_property_set/latest.sql @@ -0,0 +1,6 @@ +CREATE OR REPLACE FUNCTION pg_catalog.citus_shard_property_set(shard_id bigint, anti_affinity boolean default null) + RETURNS void + LANGUAGE C VOLATILE + AS 'MODULE_PATHNAME', $$citus_shard_property_set$$; +COMMENT ON FUNCTION pg_catalog.citus_shard_property_set(bigint, boolean) IS + 'Allows setting shard properties for all the shards within the shard group that given shard belongs to.'; diff --git a/src/backend/distributed/sql/udfs/citus_shards/12.2-1.sql b/src/backend/distributed/sql/udfs/citus_shards/12.2-1.sql new file mode 100644 index 00000000000..2c1f36e4ac6 --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_shards/12.2-1.sql @@ -0,0 +1,71 @@ +CREATE OR REPLACE VIEW citus.citus_shards AS +SELECT + pg_dist_shard.logicalrelid AS table_name, + pg_dist_shard.shardid, + shard_name(pg_dist_shard.logicalrelid, pg_dist_shard.shardid) as shard_name, + CASE WHEN colocationid IN (SELECT colocationid FROM pg_dist_schema) THEN 'schema' + WHEN partkey IS NOT NULL THEN 'distributed' + WHEN repmodel = 't' THEN 'reference' + WHEN colocationid = 0 THEN 'local' + ELSE 'distributed' END AS citus_table_type, + colocationid AS colocation_id, + pg_dist_node.nodename, + pg_dist_node.nodeport, + size as shard_size, + CASE + WHEN NOT pg_dist_shard.needsseparatenode THEN false + ELSE + -- has_separate_node = true if the node doesn't have any other shard + -- placements except the ones that belong to the same shard group. + NOT EXISTS ( + SELECT 1 + FROM pg_dist_shard pds1 + JOIN pg_dist_placement pdp1 USING (shardid) + JOIN pg_dist_partition pdp2 USING (logicalrelid) + WHERE + -- get the distributed table placements that are placed on the same node as this placement + pdp1.groupid = pg_dist_placement.groupid AND + (pdp2.partkey IS NOT NULL OR (pdp2.repmodel != 't' AND pdp2.colocationid != 0)) AND + -- filter out all the placements that belong to the same shard group + NOT ( + pdp2.colocationid = pg_dist_partition.colocationid AND + ((pds1.shardminvalue IS NULL AND pg_dist_shard.shardminvalue IS NULL) OR (pds1.shardminvalue = pg_dist_shard.shardminvalue)) + ) + ) + END AS has_separate_node +FROM + pg_dist_shard +JOIN + pg_dist_placement +ON + pg_dist_shard.shardid = pg_dist_placement.shardid +JOIN + pg_dist_node +ON + pg_dist_placement.groupid = pg_dist_node.groupid +JOIN + pg_dist_partition +ON + pg_dist_partition.logicalrelid = pg_dist_shard.logicalrelid +LEFT JOIN + (SELECT shard_id, max(size) as size from citus_shard_sizes() GROUP BY shard_id) as shard_sizes +ON + pg_dist_shard.shardid = shard_sizes.shard_id +WHERE + pg_dist_placement.shardstate = 1 +AND + -- filter out tables owned by extensions + pg_dist_partition.logicalrelid NOT IN ( + SELECT + objid + FROM + pg_depend + WHERE + classid = 'pg_class'::regclass AND refclassid = 'pg_extension'::regclass AND deptype = 'e' + ) +ORDER BY + pg_dist_shard.logicalrelid::text, shardid +; + +ALTER VIEW citus.citus_shards SET SCHEMA pg_catalog; +GRANT SELECT ON pg_catalog.citus_shards TO public; diff --git a/src/backend/distributed/sql/udfs/citus_shards/latest.sql b/src/backend/distributed/sql/udfs/citus_shards/latest.sql index f1be9219d78..2c1f36e4ac6 100644 --- a/src/backend/distributed/sql/udfs/citus_shards/latest.sql +++ b/src/backend/distributed/sql/udfs/citus_shards/latest.sql @@ -11,7 +11,28 @@ SELECT colocationid AS colocation_id, pg_dist_node.nodename, pg_dist_node.nodeport, - size as shard_size + size as shard_size, + CASE + WHEN NOT pg_dist_shard.needsseparatenode THEN false + ELSE + -- has_separate_node = true if the node doesn't have any other shard + -- placements except the ones that belong to the same shard group. + NOT EXISTS ( + SELECT 1 + FROM pg_dist_shard pds1 + JOIN pg_dist_placement pdp1 USING (shardid) + JOIN pg_dist_partition pdp2 USING (logicalrelid) + WHERE + -- get the distributed table placements that are placed on the same node as this placement + pdp1.groupid = pg_dist_placement.groupid AND + (pdp2.partkey IS NOT NULL OR (pdp2.repmodel != 't' AND pdp2.colocationid != 0)) AND + -- filter out all the placements that belong to the same shard group + NOT ( + pdp2.colocationid = pg_dist_partition.colocationid AND + ((pds1.shardminvalue IS NULL AND pg_dist_shard.shardminvalue IS NULL) OR (pds1.shardminvalue = pg_dist_shard.shardminvalue)) + ) + ) + END AS has_separate_node FROM pg_dist_shard JOIN diff --git a/src/backend/distributed/test/distribution_metadata.c b/src/backend/distributed/test/distribution_metadata.c index 01117922e36..d8da0aa95fa 100644 --- a/src/backend/distributed/test/distribution_metadata.c +++ b/src/backend/distributed/test/distribution_metadata.c @@ -230,8 +230,9 @@ create_monolithic_shard_row(PG_FUNCTION_ARGS) text *minInfoText = cstring_to_text(minInfo->data); text *maxInfoText = cstring_to_text(maxInfo->data); + bool needsSeparateNode = false; InsertShardRow(distributedTableId, newShardId, SHARD_STORAGE_TABLE, minInfoText, - maxInfoText); + maxInfoText, needsSeparateNode); PG_RETURN_INT64(newShardId); } diff --git a/src/backend/distributed/test/shard_rebalancer.c b/src/backend/distributed/test/shard_rebalancer.c index 32bfd9f463c..b534d45d081 100644 --- a/src/backend/distributed/test/shard_rebalancer.c +++ b/src/backend/distributed/test/shard_rebalancer.c @@ -45,9 +45,10 @@ static uint64 JsonFieldValueUInt64Default(Datum jsonDocument, const char *key, uint64 defaultValue); static char * JsonFieldValueString(Datum jsonDocument, const char *key); static ArrayType * PlacementUpdateListToJsonArray(List *placementUpdateList); -static bool ShardAllowedOnNode(uint64 shardId, WorkerNode *workerNode, void *context); +static bool ShardAllowedOnNode(uint64 shardId, uint64 placementId, WorkerNode *workerNode, + void *context); static float NodeCapacity(WorkerNode *workerNode, void *context); -static ShardCost GetShardCost(uint64 shardId, void *context); +static ShardCost GetShardCost(uint64 shardId, uint64 placementId, void *context); PG_FUNCTION_INFO_V1(shard_placement_rebalance_array); @@ -192,7 +193,8 @@ shard_placement_rebalance_array(PG_FUNCTION_ARGS) * a worker when running the shard rebalancer unit tests. */ static bool -ShardAllowedOnNode(uint64 shardId, WorkerNode *workerNode, void *voidContext) +ShardAllowedOnNode(uint64 shardId, uint64 placementId, WorkerNode *workerNode, + void *voidContext) { RebalancePlacementContext *context = voidContext; WorkerTestInfo *workerTestInfo = NULL; @@ -243,12 +245,13 @@ NodeCapacity(WorkerNode *workerNode, void *voidContext) * the shard rebalancer unit tests. */ static ShardCost -GetShardCost(uint64 shardId, void *voidContext) +GetShardCost(uint64 shardId, uint64 placementId, void *voidContext) { RebalancePlacementContext *context = voidContext; ShardCost shardCost; memset_struct_0(shardCost); shardCost.shardId = shardId; + shardCost.placementId = placementId; ShardPlacementTestInfo *shardPlacementTestInfo = NULL; foreach_ptr(shardPlacementTestInfo, context->shardPlacementTestInfoList) diff --git a/src/backend/distributed/utils/colocation_utils.c b/src/backend/distributed/utils/colocation_utils.c index c189195271d..4b854e2696b 100644 --- a/src/backend/distributed/utils/colocation_utils.c +++ b/src/backend/distributed/utils/colocation_utils.c @@ -1245,7 +1245,7 @@ SingleShardTableColocationNodeId(uint32 colocationId) { int workerNodeIndex = EmptySingleShardTableColocationDecideNodeId(colocationId); - List *workerNodeList = DistributedTablePlacementNodeList(RowShareLock); + List *workerNodeList = NewDistributedTablePlacementNodeList(RowShareLock); WorkerNode *workerNode = (WorkerNode *) list_nth(workerNodeList, workerNodeIndex); return workerNode->nodeId; diff --git a/src/backend/distributed/utils/listutils.c b/src/backend/distributed/utils/listutils.c index eddef1fea09..0766a9c99b1 100644 --- a/src/backend/distributed/utils/listutils.c +++ b/src/backend/distributed/utils/listutils.c @@ -270,6 +270,23 @@ GenerateListFromElement(void *listElement, int listLength) } +/* + * GenerateListFromIntElement returns a new list with length of listLength + * such that all the elements are identical with input listElement integer. + */ +List * +GenerateListFromIntElement(int listElement, int listLength) +{ + List *list = NIL; + for (int i = 0; i < listLength; i++) + { + list = lappend_int(list, listElement); + } + + return list; +} + + /* * list_filter_oid filters a list of oid-s based on a keepElement * function @@ -289,3 +306,21 @@ list_filter_oid(List *list, bool (*keepElement)(Oid element)) return result; } + + +/* + * FlattenNestedList takes a list of lists and returns a flattened list. + */ +List * +FlattenNestedList(List *nestedList) +{ + List *flattenedList = NIL; + + List *subList = NULL; + foreach_ptr(subList, nestedList) + { + flattenedList = list_concat(flattenedList, subList); + } + + return flattenedList; +} diff --git a/src/include/distributed/listutils.h b/src/include/distributed/listutils.h index 2a52cbc7527..ffcdc7e8195 100644 --- a/src/include/distributed/listutils.h +++ b/src/include/distributed/listutils.h @@ -174,6 +174,8 @@ extern List * ListTake(List *pointerList, int size); extern void * safe_list_nth(const List *list, int index); extern List * GeneratePositiveIntSequenceList(int upTo); extern List * GenerateListFromElement(void *listElement, int listLength); +extern List * GenerateListFromIntElement(int listElement, int listLength); extern List * list_filter_oid(List *list, bool (*keepElement)(Oid element)); +extern List * FlattenNestedList(List *nestedList); #endif /* CITUS_LISTUTILS_H */ diff --git a/src/include/distributed/metadata_sync.h b/src/include/distributed/metadata_sync.h index cb111e16e03..de79b17ecdd 100644 --- a/src/include/distributed/metadata_sync.h +++ b/src/include/distributed/metadata_sync.h @@ -144,6 +144,7 @@ extern char * TenantSchemaInsertCommand(Oid schemaId, uint32 colocationId); extern char * TenantSchemaDeleteCommand(char *schemaName); extern char * UpdateNoneDistTableMetadataCommand(Oid relationId, char replicationModel, uint32 colocationId, bool autoConverted); +extern char * ShardgroupSetPropertyCommand(uint64 shardId, bool *needsSeparateNodePtr); extern char * AddPlacementMetadataCommand(uint64 shardId, uint64 placementId, uint64 shardLength, int32 groupId); extern char * DeletePlacementMetadataCommand(uint64 placementId); diff --git a/src/include/distributed/metadata_utility.h b/src/include/distributed/metadata_utility.h index 737e1283ba0..511ccd04a60 100644 --- a/src/include/distributed/metadata_utility.h +++ b/src/include/distributed/metadata_utility.h @@ -68,6 +68,7 @@ typedef struct ShardInterval Datum maxValue; /* a shard's typed max value datum */ uint64 shardId; int shardIndex; + bool needsSeparateNode; } ShardInterval; @@ -104,6 +105,14 @@ typedef struct ShardPlacement } ShardPlacement; +typedef struct +{ + uint32 colocatationId; + int shardIntervalIndex; + int32 nodeGroupId; +} ShardgroupPlacement; + + typedef enum CascadeToColocatedOption { CASCADE_TO_COLOCATED_UNSPECIFIED, @@ -323,7 +332,13 @@ extern int ShardIntervalCount(Oid relationId); extern List * LoadShardList(Oid relationId); extern ShardInterval * CopyShardInterval(ShardInterval *srcInterval); extern uint64 ShardLength(uint64 shardId); +extern ShardgroupPlacement * NodeGroupGetSeparatedShardgroupPlacement(int32 groupId); +extern bool ShardgroupPlacementsSame(const ShardgroupPlacement *leftGroup, + const ShardgroupPlacement *rightGroup); extern bool NodeGroupHasShardPlacements(int32 groupId); +extern bool NodeGroupHasDistributedTableShardPlacements(int32 groupId); +extern ShardgroupPlacement * GetShardgroupPlacementForPlacement(uint64 shardId, + uint64 placementId); extern bool IsActiveShardPlacement(ShardPlacement *ShardPlacement); extern bool IsRemoteShardPlacement(ShardPlacement *shardPlacement); extern bool IsPlacementOnWorkerNode(ShardPlacement *placement, WorkerNode *workerNode); @@ -350,7 +365,8 @@ extern List * RemoveCoordinatorPlacementIfNotSingleNode(List *placementList); /* Function declarations to modify shard and shard placement data */ extern void InsertShardRow(Oid relationId, uint64 shardId, char storageType, - text *shardMinValue, text *shardMaxValue); + text *shardMinValue, text *shardMaxValue, + bool needsSeparateNode); extern void DeleteShardRow(uint64 shardId); extern ShardPlacement * InsertShardPlacementRowGlobally(uint64 shardId, uint64 placementId, @@ -434,6 +450,7 @@ extern List * SendShardStatisticsQueriesInParallel(List *citusTableIds, extern bool GetNodeDiskSpaceStatsForConnection(MultiConnection *connection, uint64 *availableBytes, uint64 *totalBytes); +extern void ShardgroupSetProperty(uint64 shardId, bool *needsSeparateNodePtr); extern void ExecuteQueryViaSPI(char *query, int SPIOK); extern void ExecuteAndLogQueryViaSPI(char *query, int SPIOK, int logLevel); extern void EnsureSequenceTypeSupported(Oid seqOid, Oid attributeTypeId, Oid diff --git a/src/include/distributed/pg_dist_shard.h b/src/include/distributed/pg_dist_shard.h index 5c98b755f3d..1a65bb4f127 100644 --- a/src/include/distributed/pg_dist_shard.h +++ b/src/include/distributed/pg_dist_shard.h @@ -30,6 +30,10 @@ typedef struct FormData_pg_dist_shard text shardminvalue; /* partition key's minimum value in shard */ text shardmaxvalue; /* partition key's maximum value in shard */ #endif + bool needsSeparateNode; /* + * Whether the placements of this shard need to be + * separated from others. + */ } FormData_pg_dist_shard; /* ---------------- @@ -43,13 +47,14 @@ typedef FormData_pg_dist_shard *Form_pg_dist_shard; * compiler constants for pg_dist_shards * ---------------- */ -#define Natts_pg_dist_shard 6 +#define Natts_pg_dist_shard 7 #define Anum_pg_dist_shard_logicalrelid 1 #define Anum_pg_dist_shard_shardid 2 #define Anum_pg_dist_shard_shardstorage 3 #define Anum_pg_dist_shard_shardalias_DROPPED 4 #define Anum_pg_dist_shard_shardminvalue 5 #define Anum_pg_dist_shard_shardmaxvalue 6 +#define Anum_pg_dist_shard_needsseparatenode 7 /* * Valid values for shard storage types include foreign table, (standard) table diff --git a/src/include/distributed/rebalancer_placement_separation.h b/src/include/distributed/rebalancer_placement_separation.h new file mode 100644 index 00000000000..8b59aad495f --- /dev/null +++ b/src/include/distributed/rebalancer_placement_separation.h @@ -0,0 +1,37 @@ +/*------------------------------------------------------------------------- + * + * rebalancer_placement_separation.h + * Routines to determine which worker node should be used to separate + * a colocated set of shard placements that need separate nodes. + * + * Copyright (c) Citus Data, Inc. + * + *------------------------------------------------------------------------- + */ + +#ifndef PLACEMENT_ISOLATION_H +#define PLACEMENT_ISOLATION_H + +#include "postgres.h" + +#include "nodes/pg_list.h" +#include "utils/hsearch.h" + +#include "distributed/metadata_utility.h" + +struct RebalancerPlacementSeparationContext; +typedef struct RebalancerPlacementSeparationContext RebalancerPlacementSeparationContext; + +/* *INDENT-OFF* */ +extern RebalancerPlacementSeparationContext * PrepareRebalancerPlacementSeparationContext( + List *activeWorkerNodeList, + List *activeShardPlacementList, + FmgrInfo shardAllowedOnNodeUDF); +extern bool RebalancerPlacementSeparationContextPlacementIsAllowedOnWorker( + RebalancerPlacementSeparationContext *context, + uint64 shardId, + uint64 placementId, + WorkerNode *workerNode); +/* *INDENT-ON* */ + +#endif /* PLACEMENT_ISOLATION_H */ diff --git a/src/include/distributed/shard_rebalancer.h b/src/include/distributed/shard_rebalancer.h index 79414eb3c88..4c7184c9c14 100644 --- a/src/include/distributed/shard_rebalancer.h +++ b/src/include/distributed/shard_rebalancer.h @@ -164,6 +164,7 @@ typedef struct NodeFillState typedef struct ShardCost { uint64 shardId; + uint64 placementId; /* * cost is the cost of the shard. This doesn't have a unit. @@ -182,9 +183,10 @@ typedef struct DisallowedPlacement typedef struct RebalancePlanFunctions { - bool (*shardAllowedOnNode)(uint64 shardId, WorkerNode *workerNode, void *context); + bool (*shardAllowedOnNode)(uint64 shardId, uint64 placementId, WorkerNode *workerNode, + void *context); float4 (*nodeCapacity)(WorkerNode *workerNode, void *context); - ShardCost (*shardCost)(uint64 shardId, void *context); + ShardCost (*shardCost)(uint64 shardId, uint64 placementId, void *context); void *context; } RebalancePlanFunctions; @@ -222,4 +224,7 @@ extern void SetupRebalanceMonitor(List *placementUpdateList, uint64 initialProgressState, PlacementUpdateStatus initialStatus); +extern List * NonColocatedDistRelationIdList(void); +extern List * FullShardPlacementList(Oid relationId, ArrayType *excludedShardArray); + #endif /* SHARD_REBALANCER_H */ diff --git a/src/include/distributed/shard_split.h b/src/include/distributed/shard_split.h index 5bd0c7686d7..f874b51515a 100644 --- a/src/include/distributed/shard_split.h +++ b/src/include/distributed/shard_split.h @@ -43,6 +43,7 @@ extern void SplitShard(SplitMode splitMode, uint64 shardIdToSplit, List *shardSplitPointsList, List *nodeIdsForPlacementList, + List *needsSeparateNodeForPlacementList, DistributionColumnMap *distributionColumnOverrides, List *colocatedShardIntervalList, uint32 targetColocationId); diff --git a/src/include/distributed/shard_transfer.h b/src/include/distributed/shard_transfer.h index c1621879bb8..503f6e8c186 100644 --- a/src/include/distributed/shard_transfer.h +++ b/src/include/distributed/shard_transfer.h @@ -30,6 +30,7 @@ extern void TransferShards(int64 shardId, extern uint64 ShardListSizeInBytes(List *colocatedShardList, char *workerNodeName, uint32 workerNodePort); extern void ErrorIfMoveUnsupportedTableType(Oid relationId); +extern void EnsureTableListOwner(List *tableIdList); extern void CopyShardsToNode(WorkerNode *sourceNode, WorkerNode *targetNode, List *shardIntervalList, char *snapshotName); extern void VerifyTablesHaveReplicaIdentity(List *colocatedTableList); @@ -41,3 +42,4 @@ extern void UpdatePlacementUpdateStatusForShardIntervalList(List *shardIntervalL extern void InsertDeferredDropCleanupRecordsForShards(List *shardIntervalList); extern void InsertCleanupRecordsForShardPlacementsOnNode(List *shardIntervalList, int32 groupId); +extern void LockColocatedRelationsForMove(List *colocatedTableList); diff --git a/src/include/distributed/worker_manager.h b/src/include/distributed/worker_manager.h index 02a43fe0b99..5e67728bbb7 100644 --- a/src/include/distributed/worker_manager.h +++ b/src/include/distributed/worker_manager.h @@ -33,6 +33,7 @@ #define WORKER_DEFAULT_CLUSTER "default" +#define INVALID_GROUP_ID -1 #define COORDINATOR_GROUP_ID 0 /* @@ -76,8 +77,8 @@ extern bool CoordinatorAddedAsWorkerNode(void); extern List * ReferenceTablePlacementNodeList(LOCKMODE lockMode); extern WorkerNode * CoordinatorNodeIfAddedAsWorkerOrError(void); extern void ErrorIfCoordinatorNotAddedAsWorkerNode(void); -extern List * DistributedTablePlacementNodeList(LOCKMODE lockMode); -extern bool NodeCanHaveDistTablePlacements(WorkerNode *node); +extern List * NewDistributedTablePlacementNodeList(LOCKMODE lockMode); +extern bool NodeCanBeUsedForNonSeparatedPlacements(WorkerNode *node); extern List * ActiveReadableNonCoordinatorNodeList(void); extern List * ActiveReadableNodeList(void); extern WorkerNode * FindWorkerNode(const char *nodeName, int32 nodePort); diff --git a/src/test/regress/after_pg_upgrade_schedule b/src/test/regress/after_pg_upgrade_schedule index 82e05cf3f79..3c46cb6bd87 100644 --- a/src/test/regress/after_pg_upgrade_schedule +++ b/src/test/regress/after_pg_upgrade_schedule @@ -1,5 +1,9 @@ test: upgrade_basic_after upgrade_ref2ref_after upgrade_type_after upgrade_distributed_function_after upgrade_rebalance_strategy_after upgrade_list_citus_objects upgrade_autoconverted_after upgrade_citus_stat_activity upgrade_citus_locks upgrade_single_shard_table_after upgrade_schema_based_sharding_after upgrade_basic_after_non_mixed +# This test temporarily renames citus_schema back to public schema, +# hence cannot be run in parallel with any other tests. +test: upgrade_isolate_placement_after + # This test cannot be run with run_test.py currently due to its dependence on # the specific PG versions that we use to run upgrade tests. For now we leave # it out of the parallel line, so that flaky test detection can at least work diff --git a/src/test/regress/before_pg_upgrade_schedule b/src/test/regress/before_pg_upgrade_schedule index 95957f8cee4..f796ef4c0db 100644 --- a/src/test/regress/before_pg_upgrade_schedule +++ b/src/test/regress/before_pg_upgrade_schedule @@ -5,7 +5,7 @@ test: upgrade_basic_before test: upgrade_ref2ref_before test: upgrade_type_before test: upgrade_distributed_function_before upgrade_rebalance_strategy_before -test: upgrade_autoconverted_before upgrade_single_shard_table_before upgrade_schema_based_sharding_before +test: upgrade_autoconverted_before upgrade_single_shard_table_before upgrade_schema_based_sharding_before upgrade_isolate_placement_before test: upgrade_citus_stat_activity test: upgrade_citus_locks test: upgrade_distributed_triggers_before diff --git a/src/test/regress/citus_tests/run_test.py b/src/test/regress/citus_tests/run_test.py index 9a648c0ab8a..b3f5400eed7 100755 --- a/src/test/regress/citus_tests/run_test.py +++ b/src/test/regress/citus_tests/run_test.py @@ -157,6 +157,7 @@ def extra_tests(self): None, ["isolation_setup", "isolation_add_remove_node"] ), "schema_based_sharding": TestDeps("minimal_schedule"), + "isolate_placement": TestDeps("minimal_schedule"), "multi_sequence_default": TestDeps( None, ["multi_test_helpers", "multi_cluster_management", "multi_table_ddl"] ), diff --git a/src/test/regress/expected/citus_shards.out b/src/test/regress/expected/citus_shards.out index b434a984b70..febb76c77ea 100644 --- a/src/test/regress/expected/citus_shards.out +++ b/src/test/regress/expected/citus_shards.out @@ -21,16 +21,16 @@ SELECT create_distributed_table('"t with space"', 'i'); INSERT INTO t1 SELECT generate_series(1, 100); INSERT INTO "t with space" SELECT generate_series(1, 1000); SELECT * FROM citus_shards; - table_name | shardid | shard_name | citus_table_type | colocation_id | nodename | nodeport | shard_size + table_name | shardid | shard_name | citus_table_type | colocation_id | nodename | nodeport | shard_size | has_separate_node --------------------------------------------------------------------- - "t with space" | 99456904 | citus_shards."t with space_99456904" | distributed | 456900 | localhost | 57637 | 40960 - "t with space" | 99456905 | citus_shards."t with space_99456905" | distributed | 456900 | localhost | 57638 | 40960 - "t with space" | 99456906 | citus_shards."t with space_99456906" | distributed | 456900 | localhost | 57637 | 40960 - "t with space" | 99456907 | citus_shards."t with space_99456907" | distributed | 456900 | localhost | 57638 | 40960 - t1 | 99456900 | citus_shards.t1_99456900 | distributed | 456900 | localhost | 57637 | 8192 - t1 | 99456901 | citus_shards.t1_99456901 | distributed | 456900 | localhost | 57638 | 8192 - t1 | 99456902 | citus_shards.t1_99456902 | distributed | 456900 | localhost | 57637 | 8192 - t1 | 99456903 | citus_shards.t1_99456903 | distributed | 456900 | localhost | 57638 | 8192 + "t with space" | 99456904 | citus_shards."t with space_99456904" | distributed | 456900 | localhost | 57637 | 40960 | f + "t with space" | 99456905 | citus_shards."t with space_99456905" | distributed | 456900 | localhost | 57638 | 40960 | f + "t with space" | 99456906 | citus_shards."t with space_99456906" | distributed | 456900 | localhost | 57637 | 40960 | f + "t with space" | 99456907 | citus_shards."t with space_99456907" | distributed | 456900 | localhost | 57638 | 40960 | f + t1 | 99456900 | citus_shards.t1_99456900 | distributed | 456900 | localhost | 57637 | 8192 | f + t1 | 99456901 | citus_shards.t1_99456901 | distributed | 456900 | localhost | 57638 | 8192 | f + t1 | 99456902 | citus_shards.t1_99456902 | distributed | 456900 | localhost | 57637 | 8192 | f + t1 | 99456903 | citus_shards.t1_99456903 | distributed | 456900 | localhost | 57638 | 8192 | f (8 rows) SET client_min_messages TO WARNING; diff --git a/src/test/regress/expected/failure_create_distributed_table_concurrently.out b/src/test/regress/expected/failure_create_distributed_table_concurrently.out index cf1df651aff..c824aa30dfe 100644 --- a/src/test/regress/expected/failure_create_distributed_table_concurrently.out +++ b/src/test/regress/expected/failure_create_distributed_table_concurrently.out @@ -189,10 +189,10 @@ SELECT create_distributed_table_concurrently('table_1', 'id'); (1 row) SELECT * FROM pg_dist_shard WHERE logicalrelid = 'table_1'::regclass; - logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue + logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode --------------------------------------------------------------------- - table_1 | 222247 | t | -2147483648 | -1 - table_1 | 222248 | t | 0 | 2147483647 + table_1 | 222247 | t | -2147483648 | -1 | f + table_1 | 222248 | t | 0 | 2147483647 | f (2 rows) DROP SCHEMA create_dist_tbl_con CASCADE; diff --git a/src/test/regress/expected/isolate_placement.out b/src/test/regress/expected/isolate_placement.out new file mode 100644 index 00000000000..88da8047a8e --- /dev/null +++ b/src/test/regress/expected/isolate_placement.out @@ -0,0 +1,2120 @@ +-- Due to a race condition that happens in TransferShards() when the same shard id +-- is used to create the same shard on a different worker node, need to call +-- citus_cleanup_orphaned_resources() to clean up any orphaned resources before +-- running the tests. +-- +-- See https://github.com/citusdata/citus/pull/7180#issuecomment-1706786615. +SET client_min_messages TO WARNING; +CALL citus_cleanup_orphaned_resources(); +RESET client_min_messages; +CREATE SCHEMA isolate_placement; +SET search_path TO isolate_placement; +-- test null input +SELECT citus_internal.shard_property_set(NULL, false); +ERROR: shard_id cannot be NULL +SET citus.shard_replication_factor TO 1; +SET citus.next_shard_id TO 2000000; +CREATE TABLE single_shard_1(a int); +SELECT create_distributed_table('single_shard_1', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- test with user that doesn't have permission to execute the function +SELECT citus_internal.shard_property_set(shardid, true) FROM pg_dist_shard WHERE logicalrelid = 'isolate_placement.single_shard_1'::regclass; +ERROR: This is an internal Citus function can only be used in a distributed transaction +DROP TABLE single_shard_1; +CREATE ROLE test_user_isolate_placement WITH LOGIN; +GRANT ALL ON SCHEMA isolate_placement TO test_user_isolate_placement; +ALTER SYSTEM SET citus.enable_manual_metadata_changes_for_user TO 'test_user_isolate_placement'; +SELECT pg_reload_conf(); + pg_reload_conf +--------------------------------------------------------------------- + t +(1 row) + +SELECT pg_sleep(0.1); + pg_sleep +--------------------------------------------------------------------- + +(1 row) + +SET ROLE test_user_isolate_placement; +-- test invalid shard id +SELECT citus_internal.shard_property_set(0, true); +ERROR: could not find valid entry for shard xxxxx +-- test null needs_separate_node +SELECT citus_internal_add_shard_metadata( + relation_id=>0, + shard_id=>0, + storage_type=>'0', + shard_min_value=>'0', + shard_max_value=>'0', + needs_separate_node=>null); +ERROR: needs separate node cannot be NULL +RESET ROLE; +REVOKE ALL ON SCHEMA isolate_placement FROM test_user_isolate_placement; +DROP USER test_user_isolate_placement; +ALTER SYSTEM RESET citus.enable_manual_metadata_changes_for_user; +SELECT pg_reload_conf(); + pg_reload_conf +--------------------------------------------------------------------- + t +(1 row) + +SELECT pg_sleep(0.1); + pg_sleep +--------------------------------------------------------------------- + +(1 row) + +SET search_path TO isolate_placement; +SET citus.shard_replication_factor TO 1; +SET citus.next_shard_id TO 2001000; +CREATE USER mysuperuser superuser; +SET ROLE mysuperuser; +CREATE TABLE single_shard_1(a int); +SELECT create_distributed_table('single_shard_1', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE USER regularuser; +GRANT USAGE ON SCHEMA isolate_placement TO regularuser; +ALTER SYSTEM SET citus.enable_manual_metadata_changes_for_user TO 'regularuser'; +SELECT pg_reload_conf(); + pg_reload_conf +--------------------------------------------------------------------- + t +(1 row) + +SELECT pg_sleep(0.1); + pg_sleep +--------------------------------------------------------------------- + +(1 row) + +SET ROLE regularuser; +-- throws an error as the user is not the owner of the table +SELECT citus_shard_property_set(shardid) FROM pg_dist_shard WHERE logicalrelid = 'isolate_placement.single_shard_1'::regclass; +ERROR: must be owner of table single_shard_1 +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid = 'isolate_placement.single_shard_1'::regclass; +ERROR: must be owner of table single_shard_1 +SELECT citus_internal.shard_property_set(shardid, true) FROM pg_dist_shard WHERE logicalrelid = 'isolate_placement.single_shard_1'::regclass; +ERROR: must be owner of table single_shard_1 +-- assign all tables to regularuser +RESET ROLE; +REASSIGN OWNED BY mysuperuser TO regularuser; +SET ROLE regularuser; +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid = 'isolate_placement.single_shard_1'::regclass; + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.single_shard_1') +$$) +ORDER BY result; + result +--------------------------------------------------------------------- + [{"1": [{"isolate_placement.single_shard_1": true}]}] + [{"1": [{"isolate_placement.single_shard_1": true}]}] + [{"1": [{"isolate_placement.single_shard_1": true}]}] +(3 rows) + +SELECT citus_internal.shard_property_set(shardid, false) FROM pg_dist_shard WHERE logicalrelid = 'isolate_placement.single_shard_1'::regclass; + shard_property_set +--------------------------------------------------------------------- + +(1 row) + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.single_shard_1') +$$) +ORDER BY result; + result +--------------------------------------------------------------------- + [{"1": [{"isolate_placement.single_shard_1": true}]}] + [{"1": [{"isolate_placement.single_shard_1": true}]}] + {} +(3 rows) + +SELECT citus_internal.shard_property_set(shardid, true) FROM pg_dist_shard WHERE logicalrelid = 'isolate_placement.single_shard_1'::regclass; + shard_property_set +--------------------------------------------------------------------- + +(1 row) + +DROP TABLE single_shard_1; +RESET ROLE; +REVOKE USAGE ON SCHEMA isolate_placement FROM regularuser; +ALTER SYSTEM RESET citus.enable_manual_metadata_changes_for_user; +SELECT pg_reload_conf(); + pg_reload_conf +--------------------------------------------------------------------- + t +(1 row) + +SELECT pg_sleep(0.1); + pg_sleep +--------------------------------------------------------------------- + +(1 row) + +DROP ROLE regularuser, mysuperuser; +SET search_path TO isolate_placement; +SET citus.next_shard_id TO 2002000; +SET citus.shard_count TO 32; +SET citus.shard_replication_factor TO 1; +SET client_min_messages TO WARNING; +SELECT 1 FROM citus_add_node('localhost', :master_port, groupid => 0); + ?column? +--------------------------------------------------------------------- + 1 +(1 row) + +SET client_min_messages TO NOTICE; +SET citus.shard_replication_factor TO 2; +CREATE TABLE dist_1(a int); +CREATE TABLE dist_2(a int); +CREATE TABLE dist_3(a int); +SELECT create_distributed_table('dist_1', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('dist_2', 'a', colocate_with=>'dist_1'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('dist_3', 'a', colocate_with=>'dist_1'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SET citus.shard_replication_factor TO 1; +-- none of the placements have been marked as needsseparatenode yet +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + result +--------------------------------------------------------------------- + {} + {} + {} +(3 rows) + +SELECT shardids[2] AS shardgroup_5_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 5 \gset +-- no-op .. +SELECT citus_shard_property_set(:shardgroup_5_shardid, NULL); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +CREATE ROLE test_user_isolate_placement WITH LOGIN; +GRANT ALL ON SCHEMA isolate_placement TO test_user_isolate_placement; +ALTER TABLE dist_1 OWNER TO test_user_isolate_placement; +ALTER TABLE dist_2 OWNER TO test_user_isolate_placement; +ALTER TABLE dist_3 OWNER TO test_user_isolate_placement; +ALTER SYSTEM SET citus.enable_manual_metadata_changes_for_user TO 'test_user_isolate_placement'; +SELECT pg_reload_conf(); + pg_reload_conf +--------------------------------------------------------------------- + t +(1 row) + +SELECT pg_sleep(0.1); + pg_sleep +--------------------------------------------------------------------- + +(1 row) + +SET ROLE test_user_isolate_placement; +-- no-op .. +SELECT citus_internal.shard_property_set(:shardgroup_5_shardid, NULL); + shard_property_set +--------------------------------------------------------------------- + +(1 row) + +RESET ROLE; +ALTER TABLE dist_1 OWNER TO current_user; +ALTER TABLE dist_2 OWNER TO current_user; +ALTER TABLE dist_3 OWNER TO current_user; +REVOKE ALL ON SCHEMA isolate_placement FROM test_user_isolate_placement; +DROP USER test_user_isolate_placement; +ALTER SYSTEM RESET citus.enable_manual_metadata_changes_for_user; +SELECT pg_reload_conf(); + pg_reload_conf +--------------------------------------------------------------------- + t +(1 row) + +SELECT pg_sleep(0.1); + pg_sleep +--------------------------------------------------------------------- + +(1 row) + +-- .. hence returns empty objects +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + result +--------------------------------------------------------------------- + {} + {} + {} +(3 rows) + +SELECT citus_shard_property_set(:shardgroup_5_shardid, anti_affinity=>true); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +SELECT shardids[3] AS shardgroup_10_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 10 \gset +SELECT citus_shard_property_set(:shardgroup_10_shardid, anti_affinity=>true); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + result +--------------------------------------------------------------------- + [{"5": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}]}, {"10": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}]}] + [{"5": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}]}, {"10": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}]}] + [{"5": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}]}, {"10": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}]}] +(3 rows) + +SELECT shardids[1] AS shardgroup_3_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 3 \gset +SELECT citus_shard_property_set(:shardgroup_3_shardid, anti_affinity=>false); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + result +--------------------------------------------------------------------- + [{"5": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}]}, {"10": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}]}] + [{"5": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}]}, {"10": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}]}] + [{"5": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}]}, {"10": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}]}] +(3 rows) + +SELECT shardids[1] AS shardgroup_10_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 10 \gset +SELECT citus_shard_property_set(:shardgroup_10_shardid, anti_affinity=>false); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + result +--------------------------------------------------------------------- + [{"5": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}]}] + [{"5": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}]}] + [{"5": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}]}] +(3 rows) + +SELECT shardids[1] AS shardgroup_5_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 5 \gset +SELECT citus_shard_property_set(:shardgroup_5_shardid, anti_affinity=>true); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + result +--------------------------------------------------------------------- + [{"5": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}]}] + [{"5": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}]}] + [{"5": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}]}] +(3 rows) + +-- test metadata sync +-- first, need to re-create them with shard_replication_factor = 1 because we will first remove worker_2 +DROP TABLE dist_1, dist_2, dist_3; +SELECT 1 FROM citus_remove_node('localhost', :worker_2_port); + ?column? +--------------------------------------------------------------------- + 1 +(1 row) + +CREATE TABLE dist_1(a int); +CREATE TABLE dist_2(a int); +CREATE TABLE dist_3(a int); +SELECT create_distributed_table('dist_1', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('dist_2', 'a', colocate_with=>'dist_1'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('dist_3', 'a', colocate_with=>'dist_1'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT 1 FROM citus_add_node('localhost', :worker_2_port); + ?column? +--------------------------------------------------------------------- + 1 +(1 row) + +SELECT shardids[1] AS shardgroup_5_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 5 \gset +SELECT citus_shard_property_set(:shardgroup_5_shardid, anti_affinity=>true); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + result +--------------------------------------------------------------------- + [{"5": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}]}] + [{"5": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}]}] + [{"5": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}]}] +(3 rows) + +CREATE TABLE dist_4(a int); +SELECT create_distributed_table('dist_4', 'a', colocate_with=>'dist_1'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE dist_4_concurrently(a int); +SELECT create_distributed_table_concurrently('dist_4_concurrently', 'a', colocate_with=>'dist_1'); +NOTICE: relation dist_4_concurrently does not have a REPLICA IDENTITY or PRIMARY KEY +DETAIL: UPDATE and DELETE commands on the relation will error out during create_distributed_table_concurrently unless there is a REPLICA IDENTITY or PRIMARY KEY. INSERT commands will still work. + create_distributed_table_concurrently +--------------------------------------------------------------------- + +(1 row) + +-- Placements of a new distributed table created within the same colocated +-- group inherit needsseparatenode from the colocated placements too. +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + result +--------------------------------------------------------------------- + [{"5": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}, {"isolate_placement.dist_4": true}, {"isolate_placement.dist_4_concurrently": true}]}] + [{"5": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}, {"isolate_placement.dist_4": true}, {"isolate_placement.dist_4_concurrently": true}]}] + [{"5": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}, {"isolate_placement.dist_4": true}, {"isolate_placement.dist_4_concurrently": true}]}] +(3 rows) + +DROP TABLE dist_4, dist_4_concurrently; +-- Returns source and target node ids that can be used to perform a +-- shard transfer for one of the placements of given shard. +CREATE OR REPLACE FUNCTION get_candidate_node_for_shard_transfer( + p_shardid bigint) +RETURNS TABLE (source_nodeid integer, target_nodeid integer) +SET search_path TO 'pg_catalog, public' +AS $func$ +DECLARE + v_source_nodeids integer[]; + v_target_nodeid integer; +BEGIN + SELECT array_agg(nodeid) INTO v_source_nodeids + FROM pg_dist_shard + JOIN pg_dist_placement USING (shardid) + JOIN pg_dist_node USING (groupid) + WHERE noderole = 'primary' AND shardid = p_shardid; + + IF v_source_nodeids IS NULL + THEN + RAISE EXCEPTION 'could not determine the source node of shard %', p_shardid; + END IF; + + SELECT nodeid INTO v_target_nodeid + FROM pg_dist_node + WHERE isactive AND shouldhaveshards AND noderole='primary' AND + nodeid NOT IN (SELECT unnest(v_source_nodeids)) + LIMIT 1; + + IF v_target_nodeid IS NULL + THEN + RAISE EXCEPTION 'could not determine a node to transfer the placement to'; + END IF; + + RETURN QUERY SELECT v_source_nodeids[1], v_target_nodeid; +END; +$func$ LANGUAGE plpgsql; +SELECT shardids[1] AS shardgroup_15_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 15 \gset +SELECT citus_move_shard_placement(:shardgroup_5_shardid, source_nodeid, target_nodeid, 'block_writes') +FROM get_candidate_node_for_shard_transfer(:shardgroup_5_shardid); + citus_move_shard_placement +--------------------------------------------------------------------- + +(1 row) + +SELECT citus_move_shard_placement(:shardgroup_15_shardid, source_nodeid, target_nodeid, 'block_writes') +FROM get_candidate_node_for_shard_transfer(:shardgroup_15_shardid); + citus_move_shard_placement +--------------------------------------------------------------------- + +(1 row) + +-- so that citus_copy_shard_placement works +UPDATE pg_dist_partition SET repmodel = 'c' WHERE logicalrelid = 'isolate_placement.dist_1'::regclass; +SELECT citus_copy_shard_placement(:shardgroup_5_shardid, source_nodeid, target_nodeid, 'block_writes') +FROM get_candidate_node_for_shard_transfer(:shardgroup_5_shardid); + citus_copy_shard_placement +--------------------------------------------------------------------- + +(1 row) + +SELECT citus_copy_shard_placement(:shardgroup_15_shardid, source_nodeid, target_nodeid, 'block_writes') +FROM get_candidate_node_for_shard_transfer(:shardgroup_15_shardid); + citus_copy_shard_placement +--------------------------------------------------------------------- + +(1 row) + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + result +--------------------------------------------------------------------- + [{"5": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}]}] + [{"5": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}]}] + [{"5": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_3": true}]}] +(3 rows) + +DROP TABLE dist_1, dist_2, dist_3; +CREATE TABLE dist_1(a int); +CREATE TABLE dist_2(a int); +SELECT create_distributed_table('dist_1', 'a', shard_count=>3); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('dist_2', 'a', colocate_with=>'dist_1'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT shardids[1] AS shardgroup_3_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 3 \gset +SELECT citus_shard_property_set(:shardgroup_3_shardid, anti_affinity=>true); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + result +--------------------------------------------------------------------- + [{"3": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}]}] + [{"3": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}]}] + [{"3": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}]}] +(3 rows) + +-- so that replicate_table_shards works +UPDATE pg_dist_partition SET repmodel = 'c' WHERE logicalrelid = 'isolate_placement.dist_1'::regclass; +SET client_min_messages TO WARNING; +SELECT replicate_table_shards('isolate_placement.dist_1', shard_replication_factor=>2, shard_transfer_mode=>'block_writes'); + replicate_table_shards +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO NOTICE; +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + result +--------------------------------------------------------------------- + [{"3": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}]}] + [{"3": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}]}] + [{"3": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}]}] +(3 rows) + +DROP TABLE dist_1, dist_2; +CREATE TABLE dist_1(a int); +CREATE TABLE dist_2(a int); +SELECT create_distributed_table('dist_1', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('dist_2', 'a', colocate_with=>'dist_1'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT shardids[1] AS shardgroup_9_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 9 \gset +SELECT citus_shard_property_set(:shardgroup_9_shardid, anti_affinity=>true); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + result +--------------------------------------------------------------------- + [{"9": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}]}] + [{"9": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}]}] + [{"9": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}]}] +(3 rows) + +SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset +SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset +SELECT pg_catalog.citus_split_shard_by_split_points( + :shardgroup_9_shardid, + ARRAY[((shardminvalue::bigint + shardmaxvalue::bigint) / 2)::text], + ARRAY[:worker_1_node, :worker_2_node], + 'block_writes') +FROM pg_dist_shard +WHERE shardid = :shardgroup_9_shardid; + citus_split_shard_by_split_points +--------------------------------------------------------------------- + +(1 row) + +-- We shouldn't see shard group 9 because shard-split operation doesn't +-- preserve needsseparatenode flag when splitting the shard. +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + result +--------------------------------------------------------------------- + {} + {} + {} +(3 rows) + +SELECT shardids[1] AS shardgroup_12_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 12 \gset +SELECT citus_shard_property_set(:shardgroup_12_shardid, anti_affinity=>true); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + result +--------------------------------------------------------------------- + [{"12": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}]}] + [{"12": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}]}] + [{"12": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}]}] +(3 rows) + +SELECT shardids[1] AS shardgroup_10_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 10 \gset +SELECT pg_catalog.citus_split_shard_by_split_points( + :shardgroup_10_shardid, + ARRAY[((shardminvalue::bigint + shardmaxvalue::bigint) / 2)::text], + ARRAY[:worker_1_node, :worker_2_node], + 'block_writes') +FROM pg_dist_shard +WHERE shardid = :shardgroup_10_shardid; + citus_split_shard_by_split_points +--------------------------------------------------------------------- + +(1 row) + +-- We should see old shard group 12 (now as 13 due to split +-- of a prior shard) because it's not the one we splitted. +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + result +--------------------------------------------------------------------- + [{"13": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}]}] + [{"13": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}]}] + [{"13": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}]}] +(3 rows) + +CREATE TABLE dist_3(a int); +SELECT create_distributed_table('dist_3', 'a', colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT shardids[1] AS shardgroup_17_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_3') +WHERE shardgroupindex = 17 \gset +SELECT citus_shard_property_set(:shardgroup_17_shardid, anti_affinity=>true); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_3') +$$) +ORDER BY result; + result +--------------------------------------------------------------------- + [{"17": [{"isolate_placement.dist_3": true}]}] + [{"17": [{"isolate_placement.dist_3": true}]}] + [{"17": [{"isolate_placement.dist_3": true}]}] +(3 rows) + +-- verify that shard key value 100 is stored on shard group 17 +select get_shard_id_for_distribution_column('dist_3', 100) = :shardgroup_17_shardid; + ?column? +--------------------------------------------------------------------- + t +(1 row) + +SELECT 1 FROM isolate_tenant_to_new_shard('dist_3', 100, shard_transfer_mode => 'block_writes'); + ?column? +--------------------------------------------------------------------- + 1 +(1 row) + +-- We shouldn't see shard group 17 because isolate_tenant_to_new_shard doesn't +-- preserve needsseparatenode flag when splitting the shard. +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_3') +$$) +ORDER BY result; + result +--------------------------------------------------------------------- + {} + {} + {} +(3 rows) + +SELECT shardids[1] AS shardgroup_18_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_3') +WHERE shardgroupindex = 18 \gset +SELECT citus_shard_property_set(:shardgroup_18_shardid, anti_affinity=>true); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_3') +$$) +ORDER BY result; + result +--------------------------------------------------------------------- + [{"18": [{"isolate_placement.dist_3": true}]}] + [{"18": [{"isolate_placement.dist_3": true}]}] + [{"18": [{"isolate_placement.dist_3": true}]}] +(3 rows) + +-- verify that shard key value 1000 is _not_ stored on shard group 18 +SELECT get_shard_id_for_distribution_column('dist_3', 1000) != :shardgroup_18_shardid; + ?column? +--------------------------------------------------------------------- + t +(1 row) + +SELECT 1 FROM isolate_tenant_to_new_shard('dist_3', 1000, shard_transfer_mode => 'block_writes'); + ?column? +--------------------------------------------------------------------- + 1 +(1 row) + +-- We should see shard group 18 (now as 20 due to split of a prior shard) +-- because it's not the one we splitted. +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_3') +$$) +ORDER BY result; + result +--------------------------------------------------------------------- + [{"20": [{"isolate_placement.dist_3": true}]}] + [{"20": [{"isolate_placement.dist_3": true}]}] + [{"20": [{"isolate_placement.dist_3": true}]}] +(3 rows) + +CREATE TABLE single_shard_1(a int); +SELECT create_distributed_table('single_shard_1', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT shardids[1] AS shardgroup_1_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.single_shard_1') +WHERE shardgroupindex = 1 \gset +SELECT citus_shard_property_set(:shardgroup_1_shardid, anti_affinity=>true); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +-- noop +SELECT citus_shard_property_set(:shardgroup_1_shardid, NULL); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +SELECT citus_shard_property_set(:shardgroup_1_shardid); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE single_shard_2(a int); +SELECT create_distributed_table('single_shard_2', null, colocate_with=>'single_shard_1'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.single_shard_1') +$$) +ORDER BY result; + result +--------------------------------------------------------------------- + [{"1": [{"isolate_placement.single_shard_1": true}, {"isolate_placement.single_shard_2": true}]}] + [{"1": [{"isolate_placement.single_shard_1": true}, {"isolate_placement.single_shard_2": true}]}] + [{"1": [{"isolate_placement.single_shard_1": true}, {"isolate_placement.single_shard_2": true}]}] +(3 rows) + +-- test invalid input +SELECT citus_shard_property_set(NULL, anti_affinity=>true); +ERROR: shard_id cannot be NULL +SELECT citus_shard_property_set(0, anti_affinity=>true); +ERROR: could not find valid entry for shard xxxxx +SELECT citus_shard_property_set(NULL, anti_affinity=>false); +ERROR: shard_id cannot be NULL +SELECT citus_shard_property_set(0, anti_affinity=>false); +ERROR: could not find valid entry for shard xxxxx +-- we verify whether shard exists even if anti_affinity is not provided +SELECT citus_shard_property_set(0, anti_affinity=>NULL); +ERROR: could not find valid entry for shard xxxxx +CREATE TABLE append_table (a int, b int); +SELECT create_distributed_table('append_table', 'a', 'append'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT 1 FROM master_create_empty_shard('append_table'); + ?column? +--------------------------------------------------------------------- + 1 +(1 row) + +CREATE TYPE composite_key_type AS (f1 int, f2 text); +CREATE TABLE range_table(key composite_key_type, value int); +SELECT create_distributed_table('range_table', 'key', 'range'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CALL public.create_range_partitioned_shards('range_table', '{"(0,a)","(25,a)"}','{"(24,z)","(49,z)"}'); +CREATE TABLE ref_table(a int); +SELECT create_reference_table('ref_table'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE local_table(a int); +SELECT citus_add_local_table_to_metadata('local_table'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +-- all should fail +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid = 'append_table'::regclass LIMIT 1; +ERROR: setting anti-affinity property is only supported for hash distributed tables +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid = 'range_table'::regclass LIMIT 1; +ERROR: setting anti-affinity property is only supported for hash distributed tables +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid = 'ref_table'::regclass LIMIT 1; +ERROR: setting anti-affinity property is only supported for hash distributed tables +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid = 'local_table'::regclass LIMIT 1; +ERROR: setting anti-affinity property is only supported for hash distributed tables +SELECT citus_shard_property_set(shardid, anti_affinity=>false) FROM pg_dist_shard WHERE logicalrelid = 'append_table'::regclass LIMIT 1; +ERROR: setting anti-affinity property is only supported for hash distributed tables +SELECT citus_shard_property_set(shardid, anti_affinity=>false) FROM pg_dist_shard WHERE logicalrelid = 'range_table'::regclass LIMIT 1; +ERROR: setting anti-affinity property is only supported for hash distributed tables +SELECT citus_shard_property_set(shardid, anti_affinity=>false) FROM pg_dist_shard WHERE logicalrelid = 'ref_table'::regclass LIMIT 1; +ERROR: setting anti-affinity property is only supported for hash distributed tables +SELECT citus_shard_property_set(shardid, anti_affinity=>false) FROM pg_dist_shard WHERE logicalrelid = 'local_table'::regclass LIMIT 1; +ERROR: setting anti-affinity property is only supported for hash distributed tables +DROP TABLE range_table; +DROP TYPE composite_key_type; +SET client_min_messages TO WARNING; +DROP SCHEMA isolate_placement CASCADE; +CREATE SCHEMA isolate_placement; +SET search_path TO isolate_placement; +SET client_min_messages TO NOTICE; +CREATE TABLE dist_1(a int); +CREATE TABLE dist_2(a int); +SELECT create_distributed_table('dist_1', 'a', shard_count=>4); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('dist_2', 'a', colocate_with=>'dist_1'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE dist_non_colocated(a int); +SELECT create_distributed_table('dist_non_colocated', 'a', shard_count=>4, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE single_shard_1(a int); +SELECT create_distributed_table('single_shard_1', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE single_shard_2(a int); +SELECT create_distributed_table('single_shard_2', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE append_table (a int, b int); +SELECT create_distributed_table('append_table', 'a', 'append'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT 1 FROM master_create_empty_shard('append_table'); + ?column? +--------------------------------------------------------------------- + 1 +(1 row) + +CREATE TABLE range_table(a int, b int); +SELECT create_distributed_table('range_table', 'a', 'range'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CALL public.create_range_partitioned_shards('range_table', '{"0","25"}','{"26","50"}'); +CREATE TABLE reference_table_1(a int); +SELECT create_reference_table('reference_table_1'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE local_table_1(a int); +SELECT citus_add_local_table_to_metadata('local_table_1'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +SELECT shardids[1] AS shardgroup_1_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 1 \gset +SELECT citus_shard_property_set(:shardgroup_1_shardid, anti_affinity=>true); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO WARNING; +SELECT rebalance_table_shards(shard_transfer_mode=>'block_writes'); + rebalance_table_shards +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO NOTICE; +CREATE TABLE reference_table_2(a int); +SELECT create_reference_table('reference_table_2'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE local_table_2(a int); +SELECT citus_add_local_table_to_metadata('local_table_2'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +-- make sure that we still have placements for both reference tables on all nodes +SELECT COUNT(DISTINCT(groupid))=3 FROM pg_dist_shard JOIN pg_dist_placement USING (shardid) WHERE logicalrelid = 'reference_table_1'::regclass; + ?column? +--------------------------------------------------------------------- + t +(1 row) + +SELECT COUNT(DISTINCT(groupid))=3 FROM pg_dist_shard JOIN pg_dist_placement USING (shardid) WHERE logicalrelid = 'reference_table_2'::regclass; + ?column? +--------------------------------------------------------------------- + t +(1 row) + +-- sanity check for local tables +SELECT groupid = 0 FROM pg_dist_shard JOIN pg_dist_placement USING (shardid) WHERE logicalrelid = 'local_table_1'::regclass; + ?column? +--------------------------------------------------------------------- + t +(1 row) + +SELECT groupid = 0 FROM pg_dist_shard JOIN pg_dist_placement USING (shardid) WHERE logicalrelid = 'local_table_2'::regclass; + ?column? +--------------------------------------------------------------------- + t +(1 row) + +CREATE TABLE dist_post_non_colocated(a int); +SELECT create_distributed_table('dist_post_non_colocated', 'a', shard_count=>4, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE dist_post_concurrently_non_colocated(a int); +SELECT create_distributed_table_concurrently('dist_post_concurrently_non_colocated', 'a', shard_count=>4, colocate_with=>'none'); +NOTICE: relation dist_post_concurrently_non_colocated does not have a REPLICA IDENTITY or PRIMARY KEY +DETAIL: UPDATE and DELETE commands on the relation will error out during create_distributed_table_concurrently unless there is a REPLICA IDENTITY or PRIMARY KEY. INSERT commands will still work. + create_distributed_table_concurrently +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE dist_post_colocated(a int); +SELECT create_distributed_table('dist_post_colocated', 'a', colocate_with=>'dist_1'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE dist_post_concurrently_colocated(a int); +SELECT create_distributed_table_concurrently('dist_post_concurrently_colocated', 'a', colocate_with=>'dist_1'); +NOTICE: relation dist_post_concurrently_colocated does not have a REPLICA IDENTITY or PRIMARY KEY +DETAIL: UPDATE and DELETE commands on the relation will error out during create_distributed_table_concurrently unless there is a REPLICA IDENTITY or PRIMARY KEY. INSERT commands will still work. + create_distributed_table_concurrently +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE single_shard_post(a int); +SELECT create_distributed_table('single_shard_post', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE append_table_post(a int, b int); +SELECT create_distributed_table('append_table_post', 'a', 'append'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT 1 FROM master_create_empty_shard('append_table_post'); + ?column? +--------------------------------------------------------------------- + 1 +(1 row) + +CREATE TABLE range_table_post(a int, b int); +SELECT create_distributed_table('range_table_post', 'a', 'range'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CALL public.create_range_partitioned_shards('range_table_post', '{"0","25"}','{"26","50"}'); +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + result +--------------------------------------------------------------------- + [{"1": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_post_colocated": true}, {"isolate_placement.dist_post_concurrently_colocated": true}]}] + [{"1": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_post_colocated": true}, {"isolate_placement.dist_post_concurrently_colocated": true}]}] + [{"1": [{"isolate_placement.dist_1": true}, {"isolate_placement.dist_2": true}, {"isolate_placement.dist_post_colocated": true}, {"isolate_placement.dist_post_concurrently_colocated": true}]}] +(3 rows) + +-- Make sure that the node that contains shard-group 1 of isolate_placement.dist_1 +-- doesn't have any other placements. +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.dist_1', 1); + verify_placements_in_shard_group_isolated +--------------------------------------------------------------------- + t +(1 row) + +SET client_min_messages TO ERROR; +SELECT citus_drain_node('localhost', :worker_1_port, shard_transfer_mode=>'block_writes'); + citus_drain_node +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO NOTICE; +SELECT citus_set_node_property('localhost', :worker_1_port, 'shouldhaveshards', true); + citus_set_node_property +--------------------------------------------------------------------- + +(1 row) + +-- drain node should have failed and the node should still have the same set of placements +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.dist_1', 1); + verify_placements_in_shard_group_isolated +--------------------------------------------------------------------- + t +(1 row) + +SET client_min_messages TO ERROR; +SELECT citus_drain_node('localhost', :worker_2_port, shard_transfer_mode=>'block_writes'); + citus_drain_node +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO NOTICE; +SELECT citus_set_node_property('localhost', :worker_2_port, 'shouldhaveshards', true); + citus_set_node_property +--------------------------------------------------------------------- + +(1 row) + +-- drain node should have failed and the node should still have the same set of placements +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.dist_1', 1); + verify_placements_in_shard_group_isolated +--------------------------------------------------------------------- + t +(1 row) + +CREATE TABLE dist_3(a int); +SELECT create_distributed_table('dist_3', 'a', colocate_with=>'dist_1'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.dist_1', 1); + verify_placements_in_shard_group_isolated +--------------------------------------------------------------------- + t +(1 row) + +SET citus.shard_replication_factor TO 2; +CREATE TABLE dist_replicated(a int); +-- fails as we only have one node that's not used to isolate a shard placement group +SELECT create_distributed_table('dist_replicated', 'a', shard_count=>4, colocate_with=>'none'); +ERROR: replication_factor (2) exceeds number of available worker nodes (1) +HINT: Add more worker nodes or try again with a lower replication factor. +SET citus.shard_replication_factor TO 1; +CREATE TABLE dist_to_be_replicated(a int); +SELECT create_distributed_table('dist_to_be_replicated', 'a', shard_count=>4, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +UPDATE pg_dist_partition SET repmodel = 'c' WHERE logicalrelid = 'isolate_placement.dist_to_be_replicated'::regclass; +SET client_min_messages TO WARNING; +-- fails as we only have one node that's not used to isolate a shard placement group +SELECT replicate_table_shards('isolate_placement.dist_to_be_replicated', shard_replication_factor=>2, shard_transfer_mode=>'block_writes'); +ERROR: could not find a target for shard xxxxx +SET client_min_messages TO NOTICE; +SELECT citus_set_node_property('localhost', :master_port, 'shouldhaveshards', true); + citus_set_node_property +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO WARNING; +-- succeeds as now we have two nodes that are not used to isolate a shard placement group +SELECT replicate_table_shards('isolate_placement.dist_to_be_replicated', shard_replication_factor=>2, shard_transfer_mode=>'block_writes'); + replicate_table_shards +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO NOTICE; +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.dist_1', 1); + verify_placements_in_shard_group_isolated +--------------------------------------------------------------------- + t +(1 row) + +SELECT DISTINCT(table_name::regclass::text) +FROM citus_shards +JOIN pg_class ON (oid = table_name) +WHERE relnamespace = 'isolate_placement'::regnamespace AND has_separate_node +ORDER BY 1; + table_name +--------------------------------------------------------------------- + dist_1 + dist_2 + dist_3 + dist_post_colocated + dist_post_concurrently_colocated +(5 rows) + +SELECT bool_or(has_separate_node) = false +FROM citus_shards +JOIN ( + SELECT unnest(shardids) shardid + FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') + WHERE shardgroupindex != 1 +) shards_except_group_1 USING (shardid); + ?column? +--------------------------------------------------------------------- + t +(1 row) + +DROP TABLE dist_to_be_replicated; +SELECT citus_drain_node('localhost', :master_port, shard_transfer_mode=>'block_writes'); + citus_drain_node +--------------------------------------------------------------------- + +(1 row) + +DROP TABLE dist_replicated; +SET client_min_messages TO WARNING; +DROP SCHEMA isolate_placement CASCADE; +CREATE SCHEMA isolate_placement; +SET search_path TO isolate_placement; +SET client_min_messages TO NOTICE; +CREATE TABLE single_shard_1(a int); +SELECT create_distributed_table('single_shard_1', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE single_shard_2(a int); +SELECT create_distributed_table('single_shard_2', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_1'::regclass, 'single_shard_2'::regclass); + citus_shard_property_set +--------------------------------------------------------------------- + + +(2 rows) + +SET client_min_messages TO WARNING; +SELECT rebalance_table_shards(shard_transfer_mode=>'block_writes'); + rebalance_table_shards +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO NOTICE; +-- fails +CREATE TABLE dist_1(a int); +SELECT create_distributed_table('dist_1', 'a', shard_count=>4); +ERROR: replication_factor (1) exceeds number of available worker nodes (0) +HINT: Add more worker nodes or try again with a lower replication factor. +CREATE TABLE single_shard_3(a int); +SELECT create_distributed_table('single_shard_3', null, colocate_with=>'none'); +ERROR: couldn't find any available worker nodes +HINT: Add more worker nodes +CREATE TABLE append_table (a int, b int); +SELECT create_distributed_table('append_table', 'a', 'append'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT 1 FROM master_create_empty_shard('append_table'); +ERROR: no worker nodes are available for placing shards +HINT: Add more worker nodes. +CREATE TABLE range_table(a int, b int); +SELECT create_distributed_table('range_table', 'a', 'range'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CALL public.create_range_partitioned_shards('range_table', '{"0","25"}','{"26","50"}'); +ERROR: no worker nodes are available for placing shards +HINT: Add more worker nodes. +CONTEXT: SQL statement "SELECT master_create_empty_shard(rel::text)" +PL/pgSQL function public.create_range_partitioned_shards(regclass,text[],text[]) line XX at SQL statement +-- succeeds +CREATE TABLE reference_table_1(a int); +SELECT create_reference_table('reference_table_1'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE local_table_1(a int); +SELECT citus_add_local_table_to_metadata('local_table_1'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE single_shard_4(a int); +SELECT create_distributed_table('single_shard_4', null, colocate_with=>'single_shard_1'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_1', 1); + verify_placements_in_shard_group_isolated +--------------------------------------------------------------------- + t +(1 row) + +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_2', 1); + verify_placements_in_shard_group_isolated +--------------------------------------------------------------------- + t +(1 row) + +SET client_min_messages TO WARNING; +DROP SCHEMA isolate_placement CASCADE; +CREATE SCHEMA isolate_placement; +SET search_path TO isolate_placement; +SET client_min_messages TO NOTICE; +CREATE TABLE single_shard_1(a int); +SELECT create_distributed_table('single_shard_1', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE single_shard_2(a int); +SELECT create_distributed_table('single_shard_2', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- Make sure that we don't assume that a node is used to isolate a shard placement +-- group just because it contains a single shard placement group. +CREATE TABLE single_shard_3(a int); +SELECT create_distributed_table('single_shard_3', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO WARNING; +DROP SCHEMA isolate_placement CASCADE; +CREATE SCHEMA isolate_placement; +SET search_path TO isolate_placement; +SET client_min_messages TO NOTICE; +SELECT citus_set_node_property('localhost', :master_port, 'shouldhaveshards', true); + citus_set_node_property +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE dist_1(a int); +CREATE TABLE dist_2(a int); -- will replicate this +CREATE TABLE dist_3(a int); +SELECT create_distributed_table('dist_1', 'a', shard_count=>1); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('dist_2', 'a', shard_count=>1, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('dist_3', 'a', shard_count=>1, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid IN ('dist_1'::regclass, 'dist_2'::regclass); + citus_shard_property_set +--------------------------------------------------------------------- + + +(2 rows) + +SET client_min_messages TO WARNING; +SELECT rebalance_table_shards(shard_transfer_mode=>'block_writes'); + rebalance_table_shards +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO NOTICE; +-- so that replicate_table_shards works +UPDATE pg_dist_partition SET repmodel = 'c' WHERE logicalrelid = 'isolate_placement.dist_2'::regclass; +SET client_min_messages TO WARNING; +-- succeeds but breaks the isolation requirement for either of dist_1 or dist_2 .. +SELECT replicate_table_shards('isolate_placement.dist_2', shard_replication_factor=>2, shard_transfer_mode=>'block_writes'); + replicate_table_shards +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO NOTICE; +-- .. so check the xor of the isolation requirements for dist_1 and dist_2 +SELECT (public.verify_placements_in_shard_group_isolated('isolate_placement.dist_1', 1) OR public.verify_placements_in_shard_group_isolated('isolate_placement.dist_2', 1)) = true AND + (public.verify_placements_in_shard_group_isolated('isolate_placement.dist_1', 1) AND public.verify_placements_in_shard_group_isolated('isolate_placement.dist_2', 1)) = false; + ?column? +--------------------------------------------------------------------- + t +(1 row) + +DROP TABLE dist_1, dist_2, dist_3; +SELECT citus_drain_node('localhost', :master_port, shard_transfer_mode=>'block_writes'); + citus_drain_node +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE single_shard_1(a int); +SELECT create_distributed_table('single_shard_1', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE single_shard_2(a int); +SELECT create_distributed_table('single_shard_2', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE dist_1(a int); +SELECT create_distributed_table('dist_1', 'a', shard_count=>4); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_1'::regclass); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +SELECT groupid AS single_shard_1_group_id FROM pg_dist_shard JOIN pg_dist_placement USING (shardid) WHERE logicalrelid = 'isolate_placement.single_shard_1'::regclass \gset +SET client_min_messages TO WARNING; +SELECT rebalance_table_shards(shard_transfer_mode=>'block_writes'); + rebalance_table_shards +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO NOTICE; +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_1', 1); + verify_placements_in_shard_group_isolated +--------------------------------------------------------------------- + t +(1 row) + +-- show that we try to isolate placements where they were staying at the time rebalancer is invoked +SELECT groupid = :single_shard_1_group_id FROM pg_dist_shard JOIN pg_dist_placement USING (shardid) WHERE logicalrelid = 'isolate_placement.single_shard_1'::regclass; + ?column? +--------------------------------------------------------------------- + t +(1 row) + +DROP TABLE dist_1, single_shard_1, single_shard_2; +SET citus.shard_replication_factor TO 2; +CREATE TABLE dist_1(a int); +CREATE TABLE dist_2(a int); +SELECT citus_set_node_property('localhost', :master_port, 'shouldhaveshards', true); + citus_set_node_property +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('dist_1', 'a', shard_count=>1); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('dist_2', 'a', colocate_with=>'dist_1'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SET citus.shard_replication_factor TO 1; +SELECT shardids[1] AS shardgroup_1_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_2') +WHERE shardgroupindex = 1 \gset +SELECT citus_shard_property_set(:shardgroup_1_shardid, anti_affinity=>true); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO WARNING; +SELECT rebalance_table_shards(shard_transfer_mode=>'block_writes'); + rebalance_table_shards +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO NOTICE; +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.dist_1', 1) = true; + ?column? +--------------------------------------------------------------------- + t +(1 row) + +DROP TABLE dist_1, dist_2; +SELECT citus_drain_node('localhost', :master_port, shard_transfer_mode=>'block_writes'); + citus_drain_node +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE single_shard_1(a int); +SELECT create_distributed_table('single_shard_1', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE single_shard_2(a int); +SELECT create_distributed_table('single_shard_2', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE single_shard_3(a int); +SELECT create_distributed_table('single_shard_3', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE single_shard_4(a int); +SELECT create_distributed_table('single_shard_4', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_1'::regclass); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO WARNING; +SELECT rebalance_table_shards(shard_transfer_mode=>'block_writes'); + rebalance_table_shards +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO NOTICE; +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_1', 1); + verify_placements_in_shard_group_isolated +--------------------------------------------------------------------- + t +(1 row) + +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_2'::regclass); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +SELECT citus_shard_property_set(shardid, anti_affinity=>false) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_1'::regclass); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO WARNING; +SELECT rebalance_table_shards(shard_transfer_mode=>'block_writes'); + rebalance_table_shards +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO NOTICE; +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_2', 1); + verify_placements_in_shard_group_isolated +--------------------------------------------------------------------- + t +(1 row) + +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_1', 1) = false; + ?column? +--------------------------------------------------------------------- + t +(1 row) + +DROP TABLE single_shard_1, single_shard_2, single_shard_3, single_shard_4; +CREATE TABLE single_shard_1(a int); +SELECT create_distributed_table('single_shard_1', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE single_shard_2(a int); +SELECT create_distributed_table('single_shard_2', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- this would be placed on the same node as single_shard_1 +CREATE TABLE single_shard_3(a int); +SELECT create_distributed_table('single_shard_3', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +DROP TABLE single_shard_2; +SELECT shardid, nodeid INTO single_shard_3_shardid_nodeid +FROM pg_dist_shard JOIN pg_dist_placement USING (shardid) JOIN pg_dist_node USING (groupid) +WHERE logicalrelid = 'isolate_placement.single_shard_3'::regclass AND noderole = 'primary'; +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_1'::regclass); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_3'::regclass); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +-- tell rebalancer that single_shard_3 cannot be placed on the node where it is currently placed +CREATE OR REPLACE FUNCTION test_shard_allowed_on_node(p_shardid bigint, p_nodeid int) + RETURNS boolean AS +$$ + SELECT + CASE + WHEN (p_shardid = shardid and p_nodeid = nodeid) THEN false + ELSE true + END + FROM single_shard_3_shardid_nodeid; +$$ LANGUAGE sql; +INSERT INTO pg_catalog.pg_dist_rebalance_strategy( + name, + default_strategy, + shard_cost_function, + node_capacity_function, + shard_allowed_on_node_function, + default_threshold, + minimum_threshold, + improvement_threshold +) +VALUES ( + 'test_isolate_placement', + false, + 'citus_shard_cost_1', + 'citus_node_capacity_1', + 'isolate_placement.test_shard_allowed_on_node', + 0, + 0, + 0 +); +SET client_min_messages TO WARNING; +SELECT rebalance_table_shards(rebalance_strategy := 'test_isolate_placement', shard_transfer_mode=>'block_writes'); + rebalance_table_shards +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO NOTICE; +-- test_shard_allowed_on_node() didn't cause rebalance_table_shards() to fail. +-- +-- Right now single_shard_1 & single_shard_3 are placed on the same node. And +-- due to order we follow when assigning nodes to placement groups that need an +-- isolated node, we will try placing single_shard_1 to the node where it is +-- currently placed, and then we will try placing single_shard_3 to some other +-- node (as its current node is already assigned to single_shard_1), not to the +-- one we disallowed in test_shard_allowed_on_node(). +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_1', 1); + verify_placements_in_shard_group_isolated +--------------------------------------------------------------------- + t +(1 row) + +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_3', 1); + verify_placements_in_shard_group_isolated +--------------------------------------------------------------------- + t +(1 row) + +DROP TABLE single_shard_3_shardid_nodeid; +DELETE FROM pg_catalog.pg_dist_rebalance_strategy WHERE name='test_isolate_placement'; +DROP TABLE single_shard_1, single_shard_3; +CREATE TABLE single_shard_1(a int); +SELECT create_distributed_table('single_shard_1', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE single_shard_2(a int); +SELECT create_distributed_table('single_shard_2', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- this would be placed on the same node as single_shard_1 +CREATE TABLE single_shard_3(a int); +SELECT create_distributed_table('single_shard_3', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +DROP TABLE single_shard_2; +SELECT shardid, nodeid INTO single_shard_3_shardid_nodeid +FROM pg_dist_shard JOIN pg_dist_placement USING (shardid) JOIN pg_dist_node USING (groupid) +WHERE logicalrelid = 'isolate_placement.single_shard_3'::regclass AND noderole = 'primary'; +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_1'::regclass); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_3'::regclass); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +-- Same test above but this time we tell rebalancer that single_shard_3 cannot be placed +-- on any node except the one where it is currently placed. +CREATE OR REPLACE FUNCTION test_shard_allowed_on_node(p_shardid bigint, p_nodeid int) + RETURNS boolean AS +$$ + SELECT + CASE + WHEN (p_shardid = shardid and p_nodeid != nodeid) THEN false + ELSE true + END + FROM single_shard_3_shardid_nodeid; +$$ LANGUAGE sql; +INSERT INTO pg_catalog.pg_dist_rebalance_strategy( + name, + default_strategy, + shard_cost_function, + node_capacity_function, + shard_allowed_on_node_function, + default_threshold, + minimum_threshold, + improvement_threshold +) +VALUES ( + 'test_isolate_placement', + false, + 'citus_shard_cost_1', + 'citus_node_capacity_1', + 'isolate_placement.test_shard_allowed_on_node', + 0, + 0, + 0 +); +SET client_min_messages TO ERROR; +SELECT rebalance_table_shards(rebalance_strategy := 'test_isolate_placement', shard_transfer_mode=>'block_writes'); + rebalance_table_shards +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO NOTICE; +-- This time, test_shard_allowed_on_node() caused rebalance_table_shards() to +-- fail. +-- +-- Right now single_shard_1 & single_shard_3 are placed on the same node. And +-- due to order we follow when assigning nodes to placement groups that need an +-- isolated node, we will try placing single_shard_1 to the node where it is +-- currently placed, and then we will try placing single_shard_3 to some other +-- node (as its current node is already assigned to single_shard_1). However, +-- test_shard_allowed_on_node() doesn't allow that. +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_1', 1) = false; + ?column? +--------------------------------------------------------------------- + t +(1 row) + +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_3', 1) = false; + ?column? +--------------------------------------------------------------------- + t +(1 row) + +DROP TABLE single_shard_3_shardid_nodeid; +DELETE FROM pg_catalog.pg_dist_rebalance_strategy WHERE name='test_isolate_placement'; +DROP TABLE single_shard_1, single_shard_3; +CREATE TABLE single_shard_1(a int); +SELECT create_distributed_table('single_shard_1', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE single_shard_2(a int); +SELECT create_distributed_table('single_shard_2', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- this would be placed on the same node as single_shard_1 +CREATE TABLE single_shard_3(a int); +SELECT create_distributed_table('single_shard_3', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +DROP TABLE single_shard_2; +SELECT shardid, nodeid INTO single_shard_1_shardid_nodeid +FROM pg_dist_shard JOIN pg_dist_placement USING (shardid) JOIN pg_dist_node USING (groupid) +WHERE logicalrelid = 'isolate_placement.single_shard_1'::regclass AND noderole = 'primary'; +SELECT shardid, nodeid INTO single_shard_3_shardid_nodeid +FROM pg_dist_shard JOIN pg_dist_placement USING (shardid) JOIN pg_dist_node USING (groupid) +WHERE logicalrelid = 'isolate_placement.single_shard_3'::regclass AND noderole = 'primary'; +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_1'::regclass); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_3'::regclass); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +-- Tell rebalancer that single_shard_1 cannot be placed on the node where it is currently placed +-- and that single_shard_3 cannot be placed on any node except the one where it is currently placed. +CREATE OR REPLACE FUNCTION test_shard_allowed_on_node(p_shardid bigint, p_nodeid int) + RETURNS boolean AS +$$ + SELECT + ( + SELECT + CASE + WHEN (p_shardid = shardid and p_nodeid = nodeid) THEN false + ELSE true + END + FROM single_shard_1_shardid_nodeid + ) AND + ( + SELECT + CASE + WHEN (p_shardid = shardid and p_nodeid != nodeid) THEN false + ELSE true + END + FROM single_shard_3_shardid_nodeid + ) +$$ LANGUAGE sql; +INSERT INTO pg_catalog.pg_dist_rebalance_strategy( + name, + default_strategy, + shard_cost_function, + node_capacity_function, + shard_allowed_on_node_function, + default_threshold, + minimum_threshold, + improvement_threshold +) +VALUES ( + 'test_isolate_placement', + false, + 'citus_shard_cost_1', + 'citus_node_capacity_1', + 'isolate_placement.test_shard_allowed_on_node', + 0, + 0, + 0 +); +SET client_min_messages TO WARNING; +SELECT rebalance_table_shards(rebalance_strategy := 'test_isolate_placement', shard_transfer_mode=>'block_writes'); + rebalance_table_shards +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO NOTICE; +-- This time, test_shard_allowed_on_node() didn't cause rebalance_table_shards() to +-- fail. +-- +-- Right now single_shard_1 & single_shard_3 are placed on the same node. And +-- due to order we follow when assigning nodes to placement groups that need an +-- isolated node, we will try placing single_shard_1 to the node where it is +-- currently placed but this is not possible due to test_shard_allowed_on_node(). +-- But this is not a problem because we will take the specified rebalancer strategy +-- into the account when assigning nodes to placements that need separate nodes and +-- will try to place it to a different node. Then we will try placing single_shard_3 +-- to the node where it is currently placed, and this is ok. +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_1', 1) = true; + ?column? +--------------------------------------------------------------------- + t +(1 row) + +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_3', 1) = true; + ?column? +--------------------------------------------------------------------- + t +(1 row) + +DROP TABLE single_shard_1_shardid_nodeid, single_shard_3_shardid_nodeid; +DELETE FROM pg_catalog.pg_dist_rebalance_strategy WHERE name='test_isolate_placement'; +DROP TABLE single_shard_1, single_shard_3; +SELECT citus_set_node_property('localhost', :master_port, 'shouldhaveshards', true); + citus_set_node_property +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE dist_1(a int); +SELECT create_distributed_table('dist_1', 'a', shard_count=>4); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT shardids[1] AS shardgroup_1_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 1 \gset +SELECT citus_shard_property_set(:shardgroup_1_shardid, anti_affinity=>true); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE single_shard_1(a int); +SELECT create_distributed_table('single_shard_1', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_1'::regclass); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO WARNING; +SELECT rebalance_table_shards(shard_transfer_mode=>'block_writes'); + rebalance_table_shards +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO NOTICE; +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_1', 1) = true; + ?column? +--------------------------------------------------------------------- + t +(1 row) + +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.dist_1', 1) = true; + ?column? +--------------------------------------------------------------------- + t +(1 row) + +SET client_min_messages TO WARNING; +SELECT rebalance_table_shards('isolate_placement.dist_1', shard_transfer_mode=>'block_writes'); + rebalance_table_shards +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO NOTICE; +-- Make sure that calling the rebalancer specifically for dist_1 doesn't +-- break the placement separation rules. +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_1', 1) = true; + ?column? +--------------------------------------------------------------------- + t +(1 row) + +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.dist_1', 1) = true; + ?column? +--------------------------------------------------------------------- + t +(1 row) + +DROP TABLE dist_1, single_shard_1; +SELECT citus_set_node_property('localhost', :master_port, 'shouldhaveshards', false); + citus_set_node_property +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE dist_1(a int); +SELECT create_distributed_table('dist_1', 'a', shard_count=>4); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT shardids[1] AS shardgroup_1_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 1 \gset +CREATE TABLE single_shard_1(a int); +SELECT create_distributed_table('single_shard_1', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- idempotantly move shard of single_shard_1 to the node that contains shard-group 1 of dist_1 +WITH + shardid_and_source_node AS ( + SELECT shardid, nodeid + FROM pg_dist_shard + JOIN pg_dist_placement USING (shardid) + JOIN pg_dist_node USING (groupid) + WHERE logicalrelid = 'isolate_placement.single_shard_1'::regclass + ), + target_node AS ( + SELECT nodeid + FROM pg_dist_shard + JOIN pg_dist_placement USING (shardid) + JOIN pg_dist_node USING (groupid) + WHERE shardid = :shardgroup_1_shardid + ) +SELECT COUNT( + citus_move_shard_placement( + shardid_and_source_node.shardid, + shardid_and_source_node.nodeid, + target_node.nodeid, + 'block_writes' + ) +) >= 0 +FROM shardid_and_source_node, target_node +WHERE shardid_and_source_node.nodeid != target_node.nodeid; + ?column? +--------------------------------------------------------------------- + t +(1 row) + +SELECT citus_shard_property_set(:shardgroup_1_shardid, anti_affinity=>true); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO ERROR; +SELECT rebalance_table_shards('isolate_placement.dist_1', shard_transfer_mode=>'block_writes'); + rebalance_table_shards +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO NOTICE; +-- Make sure that calling the rebalancer specifically for dist_1 enforces +-- placement separation rules too. +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.dist_1', 1) = true; + ?column? +--------------------------------------------------------------------- + t +(1 row) + +DROP TABLE dist_1, single_shard_1; +SET client_min_messages TO WARNING; +DROP SCHEMA isolate_placement CASCADE; +DROP FUNCTION public.verify_placements_in_shard_group_isolated(text, bigint); +SELECT citus_remove_node('localhost', :master_port); + citus_remove_node +--------------------------------------------------------------------- + +(1 row) + diff --git a/src/test/regress/expected/isolation_create_distributed_table.out b/src/test/regress/expected/isolation_create_distributed_table.out index a44d05efedf..9af52828dec 100644 --- a/src/test/regress/expected/isolation_create_distributed_table.out +++ b/src/test/regress/expected/isolation_create_distributed_table.out @@ -108,3 +108,75 @@ ERROR: table "table_to_distribute" is already distributed step s2-commit: COMMIT; + +starting permutation: s1-create_distributed_table s1-begin s2-begin s1_set-shard-property s2-create_distributed_table_colocated s1-rollback s2-commit +step s1-create_distributed_table: + SELECT create_distributed_table('table_to_distribute', 'id'); + +create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +step s1-begin: + BEGIN; + +step s2-begin: + BEGIN; + +step s1_set-shard-property: + SELECT citus_shard_property_set(shardid, anti_affinity=>'true') + FROM pg_dist_shard WHERE logicalrelid = 'table_to_distribute'::regclass + ORDER BY shardid LIMIT 1; + +citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +step s2-create_distributed_table_colocated: + SELECT create_distributed_table('table_to_colocate', 'id', colocate_with=>'table_to_distribute'); + +ERROR: could not acquire the lock required to colocate distributed table public.table_to_distribute +step s1-rollback: + ROLLBACK; + +step s2-commit: + COMMIT; + + +starting permutation: s1-create_distributed_table s1-begin s2-begin s2-create_distributed_table_colocated s1_set-shard-property s1-rollback s2-commit +step s1-create_distributed_table: + SELECT create_distributed_table('table_to_distribute', 'id'); + +create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +step s1-begin: + BEGIN; + +step s2-begin: + BEGIN; + +step s2-create_distributed_table_colocated: + SELECT create_distributed_table('table_to_colocate', 'id', colocate_with=>'table_to_distribute'); + +create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +step s1_set-shard-property: + SELECT citus_shard_property_set(shardid, anti_affinity=>'true') + FROM pg_dist_shard WHERE logicalrelid = 'table_to_distribute'::regclass + ORDER BY shardid LIMIT 1; + +ERROR: could not acquire the lock required to set a property for a shard of public.table_to_distribute +step s1-rollback: + ROLLBACK; + +step s2-commit: + COMMIT; + diff --git a/src/test/regress/expected/isolation_create_distributed_table_concurrently.out b/src/test/regress/expected/isolation_create_distributed_table_concurrently.out index d15e119c044..c9fa0541787 100644 --- a/src/test/regress/expected/isolation_create_distributed_table_concurrently.out +++ b/src/test/regress/expected/isolation_create_distributed_table_concurrently.out @@ -57,12 +57,12 @@ step s2-print-status: -- sanity check on total elements in the table SELECT COUNT(*) FROM table_1; -logicalrelid|shardid|shardstorage|shardminvalue|shardmaxvalue +logicalrelid|shardid|shardstorage|shardminvalue|shardmaxvalue|needsseparatenode --------------------------------------------------------------------- -table_1 |1400294|t | -2147483648| -1073741825 -table_1 |1400295|t | -1073741824| -1 -table_1 |1400296|t | 0| 1073741823 -table_1 |1400297|t | 1073741824| 2147483647 +table_1 |1400306|t | -2147483648| -1073741825|f +table_1 |1400307|t | -1073741824| -1|f +table_1 |1400308|t | 0| 1073741823|f +table_1 |1400309|t | 1073741824| 2147483647|f (4 rows) count @@ -125,12 +125,12 @@ step s2-print-status: -- sanity check on total elements in the table SELECT COUNT(*) FROM table_1; -logicalrelid|shardid|shardstorage|shardminvalue|shardmaxvalue +logicalrelid|shardid|shardstorage|shardminvalue|shardmaxvalue|needsseparatenode --------------------------------------------------------------------- -table_1 |1400299|t | -2147483648| -1073741825 -table_1 |1400300|t | -1073741824| -1 -table_1 |1400301|t | 0| 1073741823 -table_1 |1400302|t | 1073741824| 2147483647 +table_1 |1400311|t | -2147483648| -1073741825|f +table_1 |1400312|t | -1073741824| -1|f +table_1 |1400313|t | 0| 1073741823|f +table_1 |1400314|t | 1073741824| 2147483647|f (4 rows) count @@ -193,12 +193,12 @@ step s2-print-status: -- sanity check on total elements in the table SELECT COUNT(*) FROM table_1; -logicalrelid|shardid|shardstorage|shardminvalue|shardmaxvalue +logicalrelid|shardid|shardstorage|shardminvalue|shardmaxvalue|needsseparatenode --------------------------------------------------------------------- -table_1 |1400304|t | -2147483648| -1073741825 -table_1 |1400305|t | -1073741824| -1 -table_1 |1400306|t | 0| 1073741823 -table_1 |1400307|t | 1073741824| 2147483647 +table_1 |1400316|t | -2147483648| -1073741825|f +table_1 |1400317|t | -1073741824| -1|f +table_1 |1400318|t | 0| 1073741823|f +table_1 |1400319|t | 1073741824| 2147483647|f (4 rows) count @@ -261,12 +261,12 @@ step s2-print-status: -- sanity check on total elements in the table SELECT COUNT(*) FROM table_1; -logicalrelid|shardid|shardstorage|shardminvalue|shardmaxvalue +logicalrelid|shardid|shardstorage|shardminvalue|shardmaxvalue|needsseparatenode --------------------------------------------------------------------- -table_1 |1400309|t | -2147483648| -1073741825 -table_1 |1400310|t | -1073741824| -1 -table_1 |1400311|t | 0| 1073741823 -table_1 |1400312|t | 1073741824| 2147483647 +table_1 |1400321|t | -2147483648| -1073741825|f +table_1 |1400322|t | -1073741824| -1|f +table_1 |1400323|t | 0| 1073741823|f +table_1 |1400324|t | 1073741824| 2147483647|f (4 rows) count diff --git a/src/test/regress/expected/metadata_sync_helpers.out b/src/test/regress/expected/metadata_sync_helpers.out index 9db68eaf569..02a53d9b19e 100644 --- a/src/test/regress/expected/metadata_sync_helpers.out +++ b/src/test/regress/expected/metadata_sync_helpers.out @@ -470,7 +470,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; \set VERBOSITY terse WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('super_user_table'::regclass, 1420000::bigint, 't'::"char", '-2147483648'::text, '-1610612737'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; ERROR: must be owner of table super_user_table ROLLBACK; -- the user is only allowed to add a shard for add a table which is in pg_dist_partition @@ -485,7 +485,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; \set VERBOSITY terse WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '-2147483648'::text, '-1610612737'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; ERROR: The relation "test_2" does not have a valid entry in pg_dist_partition. ROLLBACK; -- ok, now add the table to the pg_dist_partition @@ -544,7 +544,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; \set VERBOSITY terse WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('test_2'::regclass, -1, 't'::"char", '-2147483648'::text, '-1610612737'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; ERROR: Invalid shard id: -1 ROLLBACK; -- invalid storage types are not allowed @@ -559,7 +559,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; \set VERBOSITY terse WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('test_2'::regclass, 1420000, 'X'::"char", '-2147483648'::text, '-1610612737'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; ERROR: Invalid shard storage type: X ROLLBACK; -- NULL shard ranges are not allowed for hash distributed tables @@ -574,7 +574,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; \set VERBOSITY terse WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('test_2'::regclass, 1420000, 't'::"char", NULL, '-1610612737'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; ERROR: Shards of has distributed table "test_2" cannot have NULL shard ranges ROLLBACK; -- non-integer shard ranges are not allowed @@ -589,7 +589,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; \set VERBOSITY terse WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", 'non-int'::text, '-1610612737'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; ERROR: invalid input syntax for type integer: "non-int" ROLLBACK; -- shardMinValue should be smaller than shardMaxValue @@ -604,7 +604,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; \set VERBOSITY terse WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '-1610612737'::text, '-2147483648'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; ERROR: shardMinValue=-1610612737 is greater than shardMaxValue=-2147483648 for table "test_2", which is not allowed ROLLBACK; -- we do not allow overlapping shards for the same table @@ -621,7 +621,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '10'::text, '20'::text), ('test_2'::regclass, 1420001::bigint, 't'::"char", '20'::text, '30'::text), ('test_2'::regclass, 1420002::bigint, 't'::"char", '10'::text, '50'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; ERROR: Shard intervals overlap for table "test_2": 1420001 and 1420000 ROLLBACK; -- Now let's check valid pg_dist_object updates @@ -780,7 +780,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '10'::text, '20'::text), ('test_2'::regclass, 1420001::bigint, 't'::"char", '20'::text, '30'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; ERROR: Metadata syncing is only allowed for hash, reference and local tables: X ROLLBACK; -- we do not allow NULL shardMinMax values @@ -797,8 +797,8 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; \set VERBOSITY terse WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '10'::text, '20'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; - add_shard_metadata + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; + citus_internal.add_shard_metadata --------------------------------------------------------------------- (1 row) @@ -807,7 +807,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; UPDATE pg_dist_shard SET shardminvalue = NULL WHERE shardid = 1420000; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('test_2'::regclass, 1420001::bigint, 't'::"char", '20'::text, '30'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; ERROR: Shards of has distributed table "test_2" cannot have NULL shard ranges ROLLBACK; \c - metadata_sync_helper_role - :worker_1_port @@ -830,8 +830,8 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; ('test_2'::regclass, 1420004::bigint, 't'::"char", '51'::text, '60'::text), ('test_2'::regclass, 1420005::bigint, 't'::"char", '61'::text, '70'::text), ('test_3'::regclass, 1420008::bigint, 't'::"char", '11'::text, '20'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; - add_shard_metadata + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; + citus_internal.add_shard_metadata --------------------------------------------------------------------- @@ -871,8 +871,8 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; ('test_3'::regclass, 1420011::bigint, 't'::"char", '41'::text, '50'::text), ('test_3'::regclass, 1420012::bigint, 't'::"char", '51'::text, '60'::text), ('test_3'::regclass, 1420013::bigint, 't'::"char", '61'::text, '70'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; - add_shard_metadata + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; + citus_internal.add_shard_metadata --------------------------------------------------------------------- @@ -894,7 +894,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; \set VERBOSITY terse WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('test_ref'::regclass, 1420003::bigint, 't'::"char", '-1610612737'::text, NULL)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; ERROR: Shards of reference or local table "test_ref" should have NULL shard ranges ROLLBACK; -- reference tables cannot have multiple shards @@ -910,7 +910,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('test_ref'::regclass, 1420006::bigint, 't'::"char", NULL, NULL), ('test_ref'::regclass, 1420007::bigint, 't'::"char", NULL, NULL)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; ERROR: relation "test_ref" has already at least one shard, adding more is not allowed ROLLBACK; -- finally, add a shard for reference tables @@ -925,8 +925,8 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; \set VERBOSITY terse WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('test_ref'::regclass, 1420006::bigint, 't'::"char", NULL, NULL)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; - add_shard_metadata + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; + citus_internal.add_shard_metadata --------------------------------------------------------------------- (1 row) @@ -946,8 +946,8 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; \set VERBOSITY terse WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('super_user_table'::regclass, 1420007::bigint, 't'::"char", '11'::text, '20'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; - add_shard_metadata + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; + citus_internal.add_shard_metadata --------------------------------------------------------------------- (1 row) @@ -1287,7 +1287,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; SET application_name to 'citus_internal gpid=10000000001'; -- with an ugly trick, update the vartype of table from int to bigint -- so that making two tables colocated fails - -- include varnullingrels for PG16 + -- include varnullingrels for PG16 SHOW server_version \gset SELECT substring(:'server_version', '\d+')::int >= 16 AS server_version_ge_16 \gset diff --git a/src/test/regress/expected/multi_create_shards.out b/src/test/regress/expected/multi_create_shards.out index 41e0a2460be..e271f3dfe30 100644 --- a/src/test/regress/expected/multi_create_shards.out +++ b/src/test/regress/expected/multi_create_shards.out @@ -68,7 +68,7 @@ ERROR: 0 is outside the valid range for parameter "citus.shard_replication_fact -- use a replication factor higher than shard count SET citus.shard_replication_factor TO 3; SELECT create_distributed_table('table_to_distribute', 'name', 'hash'); -ERROR: replication_factor (3) exceeds number of worker nodes (2) +ERROR: replication_factor (3) exceeds number of available worker nodes (2) HINT: Add more worker nodes or try again with a lower replication factor. RESET citus.shard_replication_factor; -- finally, create shards and inspect metadata diff --git a/src/test/regress/expected/multi_extension.out b/src/test/regress/expected/multi_extension.out index 0aecd652fbc..2e65bb0d63f 100644 --- a/src/test/regress/expected/multi_extension.out +++ b/src/test/regress/expected/multi_extension.out @@ -1420,37 +1420,41 @@ SELECT * FROM multi_extension.print_extension_changes(); -- Snapshot of state at 12.2-1 ALTER EXTENSION citus UPDATE TO '12.2-1'; SELECT * FROM multi_extension.print_extension_changes(); - previous_object | current_object ---------------------------------------------------------------------- - | function citus_internal.acquire_citus_advisory_object_class_lock(integer,cstring) void - | function citus_internal.add_colocation_metadata(integer,integer,integer,regtype,oid) void - | function citus_internal.add_object_metadata(text,text[],text[],integer,integer,boolean) void - | function citus_internal.add_partition_metadata(regclass,"char",text,integer,"char") void - | function citus_internal.add_placement_metadata(bigint,bigint,integer,bigint) void - | function citus_internal.add_shard_metadata(regclass,bigint,"char",text,text) void - | function citus_internal.add_tenant_schema(oid,integer) void - | function citus_internal.adjust_local_clock_to_remote(cluster_clock) void - | function citus_internal.commit_management_command_2pc() void - | function citus_internal.database_command(text) void - | function citus_internal.delete_colocation_metadata(integer) void - | function citus_internal.delete_partition_metadata(regclass) void - | function citus_internal.delete_placement_metadata(bigint) void - | function citus_internal.delete_shard_metadata(bigint) void - | function citus_internal.delete_tenant_schema(oid) void - | function citus_internal.execute_command_on_remote_nodes_as_user(text,text) void - | function citus_internal.global_blocked_processes() SETOF record - | function citus_internal.is_replication_origin_tracking_active() boolean - | function citus_internal.local_blocked_processes() SETOF record - | function citus_internal.mark_node_not_synced(integer,integer) void - | function citus_internal.mark_object_distributed(oid,text,oid,text) void - | function citus_internal.start_management_transaction(xid8) void - | function citus_internal.start_replication_origin_tracking() void - | function citus_internal.stop_replication_origin_tracking() void - | function citus_internal.unregister_tenant_schema_globally(oid,text) void - | function citus_internal.update_none_dist_table_metadata(oid,"char",bigint,boolean) void - | function citus_internal.update_placement_metadata(bigint,integer,integer) void - | function citus_internal.update_relation_colocation(oid,integer) void -(28 rows) + previous_object | current_object +--------------------------------------------------------------------- + function citus_internal_add_shard_metadata(regclass,bigint,"char",text,text) void | + | function citus_internal.acquire_citus_advisory_object_class_lock(integer,cstring) void + | function citus_internal.add_colocation_metadata(integer,integer,integer,regtype,oid) void + | function citus_internal.add_object_metadata(text,text[],text[],integer,integer,boolean) void + | function citus_internal.add_partition_metadata(regclass,"char",text,integer,"char") void + | function citus_internal.add_placement_metadata(bigint,bigint,integer,bigint) void + | function citus_internal.add_shard_metadata(regclass,bigint,"char",text,text,boolean) void + | function citus_internal.add_tenant_schema(oid,integer) void + | function citus_internal.adjust_local_clock_to_remote(cluster_clock) void + | function citus_internal.commit_management_command_2pc() void + | function citus_internal.database_command(text) void + | function citus_internal.delete_colocation_metadata(integer) void + | function citus_internal.delete_partition_metadata(regclass) void + | function citus_internal.delete_placement_metadata(bigint) void + | function citus_internal.delete_shard_metadata(bigint) void + | function citus_internal.delete_tenant_schema(oid) void + | function citus_internal.execute_command_on_remote_nodes_as_user(text,text) void + | function citus_internal.global_blocked_processes() SETOF record + | function citus_internal.is_replication_origin_tracking_active() boolean + | function citus_internal.local_blocked_processes() SETOF record + | function citus_internal.mark_node_not_synced(integer,integer) void + | function citus_internal.mark_object_distributed(oid,text,oid,text) void + | function citus_internal.shard_property_set(bigint,boolean) void + | function citus_internal.start_management_transaction(xid8) void + | function citus_internal.start_replication_origin_tracking() void + | function citus_internal.stop_replication_origin_tracking() void + | function citus_internal.unregister_tenant_schema_globally(oid,text) void + | function citus_internal.update_none_dist_table_metadata(oid,"char",bigint,boolean) void + | function citus_internal.update_placement_metadata(bigint,integer,integer) void + | function citus_internal.update_relation_colocation(oid,integer) void + | function citus_internal_add_shard_metadata(regclass,bigint,"char",text,text,boolean) void + | function citus_shard_property_set(bigint,boolean) void +(32 rows) DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff; -- show running version diff --git a/src/test/regress/expected/multi_fix_partition_shard_index_names.out b/src/test/regress/expected/multi_fix_partition_shard_index_names.out index 975a4935149..19a2dfb82b8 100644 --- a/src/test/regress/expected/multi_fix_partition_shard_index_names.out +++ b/src/test/regress/expected/multi_fix_partition_shard_index_names.out @@ -696,9 +696,9 @@ NOTICE: issuing SELECT citus_internal.add_partition_metadata ('fix_idx_names.p2 DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx NOTICE: issuing SELECT citus_internal.add_partition_metadata ('fix_idx_names.p2'::regclass, 'h', 'dist_col', 1370001, '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; +NOTICE: issuing WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('fix_idx_names.p2'::regclass, 915002, 't'::"char", '-2147483648', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) 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; +NOTICE: issuing WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('fix_idx_names.p2'::regclass, 915002, 't'::"char", '-2147483648', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data; DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx NOTICE: issuing WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (xxxxxx, xxxxxx, xxxxxx, xxxxxx)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data; DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx diff --git a/src/test/regress/expected/multi_join_pruning.out b/src/test/regress/expected/multi_join_pruning.out index 59c12de0a25..39c762a2f87 100644 --- a/src/test/regress/expected/multi_join_pruning.out +++ b/src/test/regress/expected/multi_join_pruning.out @@ -45,12 +45,12 @@ select * from pg_dist_shard where logicalrelid='lineitem'::regclass or logicalrelid='orders'::regclass order by shardid; - logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue + logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode --------------------------------------------------------------------- - lineitem | 360000 | t | -2147483648 | -1 - lineitem | 360001 | t | 0 | 2147483647 - orders | 360002 | t | -2147483648 | -1 - orders | 360003 | t | 0 | 2147483647 + lineitem | 360000 | t | -2147483648 | -1 | f + lineitem | 360001 | t | 0 | 2147483647 | f + orders | 360002 | t | -2147483648 | -1 | f + orders | 360003 | t | 0 | 2147483647 | f (4 rows) set client_min_messages to debug3; diff --git a/src/test/regress/expected/multi_metadata_attributes.out b/src/test/regress/expected/multi_metadata_attributes.out index b54946d3f01..897980f6d98 100644 --- a/src/test/regress/expected/multi_metadata_attributes.out +++ b/src/test/regress/expected/multi_metadata_attributes.out @@ -10,7 +10,8 @@ WHERE atthasmissing AND attrelid NOT IN ('pg_dist_node'::regclass, 'pg_dist_rebalance_strategy'::regclass, 'pg_dist_partition'::regclass, 'pg_dist_object'::regclass, - 'pg_dist_background_task'::regclass) + 'pg_dist_background_task'::regclass, + 'pg_dist_shard'::regclass) ORDER BY attrelid, attname; attrelid | attname | atthasmissing | attmissingval --------------------------------------------------------------------- diff --git a/src/test/regress/expected/multi_metadata_sync.out b/src/test/regress/expected/multi_metadata_sync.out index d15e7516c21..c3a111a263d 100644 --- a/src/test/regress/expected/multi_metadata_sync.out +++ b/src/test/regress/expected/multi_metadata_sync.out @@ -133,7 +133,7 @@ INSERT INTO single_shard_tbl VALUES (1); reset citus.shard_replication_factor; -- 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 + unnest --------------------------------------------------------------------- ALTER DATABASE regression OWNER TO postgres; ALTER SEQUENCE public.mx_test_table_col_3_seq OWNER TO postgres @@ -194,8 +194,8 @@ SELECT unnest(activate_node_snapshot()) order by 1; WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['public', 'single_shard_tbl']::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, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data; WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310008, 0, 2, 100008)) SELECT citus_internal.add_placement_metadata(shardid, 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; - WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.single_shard_tbl'::regclass, 1310008, '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, needsseparatenode) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737', false), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825', false), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913', false), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1', false), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911', false), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823', false), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735', false), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data; + WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('public.single_shard_tbl'::regclass, 1310008, 't'::"char", NULL, NULL, false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data; (61 rows) -- Drop single shard table @@ -203,7 +203,7 @@ DROP TABLE single_shard_tbl; -- Show that CREATE INDEX commands are included in the activate node snapshot CREATE INDEX mx_index ON mx_test_table(col_2); SELECT unnest(activate_node_snapshot()) order by 1; - unnest + unnest --------------------------------------------------------------------- ALTER DATABASE regression OWNER TO postgres; ALTER SEQUENCE public.mx_test_table_col_3_seq OWNER TO postgres @@ -256,14 +256,14 @@ SELECT unnest(activate_node_snapshot()) order by 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)) SELECT citus_internal.add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data; WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['public', 'mx_test_table']::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, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal.add_placement_metadata(shardid, 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; + WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737', false), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825', false), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913', false), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1', false), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911', false), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823', false), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735', false), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data; (52 rows) -- 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(activate_node_snapshot()) order by 1; - unnest + unnest --------------------------------------------------------------------- ALTER DATABASE regression OWNER TO postgres; ALTER SEQUENCE mx_testing_schema.mx_test_table_col_3_seq OWNER TO postgres @@ -318,7 +318,7 @@ SELECT unnest(activate_node_snapshot()) order by 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)) SELECT citus_internal.add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data; WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['mx_testing_schema', 'mx_test_table']::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, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal.add_placement_metadata(shardid, 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; + WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737', false), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825', false), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913', false), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1', false), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911', false), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823', false), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735', false), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data; (54 rows) -- Show that append distributed tables are not included in the activate node snapshot @@ -331,7 +331,7 @@ SELECT 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(activate_node_snapshot()) order by 1; - unnest + unnest --------------------------------------------------------------------- ALTER DATABASE regression OWNER TO postgres; ALTER SEQUENCE mx_testing_schema.mx_test_table_col_3_seq OWNER TO postgres @@ -386,13 +386,13 @@ SELECT unnest(activate_node_snapshot()) order by 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)) SELECT citus_internal.add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data; WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['mx_testing_schema', 'mx_test_table']::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, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal.add_placement_metadata(shardid, 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; + WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737', false), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825', false), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913', false), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1', false), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911', false), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823', false), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735', false), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data; (54 rows) -- 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(activate_node_snapshot()) order by 1; - unnest + unnest --------------------------------------------------------------------- ALTER DATABASE regression OWNER TO postgres; ALTER SEQUENCE mx_testing_schema.mx_test_table_col_3_seq OWNER TO postgres @@ -447,7 +447,7 @@ SELECT unnest(activate_node_snapshot()) order by 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)) SELECT citus_internal.add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data; WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['mx_testing_schema', 'mx_test_table']::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, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal.add_placement_metadata(shardid, 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; + WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737', false), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825', false), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913', false), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1', false), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911', false), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823', false), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735', false), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data; (54 rows) -- Test start_metadata_sync_to_node and citus_activate_node UDFs @@ -531,22 +531,22 @@ SELECT * FROM pg_dist_node ORDER BY nodeid; (5 rows) SELECT * FROM pg_dist_partition WHERE logicalrelid::text LIKE 'mx_testing_schema%' ORDER BY logicalrelid::text; - 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 :varnullingrels (b) :varlevelsup 0 :varnosyn 1 :varattnosyn 1 :location -1} | 2 | s | f (1 row) SELECT * FROM pg_dist_shard WHERE logicalrelid::text LIKE 'mx_testing_schema%' ORDER BY shardid; - logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue ---------------------------------------------------------------------- - mx_testing_schema.mx_test_table | 1310000 | t | -2147483648 | -1610612737 - mx_testing_schema.mx_test_table | 1310001 | t | -1610612736 | -1073741825 - mx_testing_schema.mx_test_table | 1310002 | t | -1073741824 | -536870913 - mx_testing_schema.mx_test_table | 1310003 | t | -536870912 | -1 - mx_testing_schema.mx_test_table | 1310004 | t | 0 | 536870911 - mx_testing_schema.mx_test_table | 1310005 | t | 536870912 | 1073741823 - mx_testing_schema.mx_test_table | 1310006 | t | 1073741824 | 1610612735 - mx_testing_schema.mx_test_table | 1310007 | t | 1610612736 | 2147483647 + logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode +--------------------------------------------------------------------- + mx_testing_schema.mx_test_table | 1310000 | t | -2147483648 | -1610612737 | f + mx_testing_schema.mx_test_table | 1310001 | t | -1610612736 | -1073741825 | f + mx_testing_schema.mx_test_table | 1310002 | t | -1073741824 | -536870913 | f + mx_testing_schema.mx_test_table | 1310003 | t | -536870912 | -1 | f + mx_testing_schema.mx_test_table | 1310004 | t | 0 | 536870911 | f + mx_testing_schema.mx_test_table | 1310005 | t | 536870912 | 1073741823 | f + mx_testing_schema.mx_test_table | 1310006 | t | 1073741824 | 1610612735 | f + mx_testing_schema.mx_test_table | 1310007 | t | 1610612736 | 2147483647 | f (8 rows) SELECT * FROM pg_dist_shard_placement WHERE shardid IN (SELECT shardid FROM pg_dist_shard WHERE logicalrelid::text LIKE 'mx_testing_schema%') ORDER BY shardid, nodename, nodeport; @@ -670,22 +670,22 @@ SELECT * FROM pg_dist_node ORDER BY nodeid; (5 rows) SELECT * FROM pg_dist_partition WHERE logicalrelid::text LIKE 'mx_testing_schema%' ORDER BY logicalrelid::text; - 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 :varnullingrels (b) :varlevelsup 0 :varnosyn 1 :varattnosyn 1 :location -1} | 2 | s | f (1 row) SELECT * FROM pg_dist_shard WHERE logicalrelid::text LIKE 'mx_testing_schema%' ORDER BY shardid; - logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue ---------------------------------------------------------------------- - mx_testing_schema.mx_test_table | 1310000 | t | -2147483648 | -1610612737 - mx_testing_schema.mx_test_table | 1310001 | t | -1610612736 | -1073741825 - mx_testing_schema.mx_test_table | 1310002 | t | -1073741824 | -536870913 - mx_testing_schema.mx_test_table | 1310003 | t | -536870912 | -1 - mx_testing_schema.mx_test_table | 1310004 | t | 0 | 536870911 - mx_testing_schema.mx_test_table | 1310005 | t | 536870912 | 1073741823 - mx_testing_schema.mx_test_table | 1310006 | t | 1073741824 | 1610612735 - mx_testing_schema.mx_test_table | 1310007 | t | 1610612736 | 2147483647 + logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode +--------------------------------------------------------------------- + mx_testing_schema.mx_test_table | 1310000 | t | -2147483648 | -1610612737 | f + mx_testing_schema.mx_test_table | 1310001 | t | -1610612736 | -1073741825 | f + mx_testing_schema.mx_test_table | 1310002 | t | -1073741824 | -536870913 | f + mx_testing_schema.mx_test_table | 1310003 | t | -536870912 | -1 | f + mx_testing_schema.mx_test_table | 1310004 | t | 0 | 536870911 | f + mx_testing_schema.mx_test_table | 1310005 | t | 536870912 | 1073741823 | f + mx_testing_schema.mx_test_table | 1310006 | t | 1073741824 | 1610612735 | f + mx_testing_schema.mx_test_table | 1310007 | t | 1610612736 | 2147483647 | f (8 rows) SELECT * FROM pg_dist_shard_placement WHERE shardid IN (SELECT shardid FROM pg_dist_shard WHERE logicalrelid::text LIKE 'mx_testing_schema%') ORDER BY shardid, nodename, nodeport; @@ -1021,7 +1021,7 @@ SELECT * FROM pg_dist_partition WHERE logicalrelid::text LIKE 'mx_test_schema%'; (0 rows) SELECT * FROM pg_dist_shard WHERE logicalrelid::text LIKE 'mx_test_schema%'; - logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue + logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode --------------------------------------------------------------------- (0 rows) @@ -1681,7 +1681,7 @@ SELECT "Column", "Type", "Definition" FROM index_attrs WHERE relid = 'mx_ref_index'::regclass; ERROR: relation "mx_ref_index" does not exist SELECT * FROM pg_dist_shard WHERE shardid=:ref_table_shardid; - logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue + logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode --------------------------------------------------------------------- (0 rows) @@ -1942,7 +1942,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(activate_node_snapshot()) order by 1; - unnest + unnest --------------------------------------------------------------------- ALTER DATABASE regression OWNER TO postgres; ALTER SEQUENCE mx_testing_schema.mx_test_table_col_3_seq OWNER TO postgres @@ -2056,12 +2056,12 @@ SELECT unnest(activate_node_snapshot()) order by 1; WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310075, 0, 0, 100077), (1310075, 0, 1, 100078), (1310075, 0, 5, 100079)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data; WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310076, 0, 1, 100080), (1310077, 0, 5, 100081), (1310078, 0, 1, 100082), (1310079, 0, 5, 100083)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data; WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310085, 0, 1, 100091), (1310086, 0, 5, 100092), (1310087, 0, 1, 100093), (1310088, 0, 5, 100094)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data; - WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_test_schema_1.mx_table_1'::regclass, 1310022, 't'::"char", '-2147483648', '-1288490190'), ('mx_test_schema_1.mx_table_1'::regclass, 1310023, 't'::"char", '-1288490189', '-429496731'), ('mx_test_schema_1.mx_table_1'::regclass, 1310024, 't'::"char", '-429496730', '429496728'), ('mx_test_schema_1.mx_table_1'::regclass, 1310025, 't'::"char", '429496729', '1288490187'), ('mx_test_schema_1.mx_table_1'::regclass, 1310026, 't'::"char", '1288490188', '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 ('mx_test_schema_2.mx_table_2'::regclass, 1310027, 't'::"char", '-2147483648', '-1288490190'), ('mx_test_schema_2.mx_table_2'::regclass, 1310028, 't'::"char", '-1288490189', '-429496731'), ('mx_test_schema_2.mx_table_2'::regclass, 1310029, 't'::"char", '-429496730', '429496728'), ('mx_test_schema_2.mx_table_2'::regclass, 1310030, 't'::"char", '429496729', '1288490187'), ('mx_test_schema_2.mx_table_2'::regclass, 1310031, 't'::"char", '1288490188', '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 ('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; - WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.dist_table_1'::regclass, 1310076, 't'::"char", '-2147483648', '-1073741825'), ('public.dist_table_1'::regclass, 1310077, 't'::"char", '-1073741824', '-1'), ('public.dist_table_1'::regclass, 1310078, 't'::"char", '0', '1073741823'), ('public.dist_table_1'::regclass, 1310079, '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, 1310075, '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, 1310085, 't'::"char", '-2147483648', '-1073741825'), ('public.test_table'::regclass, 1310086, 't'::"char", '-1073741824', '-1'), ('public.test_table'::regclass, 1310087, 't'::"char", '0', '1073741823'), ('public.test_table'::regclass, 1310088, '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, needsseparatenode) AS (VALUES ('mx_test_schema_1.mx_table_1'::regclass, 1310022, 't'::"char", '-2147483648', '-1288490190', false), ('mx_test_schema_1.mx_table_1'::regclass, 1310023, 't'::"char", '-1288490189', '-429496731', false), ('mx_test_schema_1.mx_table_1'::regclass, 1310024, 't'::"char", '-429496730', '429496728', false), ('mx_test_schema_1.mx_table_1'::regclass, 1310025, 't'::"char", '429496729', '1288490187', false), ('mx_test_schema_1.mx_table_1'::regclass, 1310026, 't'::"char", '1288490188', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data; + WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('mx_test_schema_2.mx_table_2'::regclass, 1310027, 't'::"char", '-2147483648', '-1288490190', false), ('mx_test_schema_2.mx_table_2'::regclass, 1310028, 't'::"char", '-1288490189', '-429496731', false), ('mx_test_schema_2.mx_table_2'::regclass, 1310029, 't'::"char", '-429496730', '429496728', false), ('mx_test_schema_2.mx_table_2'::regclass, 1310030, 't'::"char", '429496729', '1288490187', false), ('mx_test_schema_2.mx_table_2'::regclass, 1310031, 't'::"char", '1288490188', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data; + WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737', false), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825', false), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913', false), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1', false), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911', false), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823', false), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735', false), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data; + WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('public.dist_table_1'::regclass, 1310076, 't'::"char", '-2147483648', '-1073741825', false), ('public.dist_table_1'::regclass, 1310077, 't'::"char", '-1073741824', '-1', false), ('public.dist_table_1'::regclass, 1310078, 't'::"char", '0', '1073741823', false), ('public.dist_table_1'::regclass, 1310079, 't'::"char", '1073741824', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data; + WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('public.mx_ref'::regclass, 1310075, 't'::"char", NULL, NULL, false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data; + WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('public.test_table'::regclass, 1310085, 't'::"char", '-2147483648', '-1073741825', false), ('public.test_table'::regclass, 1310086, 't'::"char", '-1073741824', '-1', false), ('public.test_table'::regclass, 1310087, 't'::"char", '0', '1073741823', false), ('public.test_table'::regclass, 1310088, 't'::"char", '1073741824', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data; (118 rows) -- shouldn't work since test_table is MX diff --git a/src/test/regress/expected/multi_metadata_sync_0.out b/src/test/regress/expected/multi_metadata_sync_0.out index bc1775ada37..200de1a200f 100644 --- a/src/test/regress/expected/multi_metadata_sync_0.out +++ b/src/test/regress/expected/multi_metadata_sync_0.out @@ -133,7 +133,7 @@ INSERT INTO single_shard_tbl VALUES (1); reset citus.shard_replication_factor; -- 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 + unnest --------------------------------------------------------------------- ALTER DATABASE regression OWNER TO postgres; ALTER SEQUENCE public.mx_test_table_col_3_seq OWNER TO postgres @@ -194,8 +194,8 @@ SELECT unnest(activate_node_snapshot()) order by 1; WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['public', 'single_shard_tbl']::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, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data; WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310008, 0, 2, 100008)) SELECT citus_internal.add_placement_metadata(shardid, 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; - WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.single_shard_tbl'::regclass, 1310008, '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, needsseparatenode) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737', false), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825', false), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913', false), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1', false), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911', false), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823', false), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735', false), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data; + WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('public.single_shard_tbl'::regclass, 1310008, 't'::"char", NULL, NULL, false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data; (61 rows) -- Drop single shard table @@ -203,7 +203,7 @@ DROP TABLE single_shard_tbl; -- Show that CREATE INDEX commands are included in the activate node snapshot CREATE INDEX mx_index ON mx_test_table(col_2); SELECT unnest(activate_node_snapshot()) order by 1; - unnest + unnest --------------------------------------------------------------------- ALTER DATABASE regression OWNER TO postgres; ALTER SEQUENCE public.mx_test_table_col_3_seq OWNER TO postgres @@ -256,14 +256,14 @@ SELECT unnest(activate_node_snapshot()) order by 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)) SELECT citus_internal.add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data; WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['public', 'mx_test_table']::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, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal.add_placement_metadata(shardid, 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; + WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737', false), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825', false), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913', false), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1', false), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911', false), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823', false), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735', false), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data; (52 rows) -- 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(activate_node_snapshot()) order by 1; - unnest + unnest --------------------------------------------------------------------- ALTER DATABASE regression OWNER TO postgres; ALTER SEQUENCE mx_testing_schema.mx_test_table_col_3_seq OWNER TO postgres @@ -318,7 +318,7 @@ SELECT unnest(activate_node_snapshot()) order by 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)) SELECT citus_internal.add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data; WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['mx_testing_schema', 'mx_test_table']::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, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal.add_placement_metadata(shardid, 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; + WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737', false), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825', false), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913', false), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1', false), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911', false), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823', false), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735', false), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data; (54 rows) -- Show that append distributed tables are not included in the activate node snapshot @@ -331,7 +331,7 @@ SELECT 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(activate_node_snapshot()) order by 1; - unnest + unnest --------------------------------------------------------------------- ALTER DATABASE regression OWNER TO postgres; ALTER SEQUENCE mx_testing_schema.mx_test_table_col_3_seq OWNER TO postgres @@ -386,13 +386,13 @@ SELECT unnest(activate_node_snapshot()) order by 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)) SELECT citus_internal.add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data; WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['mx_testing_schema', 'mx_test_table']::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, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal.add_placement_metadata(shardid, 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; + WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737', false), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825', false), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913', false), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1', false), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911', false), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823', false), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735', false), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data; (54 rows) -- 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(activate_node_snapshot()) order by 1; - unnest + unnest --------------------------------------------------------------------- ALTER DATABASE regression OWNER TO postgres; ALTER SEQUENCE mx_testing_schema.mx_test_table_col_3_seq OWNER TO postgres @@ -447,7 +447,7 @@ SELECT unnest(activate_node_snapshot()) order by 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)) SELECT citus_internal.add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data; WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['mx_testing_schema', 'mx_test_table']::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, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal.add_placement_metadata(shardid, 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; + WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737', false), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825', false), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913', false), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1', false), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911', false), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823', false), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735', false), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data; (54 rows) -- Test start_metadata_sync_to_node and citus_activate_node UDFs @@ -537,16 +537,16 @@ SELECT * FROM pg_dist_partition WHERE logicalrelid::text LIKE 'mx_testing_schema (1 row) SELECT * FROM pg_dist_shard WHERE logicalrelid::text LIKE 'mx_testing_schema%' ORDER BY shardid; - logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue ---------------------------------------------------------------------- - mx_testing_schema.mx_test_table | 1310000 | t | -2147483648 | -1610612737 - mx_testing_schema.mx_test_table | 1310001 | t | -1610612736 | -1073741825 - mx_testing_schema.mx_test_table | 1310002 | t | -1073741824 | -536870913 - mx_testing_schema.mx_test_table | 1310003 | t | -536870912 | -1 - mx_testing_schema.mx_test_table | 1310004 | t | 0 | 536870911 - mx_testing_schema.mx_test_table | 1310005 | t | 536870912 | 1073741823 - mx_testing_schema.mx_test_table | 1310006 | t | 1073741824 | 1610612735 - mx_testing_schema.mx_test_table | 1310007 | t | 1610612736 | 2147483647 + logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode +--------------------------------------------------------------------- + mx_testing_schema.mx_test_table | 1310000 | t | -2147483648 | -1610612737 | f + mx_testing_schema.mx_test_table | 1310001 | t | -1610612736 | -1073741825 | f + mx_testing_schema.mx_test_table | 1310002 | t | -1073741824 | -536870913 | f + mx_testing_schema.mx_test_table | 1310003 | t | -536870912 | -1 | f + mx_testing_schema.mx_test_table | 1310004 | t | 0 | 536870911 | f + mx_testing_schema.mx_test_table | 1310005 | t | 536870912 | 1073741823 | f + mx_testing_schema.mx_test_table | 1310006 | t | 1073741824 | 1610612735 | f + mx_testing_schema.mx_test_table | 1310007 | t | 1610612736 | 2147483647 | f (8 rows) SELECT * FROM pg_dist_shard_placement WHERE shardid IN (SELECT shardid FROM pg_dist_shard WHERE logicalrelid::text LIKE 'mx_testing_schema%') ORDER BY shardid, nodename, nodeport; @@ -676,16 +676,16 @@ SELECT * FROM pg_dist_partition WHERE logicalrelid::text LIKE 'mx_testing_schema (1 row) SELECT * FROM pg_dist_shard WHERE logicalrelid::text LIKE 'mx_testing_schema%' ORDER BY shardid; - logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue ---------------------------------------------------------------------- - mx_testing_schema.mx_test_table | 1310000 | t | -2147483648 | -1610612737 - mx_testing_schema.mx_test_table | 1310001 | t | -1610612736 | -1073741825 - mx_testing_schema.mx_test_table | 1310002 | t | -1073741824 | -536870913 - mx_testing_schema.mx_test_table | 1310003 | t | -536870912 | -1 - mx_testing_schema.mx_test_table | 1310004 | t | 0 | 536870911 - mx_testing_schema.mx_test_table | 1310005 | t | 536870912 | 1073741823 - mx_testing_schema.mx_test_table | 1310006 | t | 1073741824 | 1610612735 - mx_testing_schema.mx_test_table | 1310007 | t | 1610612736 | 2147483647 + logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode +--------------------------------------------------------------------- + mx_testing_schema.mx_test_table | 1310000 | t | -2147483648 | -1610612737 | f + mx_testing_schema.mx_test_table | 1310001 | t | -1610612736 | -1073741825 | f + mx_testing_schema.mx_test_table | 1310002 | t | -1073741824 | -536870913 | f + mx_testing_schema.mx_test_table | 1310003 | t | -536870912 | -1 | f + mx_testing_schema.mx_test_table | 1310004 | t | 0 | 536870911 | f + mx_testing_schema.mx_test_table | 1310005 | t | 536870912 | 1073741823 | f + mx_testing_schema.mx_test_table | 1310006 | t | 1073741824 | 1610612735 | f + mx_testing_schema.mx_test_table | 1310007 | t | 1610612736 | 2147483647 | f (8 rows) SELECT * FROM pg_dist_shard_placement WHERE shardid IN (SELECT shardid FROM pg_dist_shard WHERE logicalrelid::text LIKE 'mx_testing_schema%') ORDER BY shardid, nodename, nodeport; @@ -1021,7 +1021,7 @@ SELECT * FROM pg_dist_partition WHERE logicalrelid::text LIKE 'mx_test_schema%'; (0 rows) SELECT * FROM pg_dist_shard WHERE logicalrelid::text LIKE 'mx_test_schema%'; - logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue + logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode --------------------------------------------------------------------- (0 rows) @@ -1681,7 +1681,7 @@ SELECT "Column", "Type", "Definition" FROM index_attrs WHERE relid = 'mx_ref_index'::regclass; ERROR: relation "mx_ref_index" does not exist SELECT * FROM pg_dist_shard WHERE shardid=:ref_table_shardid; - logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue + logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode --------------------------------------------------------------------- (0 rows) @@ -1942,7 +1942,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(activate_node_snapshot()) order by 1; - unnest + unnest --------------------------------------------------------------------- ALTER DATABASE regression OWNER TO postgres; ALTER SEQUENCE mx_testing_schema.mx_test_table_col_3_seq OWNER TO postgres @@ -2056,12 +2056,12 @@ SELECT unnest(activate_node_snapshot()) order by 1; WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310075, 0, 0, 100077), (1310075, 0, 1, 100078), (1310075, 0, 5, 100079)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data; WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310076, 0, 1, 100080), (1310077, 0, 5, 100081), (1310078, 0, 1, 100082), (1310079, 0, 5, 100083)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data; WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310085, 0, 1, 100091), (1310086, 0, 5, 100092), (1310087, 0, 1, 100093), (1310088, 0, 5, 100094)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data; - WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_test_schema_1.mx_table_1'::regclass, 1310022, 't'::"char", '-2147483648', '-1288490190'), ('mx_test_schema_1.mx_table_1'::regclass, 1310023, 't'::"char", '-1288490189', '-429496731'), ('mx_test_schema_1.mx_table_1'::regclass, 1310024, 't'::"char", '-429496730', '429496728'), ('mx_test_schema_1.mx_table_1'::regclass, 1310025, 't'::"char", '429496729', '1288490187'), ('mx_test_schema_1.mx_table_1'::regclass, 1310026, 't'::"char", '1288490188', '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 ('mx_test_schema_2.mx_table_2'::regclass, 1310027, 't'::"char", '-2147483648', '-1288490190'), ('mx_test_schema_2.mx_table_2'::regclass, 1310028, 't'::"char", '-1288490189', '-429496731'), ('mx_test_schema_2.mx_table_2'::regclass, 1310029, 't'::"char", '-429496730', '429496728'), ('mx_test_schema_2.mx_table_2'::regclass, 1310030, 't'::"char", '429496729', '1288490187'), ('mx_test_schema_2.mx_table_2'::regclass, 1310031, 't'::"char", '1288490188', '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 ('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; - WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.dist_table_1'::regclass, 1310076, 't'::"char", '-2147483648', '-1073741825'), ('public.dist_table_1'::regclass, 1310077, 't'::"char", '-1073741824', '-1'), ('public.dist_table_1'::regclass, 1310078, 't'::"char", '0', '1073741823'), ('public.dist_table_1'::regclass, 1310079, '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, 1310075, '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, 1310085, 't'::"char", '-2147483648', '-1073741825'), ('public.test_table'::regclass, 1310086, 't'::"char", '-1073741824', '-1'), ('public.test_table'::regclass, 1310087, 't'::"char", '0', '1073741823'), ('public.test_table'::regclass, 1310088, '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, needsseparatenode) AS (VALUES ('mx_test_schema_1.mx_table_1'::regclass, 1310022, 't'::"char", '-2147483648', '-1288490190', false), ('mx_test_schema_1.mx_table_1'::regclass, 1310023, 't'::"char", '-1288490189', '-429496731', false), ('mx_test_schema_1.mx_table_1'::regclass, 1310024, 't'::"char", '-429496730', '429496728', false), ('mx_test_schema_1.mx_table_1'::regclass, 1310025, 't'::"char", '429496729', '1288490187', false), ('mx_test_schema_1.mx_table_1'::regclass, 1310026, 't'::"char", '1288490188', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data; + WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('mx_test_schema_2.mx_table_2'::regclass, 1310027, 't'::"char", '-2147483648', '-1288490190', false), ('mx_test_schema_2.mx_table_2'::regclass, 1310028, 't'::"char", '-1288490189', '-429496731', false), ('mx_test_schema_2.mx_table_2'::regclass, 1310029, 't'::"char", '-429496730', '429496728', false), ('mx_test_schema_2.mx_table_2'::regclass, 1310030, 't'::"char", '429496729', '1288490187', false), ('mx_test_schema_2.mx_table_2'::regclass, 1310031, 't'::"char", '1288490188', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data; + WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737', false), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825', false), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913', false), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1', false), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911', false), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823', false), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735', false), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data; + WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('public.dist_table_1'::regclass, 1310076, 't'::"char", '-2147483648', '-1073741825', false), ('public.dist_table_1'::regclass, 1310077, 't'::"char", '-1073741824', '-1', false), ('public.dist_table_1'::regclass, 1310078, 't'::"char", '0', '1073741823', false), ('public.dist_table_1'::regclass, 1310079, 't'::"char", '1073741824', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data; + WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('public.mx_ref'::regclass, 1310075, 't'::"char", NULL, NULL, false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data; + WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('public.test_table'::regclass, 1310085, 't'::"char", '-2147483648', '-1073741825', false), ('public.test_table'::regclass, 1310086, 't'::"char", '-1073741824', '-1', false), ('public.test_table'::regclass, 1310087, 't'::"char", '0', '1073741823', false), ('public.test_table'::regclass, 1310088, 't'::"char", '1073741824', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data; (118 rows) -- shouldn't work since test_table is MX diff --git a/src/test/regress/expected/multi_row_insert.out b/src/test/regress/expected/multi_row_insert.out index caa149ce2db..3b49f1625d5 100644 --- a/src/test/regress/expected/multi_row_insert.out +++ b/src/test/regress/expected/multi_row_insert.out @@ -12,10 +12,10 @@ SELECT create_distributed_table('source_table_xyz', 'key', 'range'); CALL public.create_range_partitioned_shards('source_table_xyz', '{"(0,a)","(25,z)"}','{"(24,a)","(49,z)"}'); SELECT * FROM pg_dist_shard WHERE logicalrelid='source_table_xyz'::regclass::oid ORDER BY shardid; - logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue + logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode --------------------------------------------------------------------- - source_table_xyz | 4213581 | t | (0,a) | (24,a) - source_table_xyz | 4213582 | t | (25,z) | (49,z) + source_table_xyz | 4213581 | t | (0,a) | (24,a) | f + source_table_xyz | 4213582 | t | (25,z) | (49,z) | f (2 rows) SELECT shardid, nodename, nodeport FROM pg_dist_shard_placement WHERE EXISTS(SELECT shardid FROM pg_dist_shard WHERE shardid=pg_dist_shard_placement.shardid AND logicalrelid='source_table_xyz'::regclass::oid) ORDER BY 1, 2, 3; diff --git a/src/test/regress/expected/multi_table_ddl.out b/src/test/regress/expected/multi_table_ddl.out index 2db4a7797e3..f4997697c0f 100644 --- a/src/test/regress/expected/multi_table_ddl.out +++ b/src/test/regress/expected/multi_table_ddl.out @@ -65,7 +65,7 @@ SELECT * FROM pg_dist_partition; (0 rows) SELECT * FROM pg_dist_shard; - logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue + logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode --------------------------------------------------------------------- (0 rows) diff --git a/src/test/regress/expected/multi_tenant_isolation.out b/src/test/regress/expected/multi_tenant_isolation.out index 5af7acac8eb..eaf3def80eb 100644 --- a/src/test/regress/expected/multi_tenant_isolation.out +++ b/src/test/regress/expected/multi_tenant_isolation.out @@ -168,12 +168,12 @@ SELECT count(*) FROM orders_streaming WHERE o_orderkey = 103; SELECT * FROM pg_dist_shard WHERE logicalrelid = 'lineitem_streaming'::regclass OR logicalrelid = 'orders_streaming'::regclass ORDER BY shardminvalue::BIGINT, logicalrelid; - logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue + logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode --------------------------------------------------------------------- - lineitem_streaming | 1230000 | t | -2147483648 | -1 - orders_streaming | 1230002 | t | -2147483648 | -1 - lineitem_streaming | 1230001 | t | 0 | 2147483647 - orders_streaming | 1230003 | t | 0 | 2147483647 + lineitem_streaming | 1230000 | t | -2147483648 | -1 | f + orders_streaming | 1230002 | t | -2147483648 | -1 | f + lineitem_streaming | 1230001 | t | 0 | 2147483647 | f + orders_streaming | 1230003 | t | 0 | 2147483647 | f (4 rows) -- check without cascade option @@ -387,32 +387,32 @@ SELECT count(*) FROM orders_streaming WHERE o_orderkey = 103; SELECT * FROM pg_dist_shard WHERE logicalrelid = 'lineitem_streaming'::regclass OR logicalrelid = 'orders_streaming'::regclass ORDER BY shardminvalue::BIGINT, logicalrelid; - logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue ---------------------------------------------------------------------- - lineitem_streaming | 1230040 | t | -2147483648 | -2147483648 - orders_streaming | 1230042 | t | -2147483648 | -2147483648 - lineitem_streaming | 1230041 | t | -2147483647 | -136164586 - orders_streaming | 1230043 | t | -2147483647 | -136164586 - lineitem_streaming | 1230035 | t | -136164585 | -136164585 - orders_streaming | 1230038 | t | -136164585 | -136164585 - lineitem_streaming | 1230036 | t | -136164584 | -85071815 - orders_streaming | 1230039 | t | -136164584 | -85071815 - lineitem_streaming | 1230011 | t | -85071814 | -85071814 - orders_streaming | 1230014 | t | -85071814 | -85071814 - lineitem_streaming | 1230012 | t | -85071813 | -1 - orders_streaming | 1230015 | t | -85071813 | -1 - lineitem_streaming | 1230004 | t | 0 | 108199380 - orders_streaming | 1230007 | t | 0 | 108199380 - lineitem_streaming | 1230005 | t | 108199381 | 108199381 - orders_streaming | 1230008 | t | 108199381 | 108199381 - lineitem_streaming | 1230028 | t | 108199382 | 412880111 - orders_streaming | 1230031 | t | 108199382 | 412880111 - lineitem_streaming | 1230029 | t | 412880112 | 412880112 - orders_streaming | 1230032 | t | 412880112 | 412880112 - lineitem_streaming | 1230044 | t | 412880113 | 2147483646 - orders_streaming | 1230046 | t | 412880113 | 2147483646 - lineitem_streaming | 1230045 | t | 2147483647 | 2147483647 - orders_streaming | 1230047 | t | 2147483647 | 2147483647 + logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode +--------------------------------------------------------------------- + lineitem_streaming | 1230040 | t | -2147483648 | -2147483648 | f + orders_streaming | 1230042 | t | -2147483648 | -2147483648 | f + lineitem_streaming | 1230041 | t | -2147483647 | -136164586 | f + orders_streaming | 1230043 | t | -2147483647 | -136164586 | f + lineitem_streaming | 1230035 | t | -136164585 | -136164585 | f + orders_streaming | 1230038 | t | -136164585 | -136164585 | f + lineitem_streaming | 1230036 | t | -136164584 | -85071815 | f + orders_streaming | 1230039 | t | -136164584 | -85071815 | f + lineitem_streaming | 1230011 | t | -85071814 | -85071814 | f + orders_streaming | 1230014 | t | -85071814 | -85071814 | f + lineitem_streaming | 1230012 | t | -85071813 | -1 | f + orders_streaming | 1230015 | t | -85071813 | -1 | f + lineitem_streaming | 1230004 | t | 0 | 108199380 | f + orders_streaming | 1230007 | t | 0 | 108199380 | f + lineitem_streaming | 1230005 | t | 108199381 | 108199381 | f + orders_streaming | 1230008 | t | 108199381 | 108199381 | f + lineitem_streaming | 1230028 | t | 108199382 | 412880111 | f + orders_streaming | 1230031 | t | 108199382 | 412880111 | f + lineitem_streaming | 1230029 | t | 412880112 | 412880112 | f + orders_streaming | 1230032 | t | 412880112 | 412880112 | f + lineitem_streaming | 1230044 | t | 412880113 | 2147483646 | f + orders_streaming | 1230046 | t | 412880113 | 2147483646 | f + lineitem_streaming | 1230045 | t | 2147483647 | 2147483647 | f + orders_streaming | 1230047 | t | 2147483647 | 2147483647 | f (24 rows) SELECT * FROM pg_dist_shard_placement WHERE shardid BETWEEN 1230000 AND 1399999 ORDER BY nodeport, shardid; @@ -541,32 +541,32 @@ SET search_path to "Tenant Isolation"; SELECT * FROM pg_dist_shard WHERE logicalrelid = 'lineitem_streaming'::regclass OR logicalrelid = 'orders_streaming'::regclass ORDER BY shardminvalue::BIGINT, logicalrelid; - logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue ---------------------------------------------------------------------- - lineitem_streaming | 1230040 | t | -2147483648 | -2147483648 - orders_streaming | 1230042 | t | -2147483648 | -2147483648 - lineitem_streaming | 1230041 | t | -2147483647 | -136164586 - orders_streaming | 1230043 | t | -2147483647 | -136164586 - lineitem_streaming | 1230035 | t | -136164585 | -136164585 - orders_streaming | 1230038 | t | -136164585 | -136164585 - lineitem_streaming | 1230036 | t | -136164584 | -85071815 - orders_streaming | 1230039 | t | -136164584 | -85071815 - lineitem_streaming | 1230011 | t | -85071814 | -85071814 - orders_streaming | 1230014 | t | -85071814 | -85071814 - lineitem_streaming | 1230012 | t | -85071813 | -1 - orders_streaming | 1230015 | t | -85071813 | -1 - lineitem_streaming | 1230004 | t | 0 | 108199380 - orders_streaming | 1230007 | t | 0 | 108199380 - lineitem_streaming | 1230005 | t | 108199381 | 108199381 - orders_streaming | 1230008 | t | 108199381 | 108199381 - lineitem_streaming | 1230028 | t | 108199382 | 412880111 - orders_streaming | 1230031 | t | 108199382 | 412880111 - lineitem_streaming | 1230029 | t | 412880112 | 412880112 - orders_streaming | 1230032 | t | 412880112 | 412880112 - lineitem_streaming | 1230044 | t | 412880113 | 2147483646 - orders_streaming | 1230046 | t | 412880113 | 2147483646 - lineitem_streaming | 1230045 | t | 2147483647 | 2147483647 - orders_streaming | 1230047 | t | 2147483647 | 2147483647 + logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode +--------------------------------------------------------------------- + lineitem_streaming | 1230040 | t | -2147483648 | -2147483648 | f + orders_streaming | 1230042 | t | -2147483648 | -2147483648 | f + lineitem_streaming | 1230041 | t | -2147483647 | -136164586 | f + orders_streaming | 1230043 | t | -2147483647 | -136164586 | f + lineitem_streaming | 1230035 | t | -136164585 | -136164585 | f + orders_streaming | 1230038 | t | -136164585 | -136164585 | f + lineitem_streaming | 1230036 | t | -136164584 | -85071815 | f + orders_streaming | 1230039 | t | -136164584 | -85071815 | f + lineitem_streaming | 1230011 | t | -85071814 | -85071814 | f + orders_streaming | 1230014 | t | -85071814 | -85071814 | f + lineitem_streaming | 1230012 | t | -85071813 | -1 | f + orders_streaming | 1230015 | t | -85071813 | -1 | f + lineitem_streaming | 1230004 | t | 0 | 108199380 | f + orders_streaming | 1230007 | t | 0 | 108199380 | f + lineitem_streaming | 1230005 | t | 108199381 | 108199381 | f + orders_streaming | 1230008 | t | 108199381 | 108199381 | f + lineitem_streaming | 1230028 | t | 108199382 | 412880111 | f + orders_streaming | 1230031 | t | 108199382 | 412880111 | f + lineitem_streaming | 1230029 | t | 412880112 | 412880112 | f + orders_streaming | 1230032 | t | 412880112 | 412880112 | f + lineitem_streaming | 1230044 | t | 412880113 | 2147483646 | f + orders_streaming | 1230046 | t | 412880113 | 2147483646 | f + lineitem_streaming | 1230045 | t | 2147483647 | 2147483647 | f + orders_streaming | 1230047 | t | 2147483647 | 2147483647 | f (24 rows) -- return to master node @@ -684,32 +684,32 @@ ERROR: cannot isolate tenant because tenant isolation is only support for hash SELECT * FROM pg_dist_shard WHERE logicalrelid = 'lineitem_streaming'::regclass OR logicalrelid = 'orders_streaming'::regclass ORDER BY shardminvalue::BIGINT, logicalrelid; - logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue ---------------------------------------------------------------------- - lineitem_streaming | 1230040 | t | -2147483648 | -2147483648 - orders_streaming | 1230042 | t | -2147483648 | -2147483648 - lineitem_streaming | 1230041 | t | -2147483647 | -136164586 - orders_streaming | 1230043 | t | -2147483647 | -136164586 - lineitem_streaming | 1230035 | t | -136164585 | -136164585 - orders_streaming | 1230038 | t | -136164585 | -136164585 - lineitem_streaming | 1230036 | t | -136164584 | -85071815 - orders_streaming | 1230039 | t | -136164584 | -85071815 - lineitem_streaming | 1230011 | t | -85071814 | -85071814 - orders_streaming | 1230014 | t | -85071814 | -85071814 - lineitem_streaming | 1230012 | t | -85071813 | -1 - orders_streaming | 1230015 | t | -85071813 | -1 - lineitem_streaming | 1230004 | t | 0 | 108199380 - orders_streaming | 1230007 | t | 0 | 108199380 - lineitem_streaming | 1230005 | t | 108199381 | 108199381 - orders_streaming | 1230008 | t | 108199381 | 108199381 - lineitem_streaming | 1230028 | t | 108199382 | 412880111 - orders_streaming | 1230031 | t | 108199382 | 412880111 - lineitem_streaming | 1230029 | t | 412880112 | 412880112 - orders_streaming | 1230032 | t | 412880112 | 412880112 - lineitem_streaming | 1230044 | t | 412880113 | 2147483646 - orders_streaming | 1230046 | t | 412880113 | 2147483646 - lineitem_streaming | 1230045 | t | 2147483647 | 2147483647 - orders_streaming | 1230047 | t | 2147483647 | 2147483647 + logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode +--------------------------------------------------------------------- + lineitem_streaming | 1230040 | t | -2147483648 | -2147483648 | f + orders_streaming | 1230042 | t | -2147483648 | -2147483648 | f + lineitem_streaming | 1230041 | t | -2147483647 | -136164586 | f + orders_streaming | 1230043 | t | -2147483647 | -136164586 | f + lineitem_streaming | 1230035 | t | -136164585 | -136164585 | f + orders_streaming | 1230038 | t | -136164585 | -136164585 | f + lineitem_streaming | 1230036 | t | -136164584 | -85071815 | f + orders_streaming | 1230039 | t | -136164584 | -85071815 | f + lineitem_streaming | 1230011 | t | -85071814 | -85071814 | f + orders_streaming | 1230014 | t | -85071814 | -85071814 | f + lineitem_streaming | 1230012 | t | -85071813 | -1 | f + orders_streaming | 1230015 | t | -85071813 | -1 | f + lineitem_streaming | 1230004 | t | 0 | 108199380 | f + orders_streaming | 1230007 | t | 0 | 108199380 | f + lineitem_streaming | 1230005 | t | 108199381 | 108199381 | f + orders_streaming | 1230008 | t | 108199381 | 108199381 | f + lineitem_streaming | 1230028 | t | 108199382 | 412880111 | f + orders_streaming | 1230031 | t | 108199382 | 412880111 | f + lineitem_streaming | 1230029 | t | 412880112 | 412880112 | f + orders_streaming | 1230032 | t | 412880112 | 412880112 | f + lineitem_streaming | 1230044 | t | 412880113 | 2147483646 | f + orders_streaming | 1230046 | t | 412880113 | 2147483646 | f + lineitem_streaming | 1230045 | t | 2147483647 | 2147483647 | f + orders_streaming | 1230047 | t | 2147483647 | 2147483647 | f (24 rows) \c - postgres - :master_port diff --git a/src/test/regress/expected/multi_tenant_isolation_nonblocking.out b/src/test/regress/expected/multi_tenant_isolation_nonblocking.out index 3daac7dacd9..8e0862733c2 100644 --- a/src/test/regress/expected/multi_tenant_isolation_nonblocking.out +++ b/src/test/regress/expected/multi_tenant_isolation_nonblocking.out @@ -168,12 +168,12 @@ SELECT count(*) FROM orders_streaming WHERE o_orderkey = 103; SELECT * FROM pg_dist_shard WHERE logicalrelid = 'lineitem_streaming'::regclass OR logicalrelid = 'orders_streaming'::regclass ORDER BY shardminvalue::BIGINT, logicalrelid; - logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue + logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode --------------------------------------------------------------------- - lineitem_streaming | 1230000 | t | -2147483648 | -1 - orders_streaming | 1230002 | t | -2147483648 | -1 - lineitem_streaming | 1230001 | t | 0 | 2147483647 - orders_streaming | 1230003 | t | 0 | 2147483647 + lineitem_streaming | 1230000 | t | -2147483648 | -1 | f + orders_streaming | 1230002 | t | -2147483648 | -1 | f + lineitem_streaming | 1230001 | t | 0 | 2147483647 | f + orders_streaming | 1230003 | t | 0 | 2147483647 | f (4 rows) -- check without cascade option @@ -399,32 +399,32 @@ SELECT count(*) FROM orders_streaming WHERE o_orderkey = 103; SELECT * FROM pg_dist_shard WHERE logicalrelid = 'lineitem_streaming'::regclass OR logicalrelid = 'orders_streaming'::regclass ORDER BY shardminvalue::BIGINT, logicalrelid; - logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue ---------------------------------------------------------------------- - lineitem_streaming | 1230040 | t | -2147483648 | -2147483648 - orders_streaming | 1230042 | t | -2147483648 | -2147483648 - lineitem_streaming | 1230041 | t | -2147483647 | -136164586 - orders_streaming | 1230043 | t | -2147483647 | -136164586 - lineitem_streaming | 1230035 | t | -136164585 | -136164585 - orders_streaming | 1230038 | t | -136164585 | -136164585 - lineitem_streaming | 1230036 | t | -136164584 | -85071815 - orders_streaming | 1230039 | t | -136164584 | -85071815 - lineitem_streaming | 1230011 | t | -85071814 | -85071814 - orders_streaming | 1230014 | t | -85071814 | -85071814 - lineitem_streaming | 1230012 | t | -85071813 | -1 - orders_streaming | 1230015 | t | -85071813 | -1 - lineitem_streaming | 1230004 | t | 0 | 108199380 - orders_streaming | 1230007 | t | 0 | 108199380 - lineitem_streaming | 1230005 | t | 108199381 | 108199381 - orders_streaming | 1230008 | t | 108199381 | 108199381 - lineitem_streaming | 1230028 | t | 108199382 | 412880111 - orders_streaming | 1230031 | t | 108199382 | 412880111 - lineitem_streaming | 1230029 | t | 412880112 | 412880112 - orders_streaming | 1230032 | t | 412880112 | 412880112 - lineitem_streaming | 1230044 | t | 412880113 | 2147483646 - orders_streaming | 1230046 | t | 412880113 | 2147483646 - lineitem_streaming | 1230045 | t | 2147483647 | 2147483647 - orders_streaming | 1230047 | t | 2147483647 | 2147483647 + logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode +--------------------------------------------------------------------- + lineitem_streaming | 1230040 | t | -2147483648 | -2147483648 | f + orders_streaming | 1230042 | t | -2147483648 | -2147483648 | f + lineitem_streaming | 1230041 | t | -2147483647 | -136164586 | f + orders_streaming | 1230043 | t | -2147483647 | -136164586 | f + lineitem_streaming | 1230035 | t | -136164585 | -136164585 | f + orders_streaming | 1230038 | t | -136164585 | -136164585 | f + lineitem_streaming | 1230036 | t | -136164584 | -85071815 | f + orders_streaming | 1230039 | t | -136164584 | -85071815 | f + lineitem_streaming | 1230011 | t | -85071814 | -85071814 | f + orders_streaming | 1230014 | t | -85071814 | -85071814 | f + lineitem_streaming | 1230012 | t | -85071813 | -1 | f + orders_streaming | 1230015 | t | -85071813 | -1 | f + lineitem_streaming | 1230004 | t | 0 | 108199380 | f + orders_streaming | 1230007 | t | 0 | 108199380 | f + lineitem_streaming | 1230005 | t | 108199381 | 108199381 | f + orders_streaming | 1230008 | t | 108199381 | 108199381 | f + lineitem_streaming | 1230028 | t | 108199382 | 412880111 | f + orders_streaming | 1230031 | t | 108199382 | 412880111 | f + lineitem_streaming | 1230029 | t | 412880112 | 412880112 | f + orders_streaming | 1230032 | t | 412880112 | 412880112 | f + lineitem_streaming | 1230044 | t | 412880113 | 2147483646 | f + orders_streaming | 1230046 | t | 412880113 | 2147483646 | f + lineitem_streaming | 1230045 | t | 2147483647 | 2147483647 | f + orders_streaming | 1230047 | t | 2147483647 | 2147483647 | f (24 rows) SELECT * FROM pg_dist_shard_placement WHERE shardid BETWEEN 1230000 AND 1399999 ORDER BY nodeport, shardid; @@ -553,32 +553,32 @@ SET search_path to "Tenant Isolation"; SELECT * FROM pg_dist_shard WHERE logicalrelid = 'lineitem_streaming'::regclass OR logicalrelid = 'orders_streaming'::regclass ORDER BY shardminvalue::BIGINT, logicalrelid; - logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue ---------------------------------------------------------------------- - lineitem_streaming | 1230040 | t | -2147483648 | -2147483648 - orders_streaming | 1230042 | t | -2147483648 | -2147483648 - lineitem_streaming | 1230041 | t | -2147483647 | -136164586 - orders_streaming | 1230043 | t | -2147483647 | -136164586 - lineitem_streaming | 1230035 | t | -136164585 | -136164585 - orders_streaming | 1230038 | t | -136164585 | -136164585 - lineitem_streaming | 1230036 | t | -136164584 | -85071815 - orders_streaming | 1230039 | t | -136164584 | -85071815 - lineitem_streaming | 1230011 | t | -85071814 | -85071814 - orders_streaming | 1230014 | t | -85071814 | -85071814 - lineitem_streaming | 1230012 | t | -85071813 | -1 - orders_streaming | 1230015 | t | -85071813 | -1 - lineitem_streaming | 1230004 | t | 0 | 108199380 - orders_streaming | 1230007 | t | 0 | 108199380 - lineitem_streaming | 1230005 | t | 108199381 | 108199381 - orders_streaming | 1230008 | t | 108199381 | 108199381 - lineitem_streaming | 1230028 | t | 108199382 | 412880111 - orders_streaming | 1230031 | t | 108199382 | 412880111 - lineitem_streaming | 1230029 | t | 412880112 | 412880112 - orders_streaming | 1230032 | t | 412880112 | 412880112 - lineitem_streaming | 1230044 | t | 412880113 | 2147483646 - orders_streaming | 1230046 | t | 412880113 | 2147483646 - lineitem_streaming | 1230045 | t | 2147483647 | 2147483647 - orders_streaming | 1230047 | t | 2147483647 | 2147483647 + logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode +--------------------------------------------------------------------- + lineitem_streaming | 1230040 | t | -2147483648 | -2147483648 | f + orders_streaming | 1230042 | t | -2147483648 | -2147483648 | f + lineitem_streaming | 1230041 | t | -2147483647 | -136164586 | f + orders_streaming | 1230043 | t | -2147483647 | -136164586 | f + lineitem_streaming | 1230035 | t | -136164585 | -136164585 | f + orders_streaming | 1230038 | t | -136164585 | -136164585 | f + lineitem_streaming | 1230036 | t | -136164584 | -85071815 | f + orders_streaming | 1230039 | t | -136164584 | -85071815 | f + lineitem_streaming | 1230011 | t | -85071814 | -85071814 | f + orders_streaming | 1230014 | t | -85071814 | -85071814 | f + lineitem_streaming | 1230012 | t | -85071813 | -1 | f + orders_streaming | 1230015 | t | -85071813 | -1 | f + lineitem_streaming | 1230004 | t | 0 | 108199380 | f + orders_streaming | 1230007 | t | 0 | 108199380 | f + lineitem_streaming | 1230005 | t | 108199381 | 108199381 | f + orders_streaming | 1230008 | t | 108199381 | 108199381 | f + lineitem_streaming | 1230028 | t | 108199382 | 412880111 | f + orders_streaming | 1230031 | t | 108199382 | 412880111 | f + lineitem_streaming | 1230029 | t | 412880112 | 412880112 | f + orders_streaming | 1230032 | t | 412880112 | 412880112 | f + lineitem_streaming | 1230044 | t | 412880113 | 2147483646 | f + orders_streaming | 1230046 | t | 412880113 | 2147483646 | f + lineitem_streaming | 1230045 | t | 2147483647 | 2147483647 | f + orders_streaming | 1230047 | t | 2147483647 | 2147483647 | f (24 rows) -- return to master node @@ -728,32 +728,32 @@ ERROR: cannot isolate tenant because tenant isolation is only support for hash SELECT * FROM pg_dist_shard WHERE logicalrelid = 'lineitem_streaming'::regclass OR logicalrelid = 'orders_streaming'::regclass ORDER BY shardminvalue::BIGINT, logicalrelid; - logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue ---------------------------------------------------------------------- - lineitem_streaming | 1230040 | t | -2147483648 | -2147483648 - orders_streaming | 1230042 | t | -2147483648 | -2147483648 - lineitem_streaming | 1230041 | t | -2147483647 | -136164586 - orders_streaming | 1230043 | t | -2147483647 | -136164586 - lineitem_streaming | 1230035 | t | -136164585 | -136164585 - orders_streaming | 1230038 | t | -136164585 | -136164585 - lineitem_streaming | 1230036 | t | -136164584 | -85071815 - orders_streaming | 1230039 | t | -136164584 | -85071815 - lineitem_streaming | 1230011 | t | -85071814 | -85071814 - orders_streaming | 1230014 | t | -85071814 | -85071814 - lineitem_streaming | 1230012 | t | -85071813 | -1 - orders_streaming | 1230015 | t | -85071813 | -1 - lineitem_streaming | 1230004 | t | 0 | 108199380 - orders_streaming | 1230007 | t | 0 | 108199380 - lineitem_streaming | 1230005 | t | 108199381 | 108199381 - orders_streaming | 1230008 | t | 108199381 | 108199381 - lineitem_streaming | 1230028 | t | 108199382 | 412880111 - orders_streaming | 1230031 | t | 108199382 | 412880111 - lineitem_streaming | 1230029 | t | 412880112 | 412880112 - orders_streaming | 1230032 | t | 412880112 | 412880112 - lineitem_streaming | 1230044 | t | 412880113 | 2147483646 - orders_streaming | 1230046 | t | 412880113 | 2147483646 - lineitem_streaming | 1230045 | t | 2147483647 | 2147483647 - orders_streaming | 1230047 | t | 2147483647 | 2147483647 + logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode +--------------------------------------------------------------------- + lineitem_streaming | 1230040 | t | -2147483648 | -2147483648 | f + orders_streaming | 1230042 | t | -2147483648 | -2147483648 | f + lineitem_streaming | 1230041 | t | -2147483647 | -136164586 | f + orders_streaming | 1230043 | t | -2147483647 | -136164586 | f + lineitem_streaming | 1230035 | t | -136164585 | -136164585 | f + orders_streaming | 1230038 | t | -136164585 | -136164585 | f + lineitem_streaming | 1230036 | t | -136164584 | -85071815 | f + orders_streaming | 1230039 | t | -136164584 | -85071815 | f + lineitem_streaming | 1230011 | t | -85071814 | -85071814 | f + orders_streaming | 1230014 | t | -85071814 | -85071814 | f + lineitem_streaming | 1230012 | t | -85071813 | -1 | f + orders_streaming | 1230015 | t | -85071813 | -1 | f + lineitem_streaming | 1230004 | t | 0 | 108199380 | f + orders_streaming | 1230007 | t | 0 | 108199380 | f + lineitem_streaming | 1230005 | t | 108199381 | 108199381 | f + orders_streaming | 1230008 | t | 108199381 | 108199381 | f + lineitem_streaming | 1230028 | t | 108199382 | 412880111 | f + orders_streaming | 1230031 | t | 108199382 | 412880111 | f + lineitem_streaming | 1230029 | t | 412880112 | 412880112 | f + orders_streaming | 1230032 | t | 412880112 | 412880112 | f + lineitem_streaming | 1230044 | t | 412880113 | 2147483646 | f + orders_streaming | 1230046 | t | 412880113 | 2147483646 | f + lineitem_streaming | 1230045 | t | 2147483647 | 2147483647 | f + orders_streaming | 1230047 | t | 2147483647 | 2147483647 | f (24 rows) \c - postgres - :master_port diff --git a/src/test/regress/expected/multi_test_helpers.out b/src/test/regress/expected/multi_test_helpers.out index 70a541d2acd..331704bd696 100644 --- a/src/test/regress/expected/multi_test_helpers.out +++ b/src/test/regress/expected/multi_test_helpers.out @@ -625,4 +625,91 @@ BEGIN ) q2 JOIN pg_dist_node USING (nodeid); END; -$func$ LANGUAGE plpgsql; \ No newline at end of file +$func$ LANGUAGE plpgsql; +-- Takes a table name and returns an array of colocated shards +-- --by enumerating them based on shardminvalue-- for each shard +-- of given distributed table (including colocated shards). +CREATE OR REPLACE FUNCTION get_enumerated_shard_groups( + qualified_table_name text) +RETURNS TABLE ( + shardids bigint[], + shardgroupindex bigint +) +AS $func$ + BEGIN + RETURN QUERY + SELECT array_agg(shardid ORDER BY shardid) AS shardids, + ROW_NUMBER() OVER (ORDER BY shardminvalue) AS shardgroupindex + FROM pg_dist_shard + JOIN pg_dist_partition USING(logicalrelid) + WHERE colocationid = (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = qualified_table_name::regclass) + GROUP BY shardminvalue; + END; +$func$ LANGUAGE plpgsql; +-- Takes a table name and returns a json object for each shard group that +-- contains a shard whose placements need separate nodes. +-- +-- This does not only return the shards of input relation but also considers +-- all colocated relations. +-- +-- An example output is as follows: +-- +-- [ +-- {"10": [{"dist_1": true},{"dist_2": false}]}, +-- {"15": [{"dist_1": false},{"dist_3": true}]} +-- ] +-- +-- It only returned shard groups 10 and 15 because they are the only shard groups +-- that contain at least one shard whose placements need an isolation. +-- +-- (Innermost) Boolean values represent needsseparatenode value for given +-- shard. For example, +-- +-- {"15": [{"dist_1": false},{"dist_3": true}]} +-- +-- means that the placements of dist_3 within shard group 15 needs +-- to be isolated. +CREATE OR REPLACE FUNCTION get_colocated_shards_needisolatednode( + qualified_table_name text) +RETURNS SETOF jsonb AS $func$ + BEGIN + RETURN QUERY + SELECT + COALESCE( + jsonb_agg(jsonb_build_object(shardgroupindex, needsseparatenodejson) ORDER BY shardgroupindex), + '{}'::jsonb + ) AS result + FROM ( + SELECT shardgroupindex, + jsonb_agg(jsonb_build_object(logicalrelid, needsseparatenode) ORDER BY logicalrelid::text) AS needsseparatenodejson + FROM ( + SELECT shardgroupindex, + logicalrelid, + needsseparatenode + FROM public.get_enumerated_shard_groups(qualified_table_name) AS shardgroups + JOIN pg_dist_shard + ON shardid = ANY(shardids) + ) q1 + GROUP BY shardgroupindex + ) q2 + WHERE needsseparatenodejson::text LIKE '%true%'; + END; +$func$ LANGUAGE plpgsql; +-- Returns true if all placement groups within given shard group are isolated. +CREATE OR REPLACE FUNCTION verify_placements_in_shard_group_isolated( + qualified_table_name text, + shard_group_index bigint) +RETURNS boolean +AS $func$ +DECLARE + v_result boolean; + BEGIN + SELECT bool_and(has_separate_node) INTO v_result + FROM citus_shards + JOIN ( + SELECT shardids FROM public.get_enumerated_shard_groups(qualified_table_name) WHERE shardgroupindex = shard_group_index + ) q + ON (shardid = ANY(q.shardids)); + RETURN v_result; + END; +$func$ LANGUAGE plpgsql; diff --git a/src/test/regress/expected/upgrade_isolate_placement_after.out b/src/test/regress/expected/upgrade_isolate_placement_after.out new file mode 100644 index 00000000000..3ae5cb1b180 --- /dev/null +++ b/src/test/regress/expected/upgrade_isolate_placement_after.out @@ -0,0 +1,21 @@ +-- upgrade_columnar_before renames public to citus_schema and recreates public +-- schema. But this file depends on get_colocated_shards_needisolatednode() +-- function and get_colocated_shards_needisolatednode() depends on another +-- function --get_enumerated_shard_groups()-- that is presumably created earlier +-- within the original public schema, so we temporarily rename citus_schema to +-- public here; and revert those changes at the end of this file. +ALTER SCHEMA public RENAME TO old_public; +ALTER SCHEMA citus_schema RENAME TO public; +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('upgrade_isolate_placement_before.table_with_isolated_placements') +$$) +ORDER BY nodeid; + result +--------------------------------------------------------------------- + [{"5": [{"upgrade_isolate_placement_before.table_with_isolated_placements": true}]}] + [{"5": [{"upgrade_isolate_placement_before.table_with_isolated_placements": true}]}] + [{"5": [{"upgrade_isolate_placement_before.table_with_isolated_placements": true}]}] +(3 rows) + +ALTER SCHEMA public RENAME TO citus_schema; +ALTER SCHEMA old_public RENAME TO public; diff --git a/src/test/regress/expected/upgrade_isolate_placement_before.out b/src/test/regress/expected/upgrade_isolate_placement_before.out new file mode 100644 index 00000000000..2242ce920f8 --- /dev/null +++ b/src/test/regress/expected/upgrade_isolate_placement_before.out @@ -0,0 +1,32 @@ +SET client_min_messages TO WARNING; +DROP SCHEMA IF EXISTS upgrade_isolate_placement_before CASCADE; +CREATE SCHEMA upgrade_isolate_placement_before; +SET search_path TO upgrade_isolate_placement_before; +SET client_min_messages TO NOTICE; +CREATE TABLE table_with_isolated_placements (a int, b int); +SELECT create_distributed_table('table_with_isolated_placements', 'a', colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT shardids[1] AS shardgroup_5_shardid +FROM public.get_enumerated_shard_groups('upgrade_isolate_placement_before.table_with_isolated_placements') +WHERE shardgroupindex = 5 \gset +SELECT citus_shard_property_set(:shardgroup_5_shardid, anti_affinity=>true); + citus_shard_property_set +--------------------------------------------------------------------- + +(1 row) + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('upgrade_isolate_placement_before.table_with_isolated_placements') +$$) +ORDER BY nodeid; + result +--------------------------------------------------------------------- + [{"5": [{"upgrade_isolate_placement_before.table_with_isolated_placements": true}]}] + [{"5": [{"upgrade_isolate_placement_before.table_with_isolated_placements": true}]}] + [{"5": [{"upgrade_isolate_placement_before.table_with_isolated_placements": true}]}] +(3 rows) + diff --git a/src/test/regress/expected/upgrade_list_citus_objects.out b/src/test/regress/expected/upgrade_list_citus_objects.out index 4f17695befc..8451dcd5c12 100644 --- a/src/test/regress/expected/upgrade_list_citus_objects.out +++ b/src/test/regress/expected/upgrade_list_citus_objects.out @@ -61,7 +61,7 @@ ORDER BY 1; function citus_internal.add_object_metadata(text,text[],text[],integer,integer,boolean) function citus_internal.add_partition_metadata(regclass,"char",text,integer,"char") function citus_internal.add_placement_metadata(bigint,bigint,integer,bigint) - function citus_internal.add_shard_metadata(regclass,bigint,"char",text,text) + function citus_internal.add_shard_metadata(regclass,bigint,"char",text,text,boolean) function citus_internal.add_tenant_schema(oid,integer) function citus_internal.adjust_local_clock_to_remote(cluster_clock) function citus_internal.commit_management_command_2pc() @@ -84,6 +84,7 @@ ORDER BY 1; function citus_internal.refresh_isolation_tester_prepared_statement() function citus_internal.replace_isolation_tester_func() function citus_internal.restore_isolation_tester_func() + function citus_internal.shard_property_set(bigint,boolean) function citus_internal.start_management_transaction(xid8) function citus_internal.start_replication_origin_tracking() function citus_internal.stop_replication_origin_tracking() @@ -96,7 +97,7 @@ ORDER BY 1; function citus_internal_add_partition_metadata(regclass,"char",text,integer,"char") function citus_internal_add_placement_metadata(bigint,bigint,integer,bigint) function citus_internal_add_placement_metadata(bigint,integer,bigint,integer,bigint) - function citus_internal_add_shard_metadata(regclass,bigint,"char",text,text) + function citus_internal_add_shard_metadata(regclass,bigint,"char",text,text,boolean) function citus_internal_add_tenant_schema(oid,integer) function citus_internal_adjust_local_clock_to_remote(cluster_clock) function citus_internal_delete_colocation_metadata(integer) @@ -157,6 +158,7 @@ ORDER BY 1; function citus_shard_cost_1(bigint) function citus_shard_cost_by_disk_size(bigint) function citus_shard_indexes_on_worker() + function citus_shard_property_set(bigint,boolean) function citus_shard_sizes() function citus_shards_on_worker() function citus_split_shard_by_split_points(bigint,text[],integer[],citus.shard_transfer_mode) @@ -371,5 +373,5 @@ ORDER BY 1; view citus_stat_tenants_local view pg_dist_shard_placement view time_partitions -(361 rows) +(363 rows) diff --git a/src/test/regress/multi_1_schedule b/src/test/regress/multi_1_schedule index cfff0094255..460aab09e09 100644 --- a/src/test/regress/multi_1_schedule +++ b/src/test/regress/multi_1_schedule @@ -43,6 +43,7 @@ test: comment_on_role # don't parallelize single_shard_table_udfs to make sure colocation ids are sequential test: single_shard_table_udfs test: schema_based_sharding +test: isolate_placement test: citus_schema_distribute_undistribute test: multi_test_catalog_views diff --git a/src/test/regress/spec/isolation_create_distributed_table.spec b/src/test/regress/spec/isolation_create_distributed_table.spec index 0934df35894..e86a2210480 100644 --- a/src/test/regress/spec/isolation_create_distributed_table.spec +++ b/src/test/regress/spec/isolation_create_distributed_table.spec @@ -1,11 +1,13 @@ setup { CREATE TABLE table_to_distribute(id int); + CREATE TABLE table_to_colocate(id int); } teardown { DROP TABLE table_to_distribute CASCADE; + DROP TABLE table_to_colocate CASCADE; } session "s1" @@ -20,6 +22,13 @@ step "s1-create_distributed_table" SELECT create_distributed_table('table_to_distribute', 'id'); } +step "s1_set-shard-property" +{ + SELECT citus_shard_property_set(shardid, anti_affinity=>'true') + FROM pg_dist_shard WHERE logicalrelid = 'table_to_distribute'::regclass + ORDER BY shardid LIMIT 1; +} + step "s1-copy_to_local_table" { COPY table_to_distribute FROM PROGRAM 'echo 0 && echo 1 && echo 2 && echo 3 && echo 4 && echo 5 && echo 6 && echo 7 && echo 8'; @@ -30,6 +39,11 @@ step "s1-commit" COMMIT; } +step "s1-rollback" +{ + ROLLBACK; +} + session "s2" step "s2-begin" @@ -42,6 +56,11 @@ step "s2-create_distributed_table" SELECT create_distributed_table('table_to_distribute', 'id'); } +step "s2-create_distributed_table_colocated" +{ + SELECT create_distributed_table('table_to_colocate', 'id', colocate_with=>'table_to_distribute'); +} + step "s2-copy_to_local_table" { COPY table_to_distribute FROM PROGRAM 'echo 0 && echo 1 && echo 2 && echo 3 && echo 4 && echo 5 && echo 6 && echo 7 && echo 8'; @@ -61,3 +80,7 @@ permutation "s1-begin" "s2-begin" "s2-copy_to_local_table" "s1-create_distribute //concurrent create_distributed_table on non-empty table permutation "s1-copy_to_local_table" "s1-begin" "s2-begin" "s1-create_distributed_table" "s2-create_distributed_table" "s1-commit" "s2-commit" + +//concurrent create_distributed_table vs citus_shard_property_set +permutation "s1-create_distributed_table" "s1-begin" "s2-begin" "s1_set-shard-property" "s2-create_distributed_table_colocated" "s1-rollback" "s2-commit" +permutation "s1-create_distributed_table" "s1-begin" "s2-begin" "s2-create_distributed_table_colocated" "s1_set-shard-property" "s1-rollback" "s2-commit" diff --git a/src/test/regress/sql/isolate_placement.sql b/src/test/regress/sql/isolate_placement.sql new file mode 100644 index 00000000000..5497a9c8b89 --- /dev/null +++ b/src/test/regress/sql/isolate_placement.sql @@ -0,0 +1,1205 @@ +-- Due to a race condition that happens in TransferShards() when the same shard id +-- is used to create the same shard on a different worker node, need to call +-- citus_cleanup_orphaned_resources() to clean up any orphaned resources before +-- running the tests. +-- +-- See https://github.com/citusdata/citus/pull/7180#issuecomment-1706786615. + +SET client_min_messages TO WARNING; +CALL citus_cleanup_orphaned_resources(); +RESET client_min_messages; + +CREATE SCHEMA isolate_placement; +SET search_path TO isolate_placement; + +-- test null input +SELECT citus_internal.shard_property_set(NULL, false); + +SET citus.shard_replication_factor TO 1; +SET citus.next_shard_id TO 2000000; + +CREATE TABLE single_shard_1(a int); +SELECT create_distributed_table('single_shard_1', null, colocate_with=>'none'); + +-- test with user that doesn't have permission to execute the function +SELECT citus_internal.shard_property_set(shardid, true) FROM pg_dist_shard WHERE logicalrelid = 'isolate_placement.single_shard_1'::regclass; + +DROP TABLE single_shard_1; + +CREATE ROLE test_user_isolate_placement WITH LOGIN; +GRANT ALL ON SCHEMA isolate_placement TO test_user_isolate_placement; +ALTER SYSTEM SET citus.enable_manual_metadata_changes_for_user TO 'test_user_isolate_placement'; +SELECT pg_reload_conf(); +SELECT pg_sleep(0.1); +SET ROLE test_user_isolate_placement; + +-- test invalid shard id +SELECT citus_internal.shard_property_set(0, true); + +-- test null needs_separate_node +SELECT citus_internal_add_shard_metadata( + relation_id=>0, + shard_id=>0, + storage_type=>'0', + shard_min_value=>'0', + shard_max_value=>'0', + needs_separate_node=>null); + +RESET ROLE; +REVOKE ALL ON SCHEMA isolate_placement FROM test_user_isolate_placement; +DROP USER test_user_isolate_placement; +ALTER SYSTEM RESET citus.enable_manual_metadata_changes_for_user; +SELECT pg_reload_conf(); +SELECT pg_sleep(0.1); + +SET search_path TO isolate_placement; +SET citus.shard_replication_factor TO 1; +SET citus.next_shard_id TO 2001000; + +CREATE USER mysuperuser superuser; +SET ROLE mysuperuser; + +CREATE TABLE single_shard_1(a int); +SELECT create_distributed_table('single_shard_1', null, colocate_with=>'none'); + +CREATE USER regularuser; +GRANT USAGE ON SCHEMA isolate_placement TO regularuser; +ALTER SYSTEM SET citus.enable_manual_metadata_changes_for_user TO 'regularuser'; +SELECT pg_reload_conf(); +SELECT pg_sleep(0.1); + +SET ROLE regularuser; + +-- throws an error as the user is not the owner of the table +SELECT citus_shard_property_set(shardid) FROM pg_dist_shard WHERE logicalrelid = 'isolate_placement.single_shard_1'::regclass; +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid = 'isolate_placement.single_shard_1'::regclass; +SELECT citus_internal.shard_property_set(shardid, true) FROM pg_dist_shard WHERE logicalrelid = 'isolate_placement.single_shard_1'::regclass; + +-- assign all tables to regularuser +RESET ROLE; +REASSIGN OWNED BY mysuperuser TO regularuser; + +SET ROLE regularuser; + +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid = 'isolate_placement.single_shard_1'::regclass; + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.single_shard_1') +$$) +ORDER BY result; + +SELECT citus_internal.shard_property_set(shardid, false) FROM pg_dist_shard WHERE logicalrelid = 'isolate_placement.single_shard_1'::regclass; + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.single_shard_1') +$$) +ORDER BY result; + +SELECT citus_internal.shard_property_set(shardid, true) FROM pg_dist_shard WHERE logicalrelid = 'isolate_placement.single_shard_1'::regclass; + +DROP TABLE single_shard_1; +RESET ROLE; +REVOKE USAGE ON SCHEMA isolate_placement FROM regularuser; +ALTER SYSTEM RESET citus.enable_manual_metadata_changes_for_user; +SELECT pg_reload_conf(); +SELECT pg_sleep(0.1); +DROP ROLE regularuser, mysuperuser; + +SET search_path TO isolate_placement; + +SET citus.next_shard_id TO 2002000; +SET citus.shard_count TO 32; +SET citus.shard_replication_factor TO 1; + +SET client_min_messages TO WARNING; +SELECT 1 FROM citus_add_node('localhost', :master_port, groupid => 0); +SET client_min_messages TO NOTICE; + +SET citus.shard_replication_factor TO 2; + +CREATE TABLE dist_1(a int); +CREATE TABLE dist_2(a int); +CREATE TABLE dist_3(a int); +SELECT create_distributed_table('dist_1', 'a'); +SELECT create_distributed_table('dist_2', 'a', colocate_with=>'dist_1'); +SELECT create_distributed_table('dist_3', 'a', colocate_with=>'dist_1'); + +SET citus.shard_replication_factor TO 1; + +-- none of the placements have been marked as needsseparatenode yet +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + +SELECT shardids[2] AS shardgroup_5_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 5 \gset + +-- no-op .. +SELECT citus_shard_property_set(:shardgroup_5_shardid, NULL); + +CREATE ROLE test_user_isolate_placement WITH LOGIN; +GRANT ALL ON SCHEMA isolate_placement TO test_user_isolate_placement; +ALTER TABLE dist_1 OWNER TO test_user_isolate_placement; +ALTER TABLE dist_2 OWNER TO test_user_isolate_placement; +ALTER TABLE dist_3 OWNER TO test_user_isolate_placement; +ALTER SYSTEM SET citus.enable_manual_metadata_changes_for_user TO 'test_user_isolate_placement'; +SELECT pg_reload_conf(); +SELECT pg_sleep(0.1); +SET ROLE test_user_isolate_placement; + +-- no-op .. +SELECT citus_internal.shard_property_set(:shardgroup_5_shardid, NULL); + +RESET ROLE; +ALTER TABLE dist_1 OWNER TO current_user; +ALTER TABLE dist_2 OWNER TO current_user; +ALTER TABLE dist_3 OWNER TO current_user; +REVOKE ALL ON SCHEMA isolate_placement FROM test_user_isolate_placement; +DROP USER test_user_isolate_placement; +ALTER SYSTEM RESET citus.enable_manual_metadata_changes_for_user; +SELECT pg_reload_conf(); +SELECT pg_sleep(0.1); + +-- .. hence returns empty objects +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + +SELECT citus_shard_property_set(:shardgroup_5_shardid, anti_affinity=>true); + +SELECT shardids[3] AS shardgroup_10_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 10 \gset + +SELECT citus_shard_property_set(:shardgroup_10_shardid, anti_affinity=>true); + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + +SELECT shardids[1] AS shardgroup_3_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 3 \gset + +SELECT citus_shard_property_set(:shardgroup_3_shardid, anti_affinity=>false); + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + +SELECT shardids[1] AS shardgroup_10_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 10 \gset + +SELECT citus_shard_property_set(:shardgroup_10_shardid, anti_affinity=>false); + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + +SELECT shardids[1] AS shardgroup_5_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 5 \gset + +SELECT citus_shard_property_set(:shardgroup_5_shardid, anti_affinity=>true); + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + +-- test metadata sync + +-- first, need to re-create them with shard_replication_factor = 1 because we will first remove worker_2 + +DROP TABLE dist_1, dist_2, dist_3; + +SELECT 1 FROM citus_remove_node('localhost', :worker_2_port); + +CREATE TABLE dist_1(a int); +CREATE TABLE dist_2(a int); +CREATE TABLE dist_3(a int); +SELECT create_distributed_table('dist_1', 'a'); +SELECT create_distributed_table('dist_2', 'a', colocate_with=>'dist_1'); +SELECT create_distributed_table('dist_3', 'a', colocate_with=>'dist_1'); + +SELECT 1 FROM citus_add_node('localhost', :worker_2_port); + +SELECT shardids[1] AS shardgroup_5_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 5 \gset + +SELECT citus_shard_property_set(:shardgroup_5_shardid, anti_affinity=>true); + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + +CREATE TABLE dist_4(a int); +SELECT create_distributed_table('dist_4', 'a', colocate_with=>'dist_1'); + +CREATE TABLE dist_4_concurrently(a int); +SELECT create_distributed_table_concurrently('dist_4_concurrently', 'a', colocate_with=>'dist_1'); + +-- Placements of a new distributed table created within the same colocated +-- group inherit needsseparatenode from the colocated placements too. +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + +DROP TABLE dist_4, dist_4_concurrently; + +-- Returns source and target node ids that can be used to perform a +-- shard transfer for one of the placements of given shard. +CREATE OR REPLACE FUNCTION get_candidate_node_for_shard_transfer( + p_shardid bigint) +RETURNS TABLE (source_nodeid integer, target_nodeid integer) +SET search_path TO 'pg_catalog, public' +AS $func$ +DECLARE + v_source_nodeids integer[]; + v_target_nodeid integer; +BEGIN + SELECT array_agg(nodeid) INTO v_source_nodeids + FROM pg_dist_shard + JOIN pg_dist_placement USING (shardid) + JOIN pg_dist_node USING (groupid) + WHERE noderole = 'primary' AND shardid = p_shardid; + + IF v_source_nodeids IS NULL + THEN + RAISE EXCEPTION 'could not determine the source node of shard %', p_shardid; + END IF; + + SELECT nodeid INTO v_target_nodeid + FROM pg_dist_node + WHERE isactive AND shouldhaveshards AND noderole='primary' AND + nodeid NOT IN (SELECT unnest(v_source_nodeids)) + LIMIT 1; + + IF v_target_nodeid IS NULL + THEN + RAISE EXCEPTION 'could not determine a node to transfer the placement to'; + END IF; + + RETURN QUERY SELECT v_source_nodeids[1], v_target_nodeid; +END; +$func$ LANGUAGE plpgsql; + +SELECT shardids[1] AS shardgroup_15_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 15 \gset + +SELECT citus_move_shard_placement(:shardgroup_5_shardid, source_nodeid, target_nodeid, 'block_writes') +FROM get_candidate_node_for_shard_transfer(:shardgroup_5_shardid); + +SELECT citus_move_shard_placement(:shardgroup_15_shardid, source_nodeid, target_nodeid, 'block_writes') +FROM get_candidate_node_for_shard_transfer(:shardgroup_15_shardid); + +-- so that citus_copy_shard_placement works +UPDATE pg_dist_partition SET repmodel = 'c' WHERE logicalrelid = 'isolate_placement.dist_1'::regclass; + +SELECT citus_copy_shard_placement(:shardgroup_5_shardid, source_nodeid, target_nodeid, 'block_writes') +FROM get_candidate_node_for_shard_transfer(:shardgroup_5_shardid); + +SELECT citus_copy_shard_placement(:shardgroup_15_shardid, source_nodeid, target_nodeid, 'block_writes') +FROM get_candidate_node_for_shard_transfer(:shardgroup_15_shardid); + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + +DROP TABLE dist_1, dist_2, dist_3; + +CREATE TABLE dist_1(a int); +CREATE TABLE dist_2(a int); +SELECT create_distributed_table('dist_1', 'a', shard_count=>3); +SELECT create_distributed_table('dist_2', 'a', colocate_with=>'dist_1'); + +SELECT shardids[1] AS shardgroup_3_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 3 \gset + +SELECT citus_shard_property_set(:shardgroup_3_shardid, anti_affinity=>true); + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + +-- so that replicate_table_shards works +UPDATE pg_dist_partition SET repmodel = 'c' WHERE logicalrelid = 'isolate_placement.dist_1'::regclass; + +SET client_min_messages TO WARNING; +SELECT replicate_table_shards('isolate_placement.dist_1', shard_replication_factor=>2, shard_transfer_mode=>'block_writes'); +SET client_min_messages TO NOTICE; + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + +DROP TABLE dist_1, dist_2; + +CREATE TABLE dist_1(a int); +CREATE TABLE dist_2(a int); +SELECT create_distributed_table('dist_1', 'a'); +SELECT create_distributed_table('dist_2', 'a', colocate_with=>'dist_1'); + +SELECT shardids[1] AS shardgroup_9_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 9 \gset + +SELECT citus_shard_property_set(:shardgroup_9_shardid, anti_affinity=>true); + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + +SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset +SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset + +SELECT pg_catalog.citus_split_shard_by_split_points( + :shardgroup_9_shardid, + ARRAY[((shardminvalue::bigint + shardmaxvalue::bigint) / 2)::text], + ARRAY[:worker_1_node, :worker_2_node], + 'block_writes') +FROM pg_dist_shard +WHERE shardid = :shardgroup_9_shardid; + +-- We shouldn't see shard group 9 because shard-split operation doesn't +-- preserve needsseparatenode flag when splitting the shard. +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + +SELECT shardids[1] AS shardgroup_12_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 12 \gset + +SELECT citus_shard_property_set(:shardgroup_12_shardid, anti_affinity=>true); + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + +SELECT shardids[1] AS shardgroup_10_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 10 \gset + +SELECT pg_catalog.citus_split_shard_by_split_points( + :shardgroup_10_shardid, + ARRAY[((shardminvalue::bigint + shardmaxvalue::bigint) / 2)::text], + ARRAY[:worker_1_node, :worker_2_node], + 'block_writes') +FROM pg_dist_shard +WHERE shardid = :shardgroup_10_shardid; + +-- We should see old shard group 12 (now as 13 due to split +-- of a prior shard) because it's not the one we splitted. +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + +CREATE TABLE dist_3(a int); +SELECT create_distributed_table('dist_3', 'a', colocate_with=>'none'); + +SELECT shardids[1] AS shardgroup_17_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_3') +WHERE shardgroupindex = 17 \gset + +SELECT citus_shard_property_set(:shardgroup_17_shardid, anti_affinity=>true); + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_3') +$$) +ORDER BY result; + +-- verify that shard key value 100 is stored on shard group 17 +select get_shard_id_for_distribution_column('dist_3', 100) = :shardgroup_17_shardid; + +SELECT 1 FROM isolate_tenant_to_new_shard('dist_3', 100, shard_transfer_mode => 'block_writes'); + +-- We shouldn't see shard group 17 because isolate_tenant_to_new_shard doesn't +-- preserve needsseparatenode flag when splitting the shard. +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_3') +$$) +ORDER BY result; + +SELECT shardids[1] AS shardgroup_18_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_3') +WHERE shardgroupindex = 18 \gset + +SELECT citus_shard_property_set(:shardgroup_18_shardid, anti_affinity=>true); + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_3') +$$) +ORDER BY result; + +-- verify that shard key value 1000 is _not_ stored on shard group 18 +SELECT get_shard_id_for_distribution_column('dist_3', 1000) != :shardgroup_18_shardid; + +SELECT 1 FROM isolate_tenant_to_new_shard('dist_3', 1000, shard_transfer_mode => 'block_writes'); + +-- We should see shard group 18 (now as 20 due to split of a prior shard) +-- because it's not the one we splitted. +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_3') +$$) +ORDER BY result; + +CREATE TABLE single_shard_1(a int); +SELECT create_distributed_table('single_shard_1', null, colocate_with=>'none'); + +SELECT shardids[1] AS shardgroup_1_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.single_shard_1') +WHERE shardgroupindex = 1 \gset + +SELECT citus_shard_property_set(:shardgroup_1_shardid, anti_affinity=>true); + +-- noop +SELECT citus_shard_property_set(:shardgroup_1_shardid, NULL); +SELECT citus_shard_property_set(:shardgroup_1_shardid); + +CREATE TABLE single_shard_2(a int); +SELECT create_distributed_table('single_shard_2', null, colocate_with=>'single_shard_1'); + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.single_shard_1') +$$) +ORDER BY result; + +-- test invalid input +SELECT citus_shard_property_set(NULL, anti_affinity=>true); +SELECT citus_shard_property_set(0, anti_affinity=>true); +SELECT citus_shard_property_set(NULL, anti_affinity=>false); +SELECT citus_shard_property_set(0, anti_affinity=>false); + +-- we verify whether shard exists even if anti_affinity is not provided +SELECT citus_shard_property_set(0, anti_affinity=>NULL); + +CREATE TABLE append_table (a int, b int); +SELECT create_distributed_table('append_table', 'a', 'append'); +SELECT 1 FROM master_create_empty_shard('append_table'); + +CREATE TYPE composite_key_type AS (f1 int, f2 text); +CREATE TABLE range_table(key composite_key_type, value int); +SELECT create_distributed_table('range_table', 'key', 'range'); +CALL public.create_range_partitioned_shards('range_table', '{"(0,a)","(25,a)"}','{"(24,z)","(49,z)"}'); + +CREATE TABLE ref_table(a int); +SELECT create_reference_table('ref_table'); + +CREATE TABLE local_table(a int); +SELECT citus_add_local_table_to_metadata('local_table'); + +-- all should fail +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid = 'append_table'::regclass LIMIT 1; +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid = 'range_table'::regclass LIMIT 1; +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid = 'ref_table'::regclass LIMIT 1; +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid = 'local_table'::regclass LIMIT 1; + +SELECT citus_shard_property_set(shardid, anti_affinity=>false) FROM pg_dist_shard WHERE logicalrelid = 'append_table'::regclass LIMIT 1; +SELECT citus_shard_property_set(shardid, anti_affinity=>false) FROM pg_dist_shard WHERE logicalrelid = 'range_table'::regclass LIMIT 1; +SELECT citus_shard_property_set(shardid, anti_affinity=>false) FROM pg_dist_shard WHERE logicalrelid = 'ref_table'::regclass LIMIT 1; +SELECT citus_shard_property_set(shardid, anti_affinity=>false) FROM pg_dist_shard WHERE logicalrelid = 'local_table'::regclass LIMIT 1; + +DROP TABLE range_table; +DROP TYPE composite_key_type; + +SET client_min_messages TO WARNING; +DROP SCHEMA isolate_placement CASCADE; + +CREATE SCHEMA isolate_placement; +SET search_path TO isolate_placement; + +SET client_min_messages TO NOTICE; + +CREATE TABLE dist_1(a int); +CREATE TABLE dist_2(a int); +SELECT create_distributed_table('dist_1', 'a', shard_count=>4); +SELECT create_distributed_table('dist_2', 'a', colocate_with=>'dist_1'); + +CREATE TABLE dist_non_colocated(a int); +SELECT create_distributed_table('dist_non_colocated', 'a', shard_count=>4, colocate_with=>'none'); + +CREATE TABLE single_shard_1(a int); +SELECT create_distributed_table('single_shard_1', null, colocate_with=>'none'); + +CREATE TABLE single_shard_2(a int); +SELECT create_distributed_table('single_shard_2', null, colocate_with=>'none'); + +CREATE TABLE append_table (a int, b int); +SELECT create_distributed_table('append_table', 'a', 'append'); +SELECT 1 FROM master_create_empty_shard('append_table'); + +CREATE TABLE range_table(a int, b int); +SELECT create_distributed_table('range_table', 'a', 'range'); +CALL public.create_range_partitioned_shards('range_table', '{"0","25"}','{"26","50"}'); + +CREATE TABLE reference_table_1(a int); +SELECT create_reference_table('reference_table_1'); + +CREATE TABLE local_table_1(a int); +SELECT citus_add_local_table_to_metadata('local_table_1'); + +SELECT shardids[1] AS shardgroup_1_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 1 \gset + +SELECT citus_shard_property_set(:shardgroup_1_shardid, anti_affinity=>true); + +SET client_min_messages TO WARNING; +SELECT rebalance_table_shards(shard_transfer_mode=>'block_writes'); +SET client_min_messages TO NOTICE; + +CREATE TABLE reference_table_2(a int); +SELECT create_reference_table('reference_table_2'); + +CREATE TABLE local_table_2(a int); +SELECT citus_add_local_table_to_metadata('local_table_2'); + +-- make sure that we still have placements for both reference tables on all nodes +SELECT COUNT(DISTINCT(groupid))=3 FROM pg_dist_shard JOIN pg_dist_placement USING (shardid) WHERE logicalrelid = 'reference_table_1'::regclass; +SELECT COUNT(DISTINCT(groupid))=3 FROM pg_dist_shard JOIN pg_dist_placement USING (shardid) WHERE logicalrelid = 'reference_table_2'::regclass; + +-- sanity check for local tables +SELECT groupid = 0 FROM pg_dist_shard JOIN pg_dist_placement USING (shardid) WHERE logicalrelid = 'local_table_1'::regclass; +SELECT groupid = 0 FROM pg_dist_shard JOIN pg_dist_placement USING (shardid) WHERE logicalrelid = 'local_table_2'::regclass; + +CREATE TABLE dist_post_non_colocated(a int); +SELECT create_distributed_table('dist_post_non_colocated', 'a', shard_count=>4, colocate_with=>'none'); + +CREATE TABLE dist_post_concurrently_non_colocated(a int); +SELECT create_distributed_table_concurrently('dist_post_concurrently_non_colocated', 'a', shard_count=>4, colocate_with=>'none'); + +CREATE TABLE dist_post_colocated(a int); +SELECT create_distributed_table('dist_post_colocated', 'a', colocate_with=>'dist_1'); + +CREATE TABLE dist_post_concurrently_colocated(a int); +SELECT create_distributed_table_concurrently('dist_post_concurrently_colocated', 'a', colocate_with=>'dist_1'); + +CREATE TABLE single_shard_post(a int); +SELECT create_distributed_table('single_shard_post', null, colocate_with=>'none'); + +CREATE TABLE append_table_post(a int, b int); +SELECT create_distributed_table('append_table_post', 'a', 'append'); +SELECT 1 FROM master_create_empty_shard('append_table_post'); + +CREATE TABLE range_table_post(a int, b int); +SELECT create_distributed_table('range_table_post', 'a', 'range'); +CALL public.create_range_partitioned_shards('range_table_post', '{"0","25"}','{"26","50"}'); + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('isolate_placement.dist_1') +$$) +ORDER BY result; + +-- Make sure that the node that contains shard-group 1 of isolate_placement.dist_1 +-- doesn't have any other placements. +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.dist_1', 1); + +SET client_min_messages TO ERROR; +SELECT citus_drain_node('localhost', :worker_1_port, shard_transfer_mode=>'block_writes'); +SET client_min_messages TO NOTICE; + +SELECT citus_set_node_property('localhost', :worker_1_port, 'shouldhaveshards', true); + +-- drain node should have failed and the node should still have the same set of placements +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.dist_1', 1); + +SET client_min_messages TO ERROR; +SELECT citus_drain_node('localhost', :worker_2_port, shard_transfer_mode=>'block_writes'); +SET client_min_messages TO NOTICE; + +SELECT citus_set_node_property('localhost', :worker_2_port, 'shouldhaveshards', true); + +-- drain node should have failed and the node should still have the same set of placements +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.dist_1', 1); + +CREATE TABLE dist_3(a int); +SELECT create_distributed_table('dist_3', 'a', colocate_with=>'dist_1'); + +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.dist_1', 1); + +SET citus.shard_replication_factor TO 2; + +CREATE TABLE dist_replicated(a int); + +-- fails as we only have one node that's not used to isolate a shard placement group +SELECT create_distributed_table('dist_replicated', 'a', shard_count=>4, colocate_with=>'none'); + +SET citus.shard_replication_factor TO 1; + +CREATE TABLE dist_to_be_replicated(a int); +SELECT create_distributed_table('dist_to_be_replicated', 'a', shard_count=>4, colocate_with=>'none'); + +UPDATE pg_dist_partition SET repmodel = 'c' WHERE logicalrelid = 'isolate_placement.dist_to_be_replicated'::regclass; + +SET client_min_messages TO WARNING; +-- fails as we only have one node that's not used to isolate a shard placement group +SELECT replicate_table_shards('isolate_placement.dist_to_be_replicated', shard_replication_factor=>2, shard_transfer_mode=>'block_writes'); +SET client_min_messages TO NOTICE; + +SELECT citus_set_node_property('localhost', :master_port, 'shouldhaveshards', true); + +SET client_min_messages TO WARNING; +-- succeeds as now we have two nodes that are not used to isolate a shard placement group +SELECT replicate_table_shards('isolate_placement.dist_to_be_replicated', shard_replication_factor=>2, shard_transfer_mode=>'block_writes'); +SET client_min_messages TO NOTICE; + +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.dist_1', 1); + +SELECT DISTINCT(table_name::regclass::text) +FROM citus_shards +JOIN pg_class ON (oid = table_name) +WHERE relnamespace = 'isolate_placement'::regnamespace AND has_separate_node +ORDER BY 1; + +SELECT bool_or(has_separate_node) = false +FROM citus_shards +JOIN ( + SELECT unnest(shardids) shardid + FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') + WHERE shardgroupindex != 1 +) shards_except_group_1 USING (shardid); + +DROP TABLE dist_to_be_replicated; + +SELECT citus_drain_node('localhost', :master_port, shard_transfer_mode=>'block_writes'); + +DROP TABLE dist_replicated; + +SET client_min_messages TO WARNING; +DROP SCHEMA isolate_placement CASCADE; + +CREATE SCHEMA isolate_placement; +SET search_path TO isolate_placement; + +SET client_min_messages TO NOTICE; + +CREATE TABLE single_shard_1(a int); +SELECT create_distributed_table('single_shard_1', null, colocate_with=>'none'); + +CREATE TABLE single_shard_2(a int); +SELECT create_distributed_table('single_shard_2', null, colocate_with=>'none'); + +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_1'::regclass, 'single_shard_2'::regclass); + +SET client_min_messages TO WARNING; +SELECT rebalance_table_shards(shard_transfer_mode=>'block_writes'); +SET client_min_messages TO NOTICE; + +-- fails + +CREATE TABLE dist_1(a int); +SELECT create_distributed_table('dist_1', 'a', shard_count=>4); + +CREATE TABLE single_shard_3(a int); +SELECT create_distributed_table('single_shard_3', null, colocate_with=>'none'); + +CREATE TABLE append_table (a int, b int); +SELECT create_distributed_table('append_table', 'a', 'append'); +SELECT 1 FROM master_create_empty_shard('append_table'); + +CREATE TABLE range_table(a int, b int); +SELECT create_distributed_table('range_table', 'a', 'range'); +CALL public.create_range_partitioned_shards('range_table', '{"0","25"}','{"26","50"}'); + +-- succeeds + +CREATE TABLE reference_table_1(a int); +SELECT create_reference_table('reference_table_1'); + +CREATE TABLE local_table_1(a int); +SELECT citus_add_local_table_to_metadata('local_table_1'); + +CREATE TABLE single_shard_4(a int); +SELECT create_distributed_table('single_shard_4', null, colocate_with=>'single_shard_1'); + +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_1', 1); +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_2', 1); + +SET client_min_messages TO WARNING; +DROP SCHEMA isolate_placement CASCADE; + +CREATE SCHEMA isolate_placement; +SET search_path TO isolate_placement; + +SET client_min_messages TO NOTICE; + +CREATE TABLE single_shard_1(a int); +SELECT create_distributed_table('single_shard_1', null, colocate_with=>'none'); + +CREATE TABLE single_shard_2(a int); +SELECT create_distributed_table('single_shard_2', null, colocate_with=>'none'); + +-- Make sure that we don't assume that a node is used to isolate a shard placement +-- group just because it contains a single shard placement group. +CREATE TABLE single_shard_3(a int); +SELECT create_distributed_table('single_shard_3', null, colocate_with=>'none'); + +SET client_min_messages TO WARNING; +DROP SCHEMA isolate_placement CASCADE; + +CREATE SCHEMA isolate_placement; +SET search_path TO isolate_placement; + +SET client_min_messages TO NOTICE; + +SELECT citus_set_node_property('localhost', :master_port, 'shouldhaveshards', true); + +CREATE TABLE dist_1(a int); +CREATE TABLE dist_2(a int); -- will replicate this +CREATE TABLE dist_3(a int); +SELECT create_distributed_table('dist_1', 'a', shard_count=>1); +SELECT create_distributed_table('dist_2', 'a', shard_count=>1, colocate_with=>'none'); +SELECT create_distributed_table('dist_3', 'a', shard_count=>1, colocate_with=>'none'); + +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid IN ('dist_1'::regclass, 'dist_2'::regclass); + +SET client_min_messages TO WARNING; +SELECT rebalance_table_shards(shard_transfer_mode=>'block_writes'); +SET client_min_messages TO NOTICE; + +-- so that replicate_table_shards works +UPDATE pg_dist_partition SET repmodel = 'c' WHERE logicalrelid = 'isolate_placement.dist_2'::regclass; + +SET client_min_messages TO WARNING; +-- succeeds but breaks the isolation requirement for either of dist_1 or dist_2 .. +SELECT replicate_table_shards('isolate_placement.dist_2', shard_replication_factor=>2, shard_transfer_mode=>'block_writes'); +SET client_min_messages TO NOTICE; + +-- .. so check the xor of the isolation requirements for dist_1 and dist_2 +SELECT (public.verify_placements_in_shard_group_isolated('isolate_placement.dist_1', 1) OR public.verify_placements_in_shard_group_isolated('isolate_placement.dist_2', 1)) = true AND + (public.verify_placements_in_shard_group_isolated('isolate_placement.dist_1', 1) AND public.verify_placements_in_shard_group_isolated('isolate_placement.dist_2', 1)) = false; + +DROP TABLE dist_1, dist_2, dist_3; + +SELECT citus_drain_node('localhost', :master_port, shard_transfer_mode=>'block_writes'); + +CREATE TABLE single_shard_1(a int); +SELECT create_distributed_table('single_shard_1', null, colocate_with=>'none'); + +CREATE TABLE single_shard_2(a int); +SELECT create_distributed_table('single_shard_2', null, colocate_with=>'none'); + +CREATE TABLE dist_1(a int); +SELECT create_distributed_table('dist_1', 'a', shard_count=>4); + +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_1'::regclass); + +SELECT groupid AS single_shard_1_group_id FROM pg_dist_shard JOIN pg_dist_placement USING (shardid) WHERE logicalrelid = 'isolate_placement.single_shard_1'::regclass \gset + +SET client_min_messages TO WARNING; +SELECT rebalance_table_shards(shard_transfer_mode=>'block_writes'); +SET client_min_messages TO NOTICE; + +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_1', 1); + +-- show that we try to isolate placements where they were staying at the time rebalancer is invoked +SELECT groupid = :single_shard_1_group_id FROM pg_dist_shard JOIN pg_dist_placement USING (shardid) WHERE logicalrelid = 'isolate_placement.single_shard_1'::regclass; + +DROP TABLE dist_1, single_shard_1, single_shard_2; + +SET citus.shard_replication_factor TO 2; + +CREATE TABLE dist_1(a int); +CREATE TABLE dist_2(a int); + +SELECT citus_set_node_property('localhost', :master_port, 'shouldhaveshards', true); + +SELECT create_distributed_table('dist_1', 'a', shard_count=>1); +SELECT create_distributed_table('dist_2', 'a', colocate_with=>'dist_1'); + +SET citus.shard_replication_factor TO 1; + +SELECT shardids[1] AS shardgroup_1_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_2') +WHERE shardgroupindex = 1 \gset + +SELECT citus_shard_property_set(:shardgroup_1_shardid, anti_affinity=>true); + +SET client_min_messages TO WARNING; +SELECT rebalance_table_shards(shard_transfer_mode=>'block_writes'); +SET client_min_messages TO NOTICE; + +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.dist_1', 1) = true; + +DROP TABLE dist_1, dist_2; + +SELECT citus_drain_node('localhost', :master_port, shard_transfer_mode=>'block_writes'); + +CREATE TABLE single_shard_1(a int); +SELECT create_distributed_table('single_shard_1', null, colocate_with=>'none'); + +CREATE TABLE single_shard_2(a int); +SELECT create_distributed_table('single_shard_2', null, colocate_with=>'none'); + +CREATE TABLE single_shard_3(a int); +SELECT create_distributed_table('single_shard_3', null, colocate_with=>'none'); + +CREATE TABLE single_shard_4(a int); +SELECT create_distributed_table('single_shard_4', null, colocate_with=>'none'); + +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_1'::regclass); + +SET client_min_messages TO WARNING; +SELECT rebalance_table_shards(shard_transfer_mode=>'block_writes'); +SET client_min_messages TO NOTICE; + +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_1', 1); + +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_2'::regclass); +SELECT citus_shard_property_set(shardid, anti_affinity=>false) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_1'::regclass); + +SET client_min_messages TO WARNING; +SELECT rebalance_table_shards(shard_transfer_mode=>'block_writes'); +SET client_min_messages TO NOTICE; + +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_2', 1); +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_1', 1) = false; + +DROP TABLE single_shard_1, single_shard_2, single_shard_3, single_shard_4; + +CREATE TABLE single_shard_1(a int); +SELECT create_distributed_table('single_shard_1', null, colocate_with=>'none'); + +CREATE TABLE single_shard_2(a int); +SELECT create_distributed_table('single_shard_2', null, colocate_with=>'none'); + +-- this would be placed on the same node as single_shard_1 +CREATE TABLE single_shard_3(a int); +SELECT create_distributed_table('single_shard_3', null, colocate_with=>'none'); + +DROP TABLE single_shard_2; + +SELECT shardid, nodeid INTO single_shard_3_shardid_nodeid +FROM pg_dist_shard JOIN pg_dist_placement USING (shardid) JOIN pg_dist_node USING (groupid) +WHERE logicalrelid = 'isolate_placement.single_shard_3'::regclass AND noderole = 'primary'; + +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_1'::regclass); +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_3'::regclass); + +-- tell rebalancer that single_shard_3 cannot be placed on the node where it is currently placed +CREATE OR REPLACE FUNCTION test_shard_allowed_on_node(p_shardid bigint, p_nodeid int) + RETURNS boolean AS +$$ + SELECT + CASE + WHEN (p_shardid = shardid and p_nodeid = nodeid) THEN false + ELSE true + END + FROM single_shard_3_shardid_nodeid; +$$ LANGUAGE sql; + +INSERT INTO pg_catalog.pg_dist_rebalance_strategy( + name, + default_strategy, + shard_cost_function, + node_capacity_function, + shard_allowed_on_node_function, + default_threshold, + minimum_threshold, + improvement_threshold +) +VALUES ( + 'test_isolate_placement', + false, + 'citus_shard_cost_1', + 'citus_node_capacity_1', + 'isolate_placement.test_shard_allowed_on_node', + 0, + 0, + 0 +); + +SET client_min_messages TO WARNING; +SELECT rebalance_table_shards(rebalance_strategy := 'test_isolate_placement', shard_transfer_mode=>'block_writes'); +SET client_min_messages TO NOTICE; + +-- test_shard_allowed_on_node() didn't cause rebalance_table_shards() to fail. +-- +-- Right now single_shard_1 & single_shard_3 are placed on the same node. And +-- due to order we follow when assigning nodes to placement groups that need an +-- isolated node, we will try placing single_shard_1 to the node where it is +-- currently placed, and then we will try placing single_shard_3 to some other +-- node (as its current node is already assigned to single_shard_1), not to the +-- one we disallowed in test_shard_allowed_on_node(). +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_1', 1); +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_3', 1); + +DROP TABLE single_shard_3_shardid_nodeid; +DELETE FROM pg_catalog.pg_dist_rebalance_strategy WHERE name='test_isolate_placement'; + +DROP TABLE single_shard_1, single_shard_3; + +CREATE TABLE single_shard_1(a int); +SELECT create_distributed_table('single_shard_1', null, colocate_with=>'none'); + +CREATE TABLE single_shard_2(a int); +SELECT create_distributed_table('single_shard_2', null, colocate_with=>'none'); + +-- this would be placed on the same node as single_shard_1 +CREATE TABLE single_shard_3(a int); +SELECT create_distributed_table('single_shard_3', null, colocate_with=>'none'); + +DROP TABLE single_shard_2; + +SELECT shardid, nodeid INTO single_shard_3_shardid_nodeid +FROM pg_dist_shard JOIN pg_dist_placement USING (shardid) JOIN pg_dist_node USING (groupid) +WHERE logicalrelid = 'isolate_placement.single_shard_3'::regclass AND noderole = 'primary'; + +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_1'::regclass); +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_3'::regclass); + +-- Same test above but this time we tell rebalancer that single_shard_3 cannot be placed +-- on any node except the one where it is currently placed. +CREATE OR REPLACE FUNCTION test_shard_allowed_on_node(p_shardid bigint, p_nodeid int) + RETURNS boolean AS +$$ + SELECT + CASE + WHEN (p_shardid = shardid and p_nodeid != nodeid) THEN false + ELSE true + END + FROM single_shard_3_shardid_nodeid; +$$ LANGUAGE sql; + +INSERT INTO pg_catalog.pg_dist_rebalance_strategy( + name, + default_strategy, + shard_cost_function, + node_capacity_function, + shard_allowed_on_node_function, + default_threshold, + minimum_threshold, + improvement_threshold +) +VALUES ( + 'test_isolate_placement', + false, + 'citus_shard_cost_1', + 'citus_node_capacity_1', + 'isolate_placement.test_shard_allowed_on_node', + 0, + 0, + 0 +); + +SET client_min_messages TO ERROR; +SELECT rebalance_table_shards(rebalance_strategy := 'test_isolate_placement', shard_transfer_mode=>'block_writes'); +SET client_min_messages TO NOTICE; + +-- This time, test_shard_allowed_on_node() caused rebalance_table_shards() to +-- fail. +-- +-- Right now single_shard_1 & single_shard_3 are placed on the same node. And +-- due to order we follow when assigning nodes to placement groups that need an +-- isolated node, we will try placing single_shard_1 to the node where it is +-- currently placed, and then we will try placing single_shard_3 to some other +-- node (as its current node is already assigned to single_shard_1). However, +-- test_shard_allowed_on_node() doesn't allow that. +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_1', 1) = false; +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_3', 1) = false; + +DROP TABLE single_shard_3_shardid_nodeid; +DELETE FROM pg_catalog.pg_dist_rebalance_strategy WHERE name='test_isolate_placement'; + +DROP TABLE single_shard_1, single_shard_3; + +CREATE TABLE single_shard_1(a int); +SELECT create_distributed_table('single_shard_1', null, colocate_with=>'none'); + +CREATE TABLE single_shard_2(a int); +SELECT create_distributed_table('single_shard_2', null, colocate_with=>'none'); + +-- this would be placed on the same node as single_shard_1 +CREATE TABLE single_shard_3(a int); +SELECT create_distributed_table('single_shard_3', null, colocate_with=>'none'); + +DROP TABLE single_shard_2; + +SELECT shardid, nodeid INTO single_shard_1_shardid_nodeid +FROM pg_dist_shard JOIN pg_dist_placement USING (shardid) JOIN pg_dist_node USING (groupid) +WHERE logicalrelid = 'isolate_placement.single_shard_1'::regclass AND noderole = 'primary'; + +SELECT shardid, nodeid INTO single_shard_3_shardid_nodeid +FROM pg_dist_shard JOIN pg_dist_placement USING (shardid) JOIN pg_dist_node USING (groupid) +WHERE logicalrelid = 'isolate_placement.single_shard_3'::regclass AND noderole = 'primary'; + +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_1'::regclass); +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_3'::regclass); + +-- Tell rebalancer that single_shard_1 cannot be placed on the node where it is currently placed +-- and that single_shard_3 cannot be placed on any node except the one where it is currently placed. +CREATE OR REPLACE FUNCTION test_shard_allowed_on_node(p_shardid bigint, p_nodeid int) + RETURNS boolean AS +$$ + SELECT + ( + SELECT + CASE + WHEN (p_shardid = shardid and p_nodeid = nodeid) THEN false + ELSE true + END + FROM single_shard_1_shardid_nodeid + ) AND + ( + SELECT + CASE + WHEN (p_shardid = shardid and p_nodeid != nodeid) THEN false + ELSE true + END + FROM single_shard_3_shardid_nodeid + ) +$$ LANGUAGE sql; + +INSERT INTO pg_catalog.pg_dist_rebalance_strategy( + name, + default_strategy, + shard_cost_function, + node_capacity_function, + shard_allowed_on_node_function, + default_threshold, + minimum_threshold, + improvement_threshold +) +VALUES ( + 'test_isolate_placement', + false, + 'citus_shard_cost_1', + 'citus_node_capacity_1', + 'isolate_placement.test_shard_allowed_on_node', + 0, + 0, + 0 +); + +SET client_min_messages TO WARNING; +SELECT rebalance_table_shards(rebalance_strategy := 'test_isolate_placement', shard_transfer_mode=>'block_writes'); +SET client_min_messages TO NOTICE; + +-- This time, test_shard_allowed_on_node() didn't cause rebalance_table_shards() to +-- fail. +-- +-- Right now single_shard_1 & single_shard_3 are placed on the same node. And +-- due to order we follow when assigning nodes to placement groups that need an +-- isolated node, we will try placing single_shard_1 to the node where it is +-- currently placed but this is not possible due to test_shard_allowed_on_node(). +-- But this is not a problem because we will take the specified rebalancer strategy +-- into the account when assigning nodes to placements that need separate nodes and +-- will try to place it to a different node. Then we will try placing single_shard_3 +-- to the node where it is currently placed, and this is ok. +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_1', 1) = true; +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_3', 1) = true; + +DROP TABLE single_shard_1_shardid_nodeid, single_shard_3_shardid_nodeid; +DELETE FROM pg_catalog.pg_dist_rebalance_strategy WHERE name='test_isolate_placement'; + +DROP TABLE single_shard_1, single_shard_3; + +SELECT citus_set_node_property('localhost', :master_port, 'shouldhaveshards', true); + +CREATE TABLE dist_1(a int); +SELECT create_distributed_table('dist_1', 'a', shard_count=>4); + +SELECT shardids[1] AS shardgroup_1_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 1 \gset + +SELECT citus_shard_property_set(:shardgroup_1_shardid, anti_affinity=>true); + +CREATE TABLE single_shard_1(a int); +SELECT create_distributed_table('single_shard_1', null, colocate_with=>'none'); + +SELECT citus_shard_property_set(shardid, anti_affinity=>true) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_1'::regclass); + +SET client_min_messages TO WARNING; +SELECT rebalance_table_shards(shard_transfer_mode=>'block_writes'); +SET client_min_messages TO NOTICE; + +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_1', 1) = true; +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.dist_1', 1) = true; + +SET client_min_messages TO WARNING; +SELECT rebalance_table_shards('isolate_placement.dist_1', shard_transfer_mode=>'block_writes'); +SET client_min_messages TO NOTICE; + +-- Make sure that calling the rebalancer specifically for dist_1 doesn't +-- break the placement separation rules. +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.single_shard_1', 1) = true; +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.dist_1', 1) = true; + +DROP TABLE dist_1, single_shard_1; +SELECT citus_set_node_property('localhost', :master_port, 'shouldhaveshards', false); + +CREATE TABLE dist_1(a int); +SELECT create_distributed_table('dist_1', 'a', shard_count=>4); + +SELECT shardids[1] AS shardgroup_1_shardid +FROM public.get_enumerated_shard_groups('isolate_placement.dist_1') +WHERE shardgroupindex = 1 \gset + +CREATE TABLE single_shard_1(a int); +SELECT create_distributed_table('single_shard_1', null, colocate_with=>'none'); + +-- idempotantly move shard of single_shard_1 to the node that contains shard-group 1 of dist_1 +WITH + shardid_and_source_node AS ( + SELECT shardid, nodeid + FROM pg_dist_shard + JOIN pg_dist_placement USING (shardid) + JOIN pg_dist_node USING (groupid) + WHERE logicalrelid = 'isolate_placement.single_shard_1'::regclass + ), + target_node AS ( + SELECT nodeid + FROM pg_dist_shard + JOIN pg_dist_placement USING (shardid) + JOIN pg_dist_node USING (groupid) + WHERE shardid = :shardgroup_1_shardid + ) +SELECT COUNT( + citus_move_shard_placement( + shardid_and_source_node.shardid, + shardid_and_source_node.nodeid, + target_node.nodeid, + 'block_writes' + ) +) >= 0 +FROM shardid_and_source_node, target_node +WHERE shardid_and_source_node.nodeid != target_node.nodeid; + +SELECT citus_shard_property_set(:shardgroup_1_shardid, anti_affinity=>true); + +SET client_min_messages TO ERROR; +SELECT rebalance_table_shards('isolate_placement.dist_1', shard_transfer_mode=>'block_writes'); +SET client_min_messages TO NOTICE; + +-- Make sure that calling the rebalancer specifically for dist_1 enforces +-- placement separation rules too. +SELECT public.verify_placements_in_shard_group_isolated('isolate_placement.dist_1', 1) = true; + +DROP TABLE dist_1, single_shard_1; + +SET client_min_messages TO WARNING; +DROP SCHEMA isolate_placement CASCADE; +DROP FUNCTION public.verify_placements_in_shard_group_isolated(text, bigint); + +SELECT citus_remove_node('localhost', :master_port); diff --git a/src/test/regress/sql/metadata_sync_helpers.sql b/src/test/regress/sql/metadata_sync_helpers.sql index dae331d258b..99e36dcfc83 100644 --- a/src/test/regress/sql/metadata_sync_helpers.sql +++ b/src/test/regress/sql/metadata_sync_helpers.sql @@ -276,7 +276,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; \set VERBOSITY terse WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('super_user_table'::regclass, 1420000::bigint, 't'::"char", '-2147483648'::text, '-1610612737'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; ROLLBACK; -- the user is only allowed to add a shard for add a table which is in pg_dist_partition @@ -286,7 +286,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; \set VERBOSITY terse WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '-2147483648'::text, '-1610612737'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; ROLLBACK; -- ok, now add the table to the pg_dist_partition @@ -312,7 +312,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; \set VERBOSITY terse WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('test_2'::regclass, -1, 't'::"char", '-2147483648'::text, '-1610612737'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; ROLLBACK; -- invalid storage types are not allowed @@ -322,7 +322,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; \set VERBOSITY terse WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('test_2'::regclass, 1420000, 'X'::"char", '-2147483648'::text, '-1610612737'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; ROLLBACK; -- NULL shard ranges are not allowed for hash distributed tables @@ -332,7 +332,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; \set VERBOSITY terse WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('test_2'::regclass, 1420000, 't'::"char", NULL, '-1610612737'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; ROLLBACK; -- non-integer shard ranges are not allowed @@ -342,7 +342,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; \set VERBOSITY terse WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", 'non-int'::text, '-1610612737'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; ROLLBACK; -- shardMinValue should be smaller than shardMaxValue @@ -352,7 +352,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; \set VERBOSITY terse WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '-1610612737'::text, '-2147483648'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; ROLLBACK; -- we do not allow overlapping shards for the same table @@ -364,7 +364,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '10'::text, '20'::text), ('test_2'::regclass, 1420001::bigint, 't'::"char", '20'::text, '30'::text), ('test_2'::regclass, 1420002::bigint, 't'::"char", '10'::text, '50'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; ROLLBACK; -- Now let's check valid pg_dist_object updates @@ -482,7 +482,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '10'::text, '20'::text), ('test_2'::regclass, 1420001::bigint, 't'::"char", '20'::text, '30'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; ROLLBACK; -- we do not allow NULL shardMinMax values @@ -494,12 +494,12 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; \set VERBOSITY terse WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '10'::text, '20'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; -- manually ingest NULL values, otherwise not likely unless metadata is corrupted UPDATE pg_dist_shard SET shardminvalue = NULL WHERE shardid = 1420000; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('test_2'::regclass, 1420001::bigint, 't'::"char", '20'::text, '30'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; ROLLBACK; \c - metadata_sync_helper_role - :worker_1_port @@ -518,7 +518,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; ('test_2'::regclass, 1420004::bigint, 't'::"char", '51'::text, '60'::text), ('test_2'::regclass, 1420005::bigint, 't'::"char", '61'::text, '70'::text), ('test_3'::regclass, 1420008::bigint, 't'::"char", '11'::text, '20'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; COMMIT; -- we cannot mark these two tables colocated because they are not colocated @@ -539,7 +539,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; ('test_3'::regclass, 1420011::bigint, 't'::"char", '41'::text, '50'::text), ('test_3'::regclass, 1420012::bigint, 't'::"char", '51'::text, '60'::text), ('test_3'::regclass, 1420013::bigint, 't'::"char", '61'::text, '70'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; COMMIT; -- shardMin/MaxValues should be NULL for reference tables @@ -549,7 +549,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; \set VERBOSITY terse WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('test_ref'::regclass, 1420003::bigint, 't'::"char", '-1610612737'::text, NULL)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; ROLLBACK; -- reference tables cannot have multiple shards @@ -560,7 +560,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('test_ref'::regclass, 1420006::bigint, 't'::"char", NULL, NULL), ('test_ref'::regclass, 1420007::bigint, 't'::"char", NULL, NULL)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; ROLLBACK; -- finally, add a shard for reference tables @@ -570,7 +570,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; \set VERBOSITY terse WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('test_ref'::regclass, 1420006::bigint, 't'::"char", NULL, NULL)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; COMMIT; \c - postgres - :worker_1_port @@ -583,7 +583,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; \set VERBOSITY terse WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('super_user_table'::regclass, 1420007::bigint, 't'::"char", '11'::text, '20'::text)) - SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; + SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data; COMMIT; \c - metadata_sync_helper_role - :worker_1_port diff --git a/src/test/regress/sql/multi_metadata_attributes.sql b/src/test/regress/sql/multi_metadata_attributes.sql index 1a592d858ef..e2723574e47 100644 --- a/src/test/regress/sql/multi_metadata_attributes.sql +++ b/src/test/regress/sql/multi_metadata_attributes.sql @@ -11,5 +11,6 @@ WHERE atthasmissing AND attrelid NOT IN ('pg_dist_node'::regclass, 'pg_dist_rebalance_strategy'::regclass, 'pg_dist_partition'::regclass, 'pg_dist_object'::regclass, - 'pg_dist_background_task'::regclass) + 'pg_dist_background_task'::regclass, + 'pg_dist_shard'::regclass) ORDER BY attrelid, attname; diff --git a/src/test/regress/sql/multi_test_helpers.sql b/src/test/regress/sql/multi_test_helpers.sql index e67b782a524..340784fd140 100644 --- a/src/test/regress/sql/multi_test_helpers.sql +++ b/src/test/regress/sql/multi_test_helpers.sql @@ -652,3 +652,94 @@ BEGIN JOIN pg_dist_node USING (nodeid); END; $func$ LANGUAGE plpgsql; + +-- Takes a table name and returns an array of colocated shards +-- --by enumerating them based on shardminvalue-- for each shard +-- of given distributed table (including colocated shards). +CREATE OR REPLACE FUNCTION get_enumerated_shard_groups( + qualified_table_name text) +RETURNS TABLE ( + shardids bigint[], + shardgroupindex bigint +) +AS $func$ + BEGIN + RETURN QUERY + SELECT array_agg(shardid ORDER BY shardid) AS shardids, + ROW_NUMBER() OVER (ORDER BY shardminvalue) AS shardgroupindex + FROM pg_dist_shard + JOIN pg_dist_partition USING(logicalrelid) + WHERE colocationid = (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = qualified_table_name::regclass) + GROUP BY shardminvalue; + END; +$func$ LANGUAGE plpgsql; + + +-- Takes a table name and returns a json object for each shard group that +-- contains a shard whose placements need separate nodes. +-- +-- This does not only return the shards of input relation but also considers +-- all colocated relations. +-- +-- An example output is as follows: +-- +-- [ +-- {"10": [{"dist_1": true},{"dist_2": false}]}, +-- {"15": [{"dist_1": false},{"dist_3": true}]} +-- ] +-- +-- It only returned shard groups 10 and 15 because they are the only shard groups +-- that contain at least one shard whose placements need an isolation. +-- +-- (Innermost) Boolean values represent needsseparatenode value for given +-- shard. For example, +-- +-- {"15": [{"dist_1": false},{"dist_3": true}]} +-- +-- means that the placements of dist_3 within shard group 15 needs +-- to be isolated. +CREATE OR REPLACE FUNCTION get_colocated_shards_needisolatednode( + qualified_table_name text) +RETURNS SETOF jsonb AS $func$ + BEGIN + RETURN QUERY + SELECT + COALESCE( + jsonb_agg(jsonb_build_object(shardgroupindex, needsseparatenodejson) ORDER BY shardgroupindex), + '{}'::jsonb + ) AS result + FROM ( + SELECT shardgroupindex, + jsonb_agg(jsonb_build_object(logicalrelid, needsseparatenode) ORDER BY logicalrelid::text) AS needsseparatenodejson + FROM ( + SELECT shardgroupindex, + logicalrelid, + needsseparatenode + FROM public.get_enumerated_shard_groups(qualified_table_name) AS shardgroups + JOIN pg_dist_shard + ON shardid = ANY(shardids) + ) q1 + GROUP BY shardgroupindex + ) q2 + WHERE needsseparatenodejson::text LIKE '%true%'; + END; +$func$ LANGUAGE plpgsql; + +-- Returns true if all placement groups within given shard group are isolated. +CREATE OR REPLACE FUNCTION verify_placements_in_shard_group_isolated( + qualified_table_name text, + shard_group_index bigint) +RETURNS boolean +AS $func$ +DECLARE + v_result boolean; + BEGIN + SELECT bool_and(has_separate_node) INTO v_result + FROM citus_shards + JOIN ( + SELECT shardids FROM public.get_enumerated_shard_groups(qualified_table_name) WHERE shardgroupindex = shard_group_index + ) q + ON (shardid = ANY(q.shardids)); + RETURN v_result; + END; +$func$ LANGUAGE plpgsql; diff --git a/src/test/regress/sql/upgrade_isolate_placement_after.sql b/src/test/regress/sql/upgrade_isolate_placement_after.sql new file mode 100644 index 00000000000..5ad9d3f40ba --- /dev/null +++ b/src/test/regress/sql/upgrade_isolate_placement_after.sql @@ -0,0 +1,16 @@ +-- upgrade_columnar_before renames public to citus_schema and recreates public +-- schema. But this file depends on get_colocated_shards_needisolatednode() +-- function and get_colocated_shards_needisolatednode() depends on another +-- function --get_enumerated_shard_groups()-- that is presumably created earlier +-- within the original public schema, so we temporarily rename citus_schema to +-- public here; and revert those changes at the end of this file. +ALTER SCHEMA public RENAME TO old_public; +ALTER SCHEMA citus_schema RENAME TO public; + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('upgrade_isolate_placement_before.table_with_isolated_placements') +$$) +ORDER BY nodeid; + +ALTER SCHEMA public RENAME TO citus_schema; +ALTER SCHEMA old_public RENAME TO public; diff --git a/src/test/regress/sql/upgrade_isolate_placement_before.sql b/src/test/regress/sql/upgrade_isolate_placement_before.sql new file mode 100644 index 00000000000..83a8c09be67 --- /dev/null +++ b/src/test/regress/sql/upgrade_isolate_placement_before.sql @@ -0,0 +1,21 @@ +SET client_min_messages TO WARNING; +DROP SCHEMA IF EXISTS upgrade_isolate_placement_before CASCADE; + +CREATE SCHEMA upgrade_isolate_placement_before; +SET search_path TO upgrade_isolate_placement_before; + +SET client_min_messages TO NOTICE; + +CREATE TABLE table_with_isolated_placements (a int, b int); +SELECT create_distributed_table('table_with_isolated_placements', 'a', colocate_with=>'none'); + +SELECT shardids[1] AS shardgroup_5_shardid +FROM public.get_enumerated_shard_groups('upgrade_isolate_placement_before.table_with_isolated_placements') +WHERE shardgroupindex = 5 \gset + +SELECT citus_shard_property_set(:shardgroup_5_shardid, anti_affinity=>true); + +SELECT result FROM run_command_on_all_nodes($$ + SELECT * FROM public.get_colocated_shards_needisolatednode('upgrade_isolate_placement_before.table_with_isolated_placements') +$$) +ORDER BY nodeid;