Allow isolating shard placement groups on individual nodes

tenant-schema-isolation-complete-view
Onur Tirtir 2023-09-08 19:36:04 +03:00
parent e9035f6d32
commit 518227de38
61 changed files with 4215 additions and 220 deletions

View File

@ -134,6 +134,7 @@ static char DecideDistTableReplicationModel(char distributionMethod,
static List * HashSplitPointsForShardList(List *shardList);
static List * HashSplitPointsForShardCount(int shardCount);
static List * WorkerNodesForShardList(List *shardList);
static List * NeedsIsolatedNodeForShardList(List *shardList);
static List * RoundRobinWorkerNodeList(List *workerNodeList, int listLength);
static CitusTableParams DecideCitusTableParams(CitusTableType tableType,
DistributedTableParams *
@ -571,7 +572,7 @@ CreateDistributedTableConcurrently(Oid relationId, char *distributionColumnName,
colocatedTableId = ColocatedTableId(colocationId);
}
List *workerNodeList = DistributedTablePlacementNodeList(NoLock);
List *workerNodeList = NewDistributedTablePlacementNodeList(NoLock);
if (workerNodeList == NIL)
{
ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
@ -581,6 +582,8 @@ CreateDistributedTableConcurrently(Oid relationId, char *distributionColumnName,
List *workersForPlacementList;
List *shardSplitPointsList;
List *needsIsolatedNodeForPlacementList;
if (colocatedTableId != InvalidOid)
{
@ -595,6 +598,11 @@ CreateDistributedTableConcurrently(Oid relationId, char *distributionColumnName,
* Find the node IDs of the shard placements.
*/
workersForPlacementList = WorkerNodesForShardList(colocatedShardList);
/*
* Inherit needsisolatednode from the colocated shard placements.
*/
needsIsolatedNodeForPlacementList = NeedsIsolatedNodeForShardList(colocatedShardList);
}
else
{
@ -607,6 +615,12 @@ CreateDistributedTableConcurrently(Oid relationId, char *distributionColumnName,
* Place shards in a round-robin fashion across all data nodes.
*/
workersForPlacementList = RoundRobinWorkerNodeList(workerNodeList, shardCount);
/*
* For a new colocation group, needsisolatednode is set to false for
* all shard placements.
*/
needsIsolatedNodeForPlacementList = GenerateListFromIntElement(false, shardCount);
}
/*
@ -645,6 +659,7 @@ CreateDistributedTableConcurrently(Oid relationId, char *distributionColumnName,
shardToSplit->shardId,
shardSplitPointsList,
workersForPlacementList,
needsIsolatedNodeForPlacementList,
distributionColumnOverrides,
sourceColocatedShardIntervalList,
colocationId
@ -897,6 +912,29 @@ WorkerNodesForShardList(List *shardList)
}
/*
* NeedsIsolatedNodeForShardList returns a list of node booleans reflecting whether
* each shard in the given list needs an isolated node.
*
* Each shard within given list is assumed to have a single placement.
*/
static List *
NeedsIsolatedNodeForShardList(List *shardList)
{
List *needsIsolatedNodeList = NIL;
ShardInterval *shardInterval = NULL;
foreach_ptr(shardInterval, shardList)
{
bool missingOk = false;
ShardPlacement *shardPlacement = ActiveShardPlacement(shardInterval->shardId, missingOk);
needsIsolatedNodeList = lappend_int(needsIsolatedNodeList, shardPlacement->needsIsolatedNode);
}
return needsIsolatedNodeList;
}
/*
* RoundRobinWorkerNodeList round robins over the workers in the worker node list
* and adds node ids to a list of length listLength.

View File

@ -179,6 +179,7 @@ PG_FUNCTION_INFO_V1(citus_internal_delete_colocation_metadata);
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_shard_group_set_needsisolatednode);
static bool got_SIGTERM = false;
@ -1202,7 +1203,7 @@ ShardListInsertCommand(List *shardIntervalList)
StringInfo insertPlacementCommand = makeStringInfo();
appendStringInfo(insertPlacementCommand,
"WITH placement_data(shardid, "
"shardlength, groupid, placementid) AS (VALUES ");
"shardlength, groupid, placementid, needsisolatednode) AS (VALUES ");
ShardInterval *shardInterval = NULL;
bool firstPlacementProcessed = false;
@ -1225,11 +1226,12 @@ ShardListInsertCommand(List *shardIntervalList)
firstPlacementProcessed = true;
appendStringInfo(insertPlacementCommand,
"(%ld, %ld, %d, %ld)",
"(%ld, %ld, %d, %ld, %s)",
shardId,
placement->shardLength,
placement->groupId,
placement->placementId);
placement->placementId,
placement->needsIsolatedNode ? "true" : "false");
}
}
@ -1237,7 +1239,7 @@ ShardListInsertCommand(List *shardIntervalList)
appendStringInfo(insertPlacementCommand,
"SELECT citus_internal_add_placement_metadata("
"shardid, shardlength, groupid, placementid) "
"shardid, shardlength, groupid, placementid, needsisolatednode) "
"FROM placement_data;");
/* now add shards to insertShardCommand */
@ -1415,12 +1417,13 @@ ColocationIdUpdateCommand(Oid relationId, uint32 colocationId)
*/
char *
PlacementUpsertCommand(uint64 shardId, uint64 placementId,
uint64 shardLength, int32 groupId)
uint64 shardLength, int32 groupId,
bool needsIsolatedNode)
{
StringInfo command = makeStringInfo();
appendStringInfo(command, UPSERT_PLACEMENT, shardId, shardLength,
groupId, placementId);
groupId, placementId, needsIsolatedNode ? "true" : "false");
return command->data;
}
@ -3444,16 +3447,18 @@ citus_internal_add_placement_metadata(PG_FUNCTION_ARGS)
int64 shardLength = PG_GETARG_INT64(1);
int32 groupId = PG_GETARG_INT32(2);
int64 placementId = PG_GETARG_INT64(3);
bool needsIsolatedNode = PG_GETARG_BOOL(4);
citus_internal_add_placement_metadata_internal(shardId, shardLength,
groupId, placementId);
groupId, placementId,
needsIsolatedNode);
PG_RETURN_VOID();
}
/*
* citus_internal_add_placement_metadata is an internal UDF to
* citus_internal_delete_placement_metadata is an internal UDF to
* delete a row from pg_dist_placement.
*/
Datum
@ -3483,12 +3488,15 @@ citus_internal_add_placement_metadata_legacy(PG_FUNCTION_ARGS)
CheckCitusVersion(ERROR);
int64 shardId = PG_GETARG_INT64(0);
int64 shardLength = PG_GETARG_INT64(2);
int32 groupId = PG_GETARG_INT32(3);
int64 placementId = PG_GETARG_INT64(4);
int64 shardLength = PG_GETARG_INT64(1);
int32 groupId = PG_GETARG_INT32(2);
int64 placementId = PG_GETARG_INT64(3);
bool needsIsolatedNode = false;
citus_internal_add_placement_metadata_internal(shardId, shardLength,
groupId, placementId);
groupId, placementId,
needsIsolatedNode);
PG_RETURN_VOID();
}
@ -3499,7 +3507,8 @@ citus_internal_add_placement_metadata_legacy(PG_FUNCTION_ARGS)
*/
void
citus_internal_add_placement_metadata_internal(int64 shardId, int64 shardLength,
int32 groupId, int64 placementId)
int32 groupId, int64 placementId,
bool needsIsolatedNode)
{
bool missingOk = false;
Oid relationId = LookupShardRelationFromCatalog(shardId, missingOk);
@ -3524,7 +3533,8 @@ citus_internal_add_placement_metadata_internal(int64 shardId, int64 shardLength,
shardLength, groupId);
}
InsertShardPlacementRow(shardId, placementId, shardLength, groupId);
InsertShardPlacementRow(shardId, placementId, shardLength, groupId,
needsIsolatedNode);
}
@ -3895,6 +3905,33 @@ citus_internal_update_none_dist_table_metadata(PG_FUNCTION_ARGS)
}
/*
* citus_internal_shard_group_set_needsisolatednode is an internal UDF to
* set needsisolatednode flag for all the placements within the shard group
* that given shard belongs to.
*/
Datum
citus_internal_shard_group_set_needsisolatednode(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
PG_ENSURE_ARGNOTNULL(0, "shard_id");
uint64 shardId = PG_GETARG_INT64(0);
PG_ENSURE_ARGNOTNULL(1, "enabled");
bool enabled = PG_GETARG_BOOL(1);
if (!ShouldSkipMetadataChecks())
{
EnsureCoordinatorInitiatedOperation();
}
ShardGroupSetNeedsIsolatedNode(shardId, enabled);
PG_RETURN_VOID();
}
/*
* SyncNewColocationGroup synchronizes a new pg_dist_colocation entry to a worker.
*/
@ -4094,18 +4131,36 @@ UpdateNoneDistTableMetadataCommand(Oid relationId, char replicationModel,
}
/*
* ShardGroupSetNeedsIsolatedNodeCommand returns a command to call
* citus_internal_shard_group_set_needsisolatednode().
*/
char *
ShardGroupSetNeedsIsolatedNodeCommand(uint64 shardId, bool enabled)
{
StringInfo command = makeStringInfo();
appendStringInfo(command,
"SELECT pg_catalog.citus_internal_shard_group_set_needsisolatednode(%lu, %s)",
shardId, enabled ? "true" : "false");
return command->data;
}
/*
* AddPlacementMetadataCommand returns a command to call
* citus_internal_add_placement_metadata().
*/
char *
AddPlacementMetadataCommand(uint64 shardId, uint64 placementId,
uint64 shardLength, int32 groupId)
uint64 shardLength, int32 groupId,
bool needsIsolatedNode)
{
StringInfo command = makeStringInfo();
appendStringInfo(command,
"SELECT citus_internal_add_placement_metadata(%ld, %ld, %d, %ld)",
shardId, shardLength, groupId, placementId);
"SELECT citus_internal_add_placement_metadata(%ld, %ld, %d, %ld, %s)",
shardId, shardLength, groupId, placementId,
needsIsolatedNode ? "true" : "false");
return command->data;
}

View File

@ -35,6 +35,7 @@
#include "catalog/pg_type.h"
#include "commands/extension.h"
#include "commands/sequence.h"
#include "distributed/argutils.h"
#include "distributed/background_jobs.h"
#include "distributed/colocation_utils.h"
#include "distributed/connection_management.h"
@ -113,6 +114,9 @@ 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 ErrorIfShardIsolationNotPossible(uint64 shardId);
static void ShardGroupSetNeedsIsolatedNodeGlobally(uint64 shardId, bool enabled);
static void ShardPlacementSetNeedsIsolatedNode(uint64 placementId, bool enabled);
static BackgroundTask * DeformBackgroundTaskHeapTuple(TupleDesc tupleDescriptor,
HeapTuple taskTuple);
@ -135,6 +139,8 @@ 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_set_isolated);
PG_FUNCTION_INFO_V1(citus_shard_unset_isolated);
/*
@ -354,6 +360,170 @@ citus_relation_size(PG_FUNCTION_ARGS)
}
/*
* citus_shard_set_isolated sets the needsisolatednode flag to true for all
* the placements within the shard group that given shard belongs to.
*/
Datum
citus_shard_set_isolated(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
EnsureCoordinator();
PG_ENSURE_ARGNOTNULL(0, "shard_id");
uint64 shardId = PG_GETARG_INT64(0);
ErrorIfShardIsolationNotPossible(shardId);
bool enabled = true;
ShardGroupSetNeedsIsolatedNodeGlobally(shardId, enabled);
PG_RETURN_VOID();
}
/*
* citus_shard_unset_isolated sets the needsisolatednode flag to false for all
* the placements within the shard group that given shard belongs to.
*/
Datum
citus_shard_unset_isolated(PG_FUNCTION_ARGS)
{
PG_ENSURE_ARGNOTNULL(0, "shard_id");
uint64 shardId = PG_GETARG_INT64(0);
ErrorIfShardIsolationNotPossible(shardId);
bool enabled = false;
ShardGroupSetNeedsIsolatedNodeGlobally(shardId, enabled);
PG_RETURN_VOID();
}
/*
* ErrorIfShardIsolationNotPossible throws an error if shard isolation is not
* possible for the given shard.
*/
static void
ErrorIfShardIsolationNotPossible(uint64 shardId)
{
if (!ShardExists(shardId))
{
ereport(ERROR, (errmsg("shard %lu does not exist", shardId)));
}
Oid distributedRelationId = RelationIdForShard(shardId);
if (!IsCitusTableType(distributedRelationId, HASH_DISTRIBUTED) &&
!IsCitusTableType(distributedRelationId, SINGLE_SHARD_DISTRIBUTED))
{
ereport(ERROR, (errmsg("shard isolation is only supported for hash "
"distributed tables")));
}
}
/*
* ShardGroupSetNeedsIsolatedNodeGlobally calls ShardGroupSetNeedsIsolatedNode
* on all nodes.
*/
static void
ShardGroupSetNeedsIsolatedNodeGlobally(uint64 shardId, bool enabled)
{
ShardGroupSetNeedsIsolatedNode(shardId, enabled);
char *metadataCommand =
ShardGroupSetNeedsIsolatedNodeCommand(shardId, enabled);
SendCommandToWorkersWithMetadata(metadataCommand);
}
/*
* ShardGroupSetNeedsIsolatedNode sets the needsisolatednode flag to desired
* value for all the placements within the shard group that given shard belongs
* to.
*/
void
ShardGroupSetNeedsIsolatedNode(uint64 shardId, bool enabled)
{
ShardInterval *shardInterval = LoadShardInterval(shardId);
List *colocatedShardIntervalList = ColocatedShardIntervalList(shardInterval);
ShardInterval *colocatedShardInterval = NULL;
foreach_ptr(colocatedShardInterval, colocatedShardIntervalList)
{
List *activeShardPlacementList = ActiveShardPlacementList(
colocatedShardInterval->shardId);
ShardPlacement *activeShardPlacement = NULL;
foreach_ptr(activeShardPlacement, activeShardPlacementList)
{
ShardPlacementSetNeedsIsolatedNode(activeShardPlacement->placementId,
enabled);
}
}
}
/*
* ShardPlacementSetNeedsIsolatedNode sets the needsisolatednode flag to desired
* value for the given placement.
*/
static void
ShardPlacementSetNeedsIsolatedNode(uint64 placementId, bool enabled)
{
Relation pgDistPlacement = table_open(DistPlacementRelationId(), RowExclusiveLock);
ScanKeyData scanKey[1];
int scanKeyCount = 1;
ScanKeyInit(&scanKey[0], Anum_pg_dist_placement_placementid,
BTEqualStrategyNumber, F_INT8EQ, Int64GetDatum(placementId));
bool indexOK = true;
Oid indexId = DistPlacementPlacementidIndexId();
SysScanDesc scanDescriptor = systable_beginscan(pgDistPlacement,
indexId, indexOK, NULL,
scanKeyCount, scanKey);
HeapTuple heapTuple = systable_getnext(scanDescriptor);
if (!HeapTupleIsValid(heapTuple))
{
ereport(ERROR, (errmsg("could not find valid entry for shard placement "
UINT64_FORMAT,
placementId)));
}
Datum values[Natts_pg_dist_placement];
bool isnull[Natts_pg_dist_placement];
bool replace[Natts_pg_dist_placement];
memset(values, 0, sizeof(values));
memset(isnull, false, sizeof(isnull));
memset(replace, false, sizeof(replace));
values[Anum_pg_dist_placement_needsisolatednode - 1] = BoolGetDatum(enabled);
isnull[Anum_pg_dist_placement_needsisolatednode - 1] = false;
replace[Anum_pg_dist_placement_needsisolatednode - 1] = true;
TupleDesc tupleDescriptor = RelationGetDescr(pgDistPlacement);
heapTuple = heap_modify_tuple(heapTuple, tupleDescriptor, values, isnull, replace);
CatalogTupleUpdate(pgDistPlacement, &heapTuple->t_self, heapTuple);
bool shardIdIsNull = false;
uint64 shardId = DatumGetInt64(heap_getattr(heapTuple,
Anum_pg_dist_placement_shardid,
tupleDescriptor, &shardIdIsNull));
Assert(!shardIdIsNull);
CitusInvalidateRelcacheByShardId(shardId);
CommandCounterIncrement();
systable_endscan(scanDescriptor);
table_close(pgDistPlacement, NoLock);
}
/*
* SendShardStatisticsQueriesInParallel generates query lists for obtaining shard
* statistics and then sends the commands in parallel by opening connections
@ -1344,6 +1514,90 @@ ShardLength(uint64 shardId)
}
/*
* NodeGroupGetIsolatedShardPlacementGroup returns the shard placement group
* that given node group is used to isolate, if any. Returns NULL if this
* node is not used to a shard placement group.
*/
ShardPlacementGroup *
NodeGroupGetIsolatedShardPlacementGroup(int32 groupId)
{
ShardPlacementGroup *nodeShardPlacementGroup = NULL;
bool shardPlacementGroupNeedsIsolatedNode = false;
bool indexOK = false;
int scanKeyCount = 1;
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, scanKeyCount, scanKey);
HeapTuple heapTuple = systable_getnext(scanDescriptor);
while (HeapTupleIsValid(heapTuple))
{
TupleDesc tupleDescriptor = RelationGetDescr(pgDistPlacement);
GroupShardPlacement *placement =
TupleToGroupShardPlacement(tupleDescriptor, heapTuple);
ShardInterval *shardInterval = LoadShardInterval(placement->shardId);
Oid citusTableId = shardInterval->relationId;
if (!IsCitusTableType(citusTableId, DISTRIBUTED_TABLE))
{
heapTuple = systable_getnext(scanDescriptor);
continue;
}
ShardPlacementGroup *shardPlacementGroup =
GetShardPlacementGroupForPlacement(placement->shardId,
placement->placementId);
if (nodeShardPlacementGroup &&
!ShardPlacementGroupsSame(shardPlacementGroup,
nodeShardPlacementGroup))
{
nodeShardPlacementGroup = NULL;
break;
}
nodeShardPlacementGroup = shardPlacementGroup;
shardPlacementGroupNeedsIsolatedNode = placement->needsIsolatedNode;
heapTuple = systable_getnext(scanDescriptor);
}
systable_endscan(scanDescriptor);
table_close(pgDistPlacement, NoLock);
if (!shardPlacementGroupNeedsIsolatedNode)
{
return NULL;
}
return nodeShardPlacementGroup;
}
/*
* ShardPlacementGroupsSame returns true if two shard placement groups are the same.
*/
bool
ShardPlacementGroupsSame(const ShardPlacementGroup *leftGroup,
const ShardPlacementGroup *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
*/
@ -1376,6 +1630,25 @@ NodeGroupHasShardPlacements(int32 groupId)
}
/*
* GetShardPlacementGroupForPlacement returns ShardPlacementGroup that placement
* with given shardId & placementId belongs to.
*/
ShardPlacementGroup *
GetShardPlacementGroupForPlacement(uint64 shardId, uint64 placementId)
{
ShardPlacement *shardPlacement = LoadShardPlacement(shardId, placementId);
ShardInterval *shardInterval = LoadShardInterval(shardId);
ShardPlacementGroup *placementGroup = palloc(sizeof(ShardPlacementGroup));
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.
@ -1674,9 +1947,10 @@ TupleToGroupShardPlacement(TupleDesc tupleDescriptor, HeapTuple heapTuple)
{
bool isNullArray[Natts_pg_dist_placement];
Datum datumArray[Natts_pg_dist_placement];
memset(datumArray, 0, sizeof(datumArray));
memset(isNullArray, false, sizeof(isNullArray));
if (HeapTupleHeaderGetNatts(heapTuple->t_data) != Natts_pg_dist_placement ||
HeapTupleHasNulls(heapTuple))
if (HeapTupleHasNulls(heapTuple))
{
ereport(ERROR, (errmsg("unexpected null in pg_dist_placement tuple")));
}
@ -1696,6 +1970,8 @@ TupleToGroupShardPlacement(TupleDesc tupleDescriptor, HeapTuple heapTuple)
datumArray[Anum_pg_dist_placement_shardlength - 1]);
shardPlacement->groupId = DatumGetInt32(
datumArray[Anum_pg_dist_placement_groupid - 1]);
shardPlacement->needsIsolatedNode = DatumGetBool(
datumArray[Anum_pg_dist_placement_needsisolatednode - 1]);
return shardPlacement;
}
@ -1800,12 +2076,15 @@ InsertShardRow(Oid relationId, uint64 shardId, char storageType,
*/
ShardPlacement *
InsertShardPlacementRowGlobally(uint64 shardId, uint64 placementId,
uint64 shardLength, int32 groupId)
uint64 shardLength, int32 groupId,
bool needsIsolatedNode)
{
InsertShardPlacementRow(shardId, placementId, shardLength, groupId);
InsertShardPlacementRow(shardId, placementId, shardLength, groupId,
needsIsolatedNode);
char *insertPlacementCommand =
AddPlacementMetadataCommand(shardId, placementId, shardLength, groupId);
AddPlacementMetadataCommand(shardId, placementId, shardLength, groupId,
needsIsolatedNode);
SendCommandToWorkersWithMetadata(insertPlacementCommand);
return LoadShardPlacement(shardId, placementId);
@ -1820,7 +2099,8 @@ InsertShardPlacementRowGlobally(uint64 shardId, uint64 placementId,
*/
uint64
InsertShardPlacementRow(uint64 shardId, uint64 placementId,
uint64 shardLength, int32 groupId)
uint64 shardLength, int32 groupId,
bool needsIsolatedNode)
{
Datum values[Natts_pg_dist_placement];
bool isNulls[Natts_pg_dist_placement];
@ -1838,6 +2118,8 @@ InsertShardPlacementRow(uint64 shardId, uint64 placementId,
values[Anum_pg_dist_placement_shardstate - 1] = Int32GetDatum(1);
values[Anum_pg_dist_placement_shardlength - 1] = Int64GetDatum(shardLength);
values[Anum_pg_dist_placement_groupid - 1] = Int32GetDatum(groupId);
values[Anum_pg_dist_placement_needsisolatednode - 1] =
BoolGetDatum(needsIsolatedNode);
/* open shard placement relation and insert new tuple */
Relation pgDistPlacement = table_open(DistPlacementRelationId(), RowExclusiveLock);
@ -2075,8 +2357,7 @@ DeleteShardPlacementRow(uint64 placementId)
uint64 shardId = heap_getattr(heapTuple, Anum_pg_dist_placement_shardid,
tupleDescriptor, &isNull);
if (HeapTupleHeaderGetNatts(heapTuple->t_data) != Natts_pg_dist_placement ||
HeapTupleHasNulls(heapTuple))
if (HeapTupleHasNulls(heapTuple))
{
ereport(ERROR, (errmsg("unexpected null in pg_dist_placement tuple")));
}

View File

@ -61,8 +61,6 @@
#include "utils/rel.h"
#include "utils/relcache.h"
#define INVALID_GROUP_ID -1
/* default group size */
int GroupSize = 1;

View File

@ -21,6 +21,7 @@
#include "distributed/shardinterval_utils.h"
#include "distributed/coordinator_protocol.h"
#include "distributed/connection_management.h"
#include "distributed/listutils.h"
#include "distributed/remote_commands.h"
#include "distributed/shard_split.h"
#include "distributed/utils/distribution_column_map.h"
@ -53,6 +54,10 @@ citus_split_shard_by_split_points(PG_FUNCTION_ARGS)
Oid shardTransferModeOid = PG_GETARG_OID(3);
SplitMode shardSplitMode = LookupSplitMode(shardTransferModeOid);
/* we don't inherit needsisolatednode for new placements */
List *needsIsolatedNodeForPlacementList =
GenerateListFromIntElement(false, list_length(nodeIdsForPlacementList));
DistributionColumnMap *distributionColumnOverrides = NULL;
List *sourceColocatedShardIntervalList = NIL;
SplitShard(
@ -61,6 +66,7 @@ citus_split_shard_by_split_points(PG_FUNCTION_ARGS)
shardIdToSplit,
shardSplitPointsList,
nodeIdsForPlacementList,
needsIsolatedNodeForPlacementList,
distributionColumnOverrides,
sourceColocatedShardIntervalList,
INVALID_COLOCATION_ID);

View File

@ -145,15 +145,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.")));
}
@ -299,7 +300,8 @@ CreateColocatedShards(Oid targetRelationId, Oid sourceRelationId, bool
InsertShardPlacementRow(*newShardIdPtr,
INVALID_PLACEMENT_ID,
shardSize,
groupId);
groupId,
sourcePlacement->needsIsolatedNode);
}
}
@ -416,7 +418,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 =
@ -459,17 +461,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")));
}

View File

@ -166,6 +166,10 @@ isolate_tenant_to_new_shard(PG_FUNCTION_ARGS)
nodeIdsForPlacementList = lappend_int(nodeIdsForPlacementList, sourceNodeId);
}
/* we don't inherit needsisolatednode for new placements */
List *needsIsolatedNodeForPlacementList =
GenerateListFromIntElement(false, list_length(nodeIdsForPlacementList));
DistributionColumnMap *distributionColumnOverrides = NULL;
List *sourceColocatedShardIntervalList = NIL;
SplitMode splitMode = LookupSplitMode(shardTransferModeOid);
@ -174,6 +178,7 @@ isolate_tenant_to_new_shard(PG_FUNCTION_ARGS)
sourceShard->shardId,
shardSplitPointsList,
nodeIdsForPlacementList,
needsIsolatedNodeForPlacementList,
distributionColumnOverrides,
sourceColocatedShardIntervalList,
INVALID_COLOCATION_ID);

View File

@ -0,0 +1,420 @@
/*-------------------------------------------------------------------------
*
* rebalancer_placement_isolation.c
* Routines to determine which worker node should be used to isolate
* a colocated set of shard placements that needs isolation.
*
* 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_isolation.h"
#include "distributed/shard_rebalancer.h"
struct RebalancerPlacementIsolationContext
{
HTAB *nodePlacementGroupHash;
};
/*
* Entry of the hash table that maps each primary worker node to a shard
* placement group that is determined to be isolated on that node.
*/
typedef struct
{
/* hash key -- group id of the node */
int32 nodeGroupId;
/*
* Whether given node is allowed to have any shards.
*
* This is not just WorkerNode->shouldHaveShards but also takes into account
* whether the node is being drained.
*/
bool shouldHaveShards;
/*
* Whether given node is allowed to isolate any shard placement groups.
*
* This is set only if we're draining a single node because otherwise
* we have the control to isolate shard placement groups on any node.
*
* However if we're draining a single node, we cannot isolate shard
* placement groups on the node that already has some placements because
* we cannot move the existing placements from a node that we're not
* draining to another node when we're draining a single node.
*/
bool allowedToIsolateAnyPlacementGroup;
/*
* Shard placement group that is assigned to this node to be isolated.
*
* NULL if no shard placement group is assigned yet.
*/
ShardPlacementGroup *assignedPlacementGroup;
} NodePlacementGroupHashEntry;
/*
* Routines to prepare a hash table where each entry is of type
* NodePlacementGroupHashEntry.
*/
static void NodePlacementGroupHashInit(HTAB *nodePlacementGroupHash,
List *workerNodeList,
WorkerNode *drainWorkerNode);
static void NodePlacementGroupHashAssignNodes(HTAB *nodePlacementGroupHash,
List *workerNodeList,
List *shardPlacementList);
static bool NodePlacementGroupHashAssignNode(HTAB *nodePlacementGroupHash,
int32 nodeGroupId,
ShardPlacement *shardPlacement);
static NodePlacementGroupHashEntry * NodePlacementGroupHashGetNodeWithGroupId(
HTAB *nodePlacementGroupHash,
int32
nodeGroupId);
/* other helpers */
static int WorkerNodeListGetNodeWithGroupId(List *workerNodeList, int32 nodeGroupId);
/*
* PrepareRebalancerPlacementIsolationContext creates RebalancerPlacementIsolationContext
* that keeps track of which worker nodes are used to isolate which shard placement groups
* that need an isolated node.
*/
RebalancerPlacementIsolationContext *
PrepareRebalancerPlacementIsolationContext(List *activeWorkerNodeList,
List *activeShardPlacementList,
WorkerNode *drainWorkerNode)
{
HTAB *nodePlacementGroupHash =
CreateSimpleHashWithNameAndSize(uint32, NodePlacementGroupHashEntry,
"NodePlacementGroupHash",
list_length(activeWorkerNodeList));
activeWorkerNodeList = SortList(activeWorkerNodeList, CompareWorkerNodes);
activeShardPlacementList = SortList(activeShardPlacementList, CompareShardPlacements);
NodePlacementGroupHashInit(nodePlacementGroupHash, activeWorkerNodeList,
drainWorkerNode);
NodePlacementGroupHashAssignNodes(nodePlacementGroupHash,
activeWorkerNodeList,
activeShardPlacementList);
RebalancerPlacementIsolationContext *context =
palloc(sizeof(RebalancerPlacementIsolationContext));
context->nodePlacementGroupHash = nodePlacementGroupHash;
return context;
}
/*
* NodePlacementGroupHashInit initializes given hash table where each
* entry is of type NodePlacementGroupHashEntry by using given list
* of worker nodes and the worker node that is being drained, if specified.
*/
static void
NodePlacementGroupHashInit(HTAB *nodePlacementGroupHash, List *workerNodeList,
WorkerNode *drainWorkerNode)
{
bool drainSingleNode = drainWorkerNode != NULL;
WorkerNode *workerNode = NULL;
foreach_ptr(workerNode, workerNodeList)
{
NodePlacementGroupHashEntry *nodePlacementGroupHashEntry =
hash_search(nodePlacementGroupHash, &workerNode->groupId, HASH_ENTER,
NULL);
nodePlacementGroupHashEntry->nodeGroupId = workerNode->groupId;
bool shouldHaveShards = workerNode->shouldHaveShards;
if (drainWorkerNode && drainWorkerNode->groupId == workerNode->groupId)
{
shouldHaveShards = false;
}
nodePlacementGroupHashEntry->shouldHaveShards = shouldHaveShards;
nodePlacementGroupHashEntry->allowedToIsolateAnyPlacementGroup = shouldHaveShards;
nodePlacementGroupHashEntry->assignedPlacementGroup = NULL;
/*
* For the rest of the comment, assume that:
* Node D: the node we're draining
* Node I: a node that is not D and that has a shard placement group
* that needs an isolated node
* Node R: a node that is not D and that has some regular shard
* placements
*
* If we're draining a single node, then we don't know whether other
* nodes have any regular shard placements or any that need an isolated
* node because in that case GetRebalanceSteps() would provide a list of
* shard placements that are stored on D, not a list that contains all
* the placements accross the cluster (because we want to limit node
* draining to that node in that case). Note that when all shard
* placements in the cluster are provided, NodePlacementGroupHashAssignNodes()
* would already be aware of which node is used to isolate which shard
* placement group or which node is used to store some regular shard
* placements. That is why we skip below code if we're not draining a
* single node. It's not only inefficient to run below code when we're
* not draining a single node, but also it's not correct because otherwise
* rebalancer might decide to move some shard placements between any
* nodes in the cluster and it would be incorrect to assume that current
* placement distribution would be the same after the rebalancer plans the
* moves.
*
* Below we find out the assigned placement groups for nodes of type
* I because we want to avoid from moving the placements (if any) from
* node D to node I. We also set allowedToIsolateAnyPlacementGroup to
* false for the nodes that already have some shard placements because
* we want to avoid from moving the placements that need an isolated node
* (if any) from node D to node R.
*/
if (!(shouldHaveShards && drainSingleNode))
{
continue;
}
ShardPlacementGroup *isolatedShardPlacementGroup =
NodeGroupGetIsolatedShardPlacementGroup(
nodePlacementGroupHashEntry->nodeGroupId);
if (isolatedShardPlacementGroup)
{
nodePlacementGroupHashEntry->assignedPlacementGroup =
isolatedShardPlacementGroup;
}
else
{
nodePlacementGroupHashEntry->allowedToIsolateAnyPlacementGroup =
!NodeGroupHasShardPlacements(nodePlacementGroupHashEntry->nodeGroupId);
}
}
}
/*
* NodePlacementGroupHashAssignNodes assigns all active shard placements in
* the cluster that need to be isolated to individual worker nodes.
*/
static void
NodePlacementGroupHashAssignNodes(HTAB *nodePlacementGroupHash,
List *workerNodeList,
List *shardPlacementList)
{
List *availableWorkerList = list_copy(workerNodeList);
List *unassignedShardPlacementList = NIL;
/*
* Assign as much as possible shard placement groups to worker nodes where
* they are stored already.
*/
ShardPlacement *shardPlacement = NULL;
foreach_ptr(shardPlacement, shardPlacementList)
{
if (!shardPlacement->needsIsolatedNode)
{
continue;
}
int32 assignGroupId = shardPlacement->groupId;
if (NodePlacementGroupHashAssignNode(nodePlacementGroupHash,
assignGroupId,
shardPlacement))
{
int currentPlacementNodeIdx =
WorkerNodeListGetNodeWithGroupId(availableWorkerList,
assignGroupId);
availableWorkerList = list_delete_nth_cell(availableWorkerList,
currentPlacementNodeIdx);
}
else
{
unassignedShardPlacementList =
lappend(unassignedShardPlacementList, shardPlacement);
}
}
/*
* For the shard placement groups that could not be assigned to their
* current node, assign them to any other node that is available.
*/
int availableNodeIdx = 0;
ShardPlacement *unassignedShardPlacement = NULL;
foreach_ptr(unassignedShardPlacement, unassignedShardPlacementList)
{
Assert(unassignedShardPlacement->needsIsolatedNode);
bool isolated = false;
while (!isolated && availableNodeIdx < list_length(availableWorkerList))
{
WorkerNode *availableWorkerNode =
(WorkerNode *) list_nth(availableWorkerList, availableNodeIdx);
availableNodeIdx++;
if (NodePlacementGroupHashAssignNode(nodePlacementGroupHash,
availableWorkerNode->groupId,
unassignedShardPlacement))
{
isolated = true;
break;
}
}
if (!isolated)
{
ereport(WARNING, (errmsg("could not isolate all shard placements "
"that need an isolated node")));
return;
}
}
}
/*
* NodePlacementGroupHashAssignNode is an helper to
* NodePlacementGroupHashAssignNodes that tries to assign given
* shard placement to given node and returns true if it succeeds.
*/
static bool
NodePlacementGroupHashAssignNode(HTAB *nodePlacementGroupHash,
int32 nodeGroupId,
ShardPlacement *shardPlacement)
{
NodePlacementGroupHashEntry *nodePlacementGroupHashEntry =
NodePlacementGroupHashGetNodeWithGroupId(nodePlacementGroupHash, nodeGroupId);
ShardPlacementGroup *placementGroup =
GetShardPlacementGroupForPlacement(shardPlacement->shardId,
shardPlacement->placementId);
if (nodePlacementGroupHashEntry->assignedPlacementGroup)
{
/*
* Right now callers of this function call it once for each distinct
* shard placement group, hence we assume that placementGroup and
* nodePlacementGroupHashEntry->assignedPlacementGroup cannot be the
* same already, unless no shard placement group is assigned to this
* node yet.
*/
Assert(!ShardPlacementGroupsSame(placementGroup,
nodePlacementGroupHashEntry->
assignedPlacementGroup));
return false;
}
if (!nodePlacementGroupHashEntry->allowedToIsolateAnyPlacementGroup)
{
return false;
}
if (!nodePlacementGroupHashEntry->shouldHaveShards)
{
return false;
}
nodePlacementGroupHashEntry->assignedPlacementGroup = placementGroup;
return true;
}
/*
* RebalancerPlacementIsolationContextPlacementIsAllowedOnWorker returns true
* if shard placement with given shardId & placementId is allowed to be stored
* on given worker node.
*/
bool
RebalancerPlacementIsolationContextPlacementIsAllowedOnWorker(
RebalancerPlacementIsolationContext *context,
uint64 shardId,
uint64 placementId,
WorkerNode *workerNode)
{
HTAB *nodePlacementGroupHash = context->nodePlacementGroupHash;
NodePlacementGroupHashEntry *nodePlacementGroupHashEntry =
NodePlacementGroupHashGetNodeWithGroupId(nodePlacementGroupHash,
workerNode->groupId);
ShardPlacement *shardPlacement = LoadShardPlacement(shardId, placementId);
if (!shardPlacement->needsIsolatedNode)
{
/*
* It doesn't need an isolated node, but is the node used to isolate
* a shard placement group? If so, we cannot store it on this node.
*/
return nodePlacementGroupHashEntry->shouldHaveShards &&
nodePlacementGroupHashEntry->assignedPlacementGroup == NULL;
}
/*
* Given shard placement needs an isolated node.
* Check if given worker node is the one that is assigned to isolate it.
*/
ShardPlacementGroup *placementGroup =
GetShardPlacementGroupForPlacement(shardId, placementId);
return nodePlacementGroupHashEntry->assignedPlacementGroup != NULL &&
ShardPlacementGroupsSame(nodePlacementGroupHashEntry->assignedPlacementGroup,
placementGroup);
}
/*
* NodePlacementGroupHashGetNodeWithGroupId searches given hash table for
* NodePlacementGroupHashEntry with given node id and returns it.
*
* Throws an error if no such entry is found.
*/
static NodePlacementGroupHashEntry *
NodePlacementGroupHashGetNodeWithGroupId(HTAB *nodePlacementGroupHash, int32 nodeGroupId)
{
NodePlacementGroupHashEntry *nodePlacementGroupHashEntry =
hash_search(nodePlacementGroupHash, &nodeGroupId, HASH_FIND, NULL);
if (nodePlacementGroupHashEntry == NULL)
{
ereport(ERROR, (errmsg("no such node is found")));
}
return nodePlacementGroupHashEntry;
}
/*
* WorkerNodeListGetNodeWithGroupId returns the index of worker node with given id
* in given worker node list.
*
* Throws an error if no such node is found.
*/
static int
WorkerNodeListGetNodeWithGroupId(List *workerNodeList, int32 nodeGroupId)
{
int workerNodeIndex = 0;
WorkerNode *workerNode = NULL;
foreach_ptr(workerNode, workerNodeList)
{
if (workerNode->groupId == nodeGroupId)
{
return workerNodeIndex;
}
workerNodeIndex++;
}
ereport(ERROR, (errmsg("no such node is found")));
}

View File

@ -4,6 +4,11 @@
* Routines to replicate shard of none-distributed table to
* a remote node.
*
* Procedures defined in this file assume that given none-distributed
* table was a Citus local table, caller updated the metadata to convert
* it to another none-distributed table type, and now wants to replicate
* the shard of the table to a remote node as part of the conversion.
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
@ -58,14 +63,25 @@ NoneDistTableReplicateCoordinatorPlacement(Oid noneDistTableId,
uint64 shardLength = ShardLength(shardId);
/* we've already verified that table has a coordinator placement */
ShardPlacement *coordinatorPlacement =
linitial(ActiveShardPlacementListOnGroup(shardId, COORDINATOR_GROUP_ID));
/* insert new placements to pg_dist_placement */
List *insertedPlacementList = NIL;
WorkerNode *targetNode = NULL;
foreach_ptr(targetNode, targetNodeList)
{
/*
* needsIsolatedNode cannot be true because the input table was
* originally a Citus local table.
*/
Assert(!coordinatorPlacement->needsIsolatedNode);
ShardPlacement *shardPlacement =
InsertShardPlacementRowGlobally(shardId, GetNextPlacementId(),
shardLength, targetNode->groupId);
shardLength, targetNode->groupId,
coordinatorPlacement->needsIsolatedNode);
/* and save the placement for shard creation on workers */
insertedPlacementList = lappend(insertedPlacementList, shardPlacement);
@ -78,8 +94,6 @@ NoneDistTableReplicateCoordinatorPlacement(Oid noneDistTableId,
/* fetch coordinator placement before deleting it */
Oid localPlacementTableId = GetTableLocalShardOid(noneDistTableId, shardId);
ShardPlacement *coordinatorPlacement =
linitial(ActiveShardPlacementListOnGroup(shardId, COORDINATOR_GROUP_ID));
/*
* CreateForeignKeysFromReferenceTablesOnShards and CopyFromLocalTableIntoDistTable
@ -106,9 +120,16 @@ NoneDistTableReplicateCoordinatorPlacement(Oid noneDistTableId,
*/
CreateForeignKeysFromReferenceTablesOnShards(noneDistTableId);
/* using the same placement id, re-insert the deleted placement */
/*
* Using the same placement id, re-insert the deleted placement.
*
* needsIsolatedNode cannot be true because the input table was originally
* a Citus local table.
*/
Assert(!coordinatorPlacement->needsIsolatedNode);
InsertShardPlacementRowGlobally(shardId, coordinatorPlacement->placementId,
shardLength, COORDINATOR_GROUP_ID);
shardLength, COORDINATOR_GROUP_ID,
coordinatorPlacement->needsIsolatedNode);
}

View File

@ -44,6 +44,7 @@
#include "distributed/pg_dist_rebalance_strategy.h"
#include "distributed/reference_table_utils.h"
#include "distributed/remote_commands.h"
#include "distributed/rebalancer_placement_isolation.h"
#include "distributed/resource_lock.h"
#include "distributed/shard_rebalancer.h"
#include "distributed/shard_cleaner.h"
@ -145,6 +146,8 @@ typedef struct RebalanceContext
FmgrInfo shardCostUDF;
FmgrInfo nodeCapacityUDF;
FmgrInfo shardAllowedOnNodeUDF;
RebalancerPlacementIsolationContext *rebalancerPlacementGroupIsolationContext;
} RebalanceContext;
/* WorkerHashKey contains hostname and port to be used as a key in a hash */
@ -253,7 +256,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);
@ -262,9 +266,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 ShardCost GetShardCost(uint64 shardId, uint64 placementId, void *context);
static List * NonColocatedDistRelationIdList(void);
static void RebalanceTableShards(RebalanceOptions *options, Oid shardReplicationModeOid);
static int64 RebalanceTableShardsBackground(RebalanceOptions *options, Oid
@ -465,7 +470,9 @@ FullShardPlacementList(Oid relationId, ArrayType *excludedShardArray)
ShardPlacement *placement = CitusMakeNode(ShardPlacement);
placement->shardId = groupPlacement->shardId;
placement->shardLength = groupPlacement->shardLength;
placement->groupId = groupPlacement->groupId;
placement->nodeId = worker->nodeId;
placement->needsIsolatedNode = groupPlacement->needsIsolatedNode;
placement->nodeName = pstrdup(worker->workerName);
placement->nodePort = worker->workerPort;
placement->placementId = groupPlacement->placementId;
@ -588,6 +595,12 @@ GetRebalanceSteps(RebalanceOptions *options)
options->threshold = options->rebalanceStrategy->minimumThreshold;
}
context.rebalancerPlacementGroupIsolationContext =
PrepareRebalancerPlacementIsolationContext(
activeWorkerList,
FlattenNestedList(activeShardPlacementListList),
options->workerNode);
return RebalancePlacementUpdates(activeWorkerList,
activeShardPlacementListList,
options->threshold,
@ -602,7 +615,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)
{
@ -610,6 +624,14 @@ ShardAllowedOnNode(uint64 shardId, WorkerNode *workerNode, void *voidContext)
}
RebalanceContext *context = voidContext;
if (!RebalancerPlacementIsolationContextPlacementIsAllowedOnWorker(
context->rebalancerPlacementGroupIsolationContext,
shardId, placementId, workerNode))
{
return false;
}
Datum allowed = FunctionCall2(&context->shardAllowedOnNodeUDF, shardId,
workerNode->nodeId);
return DatumGetBool(allowed);
@ -643,10 +665,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);
@ -2560,7 +2583,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,
@ -2572,8 +2596,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;
@ -2733,7 +2757,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(
@ -2782,7 +2807,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)
@ -2793,6 +2818,7 @@ FindAllowedTargetFillState(RebalanceState *state, uint64 shardId)
targetFillState->node);
if (!hasShard && state->functions->shardAllowedOnNode(
shardId,
placementId,
targetFillState->node,
state->functions->context))
{
@ -2967,6 +2993,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))
{
@ -3163,7 +3190,7 @@ ReplicationPlacementUpdates(List *workerNodeList, List *activeShardPlacementList
{
WorkerNode *workerNode = list_nth(workerNodeList, workerNodeIndex);
if (!NodeCanHaveDistTablePlacements(workerNode))
if (!NodeCanBeUsedForNonIsolatedPlacements(workerNode))
{
/* never replicate placements to nodes that should not have placements */
continue;

View File

@ -102,12 +102,14 @@ static void BlockingShardSplit(SplitOperation splitOperation,
List *sourceColocatedShardIntervalList,
List *shardSplitPointsList,
List *workersForPlacementList,
List *needsIsolatedNodeForPlacementList,
DistributionColumnMap *distributionColumnOverrides);
static void NonBlockingShardSplit(SplitOperation splitOperation,
uint64 splitWorkflowId,
List *sourceColocatedShardIntervalList,
List *shardSplitPointsList,
List *workersForPlacementList,
List *needsIsolatedNodeForPlacementList,
DistributionColumnMap *distributionColumnOverrides,
uint32 targetColocationId);
static void DoSplitCopy(WorkerNode *sourceShardNode,
@ -128,7 +130,8 @@ static void UpdateDistributionColumnsForShardGroup(List *colocatedShardList,
int shardCount,
uint32 colocationId);
static void InsertSplitChildrenShardMetadata(List *shardGroupSplitIntervalListList,
List *workersForPlacementList);
List *workersForPlacementList,
List *needsIsolatedNodeForPlacementList);
static void CreatePartitioningHierarchyForBlockingSplit(
List *shardGroupSplitIntervalListList,
List *workersForPlacementList);
@ -420,18 +423,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.
* 'needsIsolatedNodeForPlacementList' : Whether each split children needs an isolated 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,
@ -439,6 +443,7 @@ SplitShard(SplitMode splitMode,
uint64 shardIdToSplit,
List *shardSplitPointsList,
List *nodeIdsForPlacementList,
List *needsIsolatedNodeForPlacementList,
DistributionColumnMap *distributionColumnOverrides,
List *colocatedShardIntervalList,
uint32 targetColocationId)
@ -512,6 +517,7 @@ SplitShard(SplitMode splitMode,
sourceColocatedShardIntervalList,
shardSplitPointsList,
workersForPlacementList,
needsIsolatedNodeForPlacementList,
distributionColumnOverrides);
}
else
@ -524,6 +530,7 @@ SplitShard(SplitMode splitMode,
sourceColocatedShardIntervalList,
shardSplitPointsList,
workersForPlacementList,
needsIsolatedNodeForPlacementList,
distributionColumnOverrides,
targetColocationId);
@ -540,11 +547,12 @@ SplitShard(SplitMode splitMode,
/*
* SplitShard API to split a given shard (or shard group) in blocking fashion
* based on specified split points to a set of destination nodes.
* splitOperation : Customer operation that triggered split.
* splitWorkflowId : Number used to identify split workflow in names.
* sourceColocatedShardIntervalList : Source shard group to be split.
* shardSplitPointsList : Split Points list for the source 'shardInterval'.
* workersForPlacementList : Placement list corresponding to split children.
* splitOperation : Customer operation that triggered split.
* splitWorkflowId : Number used to identify split workflow in names.
* sourceColocatedShardIntervalList : Source shard group to be split.
* shardSplitPointsList : Split Points list for the source 'shardInterval'.
* workersForPlacementList : Placement list corresponding to split children.
* needsIsolatedNodeForPlacementList : Whether each split children needs an isolated node.
*/
static void
BlockingShardSplit(SplitOperation splitOperation,
@ -552,6 +560,7 @@ BlockingShardSplit(SplitOperation splitOperation,
List *sourceColocatedShardIntervalList,
List *shardSplitPointsList,
List *workersForPlacementList,
List *needsIsolatedNodeForPlacementList,
DistributionColumnMap *distributionColumnOverrides)
{
const char *operationName = SplitOperationAPIName[splitOperation];
@ -615,7 +624,8 @@ BlockingShardSplit(SplitOperation splitOperation,
/* Insert new shard and placement metdata */
InsertSplitChildrenShardMetadata(shardGroupSplitIntervalListList,
workersForPlacementList);
workersForPlacementList,
needsIsolatedNodeForPlacementList);
/* create partitioning hierarchy, if any */
CreatePartitioningHierarchyForBlockingSplit(
@ -1152,7 +1162,8 @@ UpdateDistributionColumnsForShardGroup(List *colocatedShardList,
*/
static void
InsertSplitChildrenShardMetadata(List *shardGroupSplitIntervalListList,
List *workersForPlacementList)
List *workersForPlacementList,
List *needsIsolatedNodeForPlacementList)
{
List *shardIntervalList = NIL;
List *syncedShardList = NIL;
@ -1165,11 +1176,12 @@ InsertSplitChildrenShardMetadata(List *shardGroupSplitIntervalListList,
/*
* Iterate on split shards list for a given shard and insert metadata.
*/
ShardInterval *shardInterval = NULL;
WorkerNode *workerPlacementNode = NULL;
forboth_ptr(shardInterval, shardIntervalList, workerPlacementNode,
workersForPlacementList)
for (int i = 0; i < list_length(shardIntervalList); i++)
{
ShardInterval *shardInterval = list_nth(shardIntervalList, i);
WorkerNode *workerPlacementNode = list_nth(workersForPlacementList, i);
bool needsIsolatedNode = list_nth_int(needsIsolatedNodeForPlacementList, i);
InsertShardRow(
shardInterval->relationId,
shardInterval->shardId,
@ -1181,7 +1193,8 @@ InsertSplitChildrenShardMetadata(List *shardGroupSplitIntervalListList,
shardInterval->shardId,
INVALID_PLACEMENT_ID, /* triggers generation of new id */
0, /* shard length (zero for HashDistributed Table) */
workerPlacementNode->groupId);
workerPlacementNode->groupId,
needsIsolatedNode);
if (ShouldSyncTableMetadata(shardInterval->relationId))
{
@ -1366,16 +1379,17 @@ AcquireNonblockingSplitLock(Oid relationId)
/*
* SplitShard API to split a given shard (or shard group) in non-blocking fashion
* based on specified split points to a set of destination nodes.
* splitOperation : Customer operation that triggered split.
* splitWorkflowId : Number used to identify split workflow in names.
* sourceColocatedShardIntervalList : Source shard group to be split.
* shardSplitPointsList : Split Points list for the source 'shardInterval'.
* workersForPlacementList : Placement list corresponding to split children.
* distributionColumnList : Maps relation IDs to distribution columns.
* If not specified, the distribution column is read
* from the metadata.
* targetColocationId : Specifies the colocation ID (only used for
* create_distributed_table_concurrently).
* splitOperation : Customer operation that triggered split.
* splitWorkflowId : Number used to identify split workflow in names.
* sourceColocatedShardIntervalList : Source shard group to be split.
* shardSplitPointsList : Split Points list for the source 'shardInterval'.
* workersForPlacementList : Placement list corresponding to split children.
* needsIsolatedNodeForPlacementList : Whether each split children needs an isolated node.
* distributionColumnList : Maps relation IDs to distribution columns.
* If not specified, the distribution column is read
* from the metadata.
* targetColocationId : Specifies the colocation ID (only used for
* create_distributed_table_concurrently).
*/
void
NonBlockingShardSplit(SplitOperation splitOperation,
@ -1383,6 +1397,7 @@ NonBlockingShardSplit(SplitOperation splitOperation,
List *sourceColocatedShardIntervalList,
List *shardSplitPointsList,
List *workersForPlacementList,
List *needsIsolatedNodeForPlacementList,
DistributionColumnMap *distributionColumnOverrides,
uint32 targetColocationId)
{
@ -1599,7 +1614,8 @@ NonBlockingShardSplit(SplitOperation splitOperation,
/* 12) Insert new shard and placement metdata */
InsertSplitChildrenShardMetadata(shardGroupSplitIntervalListList,
workersForPlacementList);
workersForPlacementList,
needsIsolatedNodeForPlacementList);
/* 13) create partitioning hierarchy, if any, this needs to be done
* after the metadata is correct, because it fails for some

View File

@ -122,7 +122,9 @@ static void EnsureShardCanBeCopied(int64 shardId, const char *sourceNodeName,
static List * RecreateTableDDLCommandList(Oid relationId);
static void EnsureTableListOwner(List *tableIdList);
static void ErrorIfReplicatingDistributedTableWithFKeys(List *tableIdList);
static bool NewPlacementNeedsIsolatedNode(uint64 shardId,
char *sourceNodeName,
int32 sourceNodePort);
static void DropShardPlacementsFromMetadata(List *shardList,
char *nodeName,
int32 nodePort);
@ -524,15 +526,34 @@ TransferShards(int64 shardId, char *sourceNodeName,
uint32 groupId = GroupForNode(targetNodeName, targetNodePort);
uint64 placementId = GetNextPlacementId();
/*
* Decide whether the new placement needs isolated node or not.
*
* Note that even if the new placement needs isolated node, we don't
* enforce it here because we assume that user is aware of what they're
* doing if this shard transfer operation is initiated by the user.
* Consequence of this assumption is that if user is transferring a
* placement that needs isolated node to a node that has some other
* placements, then the next call made to rebalancer would overwrite
* this operation (by moving the placement to an appropriate node).
*
* Otherwise, i.e., if this is initiated by the rebalancer, rebalancer
* anyway enforces isolation by choosing an appropriate node.
*/
bool newPlacementNeedsIsolatedNode = NewPlacementNeedsIsolatedNode(
colocatedShardId,
sourceNodeName,
sourceNodePort);
InsertShardPlacementRow(colocatedShardId, placementId,
ShardLength(colocatedShardId),
groupId);
groupId, newPlacementNeedsIsolatedNode);
if (transferType == SHARD_TRANSFER_COPY &&
ShouldSyncTableMetadata(colocatedShard->relationId))
{
char *placementCommand = PlacementUpsertCommand(colocatedShardId, placementId,
0, groupId);
0, groupId,
newPlacementNeedsIsolatedNode);
SendCommandToWorkersWithMetadata(placementCommand);
}
@ -1979,6 +2000,24 @@ RecreateTableDDLCommandList(Oid relationId)
}
/*
* NewPlacementNeedsIsolatedNode if the placement we're creating based on the
* placement we're replicating from sourceNodeName/sourceNodePort needs
* isolation.
*/
static bool
NewPlacementNeedsIsolatedNode(uint64 shardId, char *sourceNodeName, int32 sourceNodePort)
{
/* assume we're transferring the first placement */
uint32 groupId = GroupForNode(sourceNodeName, sourceNodePort);
List *activeShardPlacementListOnGroup = ActiveShardPlacementListOnGroup(shardId,
groupId);
ShardPlacement *firstPlacementOnGroup = linitial(activeShardPlacementListOnGroup);
return firstPlacementOnGroup->needsIsolatedNode;
}
/*
* DropShardPlacementsFromMetadata drops the shard placement metadata for
* the shard placements of given shard interval list from pg_dist_placement.

View File

@ -164,7 +164,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)
{
@ -362,8 +369,9 @@ CreateAppendDistributedShardPlacements(Oid relationId, int64 shardId,
ExecuteCriticalRemoteCommandList(connection, commandList);
bool needsIsolatedNode = false;
InsertShardPlacementRow(shardId, INVALID_PLACEMENT_ID, shardSize,
nodeGroupId);
nodeGroupId, needsIsolatedNode);
placementsCreated++;
if (placementsCreated >= replicationFactor)
@ -384,6 +392,9 @@ CreateAppendDistributedShardPlacements(Oid relationId, int64 shardId,
/*
* InsertShardPlacementRows inserts shard placements to the metadata table on
* the coordinator node.
*
* This function assumes that the caller is inserting the placements for a
* newly created shard. As a result, always sets needsisolatednode to false.
*/
void
InsertShardPlacementRows(Oid relationId, int64 shardId, List *workerNodeList,
@ -398,10 +409,12 @@ InsertShardPlacementRows(Oid relationId, int64 shardId, List *workerNodeList,
uint32 nodeGroupId = workerNode->groupId;
const uint64 shardSize = 0;
bool needsIsolatedNode = false;
InsertShardPlacementRow(shardId,
INVALID_PLACEMENT_ID,
shardSize,
nodeGroupId);
nodeGroupId,
needsIsolatedNode);
}
}
@ -811,9 +824,11 @@ UpdateShardSize(uint64 shardId, ShardInterval *shardInterval, Oid relationId,
{
uint64 placementId = placement->placementId;
int32 groupId = placement->groupId;
bool needsIsolatedNode = placement->needsIsolatedNode;
DeleteShardPlacementRow(placementId);
InsertShardPlacementRow(shardId, placementId, shardSize, groupId);
InsertShardPlacementRow(shardId, placementId, shardSize, groupId,
needsIsolatedNode);
}
}

View File

@ -271,30 +271,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 shard placement group.
*/
List *
DistributedTablePlacementNodeList(LOCKMODE lockMode)
NewDistributedTablePlacementNodeList(LOCKMODE lockMode)
{
EnsureModificationsCanRun();
return FilterActiveNodeListFunc(lockMode, NodeCanHaveDistTablePlacements);
return FilterActiveNodeListFunc(lockMode, NodeCanBeUsedForNonIsolatedPlacements);
}
/*
* NodeCanHaveDistTablePlacements returns true if the given node can have
* shards of a distributed table.
* NodeCanBeUsedForNonIsolatedPlacements returns true if given node can be
* used to store shard placements that don't need isolation.
*/
bool
NodeCanHaveDistTablePlacements(WorkerNode *node)
NodeCanBeUsedForNonIsolatedPlacements(WorkerNode *node)
{
if (!NodeIsPrimary(node))
{
return false;
}
return node->shouldHaveShards;
if (!node->shouldHaveShards)
{
return false;
}
return NodeGroupGetIsolatedShardPlacementGroup(node->groupId) == NULL;
}

View File

@ -3,3 +3,21 @@
-- bump version to 12.2-1
#include "udfs/citus_add_rebalance_strategy/12.2-1.sql"
ALTER TABLE pg_dist_placement ADD COLUMN needsisolatednode boolean NOT NULL DEFAULT false;
-- Drop the legacy one that survived from 10.2-1, not the one created in 11.2-1.
--
-- And as we did when upgrading from 10.2-1 to 11.2-1, citus_internal_add_placement_metadata/12.2-1.sql
-- preserves the one created in 11.2-1 as the "new legacy" one.
DROP FUNCTION pg_catalog.citus_internal_add_placement_metadata(
shard_id bigint, shard_state integer,
shard_length bigint, group_id integer,
placement_id bigint);
#include "udfs/citus_internal_add_placement_metadata/12.2-1.sql"
#include "udfs/citus_internal_shard_group_set_needsisolatednode/12.2-1.sql"
#include "udfs/citus_shard_set_isolated/12.2-1.sql"
#include "udfs/citus_shard_unset_isolated/12.2-1.sql"

View File

@ -1,3 +1,20 @@
-- citus--12.2-1--12.1-1
#include "../udfs/citus_add_rebalance_strategy/10.1-1.sql"
ALTER TABLE pg_dist_placement DROP COLUMN needsisolatednode;
DROP FUNCTION pg_catalog.citus_internal_add_placement_metadata(
shard_id bigint,
shard_length bigint, group_id integer,
placement_id bigint,
needs_isolated_node boolean);
#include "../udfs/citus_internal_add_placement_metadata/11.2-1.sql"
DROP FUNCTION pg_catalog.citus_shard_set_isolated(shard_id bigint);
DROP FUNCTION pg_catalog.citus_shard_unset_isolated(shard_id bigint);
DROP FUNCTION pg_catalog.citus_internal_shard_group_set_needsisolatednode(
shard_id bigint,
enabled boolean);

View File

@ -0,0 +1,24 @@
-- create a new function, with needs_isolated_node
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_add_placement_metadata(
shard_id bigint,
shard_length bigint, group_id integer,
placement_id bigint,
needs_isolated_node boolean)
RETURNS void
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$citus_internal_add_placement_metadata$$;
COMMENT ON FUNCTION pg_catalog.citus_internal_add_placement_metadata(bigint, bigint, integer, bigint, boolean) IS
'Inserts into pg_dist_shard_placement with user checks';
-- replace the old one so it would call the old C function without needs_isolated_node
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_add_placement_metadata(
shard_id bigint,
shard_length bigint, group_id integer,
placement_id bigint)
RETURNS void
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$citus_internal_add_placement_metadata_legacy$$;
COMMENT ON FUNCTION pg_catalog.citus_internal_add_placement_metadata(bigint, bigint, integer, bigint) IS
'Inserts into pg_dist_shard_placement with user checks';

View File

@ -1,24 +1,24 @@
-- create a new function, without shardstate
-- create a new function, with needs_isolated_node
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_add_placement_metadata(
shard_id bigint,
shard_length bigint, group_id integer,
placement_id bigint,
needs_isolated_node boolean)
RETURNS void
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$citus_internal_add_placement_metadata$$;
COMMENT ON FUNCTION pg_catalog.citus_internal_add_placement_metadata(bigint, bigint, integer, bigint, boolean) IS
'Inserts into pg_dist_shard_placement with user checks';
-- replace the old one so it would call the old C function without needs_isolated_node
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_add_placement_metadata(
shard_id bigint,
shard_length bigint, group_id integer,
placement_id bigint)
RETURNS void
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$citus_internal_add_placement_metadata$$;
AS 'MODULE_PATHNAME', $$citus_internal_add_placement_metadata_legacy$$;
COMMENT ON FUNCTION pg_catalog.citus_internal_add_placement_metadata(bigint, bigint, integer, bigint) IS
'Inserts into pg_dist_shard_placement with user checks';
-- replace the old one so it would call the old C function with shard_state
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_add_placement_metadata(
shard_id bigint, shard_state integer,
shard_length bigint, group_id integer,
placement_id bigint)
RETURNS void
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$citus_internal_add_placement_metadata_legacy$$;
COMMENT ON FUNCTION pg_catalog.citus_internal_add_placement_metadata(bigint, integer, bigint, integer, bigint) IS
'Inserts into pg_dist_shard_placement with user checks';

View File

@ -0,0 +1,6 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_shard_group_set_needsisolatednode(
shard_id bigint,
enabled boolean)
RETURNS void
LANGUAGE C VOLATILE
AS 'MODULE_PATHNAME', $$citus_internal_shard_group_set_needsisolatednode$$;

View File

@ -0,0 +1,6 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_shard_group_set_needsisolatednode(
shard_id bigint,
enabled boolean)
RETURNS void
LANGUAGE C VOLATILE
AS 'MODULE_PATHNAME', $$citus_internal_shard_group_set_needsisolatednode$$;

View File

@ -0,0 +1,4 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_shard_set_isolated(shard_id bigint)
RETURNS void
LANGUAGE C VOLATILE
AS 'MODULE_PATHNAME', $$citus_shard_set_isolated$$;

View File

@ -0,0 +1,4 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_shard_set_isolated(shard_id bigint)
RETURNS void
LANGUAGE C VOLATILE
AS 'MODULE_PATHNAME', $$citus_shard_set_isolated$$;

View File

@ -0,0 +1,4 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_shard_unset_isolated(shard_id bigint)
RETURNS void
LANGUAGE C VOLATILE
AS 'MODULE_PATHNAME', $$citus_shard_unset_isolated$$;

View File

@ -0,0 +1,4 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_shard_unset_isolated(shard_id bigint)
RETURNS void
LANGUAGE C VOLATILE
AS 'MODULE_PATHNAME', $$citus_shard_unset_isolated$$;

View File

@ -44,9 +44,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);
@ -191,7 +192,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;
@ -242,12 +244,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)

View File

@ -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;

View File

@ -268,6 +268,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
@ -287,3 +304,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;
}

View File

@ -172,6 +172,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 */

View File

@ -73,7 +73,8 @@ typedef struct SequenceInfo
extern void citus_internal_add_placement_metadata_internal(int64 shardId,
int64 shardLength,
int32 groupId,
int64 placementId);
int64 placementId,
bool needsIsolatedNode);
extern void SyncCitusTableMetadata(Oid relationId);
extern void EnsureSequentialModeMetadataOperations(void);
extern bool ClusterHasKnownMetadataWorkers(void);
@ -112,7 +113,8 @@ extern List * GenerateGrantOnForeignServerQueriesFromAclItem(Oid serverId,
AclItem *aclItem);
extern List * GenerateGrantOnFDWQueriesFromAclItem(Oid serverId, AclItem *aclItem);
extern char * PlacementUpsertCommand(uint64 shardId, uint64 placementId,
uint64 shardLength, int32 groupId);
uint64 shardLength, int32 groupId,
bool needsIsolatedNode);
extern TableDDLCommand * TruncateTriggerCreateCommand(Oid relationId);
extern void CreateInterTableRelationshipOfRelationOnWorkers(Oid relationId);
extern List * InterTableRelationshipOfRelationCommandList(Oid relationId);
@ -141,8 +143,10 @@ 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 * ShardGroupSetNeedsIsolatedNodeCommand(uint64 shardId, bool enabled);
extern char * AddPlacementMetadataCommand(uint64 shardId, uint64 placementId,
uint64 shardLength, int32 groupId);
uint64 shardLength, int32 groupId,
bool needsIsolatedNode);
extern char * DeletePlacementMetadataCommand(uint64 placementId);
extern MetadataSyncContext * CreateMetadataSyncContext(List *nodeList,
@ -203,10 +207,10 @@ extern void SendInterTableRelationshipCommands(MetadataSyncContext *context);
#define UPSERT_PLACEMENT \
"INSERT INTO pg_dist_placement " \
"(shardid, shardstate, shardlength, " \
"groupid, placementid) " \
"groupid, placementid, needsisolatednode) " \
"VALUES (" UINT64_FORMAT ", 1, " UINT64_FORMAT \
", %d, " UINT64_FORMAT \
") " \
", %s) " \
"ON CONFLICT (shardid, groupid) DO UPDATE SET " \
"shardstate = EXCLUDED.shardstate, " \
"shardlength = EXCLUDED.shardlength, " \

View File

@ -78,6 +78,7 @@ typedef struct GroupShardPlacement
uint64 shardId;
uint64 shardLength;
int32 groupId;
bool needsIsolatedNode;
} GroupShardPlacement;
@ -92,6 +93,7 @@ typedef struct ShardPlacement
uint64 shardId;
uint64 shardLength;
int32 groupId;
bool needsIsolatedNode;
/* the rest of the fields aren't from pg_dist_placement */
char *nodeName;
@ -103,6 +105,14 @@ typedef struct ShardPlacement
} ShardPlacement;
typedef struct
{
uint32 colocatationId;
int shardIntervalIndex;
int32 nodeGroupId;
} ShardPlacementGroup;
typedef enum CascadeToColocatedOption
{
CASCADE_TO_COLOCATED_UNSPECIFIED,
@ -322,7 +332,12 @@ extern int ShardIntervalCount(Oid relationId);
extern List * LoadShardList(Oid relationId);
extern ShardInterval * CopyShardInterval(ShardInterval *srcInterval);
extern uint64 ShardLength(uint64 shardId);
extern ShardPlacementGroup * NodeGroupGetIsolatedShardPlacementGroup(int32 groupId);
extern bool ShardPlacementGroupsSame(const ShardPlacementGroup *leftGroup,
const ShardPlacementGroup *rightGroup);
extern bool NodeGroupHasShardPlacements(int32 groupId);
extern ShardPlacementGroup * GetShardPlacementGroupForPlacement(uint64 shardId,
uint64 placementId);
extern bool IsActiveShardPlacement(ShardPlacement *ShardPlacement);
extern bool IsRemoteShardPlacement(ShardPlacement *shardPlacement);
extern bool IsPlacementOnWorkerNode(ShardPlacement *placement, WorkerNode *workerNode);
@ -353,9 +368,11 @@ extern void DeleteShardRow(uint64 shardId);
extern ShardPlacement * InsertShardPlacementRowGlobally(uint64 shardId,
uint64 placementId,
uint64 shardLength,
int32 groupId);
int32 groupId,
bool needsIsolatedNode);
extern uint64 InsertShardPlacementRow(uint64 shardId, uint64 placementId,
uint64 shardLength, int32 groupId);
uint64 shardLength, int32 groupId,
bool needsIsolatedNode);
extern void InsertIntoPgDistPartition(Oid relationId, char distributionMethod,
Var *distributionColumn, uint32 colocationId,
char replicationModel, bool autoConverted);
@ -431,6 +448,7 @@ extern List * SendShardStatisticsQueriesInParallel(List *citusTableIds,
extern bool GetNodeDiskSpaceStatsForConnection(MultiConnection *connection,
uint64 *availableBytes,
uint64 *totalBytes);
extern void ShardGroupSetNeedsIsolatedNode(uint64 shardId, bool enabled);
extern void ExecuteQueryViaSPI(char *query, int SPIOK);
extern void ExecuteAndLogQueryViaSPI(char *query, int SPIOK, int logLevel);
extern void EnsureSequenceTypeSupported(Oid seqOid, Oid attributeTypeId, Oid

View File

@ -28,6 +28,7 @@ typedef struct FormData_pg_dist_placement
int32 shardstate; /* shard state on remote node; see ShardState */
int64 shardlength; /* shard length on remote node; stored as bigint */
int32 groupid; /* the group the shard is placed on */
bool needsisolatednode; /* whether the placement group needs to be isolated from others */
} FormData_pg_dist_placement;
/* ----------------
@ -41,12 +42,13 @@ typedef FormData_pg_dist_placement *Form_pg_dist_placement;
* compiler constants for pg_dist_placement
* ----------------
*/
#define Natts_pg_dist_placement 5
#define Natts_pg_dist_placement 6
#define Anum_pg_dist_placement_placementid 1
#define Anum_pg_dist_placement_shardid 2
#define Anum_pg_dist_placement_shardstate 3
#define Anum_pg_dist_placement_shardlength 4
#define Anum_pg_dist_placement_groupid 5
#define Anum_pg_dist_placement_needsisolatednode 6
#endif /* PG_DIST_PLACEMENT_H */

View File

@ -0,0 +1,40 @@
/*-------------------------------------------------------------------------
*
* rebalancer_placement_isolation.h
* Routines to determine which worker node should be used to isolate
* a colocated set of shard placements that needs isolation.
*
* 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 RebalancerPlacementIsolationContext;
typedef struct RebalancerPlacementIsolationContext RebalancerPlacementIsolationContext;
extern RebalancerPlacementIsolationContext * PrepareRebalancerPlacementIsolationContext(
List *activeWorkerNodeList,
List
*
activeShardPlacementList,
WorkerNode
*
drainWorkerNode);
extern bool RebalancerPlacementIsolationContextPlacementIsAllowedOnWorker(
RebalancerPlacementIsolationContext *context,
uint64 shardId,
uint64
placementId,
WorkerNode *
workerNode);
#endif /* PLACEMENT_ISOLATION_H */

View File

@ -162,6 +162,7 @@ typedef struct NodeFillState
typedef struct ShardCost
{
uint64 shardId;
uint64 placementId;
/*
* cost is the cost of the shard. This doesn't have a unit.
@ -180,9 +181,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;

View File

@ -43,6 +43,7 @@ extern void SplitShard(SplitMode splitMode,
uint64 shardIdToSplit,
List *shardSplitPointsList,
List *nodeIdsForPlacementList,
List *needsIsolatedNodeForPlacementList,
DistributionColumnMap *distributionColumnOverrides,
List *colocatedShardIntervalList,
uint32 targetColocationId);

View File

@ -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 NodeCanBeUsedForNonIsolatedPlacements(WorkerNode *node);
extern List * ActiveReadableNonCoordinatorNodeList(void);
extern List * ActiveReadableNodeList(void);
extern WorkerNode * FindWorkerNode(const char *nodeName, int32 nodePort);

View File

@ -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

View File

@ -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

View File

@ -232,7 +232,7 @@ s/CREATE TABLESPACE test_tablespace LOCATION.*/CREATE TABLESPACE test_tablespace
s/(.*absolute correlation \()([0,1]\.[0-9]+)(\) of var attribute [0-9]+ is smaller than.*)/\1X\.YZ\3/g
# normalize differences in multi_fix_partition_shard_index_names test
s/NOTICE: issuing WITH placement_data\(shardid, shardlength, groupid, placementid\) AS \(VALUES \([0-9]+, [0-9]+, [0-9]+, [0-9]+\)\)/NOTICE: issuing WITH placement_data\(shardid, shardlength, groupid, placementid\) AS \(VALUES \(xxxxxx, xxxxxx, xxxxxx, xxxxxx\)\)/g
s/NOTICE: issuing WITH placement_data\(shardid, shardlength, groupid, placementid, needsisolatednode\) AS \(VALUES \([0-9]+, [0-9]+, [0-9]+, [0-9]+, (true|false)\)\)/NOTICE: issuing WITH placement_data\(shardid, shardlength, groupid, placementid, needsisolatednode\) AS \(VALUES \(xxxxxx, xxxxxx, xxxxxx, xxxxxx, xxxxxx\)\)/g
# global_pid when pg_cancel_backend is sent to workers
s/pg_cancel_backend\('[0-9]+'::bigint\)/pg_cancel_backend('xxxxx'::bigint)/g

View File

@ -164,6 +164,7 @@ DEPS = {
None, ["isolation_setup", "isolation_add_remove_node"]
),
"schema_based_sharding": TestDeps("minimal_schedule"),
"isolate_placement": TestDeps("minimal_schedule"),
"multi_sequence_default": TestDeps(
None,
[
@ -174,6 +175,13 @@ DEPS = {
),
"grant_on_schema_propagation": TestDeps("minimal_schedule"),
"propagate_extension_commands": TestDeps("minimal_schedule"),
"metadata_sync_helpers": TestDeps(
None,
[
"multi_test_helpers",
"multi_cluster_management",
],
),
}

File diff suppressed because it is too large Load Diff

View File

@ -963,9 +963,9 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS
(VALUES (-10, 1, 0::bigint, 1::int, 1500000::bigint))
SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS
(VALUES (-10, 0::bigint, 1::int, 1500000::bigint))
SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
ERROR: could not find valid entry for shard xxxxx
ROLLBACK;
-- invalid placementid
@ -980,7 +980,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH placement_data(shardid, shardlength, groupid, placementid) AS
(VALUES (1420000, 0::bigint, 1::int, -10))
SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, false) FROM placement_data;
ERROR: Shard placement has invalid placement id (-10) for shard(1420000)
ROLLBACK;
-- non-existing shard
@ -995,7 +995,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH placement_data(shardid, shardlength, groupid, placementid) AS
(VALUES (1430100, 0::bigint, 1::int, 10))
SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, false) FROM placement_data;
ERROR: could not find valid entry for shard xxxxx
ROLLBACK;
-- non-existing node with non-existing node-id 123123123
@ -1010,7 +1010,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH placement_data(shardid, shardlength, groupid, placementid) AS
(VALUES ( 1420000, 0::bigint, 123123123::int, 1500000))
SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, false) FROM placement_data;
ERROR: Node with group id 123123123 for shard placement xxxxx does not exist
ROLLBACK;
-- create a volatile function that returns the local node id
@ -1041,7 +1041,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
WITH placement_data(shardid, shardlength, groupid, placementid) AS
(VALUES (1420000, 0::bigint, get_node_id(), 1500000),
(1420000, 0::bigint, get_node_id(), 1500001))
SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, false) FROM placement_data;
ERROR: duplicate key value violates unique constraint "placement_shardid_groupid_unique_index"
ROLLBACK;
-- shard is not owned by us
@ -1056,7 +1056,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH placement_data(shardid, shardlength, groupid, placementid) AS
(VALUES (1420007, 0::bigint, get_node_id(), 1500000))
SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, false) FROM placement_data;
ERROR: must be owner of table super_user_table
ROLLBACK;
-- sucessfully add placements
@ -1082,7 +1082,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1420011, 0::bigint, get_node_id(), 1500009),
(1420012, 0::bigint, get_node_id(), 1500010),
(1420013, 0::bigint, get_node_id(), 1500011))
SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, false) FROM placement_data;
citus_internal_add_placement_metadata
---------------------------------------------------------------------
@ -1284,7 +1284,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

View File

@ -1,3 +1,5 @@
CREATE SCHEMA multi_create_shards;
SET search_path TO multi_create_shards;
SET citus.next_shard_id TO 370000;
-- ===================================================================
-- create test functions and types needed for tests
@ -68,7 +70,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
@ -163,3 +165,5 @@ SELECT shardmaxvalue::integer - shardminvalue::integer AS shard_size
613566759
(7 rows)
SET client_min_messages TO WARNING;
DROP SCHEMA multi_create_shards CASCADE;

View File

@ -60,7 +60,7 @@ BEGIN
FROM current_objects c FULL JOIN prev_objects p
ON p.description = c.description
WHERE (p.description is null OR c.description is null)
AND c.description IS DISTINCT FROM 'function any_value(anyelement) anyelement'
AND c.description IS DISTINCT FROM 'function any_value(anyelement) anyelement'
AND c.description IS DISTINCT FROM 'function any_value_agg(anyelement,anyelement) anyelement';
DROP TABLE prev_objects;
@ -1420,9 +1420,14 @@ 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
previous_object | current_object
---------------------------------------------------------------------
(0 rows)
function citus_internal_add_placement_metadata(bigint,integer,bigint,integer,bigint) void |
| function citus_internal_add_placement_metadata(bigint,bigint,integer,bigint,boolean) void
| function citus_internal_shard_group_set_needsisolatednode(bigint,boolean) void
| function citus_shard_set_isolated(bigint) void
| function citus_shard_unset_isolated(bigint) void
(5 rows)
DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff;
-- show running version

View File

@ -5,6 +5,7 @@
-- https://github.com/citusdata/citus/issues/5138
---------------------------------------------------------------------
SET citus.next_shard_id TO 910000;
SET citus.next_placement_id TO 910000;
SET citus.shard_replication_factor TO 1;
CREATE SCHEMA fix_idx_names;
SET search_path TO fix_idx_names, public;
@ -454,6 +455,7 @@ SET ROLE user1;
SELECT fix_partition_shard_index_names('fix_idx_names.dist_partitioned_table'::regclass);
ERROR: permission denied for schema fix_idx_names
RESET ROLE;
DROP ROLE user1;
SET search_path TO fix_idx_names, public;
DROP TABLE dist_partitioned_table;
-- We can do any further operations (e.g. rename) on the indexes of partitions because
@ -491,6 +493,7 @@ SELECT tablename, indexname FROM pg_indexes WHERE schemaname = 'fix_idx_names' A
(2 rows)
\c - - - :master_port
SET citus.next_placement_id TO 920000;
SET search_path TO fix_idx_names, public;
DROP TABLE dist_partitioned_table;
SET citus.next_shard_id TO 910040;
@ -634,40 +637,24 @@ ALTER INDEX p1_dist_col_idx3 RENAME TO p1_dist_col_idx3_renamed;
ALTER INDEX p1_pkey RENAME TO p1_pkey_renamed;
ALTER INDEX p1_dist_col_partition_col_key RENAME TO p1_dist_col_partition_col_key_renamed;
ALTER INDEX p1_dist_col_idx RENAME TO p1_dist_col_idx_renamed;
-- create columnar extension idempotently
SET client_min_messages TO WARNING;
CREATE EXTENSION IF NOT EXISTS citus_columnar;
RESET client_min_messages;
-- should be able to create a new partition that is columnar
SET citus.log_remote_commands TO ON;
CREATE TABLE p2(dist_col int NOT NULL, another_col int, partition_col timestamp NOT NULL, name text) USING columnar;
ALTER TABLE parent_table ATTACH PARTITION p2 FOR VALUES FROM ('2019-01-01') TO ('2020-01-01');
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing SET citus.enable_ddl_propagation TO 'off'
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing CREATE EXTENSION IF NOT EXISTS citus_columnar WITH SCHEMA pg_catalog VERSION "x.y-z";
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing COMMIT
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing SET citus.enable_ddl_propagation TO 'off'
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing CREATE EXTENSION IF NOT EXISTS citus_columnar WITH SCHEMA pg_catalog VERSION "x.y-z";
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing COMMIT
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('extension', ARRAY['citus_columnar']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('extension', ARRAY['citus_columnar']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing SELECT worker_apply_shard_ddl_command (915002, 'fix_idx_names', 'CREATE TABLE fix_idx_names.p2 (dist_col integer NOT NULL, another_col integer, partition_col timestamp without time zone NOT NULL, name text) USING columnar')
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing ALTER TABLE fix_idx_names.p2_915002 SET (columnar.chunk_group_row_limit = 10000, columnar.stripe_row_limit = 150000, columnar.compression_level = 3, columnar.compression = 'zstd');
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing SELECT worker_apply_shard_ddl_command (915002, 'fix_idx_names', 'ALTER TABLE fix_idx_names.p2 OWNER TO postgres')
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing SET citus.enable_ddl_propagation TO 'off'
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing SET citus.enable_ddl_propagation TO 'off'
@ -700,9 +687,9 @@ NOTICE: issuing WITH shard_data(relationname, shardid, storagetype, shardminval
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('fix_idx_names.p2'::regclass, 915002, 't'::"char", '-2147483648', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing WITH 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;
NOTICE: issuing WITH placement_data(shardid, shardlength, groupid, placementid, needsisolatednode) AS (VALUES (xxxxxx, xxxxxx, xxxxxx, xxxxxx, xxxxxx)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, needsisolatednode) FROM placement_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;
NOTICE: issuing WITH placement_data(shardid, shardlength, groupid, placementid, needsisolatednode) AS (VALUES (xxxxxx, xxxxxx, xxxxxx, xxxxxx, xxxxxx)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, needsisolatednode) FROM placement_data;
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing SET citus.enable_ddl_propagation TO 'off'
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx

View File

@ -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_placement'::regclass)
ORDER BY attrelid, attname;
attrelid | attname | atthasmissing | attmissingval
---------------------------------------------------------------------

View File

@ -192,8 +192,8 @@ 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 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 placement_data(shardid, shardlength, groupid, placementid, needsisolatednode) AS (VALUES (1310000, 0, 1, 100000, false), (1310001, 0, 2, 100001, false), (1310002, 0, 1, 100002, false), (1310003, 0, 2, 100003, false), (1310004, 0, 1, 100004, false), (1310005, 0, 2, 100005, false), (1310006, 0, 1, 100006, false), (1310007, 0, 2, 100007, false)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, needsisolatednode) FROM placement_data;
WITH placement_data(shardid, shardlength, groupid, placementid, needsisolatednode) AS (VALUES (1310008, 0, 2, 100008, false)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, needsisolatednode) 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;
(61 rows)
@ -255,7 +255,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', 'mx_test_table_col_3_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 ('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 placement_data(shardid, shardlength, groupid, placementid, needsisolatednode) AS (VALUES (1310000, 0, 1, 100000, false), (1310001, 0, 2, 100001, false), (1310002, 0, 1, 100002, false), (1310003, 0, 2, 100003, false), (1310004, 0, 1, 100004, false), (1310005, 0, 2, 100005, false), (1310006, 0, 1, 100006, false), (1310007, 0, 2, 100007, false)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, needsisolatednode) 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;
(52 rows)
@ -317,7 +317,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['mx_testing_schema', 'mx_test_table_col_3_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 ('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 placement_data(shardid, shardlength, groupid, placementid, needsisolatednode) AS (VALUES (1310000, 0, 1, 100000, false), (1310001, 0, 2, 100001, false), (1310002, 0, 1, 100002, false), (1310003, 0, 2, 100003, false), (1310004, 0, 1, 100004, false), (1310005, 0, 2, 100005, false), (1310006, 0, 1, 100006, false), (1310007, 0, 2, 100007, false)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, needsisolatednode) 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;
(54 rows)
@ -385,7 +385,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['mx_testing_schema', 'mx_test_table_col_3_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 ('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 placement_data(shardid, shardlength, groupid, placementid, needsisolatednode) AS (VALUES (1310000, 0, 1, 100000, false), (1310001, 0, 2, 100001, false), (1310002, 0, 1, 100002, false), (1310003, 0, 2, 100003, false), (1310004, 0, 1, 100004, false), (1310005, 0, 2, 100005, false), (1310006, 0, 1, 100006, false), (1310007, 0, 2, 100007, false)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, needsisolatednode) 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;
(54 rows)
@ -446,7 +446,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['mx_testing_schema', 'mx_test_table_col_3_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 ('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 placement_data(shardid, shardlength, groupid, placementid, needsisolatednode) AS (VALUES (1310000, 0, 1, 100000, false), (1310001, 0, 2, 100001, false), (1310002, 0, 1, 100002, false), (1310003, 0, 2, 100003, false), (1310004, 0, 1, 100004, false), (1310005, 0, 2, 100005, false), (1310006, 0, 1, 100006, false), (1310007, 0, 2, 100007, false)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, needsisolatednode) 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;
(54 rows)
@ -2050,12 +2050,12 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['public', 'dist_table_1']::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_ref']::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', '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, 5, 100001), (1310002, 0, 1, 100002), (1310003, 0, 5, 100003), (1310004, 0, 1, 100004), (1310005, 0, 5, 100005), (1310006, 0, 1, 100006), (1310007, 0, 5, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310022, 0, 1, 100022), (1310023, 0, 5, 100023), (1310024, 0, 1, 100024), (1310025, 0, 5, 100025), (1310026, 0, 1, 100026)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310027, 0, 1, 100027), (1310028, 0, 5, 100028), (1310029, 0, 1, 100029), (1310030, 0, 5, 100030), (1310031, 0, 1, 100031)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
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 placement_data(shardid, shardlength, groupid, placementid, needsisolatednode) AS (VALUES (1310000, 0, 1, 100000, false), (1310001, 0, 5, 100001, false), (1310002, 0, 1, 100002, false), (1310003, 0, 5, 100003, false), (1310004, 0, 1, 100004, false), (1310005, 0, 5, 100005, false), (1310006, 0, 1, 100006, false), (1310007, 0, 5, 100007, false)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, needsisolatednode) FROM placement_data;
WITH placement_data(shardid, shardlength, groupid, placementid, needsisolatednode) AS (VALUES (1310022, 0, 1, 100022, false), (1310023, 0, 5, 100023, false), (1310024, 0, 1, 100024, false), (1310025, 0, 5, 100025, false), (1310026, 0, 1, 100026, false)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, needsisolatednode) FROM placement_data;
WITH placement_data(shardid, shardlength, groupid, placementid, needsisolatednode) AS (VALUES (1310027, 0, 1, 100027, false), (1310028, 0, 5, 100028, false), (1310029, 0, 1, 100029, false), (1310030, 0, 5, 100030, false), (1310031, 0, 1, 100031, false)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, needsisolatednode) FROM placement_data;
WITH placement_data(shardid, shardlength, groupid, placementid, needsisolatednode) AS (VALUES (1310075, 0, 0, 100077, false), (1310075, 0, 1, 100078, false), (1310075, 0, 5, 100079, false)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, needsisolatednode) FROM placement_data;
WITH placement_data(shardid, shardlength, groupid, placementid, needsisolatednode) AS (VALUES (1310076, 0, 1, 100080, false), (1310077, 0, 5, 100081, false), (1310078, 0, 1, 100082, false), (1310079, 0, 5, 100083, false)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, needsisolatednode) FROM placement_data;
WITH placement_data(shardid, shardlength, groupid, placementid, needsisolatednode) AS (VALUES (1310085, 0, 1, 100091, false), (1310086, 0, 5, 100092, false), (1310087, 0, 1, 100093, false), (1310088, 0, 5, 100094, false)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, needsisolatednode) 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;

View File

@ -192,8 +192,8 @@ 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 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 placement_data(shardid, shardlength, groupid, placementid, needsisolatednode) AS (VALUES (1310000, 0, 1, 100000, false), (1310001, 0, 2, 100001, false), (1310002, 0, 1, 100002, false), (1310003, 0, 2, 100003, false), (1310004, 0, 1, 100004, false), (1310005, 0, 2, 100005, false), (1310006, 0, 1, 100006, false), (1310007, 0, 2, 100007, false)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, needsisolatednode) FROM placement_data;
WITH placement_data(shardid, shardlength, groupid, placementid, needsisolatednode) AS (VALUES (1310008, 0, 2, 100008, false)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, needsisolatednode) 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;
(61 rows)
@ -255,7 +255,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', 'mx_test_table_col_3_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 ('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 placement_data(shardid, shardlength, groupid, placementid, needsisolatednode) AS (VALUES (1310000, 0, 1, 100000, false), (1310001, 0, 2, 100001, false), (1310002, 0, 1, 100002, false), (1310003, 0, 2, 100003, false), (1310004, 0, 1, 100004, false), (1310005, 0, 2, 100005, false), (1310006, 0, 1, 100006, false), (1310007, 0, 2, 100007, false)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, needsisolatednode) 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;
(52 rows)
@ -317,7 +317,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['mx_testing_schema', 'mx_test_table_col_3_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 ('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 placement_data(shardid, shardlength, groupid, placementid, needsisolatednode) AS (VALUES (1310000, 0, 1, 100000, false), (1310001, 0, 2, 100001, false), (1310002, 0, 1, 100002, false), (1310003, 0, 2, 100003, false), (1310004, 0, 1, 100004, false), (1310005, 0, 2, 100005, false), (1310006, 0, 1, 100006, false), (1310007, 0, 2, 100007, false)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, needsisolatednode) 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;
(54 rows)
@ -385,7 +385,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['mx_testing_schema', 'mx_test_table_col_3_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 ('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 placement_data(shardid, shardlength, groupid, placementid, needsisolatednode) AS (VALUES (1310000, 0, 1, 100000, false), (1310001, 0, 2, 100001, false), (1310002, 0, 1, 100002, false), (1310003, 0, 2, 100003, false), (1310004, 0, 1, 100004, false), (1310005, 0, 2, 100005, false), (1310006, 0, 1, 100006, false), (1310007, 0, 2, 100007, false)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, needsisolatednode) 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;
(54 rows)
@ -446,7 +446,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['mx_testing_schema', 'mx_test_table_col_3_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 ('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 placement_data(shardid, shardlength, groupid, placementid, needsisolatednode) AS (VALUES (1310000, 0, 1, 100000, false), (1310001, 0, 2, 100001, false), (1310002, 0, 1, 100002, false), (1310003, 0, 2, 100003, false), (1310004, 0, 1, 100004, false), (1310005, 0, 2, 100005, false), (1310006, 0, 1, 100006, false), (1310007, 0, 2, 100007, false)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, needsisolatednode) 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;
(54 rows)
@ -2050,12 +2050,12 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['public', 'dist_table_1']::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_ref']::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', '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, 5, 100001), (1310002, 0, 1, 100002), (1310003, 0, 5, 100003), (1310004, 0, 1, 100004), (1310005, 0, 5, 100005), (1310006, 0, 1, 100006), (1310007, 0, 5, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310022, 0, 1, 100022), (1310023, 0, 5, 100023), (1310024, 0, 1, 100024), (1310025, 0, 5, 100025), (1310026, 0, 1, 100026)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310027, 0, 1, 100027), (1310028, 0, 5, 100028), (1310029, 0, 1, 100029), (1310030, 0, 5, 100030), (1310031, 0, 1, 100031)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
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 placement_data(shardid, shardlength, groupid, placementid, needsisolatednode) AS (VALUES (1310000, 0, 1, 100000, false), (1310001, 0, 5, 100001, false), (1310002, 0, 1, 100002, false), (1310003, 0, 5, 100003, false), (1310004, 0, 1, 100004, false), (1310005, 0, 5, 100005, false), (1310006, 0, 1, 100006, false), (1310007, 0, 5, 100007, false)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, needsisolatednode) FROM placement_data;
WITH placement_data(shardid, shardlength, groupid, placementid, needsisolatednode) AS (VALUES (1310022, 0, 1, 100022, false), (1310023, 0, 5, 100023, false), (1310024, 0, 1, 100024, false), (1310025, 0, 5, 100025, false), (1310026, 0, 1, 100026, false)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, needsisolatednode) FROM placement_data;
WITH placement_data(shardid, shardlength, groupid, placementid, needsisolatednode) AS (VALUES (1310027, 0, 1, 100027, false), (1310028, 0, 5, 100028, false), (1310029, 0, 1, 100029, false), (1310030, 0, 5, 100030, false), (1310031, 0, 1, 100031, false)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, needsisolatednode) FROM placement_data;
WITH placement_data(shardid, shardlength, groupid, placementid, needsisolatednode) AS (VALUES (1310075, 0, 0, 100077, false), (1310075, 0, 1, 100078, false), (1310075, 0, 5, 100079, false)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, needsisolatednode) FROM placement_data;
WITH placement_data(shardid, shardlength, groupid, placementid, needsisolatednode) AS (VALUES (1310076, 0, 1, 100080, false), (1310077, 0, 5, 100081, false), (1310078, 0, 1, 100082, false), (1310079, 0, 5, 100083, false)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, needsisolatednode) FROM placement_data;
WITH placement_data(shardid, shardlength, groupid, placementid, needsisolatednode) AS (VALUES (1310085, 0, 1, 100091, false), (1310086, 0, 5, 100092, false), (1310087, 0, 1, 100093, false), (1310088, 0, 5, 100094, false)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, needsisolatednode) 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;

View File

@ -526,3 +526,112 @@ BEGIN
RETURN result;
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 placement that needs an isolated node.
--
-- This does not only return the placements of input relation but also considers
-- all colocated relations.
--
-- An example output is as follows:
--
-- [
-- {"10": [{"dist_1": [true,true]},{"dist_2": [false,false]}]},
-- {"15": [{"dist_1": [false,false]},{"dist_3": [true,false]}]}
-- ]
--
-- It only returned shard groups 10 and 15 because they are the only shard groups
-- that contain at least one shard placement that needs an isolated node.
--
-- (Innermost) Boolean arrays represent needsisolatednode values for different
-- placements of given shard. For example,
--
-- {"15": [{"dist_1": [false,false]},{"dist_3": [true,false]}]}
--
-- means that the first shard placement of dist_3 within shard group 15 needs
-- to be isolated but the other placement doesn't. Also, the first placement
-- is on the node that has a lower groupid than the second one because we order
-- them by groupid.
CREATE OR REPLACE FUNCTION get_colocated_placements_needisolatednode(
qualified_table_name text)
RETURNS SETOF jsonb AS $func$
BEGIN
RETURN QUERY
SELECT
COALESCE(
jsonb_agg(jsonb_build_object(shardgroupindex, needsisolatednodejson) ORDER BY shardgroupindex),
'{}'::jsonb
) AS result
FROM (
SELECT shardgroupindex,
jsonb_agg(jsonb_build_object(logicalrelid, needsisolatednodearr) ORDER BY logicalrelid::text) AS needsisolatednodejson
FROM (
SELECT logicalrelid,
shardgroupindex,
array_agg(needsisolatednode ORDER BY shardgroupnodegroupid) AS needsisolatednodearr
FROM (
SELECT shardgroupindex,
groupid AS shardgroupnodegroupid,
logicalrelid,
needsisolatednode
FROM public.get_enumerated_shard_groups(qualified_table_name) AS shardgroups
JOIN pg_dist_placement
ON shardid = ANY(shardids)
JOIN pg_dist_shard USING(shardid)
) q1
GROUP BY logicalrelid, shardgroupindex
) q2
GROUP BY shardgroupindex
) q3
WHERE needsisolatednodejson::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(ok_for_nodegroup) INTO v_result FROM (
SELECT array_agg(shardid ORDER BY shardid) =
(SELECT shardids FROM public.get_enumerated_shard_groups(qualified_table_name) WHERE shardgroupindex = shard_group_index)
AS ok_for_nodegroup -- check whether each of those nodes only contain placements of given shard group
FROM citus_shards
JOIN pg_dist_node USING (nodename, nodeport)
WHERE citus_table_type = 'distributed' AND -- only interested in distributed table shards on the nodes we're interested in
groupid IN ( -- only interested in the nodes that contain placements of given shard group
SELECT DISTINCT(pdn.groupid)
FROM citus_shards cs
JOIN pg_dist_node pdn USING (nodename, nodeport)
WHERE cs.shardid IN (
SELECT unnest(shardids) FROM public.get_enumerated_shard_groups(qualified_table_name) WHERE shardgroupindex = shard_group_index
)
)
GROUP BY groupid
) q;
RETURN v_result;
END;
$func$ LANGUAGE plpgsql;

View File

@ -1153,16 +1153,16 @@ SELECT * FROM public.table_placements_per_node;
CALL citus_cleanup_orphaned_resources();
select * from pg_dist_placement ORDER BY placementid;
placementid | shardid | shardstate | shardlength | groupid
placementid | shardid | shardstate | shardlength | groupid | needsisolatednode
---------------------------------------------------------------------
138 | 123023 | 1 | 0 | 14
141 | 123024 | 1 | 0 | 14
144 | 123027 | 1 | 0 | 14
145 | 123028 | 1 | 0 | 14
146 | 123021 | 1 | 0 | 16
147 | 123025 | 1 | 0 | 16
148 | 123022 | 1 | 0 | 16
149 | 123026 | 1 | 0 | 16
138 | 123023 | 1 | 0 | 14 | f
141 | 123024 | 1 | 0 | 14 | f
144 | 123027 | 1 | 0 | 14 | f
145 | 123028 | 1 | 0 | 14 | f
146 | 123021 | 1 | 0 | 16 | f
147 | 123025 | 1 | 0 | 16 | f
148 | 123022 | 1 | 0 | 16 | f
149 | 123026 | 1 | 0 | 16 | f
(8 rows)
-- Move all shards to worker1 again

View File

@ -0,0 +1,19 @@
-- upgrade_columnar_before renames public to citus_schema and recreates public
-- schema. But this file depends some helper functions 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_placements_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;

View File

@ -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_set_isolated(:shardgroup_5_shardid);
citus_shard_set_isolated
---------------------------------------------------------------------
(1 row)
SELECT result FROM run_command_on_all_nodes($$
SELECT * FROM public.get_colocated_placements_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)

View File

@ -67,7 +67,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_placement_metadata(bigint,integer,bigint,integer,bigint)
function citus_internal_add_placement_metadata(bigint,bigint,integer,bigint,boolean)
function citus_internal_add_shard_metadata(regclass,bigint,"char",text,text)
function citus_internal_add_tenant_schema(oid,integer)
function citus_internal_adjust_local_clock_to_remote(cluster_clock)
@ -80,6 +80,7 @@ ORDER BY 1;
function citus_internal_is_replication_origin_tracking_active()
function citus_internal_local_blocked_processes()
function citus_internal_mark_node_not_synced(integer,integer)
function citus_internal_shard_group_set_needsisolatednode(bigint,boolean)
function citus_internal_start_replication_origin_tracking()
function citus_internal_stop_replication_origin_tracking()
function citus_internal_unregister_tenant_schema_globally(oid,text)
@ -129,7 +130,9 @@ 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_set_isolated(bigint)
function citus_shard_sizes()
function citus_shard_unset_isolated(bigint)
function citus_shards_on_worker()
function citus_split_shard_by_split_points(bigint,text[],integer[],citus.shard_transfer_mode)
function citus_stat_activity()
@ -343,5 +346,5 @@ ORDER BY 1;
view citus_stat_tenants_local
view pg_dist_shard_placement
view time_partitions
(333 rows)
(336 rows)

View File

@ -36,6 +36,7 @@ test: create_single_shard_table
# 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

View File

@ -0,0 +1,908 @@
-- 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_group_set_needsisolatednode(0, NULL);
SELECT citus_internal_shard_group_set_needsisolatednode(NULL, false);
-- test with user that doesn't have permission to execute the function
SELECT citus_internal_shard_group_set_needsisolatednode(0, true);
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_group_set_needsisolatednode(0, true);
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.next_shard_id TO 2000000;
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 needsisolatednode yet
SELECT result FROM run_command_on_all_nodes($$
SELECT * FROM public.get_colocated_placements_needisolatednode('isolate_placement.dist_1')
$$)
ORDER BY nodeid;
SELECT shardids[2] AS shardgroup_5_shardid
FROM public.get_enumerated_shard_groups('isolate_placement.dist_1')
WHERE shardgroupindex = 5 \gset
SELECT citus_shard_set_isolated(:shardgroup_5_shardid);
SELECT shardids[3] AS shardgroup_10_shardid
FROM public.get_enumerated_shard_groups('isolate_placement.dist_1')
WHERE shardgroupindex = 10 \gset
SELECT citus_shard_set_isolated(:shardgroup_10_shardid);
SELECT result FROM run_command_on_all_nodes($$
SELECT * FROM public.get_colocated_placements_needisolatednode('isolate_placement.dist_1')
$$)
ORDER BY nodeid;
SELECT shardids[1] AS shardgroup_3_shardid
FROM public.get_enumerated_shard_groups('isolate_placement.dist_1')
WHERE shardgroupindex = 3 \gset
SELECT citus_shard_unset_isolated(:shardgroup_3_shardid);
SELECT result FROM run_command_on_all_nodes($$
SELECT * FROM public.get_colocated_placements_needisolatednode('isolate_placement.dist_1')
$$)
ORDER BY nodeid;
SELECT shardids[1] AS shardgroup_10_shardid
FROM public.get_enumerated_shard_groups('isolate_placement.dist_1')
WHERE shardgroupindex = 10 \gset
SELECT citus_shard_unset_isolated(:shardgroup_10_shardid);
SELECT result FROM run_command_on_all_nodes($$
SELECT * FROM public.get_colocated_placements_needisolatednode('isolate_placement.dist_1')
$$)
ORDER BY nodeid;
SELECT shardids[1] AS shardgroup_5_shardid
FROM public.get_enumerated_shard_groups('isolate_placement.dist_1')
WHERE shardgroupindex = 5 \gset
SELECT citus_shard_set_isolated(:shardgroup_5_shardid);
SELECT result FROM run_command_on_all_nodes($$
SELECT * FROM public.get_colocated_placements_needisolatednode('isolate_placement.dist_1')
$$)
ORDER BY nodeid;
-- 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_set_isolated(:shardgroup_5_shardid);
SELECT result FROM run_command_on_all_nodes($$
SELECT * FROM public.get_colocated_placements_needisolatednode('isolate_placement.dist_1')
$$)
ORDER BY nodeid;
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 needsisolatednode from the colocated placements too.
SELECT result FROM run_command_on_all_nodes($$
SELECT * FROM public.get_colocated_placements_needisolatednode('isolate_placement.dist_1')
$$)
ORDER BY nodeid;
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_placements_needisolatednode('isolate_placement.dist_1')
$$)
ORDER BY nodeid;
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_set_isolated(:shardgroup_3_shardid);
SELECT result FROM run_command_on_all_nodes($$
SELECT * FROM public.get_colocated_placements_needisolatednode('isolate_placement.dist_1')
$$)
ORDER BY nodeid;
-- 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_placements_needisolatednode('isolate_placement.dist_1')
$$)
ORDER BY nodeid;
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_set_isolated(:shardgroup_9_shardid);
SELECT result FROM run_command_on_all_nodes($$
SELECT * FROM public.get_colocated_placements_needisolatednode('isolate_placement.dist_1')
$$)
ORDER BY nodeid;
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 needsisolatednode flag when splitting the shard.
SELECT result FROM run_command_on_all_nodes($$
SELECT * FROM public.get_colocated_placements_needisolatednode('isolate_placement.dist_1')
$$)
ORDER BY nodeid;
SELECT shardids[1] AS shardgroup_12_shardid
FROM public.get_enumerated_shard_groups('isolate_placement.dist_1')
WHERE shardgroupindex = 12 \gset
SELECT citus_shard_set_isolated(:shardgroup_12_shardid);
SELECT result FROM run_command_on_all_nodes($$
SELECT * FROM public.get_colocated_placements_needisolatednode('isolate_placement.dist_1')
$$)
ORDER BY nodeid;
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_placements_needisolatednode('isolate_placement.dist_1')
$$)
ORDER BY nodeid;
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_set_isolated(:shardgroup_17_shardid);
SELECT result FROM run_command_on_all_nodes($$
SELECT * FROM public.get_colocated_placements_needisolatednode('isolate_placement.dist_3')
$$)
ORDER BY nodeid;
-- 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 needsisolatednode flag when splitting the shard.
SELECT result FROM run_command_on_all_nodes($$
SELECT * FROM public.get_colocated_placements_needisolatednode('isolate_placement.dist_3')
$$)
ORDER BY nodeid;
SELECT shardids[1] AS shardgroup_18_shardid
FROM public.get_enumerated_shard_groups('isolate_placement.dist_3')
WHERE shardgroupindex = 18 \gset
SELECT citus_shard_set_isolated(:shardgroup_18_shardid);
SELECT result FROM run_command_on_all_nodes($$
SELECT * FROM public.get_colocated_placements_needisolatednode('isolate_placement.dist_3')
$$)
ORDER BY nodeid;
-- 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_placements_needisolatednode('isolate_placement.dist_3')
$$)
ORDER BY nodeid;
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_set_isolated(: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_placements_needisolatednode('isolate_placement.single_shard_1')
$$)
ORDER BY nodeid;
-- test invalid input
SELECT citus_shard_set_isolated(NULL);
SELECT citus_shard_set_isolated(0);
SELECT citus_shard_unset_isolated(NULL);
SELECT citus_shard_unset_isolated(0);
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_set_isolated(shardid) FROM pg_dist_shard WHERE logicalrelid = 'append_table'::regclass LIMIT 1;
SELECT citus_shard_set_isolated(shardid) FROM pg_dist_shard WHERE logicalrelid = 'range_table'::regclass LIMIT 1;
SELECT citus_shard_set_isolated(shardid) FROM pg_dist_shard WHERE logicalrelid = 'ref_table'::regclass LIMIT 1;
SELECT citus_shard_set_isolated(shardid) FROM pg_dist_shard WHERE logicalrelid = 'local_table'::regclass LIMIT 1;
SELECT citus_shard_unset_isolated(shardid) FROM pg_dist_shard WHERE logicalrelid = 'append_table'::regclass LIMIT 1;
SELECT citus_shard_unset_isolated(shardid) FROM pg_dist_shard WHERE logicalrelid = 'range_table'::regclass LIMIT 1;
SELECT citus_shard_unset_isolated(shardid) FROM pg_dist_shard WHERE logicalrelid = 'ref_table'::regclass LIMIT 1;
SELECT citus_shard_unset_isolated(shardid) 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_set_isolated(:shardgroup_1_shardid);
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_placements_needisolatednode('isolate_placement.dist_1')
$$)
ORDER BY nodeid;
-- 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);
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_set_isolated(shardid) 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_set_isolated(shardid) 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_set_isolated(shardid) 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_set_isolated(:shardgroup_1_shardid);
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_set_isolated(shardid) 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_set_isolated(shardid) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_2'::regclass);
SELECT citus_shard_unset_isolated(shardid) 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_set_isolated(shardid) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_1'::regclass);
SELECT citus_shard_set_isolated(shardid) 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 ERROR;
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_set_isolated(shardid) FROM pg_dist_shard WHERE logicalrelid IN ('single_shard_1'::regclass);
SELECT citus_shard_set_isolated(shardid) 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;
SET client_min_messages TO WARNING;
DROP SCHEMA isolate_placement CASCADE;
SELECT citus_remove_node('localhost', :master_port);

View File

@ -597,9 +597,9 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS
(VALUES (-10, 1, 0::bigint, 1::int, 1500000::bigint))
SELECT citus_internal_add_placement_metadata(shardid, shardstate, shardlength, groupid, placementid) FROM placement_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS
(VALUES (-10, 0::bigint, 1::int, 1500000::bigint))
SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
ROLLBACK;
-- invalid placementid
@ -609,7 +609,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH placement_data(shardid, shardlength, groupid, placementid) AS
(VALUES (1420000, 0::bigint, 1::int, -10))
SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, false) FROM placement_data;
ROLLBACK;
-- non-existing shard
@ -619,7 +619,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH placement_data(shardid, shardlength, groupid, placementid) AS
(VALUES (1430100, 0::bigint, 1::int, 10))
SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, false) FROM placement_data;
ROLLBACK;
-- non-existing node with non-existing node-id 123123123
@ -629,7 +629,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH placement_data(shardid, shardlength, groupid, placementid) AS
(VALUES ( 1420000, 0::bigint, 123123123::int, 1500000))
SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, false) FROM placement_data;
ROLLBACK;
-- create a volatile function that returns the local node id
@ -656,7 +656,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
WITH placement_data(shardid, shardlength, groupid, placementid) AS
(VALUES (1420000, 0::bigint, get_node_id(), 1500000),
(1420000, 0::bigint, get_node_id(), 1500001))
SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, false) FROM placement_data;
ROLLBACK;
-- shard is not owned by us
@ -666,7 +666,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH placement_data(shardid, shardlength, groupid, placementid) AS
(VALUES (1420007, 0::bigint, get_node_id(), 1500000))
SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, false) FROM placement_data;
ROLLBACK;
-- sucessfully add placements
@ -687,7 +687,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1420011, 0::bigint, get_node_id(), 1500009),
(1420012, 0::bigint, get_node_id(), 1500010),
(1420013, 0::bigint, get_node_id(), 1500011))
SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid, false) FROM placement_data;
COMMIT;
-- we should be able to colocate both tables now

View File

@ -1,3 +1,5 @@
CREATE SCHEMA multi_create_shards;
SET search_path TO multi_create_shards;
SET citus.next_shard_id TO 370000;
@ -115,3 +117,6 @@ SELECT shardmaxvalue::integer - shardminvalue::integer AS shard_size
FROM pg_dist_shard
WHERE logicalrelid = 'weird_shard_count'::regclass
ORDER BY shardminvalue::integer ASC;
SET client_min_messages TO WARNING;
DROP SCHEMA multi_create_shards CASCADE;

View File

@ -5,6 +5,7 @@
-- https://github.com/citusdata/citus/issues/5138
----------------------------------------------------
SET citus.next_shard_id TO 910000;
SET citus.next_placement_id TO 910000;
SET citus.shard_replication_factor TO 1;
CREATE SCHEMA fix_idx_names;
SET search_path TO fix_idx_names, public;
@ -240,6 +241,7 @@ SET ROLE user1;
SELECT fix_partition_shard_index_names('fix_idx_names.dist_partitioned_table'::regclass);
RESET ROLE;
DROP ROLE user1;
SET search_path TO fix_idx_names, public;
DROP TABLE dist_partitioned_table;
@ -273,6 +275,7 @@ DROP INDEX p_another_col_partition_col_idx;
SELECT tablename, indexname FROM pg_indexes WHERE schemaname = 'fix_idx_names' AND tablename SIMILAR TO '%\_\d*' ORDER BY 1, 2;
\c - - - :master_port
SET citus.next_placement_id TO 920000;
SET search_path TO fix_idx_names, public;
DROP TABLE dist_partitioned_table;
SET citus.next_shard_id TO 910040;
@ -329,6 +332,11 @@ ALTER INDEX p1_pkey RENAME TO p1_pkey_renamed;
ALTER INDEX p1_dist_col_partition_col_key RENAME TO p1_dist_col_partition_col_key_renamed;
ALTER INDEX p1_dist_col_idx RENAME TO p1_dist_col_idx_renamed;
-- create columnar extension idempotently
SET client_min_messages TO WARNING;
CREATE EXTENSION IF NOT EXISTS citus_columnar;
RESET client_min_messages;
-- should be able to create a new partition that is columnar
SET citus.log_remote_commands TO ON;
CREATE TABLE p2(dist_col int NOT NULL, another_col int, partition_col timestamp NOT NULL, name text) USING columnar;

View File

@ -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_placement'::regclass)
ORDER BY attrelid, attname;

View File

@ -550,3 +550,117 @@ BEGIN
RETURN result;
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 placement that needs an isolated node.
--
-- This does not only return the placements of input relation but also considers
-- all colocated relations.
--
-- An example output is as follows:
--
-- [
-- {"10": [{"dist_1": [true,true]},{"dist_2": [false,false]}]},
-- {"15": [{"dist_1": [false,false]},{"dist_3": [true,false]}]}
-- ]
--
-- It only returned shard groups 10 and 15 because they are the only shard groups
-- that contain at least one shard placement that needs an isolated node.
--
-- (Innermost) Boolean arrays represent needsisolatednode values for different
-- placements of given shard. For example,
--
-- {"15": [{"dist_1": [false,false]},{"dist_3": [true,false]}]}
--
-- means that the first shard placement of dist_3 within shard group 15 needs
-- to be isolated but the other placement doesn't. Also, the first placement
-- is on the node that has a lower groupid than the second one because we order
-- them by groupid.
CREATE OR REPLACE FUNCTION get_colocated_placements_needisolatednode(
qualified_table_name text)
RETURNS SETOF jsonb AS $func$
BEGIN
RETURN QUERY
SELECT
COALESCE(
jsonb_agg(jsonb_build_object(shardgroupindex, needsisolatednodejson) ORDER BY shardgroupindex),
'{}'::jsonb
) AS result
FROM (
SELECT shardgroupindex,
jsonb_agg(jsonb_build_object(logicalrelid, needsisolatednodearr) ORDER BY logicalrelid::text) AS needsisolatednodejson
FROM (
SELECT logicalrelid,
shardgroupindex,
array_agg(needsisolatednode ORDER BY shardgroupnodegroupid) AS needsisolatednodearr
FROM (
SELECT shardgroupindex,
groupid AS shardgroupnodegroupid,
logicalrelid,
needsisolatednode
FROM public.get_enumerated_shard_groups(qualified_table_name) AS shardgroups
JOIN pg_dist_placement
ON shardid = ANY(shardids)
JOIN pg_dist_shard USING(shardid)
) q1
GROUP BY logicalrelid, shardgroupindex
) q2
GROUP BY shardgroupindex
) q3
WHERE needsisolatednodejson::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(ok_for_nodegroup) INTO v_result FROM (
SELECT array_agg(shardid ORDER BY shardid) =
(SELECT shardids FROM public.get_enumerated_shard_groups(qualified_table_name) WHERE shardgroupindex = shard_group_index)
AS ok_for_nodegroup -- check whether each of those nodes only contain placements of given shard group
FROM citus_shards
JOIN pg_dist_node USING (nodename, nodeport)
WHERE citus_table_type = 'distributed' AND -- only interested in distributed table shards on the nodes we're interested in
groupid IN ( -- only interested in the nodes that contain placements of given shard group
SELECT DISTINCT(pdn.groupid)
FROM citus_shards cs
JOIN pg_dist_node pdn USING (nodename, nodeport)
WHERE cs.shardid IN (
SELECT unnest(shardids) FROM public.get_enumerated_shard_groups(qualified_table_name) WHERE shardgroupindex = shard_group_index
)
)
GROUP BY groupid
) q;
RETURN v_result;
END;
$func$ LANGUAGE plpgsql;

View File

@ -0,0 +1,14 @@
-- upgrade_columnar_before renames public to citus_schema and recreates public
-- schema. But this file depends some helper functions 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_placements_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;

View File

@ -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_set_isolated(:shardgroup_5_shardid);
SELECT result FROM run_command_on_all_nodes($$
SELECT * FROM public.get_colocated_placements_needisolatednode('upgrade_isolate_placement_before.table_with_isolated_placements')
$$)
ORDER BY nodeid;