Allow isolating shard placement groups on individual nodes

tenant-schema-isolation
Onur Tirtir 2024-02-09 12:15:10 +03:00
parent 2fae91c5df
commit f1f72f3194
71 changed files with 5338 additions and 454 deletions

View File

@ -1454,8 +1454,9 @@ InsertMetadataForCitusLocalTable(Oid citusLocalTableId, uint64 shardId,
text *shardMinValue = NULL;
text *shardMaxValue = NULL;
bool needsSeparateNode = false;
InsertShardRow(citusLocalTableId, shardId, shardStorageType,
shardMinValue, shardMaxValue);
shardMinValue, shardMaxValue, needsSeparateNode);
List *nodeList = list_make1(CoordinatorNodeIfAddedAsWorkerOrError());

View File

@ -135,6 +135,7 @@ static char DecideDistTableReplicationModel(char distributionMethod,
static List * HashSplitPointsForShardList(List *shardList);
static List * HashSplitPointsForShardCount(int shardCount);
static List * WorkerNodesForShardList(List *shardList);
static List * NeedsSeparateNodeForShardList(List *shardList);
static List * RoundRobinWorkerNodeList(List *workerNodeList, int listLength);
static CitusTableParams DecideCitusTableParams(CitusTableType tableType,
DistributedTableParams *
@ -572,16 +573,10 @@ CreateDistributedTableConcurrently(Oid relationId, char *distributionColumnName,
colocatedTableId = ColocatedTableId(colocationId);
}
List *workerNodeList = DistributedTablePlacementNodeList(NoLock);
if (workerNodeList == NIL)
{
ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("no worker nodes are available for placing shards"),
errhint("Add more worker nodes.")));
}
List *workersForPlacementList;
List *shardSplitPointsList;
List *needsSeparateNodeForPlacementList;
if (colocatedTableId != InvalidOid)
{
@ -596,6 +591,12 @@ CreateDistributedTableConcurrently(Oid relationId, char *distributionColumnName,
* Find the node IDs of the shard placements.
*/
workersForPlacementList = WorkerNodesForShardList(colocatedShardList);
/*
* Inherit needsseparatenode from the colocated shards.
*/
needsSeparateNodeForPlacementList =
NeedsSeparateNodeForShardList(colocatedShardList);
}
else
{
@ -607,7 +608,21 @@ CreateDistributedTableConcurrently(Oid relationId, char *distributionColumnName,
/*
* Place shards in a round-robin fashion across all data nodes.
*/
List *workerNodeList = NewDistributedTablePlacementNodeList(NoLock);
if (workerNodeList == NIL)
{
ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("no worker nodes are available for placing shards"),
errhint("Add more worker nodes.")));
}
workersForPlacementList = RoundRobinWorkerNodeList(workerNodeList, shardCount);
/*
* For a new colocation group, needsseparatenode is set to false for
* all shards.
*/
needsSeparateNodeForPlacementList = GenerateListFromIntElement(false, shardCount);
}
/*
@ -646,6 +661,7 @@ CreateDistributedTableConcurrently(Oid relationId, char *distributionColumnName,
shardToSplit->shardId,
shardSplitPointsList,
workersForPlacementList,
needsSeparateNodeForPlacementList,
distributionColumnOverrides,
sourceColocatedShardIntervalList,
colocationId
@ -898,6 +914,26 @@ WorkerNodesForShardList(List *shardList)
}
/*
* NeedsSeparateNodeForShardList returns a list of node booleans reflecting whether
* each shard in the given list needs a separate node.
*/
static List *
NeedsSeparateNodeForShardList(List *shardList)
{
List *needsSeparateNodeList = NIL;
ShardInterval *shardInterval = NULL;
foreach_ptr(shardInterval, shardList)
{
needsSeparateNodeList = lappend_int(needsSeparateNodeList,
shardInterval->needsSeparateNode);
}
return needsSeparateNodeList;
}
/*
* RoundRobinWorkerNodeList round robins over the workers in the worker node list
* and adds node ids to a list of length listLength.

View File

@ -5379,6 +5379,8 @@ DeformedDistShardTupleToShardInterval(Datum *datumArray, bool *isNullArray,
char storageType = DatumGetChar(datumArray[Anum_pg_dist_shard_shardstorage - 1]);
Datum minValueTextDatum = datumArray[Anum_pg_dist_shard_shardminvalue - 1];
Datum maxValueTextDatum = datumArray[Anum_pg_dist_shard_shardmaxvalue - 1];
bool needsSeparateNode = DatumGetBool(
datumArray[Anum_pg_dist_shard_needsseparatenode - 1]);
bool minValueNull = isNullArray[Anum_pg_dist_shard_shardminvalue - 1];
bool maxValueNull = isNullArray[Anum_pg_dist_shard_shardmaxvalue - 1];
@ -5415,6 +5417,7 @@ DeformedDistShardTupleToShardInterval(Datum *datumArray, bool *isNullArray,
shardInterval->minValue = minValue;
shardInterval->maxValue = maxValue;
shardInterval->shardId = shardId;
shardInterval->needsSeparateNode = needsSeparateNode;
return shardInterval;
}

View File

@ -185,6 +185,7 @@ PG_FUNCTION_INFO_V1(citus_internal_add_tenant_schema);
PG_FUNCTION_INFO_V1(citus_internal_delete_tenant_schema);
PG_FUNCTION_INFO_V1(citus_internal_update_none_dist_table_metadata);
PG_FUNCTION_INFO_V1(citus_internal_database_command);
PG_FUNCTION_INFO_V1(citus_internal_shard_property_set);
static bool got_SIGTERM = false;
@ -1267,7 +1268,7 @@ ShardListInsertCommand(List *shardIntervalList)
StringInfo insertShardCommand = makeStringInfo();
appendStringInfo(insertShardCommand,
"WITH shard_data(relationname, shardid, storagetype, "
"shardminvalue, shardmaxvalue) AS (VALUES ");
"shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ");
foreach_ptr(shardInterval, shardIntervalList)
{
@ -1299,12 +1300,13 @@ ShardListInsertCommand(List *shardIntervalList)
}
appendStringInfo(insertShardCommand,
"(%s::regclass, %ld, '%c'::\"char\", %s, %s)",
"(%s::regclass, %ld, '%c'::\"char\", %s, %s, %s)",
quote_literal_cstr(qualifiedRelationName),
shardId,
shardInterval->storageType,
minHashToken->data,
maxHashToken->data);
maxHashToken->data,
shardInterval->needsSeparateNode ? "true" : "false");
if (llast(shardIntervalList) != shardInterval)
{
@ -1316,7 +1318,7 @@ ShardListInsertCommand(List *shardIntervalList)
appendStringInfo(insertShardCommand,
"SELECT citus_internal.add_shard_metadata(relationname, shardid, "
"storagetype, shardminvalue, shardmaxvalue) "
"storagetype, shardminvalue, shardmaxvalue, needsseparatenode) "
"FROM shard_data;");
/*
@ -3359,6 +3361,9 @@ citus_internal_add_shard_metadata(PG_FUNCTION_ARGS)
shardMaxValue = PG_GETARG_TEXT_P(4);
}
PG_ENSURE_ARGNOTNULL(5, "needs separate node");
bool needsSeparateNode = PG_GETARG_BOOL(5);
/* only owner of the table (or superuser) is allowed to add the Citus metadata */
EnsureTableOwner(relationId);
@ -3379,7 +3384,8 @@ citus_internal_add_shard_metadata(PG_FUNCTION_ARGS)
shardMaxValue);
}
InsertShardRow(relationId, shardId, storageType, shardMinValue, shardMaxValue);
InsertShardRow(relationId, shardId, storageType, shardMinValue, shardMaxValue,
needsSeparateNode);
PG_RETURN_VOID();
}
@ -4067,6 +4073,45 @@ citus_internal_database_command(PG_FUNCTION_ARGS)
}
/*
* citus_internal_shard_property_set is an internal UDF to
* set shard properties for all the shards within the shard group
* that given shard belongs to.
*/
Datum
citus_internal_shard_property_set(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
PG_ENSURE_ARGNOTNULL(0, "shard_id");
uint64 shardId = PG_GETARG_INT64(0);
/* only owner of the table (or superuser) is allowed to modify the Citus metadata */
Oid distributedRelationId = RelationIdForShard(shardId);
EnsureTableOwner(distributedRelationId);
/* we want to serialize all the metadata changes to this table */
LockRelationOid(distributedRelationId, ShareUpdateExclusiveLock);
if (!ShouldSkipMetadataChecks())
{
EnsureCitusInitiatedOperation();
}
bool *needsSeparateNodePtr = NULL;
if (!PG_ARGISNULL(1))
{
needsSeparateNodePtr = palloc(sizeof(bool));
*needsSeparateNodePtr = PG_GETARG_BOOL(1);
}
ShardgroupSetProperty(shardId, needsSeparateNodePtr);
PG_RETURN_VOID();
}
/*
* SyncNewColocationGroup synchronizes a new pg_dist_colocation entry to a worker.
*/
@ -4266,6 +4311,24 @@ UpdateNoneDistTableMetadataCommand(Oid relationId, char replicationModel,
}
/*
* ShardgroupSetPropertyCommand returns a command to call
* citus_internal.shard_property_set().
*/
char *
ShardgroupSetPropertyCommand(uint64 shardId, bool *needsSeparateNodePtr)
{
char *needsSeparateNodeStr = !needsSeparateNodePtr ? "null" :
(*needsSeparateNodePtr ? "true" : "false");
StringInfo command = makeStringInfo();
appendStringInfo(command,
"SELECT citus_internal.shard_property_set(%lu, %s)",
shardId, needsSeparateNodeStr);
return command->data;
}
/*
* AddPlacementMetadataCommand returns a command to call
* citus_internal_add_placement_metadata().

View File

@ -48,6 +48,7 @@
#include "pg_version_constants.h"
#include "distributed/argutils.h"
#include "distributed/background_jobs.h"
#include "distributed/citus_nodes.h"
#include "distributed/citus_safe_lib.h"
@ -75,6 +76,7 @@
#include "distributed/remote_commands.h"
#include "distributed/resource_lock.h"
#include "distributed/shard_rebalancer.h"
#include "distributed/shard_transfer.h"
#include "distributed/tuplestore.h"
#include "distributed/utils/array_type.h"
#include "distributed/version_compat.h"
@ -117,6 +119,7 @@ static void AppendShardIdNameValues(StringInfo selectQuery, ShardInterval *shard
static HeapTuple CreateDiskSpaceTuple(TupleDesc tupleDesc, uint64 availableBytes,
uint64 totalBytes);
static bool GetLocalDiskSpaceStats(uint64 *availableBytes, uint64 *totalBytes);
static void ShardgroupSetPropertyGlobally(uint64 shardId, bool *needsSeparateNodePtr);
static BackgroundTask * DeformBackgroundTaskHeapTuple(TupleDesc tupleDescriptor,
HeapTuple taskTuple);
@ -139,6 +142,7 @@ PG_FUNCTION_INFO_V1(citus_table_size);
PG_FUNCTION_INFO_V1(citus_total_relation_size);
PG_FUNCTION_INFO_V1(citus_relation_size);
PG_FUNCTION_INFO_V1(citus_shard_sizes);
PG_FUNCTION_INFO_V1(citus_shard_property_set);
/*
@ -361,6 +365,169 @@ citus_relation_size(PG_FUNCTION_ARGS)
}
/*
* citus_shard_property_set allows setting shard properties for all
* the shards within the shard group that given shard belongs to.
*/
Datum
citus_shard_property_set(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
EnsureCoordinator();
PG_ENSURE_ARGNOTNULL(0, "shard_id");
uint64 shardId = PG_GETARG_INT64(0);
/* RelationIdForShard() first checks whether the shard id is valid */
Oid distributedRelationId = RelationIdForShard(shardId);
List *colocatedTableList = ColocatedTableList(distributedRelationId);
colocatedTableList = SortList(colocatedTableList, CompareOids);
EnsureTableListOwner(colocatedTableList);
AcquirePlacementColocationLock(distributedRelationId, ExclusiveLock,
"set a property for a shard of");
Oid colocatedTableId = InvalidOid;
foreach_oid(colocatedTableId, colocatedTableList)
{
/*
* Prevent relations from being dropped while we are setting the
* property.
*/
LockRelationOid(colocatedTableId, AccessShareLock);
}
bool *needsSeparateNodePtr = NULL;
if (!PG_ARGISNULL(1))
{
if (!IsCitusTableType(distributedRelationId, HASH_DISTRIBUTED) &&
!IsCitusTableType(distributedRelationId, SINGLE_SHARD_DISTRIBUTED))
{
ereport(ERROR, (errmsg("setting anti-affinity property is only "
"supported for hash distributed tables")));
}
needsSeparateNodePtr = palloc(sizeof(bool));
*needsSeparateNodePtr = PG_GETARG_BOOL(1);
}
ShardgroupSetPropertyGlobally(shardId, needsSeparateNodePtr);
PG_RETURN_VOID();
}
/*
* ShardgroupSetPropertyGlobally calls ShardgroupSetProperty
* on all nodes.
*/
static void
ShardgroupSetPropertyGlobally(uint64 shardId, bool *needsSeparateNodePtr)
{
ShardgroupSetProperty(shardId, needsSeparateNodePtr);
char *metadataCommand =
ShardgroupSetPropertyCommand(shardId, needsSeparateNodePtr);
SendCommandToWorkersWithMetadata(metadataCommand);
}
/*
* ShardgroupSetProperty sets shard properties for all the shards within
* the shard group that given shard belongs to.
*/
void
ShardgroupSetProperty(uint64 shardId, bool *needsSeparateNodePtr)
{
ShardInterval *shardInterval = LoadShardInterval(shardId);
List *colocatedShardIntervalList = ColocatedShardIntervalList(shardInterval);
int nShardInterval = list_length(colocatedShardIntervalList);
Datum *shardIdDatumArray = (Datum *) palloc(nShardInterval * sizeof(Datum));
int shardIndex = 0;
ShardInterval *colocatedShardInterval = NULL;
foreach_ptr(colocatedShardInterval, colocatedShardIntervalList)
{
shardIdDatumArray[shardIndex] = UInt64GetDatum(colocatedShardInterval->shardId);
shardIndex++;
}
ArrayType *shardIdArrayDatum = DatumArrayToArrayType(shardIdDatumArray,
nShardInterval, INT8OID);
Relation pgDistShard = table_open(DistShardRelationId(), RowExclusiveLock);
ScanKeyData scanKey[1];
int scanKeyCount = 1;
ScanKeyInit(&scanKey[0], Anum_pg_dist_shard_shardid,
BTEqualStrategyNumber, F_INT8EQ, PointerGetDatum(shardIdArrayDatum));
scanKey[0].sk_flags |= SK_SEARCHARRAY;
bool indexOK = true;
Oid indexId = DistShardShardidIndexId();
SysScanDesc scanDescriptor = systable_beginscan(pgDistShard,
indexId, indexOK, NULL,
scanKeyCount, scanKey);
Datum values[Natts_pg_dist_shard];
bool isnull[Natts_pg_dist_shard];
bool replace[Natts_pg_dist_shard];
memset(values, 0, sizeof(values));
memset(isnull, false, sizeof(isnull));
memset(replace, false, sizeof(replace));
if (needsSeparateNodePtr)
{
values[Anum_pg_dist_shard_needsseparatenode - 1] = BoolGetDatum(
*needsSeparateNodePtr);
isnull[Anum_pg_dist_shard_needsseparatenode - 1] = false;
replace[Anum_pg_dist_shard_needsseparatenode - 1] = true;
}
bool updatedAny = false;
CatalogIndexState indexState = CatalogOpenIndexes(pgDistShard);
HeapTuple heapTuple = NULL;
while (HeapTupleIsValid(heapTuple = systable_getnext(scanDescriptor)))
{
TupleDesc tupleDescriptor = RelationGetDescr(pgDistShard);
heapTuple = heap_modify_tuple(heapTuple, tupleDescriptor, values, isnull,
replace);
CatalogTupleUpdateWithInfo(pgDistShard, &heapTuple->t_self, heapTuple,
indexState);
updatedAny = true;
}
if (!updatedAny)
{
ereport(ERROR, (errmsg("could not find valid entry for shard "
UINT64_FORMAT,
shardId)));
}
CatalogCloseIndexes(indexState);
/*
* We don't need to send invalidations for all the shards as
* CitusInvalidateRelcacheByShardId() will send the invalidation based on
* id of the belonging distributed table, not just for the input shard.
*/
CitusInvalidateRelcacheByShardId(shardInterval->shardId);
CommandCounterIncrement();
systable_endscan(scanDescriptor);
table_close(pgDistShard, NoLock);
}
/*
* SendShardStatisticsQueriesInParallel generates query lists for obtaining shard
* statistics and then sends the commands in parallel by opening connections
@ -1350,6 +1517,7 @@ CopyShardInterval(ShardInterval *srcInterval)
destInterval->maxValueExists = srcInterval->maxValueExists;
destInterval->shardId = srcInterval->shardId;
destInterval->shardIndex = srcInterval->shardIndex;
destInterval->needsSeparateNode = srcInterval->needsSeparateNode;
destInterval->minValue = 0;
if (destInterval->minValueExists)
@ -1397,6 +1565,92 @@ ShardLength(uint64 shardId)
}
/*
* NodeGroupGetSeparatedShardgroupPlacement returns the shard group placement
* that given node group is used to separate from others. Returns NULL if this
* node is not used to separate a shard group placement.
*/
ShardgroupPlacement *
NodeGroupGetSeparatedShardgroupPlacement(int32 groupId)
{
ShardgroupPlacement *nodeShardgroupPlacement = NULL;
bool shardgroupPlacementNeedsSeparateNode = false;
bool indexOK = true;
ScanKeyData scanKey[1];
Relation pgDistPlacement = table_open(DistPlacementRelationId(),
AccessShareLock);
ScanKeyInit(&scanKey[0], Anum_pg_dist_placement_groupid,
BTEqualStrategyNumber, F_INT4EQ, Int32GetDatum(groupId));
SysScanDesc scanDescriptor = systable_beginscan(pgDistPlacement,
DistPlacementGroupidIndexId(),
indexOK,
NULL, lengthof(scanKey), scanKey);
HeapTuple heapTuple = NULL;
while (HeapTupleIsValid(heapTuple = systable_getnext(scanDescriptor)))
{
TupleDesc tupleDescriptor = RelationGetDescr(pgDistPlacement);
GroupShardPlacement *placement =
TupleToGroupShardPlacement(tupleDescriptor, heapTuple);
ShardInterval *shardInterval = LoadShardInterval(placement->shardId);
Oid citusTableId = shardInterval->relationId;
if (!IsCitusTableType(citusTableId, DISTRIBUTED_TABLE))
{
continue;
}
ShardgroupPlacement *shardgroupPlacement =
GetShardgroupPlacementForPlacement(placement->shardId,
placement->placementId);
if (nodeShardgroupPlacement &&
!ShardgroupPlacementsSame(shardgroupPlacement,
nodeShardgroupPlacement))
{
/*
* If we have more than one shardgroup placement on the node,
* then this means that the node is not actually used to separate
* a shardgroup placement.
*/
nodeShardgroupPlacement = NULL;
shardgroupPlacementNeedsSeparateNode = false;
break;
}
nodeShardgroupPlacement = shardgroupPlacement;
shardgroupPlacementNeedsSeparateNode = shardInterval->needsSeparateNode;
}
systable_endscan(scanDescriptor);
table_close(pgDistPlacement, NoLock);
if (!shardgroupPlacementNeedsSeparateNode)
{
return NULL;
}
return nodeShardgroupPlacement;
}
/*
* ShardgroupPlacementsSame returns true if two shardgroup placements are the same.
*/
bool
ShardgroupPlacementsSame(const ShardgroupPlacement *leftGroup,
const ShardgroupPlacement *rightGroup)
{
return leftGroup->colocatationId == rightGroup->colocatationId &&
leftGroup->shardIntervalIndex == rightGroup->shardIntervalIndex &&
leftGroup->nodeGroupId == rightGroup->nodeGroupId;
}
/*
* NodeGroupHasShardPlacements returns whether any active shards are placed on the group
*/
@ -1429,6 +1683,70 @@ NodeGroupHasShardPlacements(int32 groupId)
}
/*
* NodeGroupHasDistributedTableShardPlacements returns whether any active
* distributed table shards are placed on the group
*/
bool
NodeGroupHasDistributedTableShardPlacements(int32 groupId)
{
bool nodeGroupHasDistributedTableShardPlacements = false;
Relation pgPlacement = table_open(DistPlacementRelationId(), AccessShareLock);
ScanKeyData scanKey[1];
ScanKeyInit(&scanKey[0], Anum_pg_dist_placement_groupid,
BTEqualStrategyNumber, F_INT4EQ, Int32GetDatum(groupId));
bool indexOK = true;
SysScanDesc scanDescriptor = systable_beginscan(pgPlacement,
DistPlacementGroupidIndexId(),
indexOK,
NULL, lengthof(scanKey), scanKey);
HeapTuple heapTuple = NULL;
while (HeapTupleIsValid(heapTuple = systable_getnext(scanDescriptor)))
{
TupleDesc tupleDescriptor = RelationGetDescr(pgPlacement);
GroupShardPlacement *placement =
TupleToGroupShardPlacement(tupleDescriptor, heapTuple);
ShardInterval *shardInterval = LoadShardInterval(placement->shardId);
Oid citusTableId = shardInterval->relationId;
if (IsCitusTableType(citusTableId, DISTRIBUTED_TABLE))
{
nodeGroupHasDistributedTableShardPlacements = true;
break;
}
}
systable_endscan(scanDescriptor);
table_close(pgPlacement, NoLock);
return nodeGroupHasDistributedTableShardPlacements;
}
/*
* GetShardgroupPlacementForPlacement returns ShardgroupPlacement that placement
* with given shardId & placementId belongs to.
*/
ShardgroupPlacement *
GetShardgroupPlacementForPlacement(uint64 shardId, uint64 placementId)
{
ShardPlacement *shardPlacement = LoadShardPlacement(shardId, placementId);
ShardInterval *shardInterval = LoadShardInterval(shardId);
ShardgroupPlacement *placementGroup = palloc(sizeof(ShardgroupPlacement));
placementGroup->colocatationId = shardPlacement->colocationGroupId;
placementGroup->shardIntervalIndex = shardInterval->shardIndex;
placementGroup->nodeGroupId = shardPlacement->groupId;
return placementGroup;
}
/*
* IsActiveShardPlacement checks if the shard placement is labelled as
* active, and that it is placed in an active worker.
@ -1803,7 +2121,8 @@ IsDummyPlacement(ShardPlacement *taskPlacement)
*/
void
InsertShardRow(Oid relationId, uint64 shardId, char storageType,
text *shardMinValue, text *shardMaxValue)
text *shardMinValue, text *shardMaxValue,
bool needsSeparateNode)
{
Datum values[Natts_pg_dist_shard];
bool isNulls[Natts_pg_dist_shard];
@ -1815,6 +2134,7 @@ InsertShardRow(Oid relationId, uint64 shardId, char storageType,
values[Anum_pg_dist_shard_logicalrelid - 1] = ObjectIdGetDatum(relationId);
values[Anum_pg_dist_shard_shardid - 1] = Int64GetDatum(shardId);
values[Anum_pg_dist_shard_shardstorage - 1] = CharGetDatum(storageType);
values[Anum_pg_dist_shard_needsseparatenode - 1] = BoolGetDatum(needsSeparateNode);
/* dropped shardalias column must also be set; it is still part of the tuple */
isNulls[Anum_pg_dist_shard_shardalias_DROPPED - 1] = true;

View File

@ -62,8 +62,6 @@
#include "distributed/worker_manager.h"
#include "distributed/worker_transaction.h"
#define INVALID_GROUP_ID -1
/* default group size */
int GroupSize = 1;

View File

@ -20,6 +20,7 @@
#include "distributed/colocation_utils.h"
#include "distributed/connection_management.h"
#include "distributed/coordinator_protocol.h"
#include "distributed/listutils.h"
#include "distributed/metadata_cache.h"
#include "distributed/remote_commands.h"
#include "distributed/shard_split.h"
@ -55,6 +56,10 @@ citus_split_shard_by_split_points(PG_FUNCTION_ARGS)
Oid shardTransferModeOid = PG_GETARG_OID(3);
SplitMode shardSplitMode = LookupSplitMode(shardTransferModeOid);
/* we don't inherit needsseparatenode for new shards */
List *needsSeparateNodeForPlacementList =
GenerateListFromIntElement(false, list_length(nodeIdsForPlacementList));
DistributionColumnMap *distributionColumnOverrides = NULL;
List *sourceColocatedShardIntervalList = NIL;
SplitShard(
@ -63,6 +68,7 @@ citus_split_shard_by_split_points(PG_FUNCTION_ARGS)
shardIdToSplit,
shardSplitPointsList,
nodeIdsForPlacementList,
needsSeparateNodeForPlacementList,
distributionColumnOverrides,
sourceColocatedShardIntervalList,
INVALID_COLOCATION_ID);

View File

@ -147,15 +147,16 @@ CreateShardsWithRoundRobinPolicy(Oid distributedTableId, int32 shardCount,
LockRelationOid(DistNodeRelationId(), RowShareLock);
/* load and sort the worker node list for deterministic placement */
List *workerNodeList = DistributedTablePlacementNodeList(NoLock);
List *workerNodeList = NewDistributedTablePlacementNodeList(NoLock);
workerNodeList = SortList(workerNodeList, CompareWorkerNodes);
int32 workerNodeCount = list_length(workerNodeList);
if (replicationFactor > workerNodeCount)
{
ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("replication_factor (%d) exceeds number of worker nodes "
"(%d)", replicationFactor, workerNodeCount),
errmsg("replication_factor (%d) exceeds number of "
"available worker nodes (%d)",
replicationFactor, workerNodeCount),
errhint("Add more worker nodes or try again with a lower "
"replication factor.")));
}
@ -184,8 +185,9 @@ CreateShardsWithRoundRobinPolicy(Oid distributedTableId, int32 shardCount,
text *minHashTokenText = IntegerToText(shardMinHashToken);
text *maxHashTokenText = IntegerToText(shardMaxHashToken);
bool needsSeparateNode = false;
InsertShardRow(distributedTableId, *shardIdPtr, shardStorageType,
minHashTokenText, maxHashTokenText);
minHashTokenText, maxHashTokenText, needsSeparateNode);
InsertShardPlacementRows(distributedTableId,
*shardIdPtr,
@ -282,8 +284,10 @@ CreateColocatedShards(Oid targetRelationId, Oid sourceRelationId, bool
List *sourceShardPlacementList = ShardPlacementListSortedByWorker(
sourceShardId);
/* inherit from the colocated shard */
bool needsSeparateNode = sourceShardInterval->needsSeparateNode;
InsertShardRow(targetRelationId, *newShardIdPtr, targetShardStorageType,
shardMinValueText, shardMaxValueText);
shardMinValueText, shardMaxValueText, needsSeparateNode);
ShardPlacement *sourcePlacement = NULL;
foreach_ptr(sourcePlacement, sourceShardPlacementList)
@ -365,8 +369,9 @@ CreateReferenceTableShard(Oid distributedTableId)
/* get the next shard id */
uint64 shardId = GetNextShardId();
bool needsSeparateNode = false;
InsertShardRow(distributedTableId, shardId, shardStorageType, shardMinValue,
shardMaxValue);
shardMaxValue, needsSeparateNode);
InsertShardPlacementRows(distributedTableId,
shardId,
@ -411,7 +416,7 @@ CreateSingleShardTableShardWithRoundRobinPolicy(Oid relationId, uint32 colocatio
* Also take a RowShareLock on pg_dist_node to disallow concurrent
* node list changes that require an exclusive lock.
*/
List *workerNodeList = DistributedTablePlacementNodeList(RowShareLock);
List *workerNodeList = NewDistributedTablePlacementNodeList(RowShareLock);
workerNodeList = SortList(workerNodeList, CompareWorkerNodes);
int roundRobinNodeIdx =
@ -421,8 +426,9 @@ CreateSingleShardTableShardWithRoundRobinPolicy(Oid relationId, uint32 colocatio
text *minHashTokenText = NULL;
text *maxHashTokenText = NULL;
uint64 shardId = GetNextShardId();
bool needsSeparateNode = false;
InsertShardRow(relationId, shardId, shardStorageType,
minHashTokenText, maxHashTokenText);
minHashTokenText, maxHashTokenText, needsSeparateNode);
int replicationFactor = 1;
InsertShardPlacementRows(relationId,
@ -454,17 +460,17 @@ CreateSingleShardTableShardWithRoundRobinPolicy(Oid relationId, uint32 colocatio
* group" should be placed on.
*
* This is determined by modulo of the colocation id by the length of the
* list returned by DistributedTablePlacementNodeList().
* list returned by NewDistributedTablePlacementNodeList().
*/
int
EmptySingleShardTableColocationDecideNodeId(uint32 colocationId)
{
List *workerNodeList = DistributedTablePlacementNodeList(RowShareLock);
List *workerNodeList = NewDistributedTablePlacementNodeList(RowShareLock);
int32 workerNodeCount = list_length(workerNodeList);
if (workerNodeCount == 0)
{
ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("couldn't find any worker nodes"),
errmsg("couldn't find any available worker nodes"),
errhint("Add more worker nodes")));
}

View File

@ -168,6 +168,10 @@ isolate_tenant_to_new_shard(PG_FUNCTION_ARGS)
nodeIdsForPlacementList = lappend_int(nodeIdsForPlacementList, sourceNodeId);
}
/* we don't inherit needsseparatenode for new shards */
List *needsSeparateNodeForPlacementList =
GenerateListFromIntElement(false, list_length(nodeIdsForPlacementList));
DistributionColumnMap *distributionColumnOverrides = NULL;
List *sourceColocatedShardIntervalList = NIL;
SplitMode splitMode = LookupSplitMode(shardTransferModeOid);
@ -176,6 +180,7 @@ isolate_tenant_to_new_shard(PG_FUNCTION_ARGS)
sourceShard->shardId,
shardSplitPointsList,
nodeIdsForPlacementList,
needsSeparateNodeForPlacementList,
distributionColumnOverrides,
sourceColocatedShardIntervalList,
INVALID_COLOCATION_ID);

View File

@ -0,0 +1,433 @@
/*-------------------------------------------------------------------------
*
* rebalancer_placement_separation.c
* Routines to determine which worker node should be used to separate
* a colocated set of shard placements that need separate nodes.
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "nodes/pg_list.h"
#include "utils/hsearch.h"
#include "utils/lsyscache.h"
#include "distributed/colocation_utils.h"
#include "distributed/hash_helpers.h"
#include "distributed/listutils.h"
#include "distributed/metadata_cache.h"
#include "distributed/metadata_utility.h"
#include "distributed/multi_physical_planner.h"
#include "distributed/rebalancer_placement_separation.h"
#include "distributed/shard_rebalancer.h"
typedef struct RebalancerPlacementSeparationContext
{
/*
* Hash table where each entry is of the form NodeToPlacementGroupHashEntry,
* meaning that each entry maps the node with nodeGroupId to
* a NodeToPlacementGroupHashEntry.
*/
HTAB *nodePlacementGroupHash;
} RebalancerPlacementSeparationContext;
/*
* Entry of the hash table that maps each primary worker node to a shard
* placement group that is determined to be separated from other shards in
* the cluster via that node.
*/
typedef struct NodeToPlacementGroupHashEntry
{
/* hash key -- group id of the node */
int32 nodeGroupId;
/*
* Whether given node is allowed to have any shards.
*
* Inherited from WorkerNode->shouldHaveShards.
*/
bool shouldHaveShards;
/*
* Whether given node has some shard placements that cannot be moved away.
*
* For the nodes that this rebalancer-run is not allowed to move the
* placements away from, InitRebalancerPlacementSeparationContext() sets
* this to true if the node has some shard placements already. And if the
* node has a single shard placement group that needs a separate node, it
* also sets assignedPlacementGroup.
*
* We do so to prevent TryAssignPlacementGroupsToNodeGroups() making
* incorrect assignments later on.
*
* See InitRebalancerPlacementSeparationContext() for more details.
*/
bool hasPlacementsThatCannotBeMovedAway;
/*
* Shardgroup placement that is assigned to this node to be separated
* from others in the cluster.
*
* NULL if no shardgroup placement is not assigned yet.
*/
ShardgroupPlacement *assignedPlacementGroup;
} NodeToPlacementGroupHashEntry;
/*
* Routines to prepare RebalancerPlacementSeparationContext.
*/
static void InitRebalancerPlacementSeparationContext(
RebalancerPlacementSeparationContext *context,
List *activeWorkerNodeList,
List *rebalancePlacementList);
static void TryAssignPlacementGroupsToNodeGroups(
RebalancerPlacementSeparationContext *context,
List *activeWorkerNodeList,
List *rebalancePlacementList,
FmgrInfo shardAllowedOnNodeUDF);
static bool TryAssignPlacementGroupToNodeGroup(
RebalancerPlacementSeparationContext *context,
int32 candidateNodeGroupId,
ShardPlacement *shardPlacement,
FmgrInfo shardAllowedOnNodeUDF);
/* other helpers */
static List * LoadAllShardgroupPlacements(void);
static HTAB * ShardPlacementListToShardgroupPlacementSet(List *shardPlacementList);
/*
* PrepareRebalancerPlacementSeparationContext creates RebalancerPlacementSeparationContext
* that keeps track of which worker nodes are used to separate which shardgroup placements
* that need separate nodes.
*/
RebalancerPlacementSeparationContext *
PrepareRebalancerPlacementSeparationContext(List *activeWorkerNodeList,
List *rebalancePlacementList,
FmgrInfo shardAllowedOnNodeUDF)
{
HTAB *nodePlacementGroupHash =
CreateSimpleHashWithNameAndSize(int32, NodeToPlacementGroupHashEntry,
"NodeToPlacementGroupHash",
list_length(activeWorkerNodeList));
RebalancerPlacementSeparationContext *context =
palloc0(sizeof(RebalancerPlacementSeparationContext));
context->nodePlacementGroupHash = nodePlacementGroupHash;
activeWorkerNodeList = SortList(activeWorkerNodeList, CompareWorkerNodes);
rebalancePlacementList = SortList(rebalancePlacementList, CompareShardPlacements);
InitRebalancerPlacementSeparationContext(context, activeWorkerNodeList,
rebalancePlacementList);
TryAssignPlacementGroupsToNodeGroups(context,
activeWorkerNodeList,
rebalancePlacementList,
shardAllowedOnNodeUDF);
return context;
}
/*
* InitRebalancerPlacementSeparationContext initializes given
* RebalancerPlacementSeparationContext by using given list
* of worker nodes and the worker node that is being drained,
* if specified.
*/
static void
InitRebalancerPlacementSeparationContext(RebalancerPlacementSeparationContext *context,
List *activeWorkerNodeList,
List *rebalancePlacementList)
{
HTAB *nodePlacementGroupHash = context->nodePlacementGroupHash;
WorkerNode *workerNode = NULL;
foreach_ptr(workerNode, activeWorkerNodeList)
{
NodeToPlacementGroupHashEntry *nodePlacementGroupHashEntry =
hash_search(nodePlacementGroupHash, &workerNode->groupId, HASH_ENTER,
NULL);
nodePlacementGroupHashEntry->shouldHaveShards = workerNode->shouldHaveShards;
nodePlacementGroupHashEntry->hasPlacementsThatCannotBeMovedAway = false;
nodePlacementGroupHashEntry->assignedPlacementGroup = NULL;
if (!nodePlacementGroupHashEntry->shouldHaveShards)
{
continue;
}
nodePlacementGroupHashEntry->assignedPlacementGroup =
NodeGroupGetSeparatedShardgroupPlacement(
nodePlacementGroupHashEntry->nodeGroupId);
}
HTAB *balancingShardgroupPlacementsSet =
ShardPlacementListToShardgroupPlacementSet(rebalancePlacementList);
/* iterate over all shardgroups to find nodes that have shardgroups not balancing */
List *allShardgroupPlacements = LoadAllShardgroupPlacements();
ShardgroupPlacement *shardgroupPlacement = NULL;
foreach_ptr(shardgroupPlacement, allShardgroupPlacements)
{
bool found = false;
hash_search(balancingShardgroupPlacementsSet, shardgroupPlacement, HASH_FIND,
&found);
if (found)
{
/* we are balancing this shardgroup placement, skip */
continue;
}
/* we have a shardgroupPlacement we are not balancing, marking node as such */
NodeToPlacementGroupHashEntry *nodePlacementGroupHashEntry =
hash_search(nodePlacementGroupHash, &shardgroupPlacement->nodeGroupId,
HASH_ENTER, NULL);
nodePlacementGroupHashEntry->hasPlacementsThatCannotBeMovedAway = true;
}
}
/*
* TryAssignPlacementGroupsToNodeGroups tries to assign placements that need
* separate nodes within given placement list to individual worker nodes.
*/
static void
TryAssignPlacementGroupsToNodeGroups(RebalancerPlacementSeparationContext *context,
List *activeWorkerNodeList,
List *rebalancePlacementList,
FmgrInfo shardAllowedOnNodeUDF)
{
List *unassignedPlacementList = NIL;
/*
* Assign as much as possible shardgroup placements to worker nodes where
* they are stored already.
*/
ShardPlacement *shardPlacement = NULL;
foreach_ptr(shardPlacement, rebalancePlacementList)
{
ShardInterval *shardInterval = LoadShardInterval(shardPlacement->shardId);
if (!shardInterval->needsSeparateNode)
{
continue;
}
int32 currentNodeGroupId = shardPlacement->groupId;
if (!TryAssignPlacementGroupToNodeGroup(context,
currentNodeGroupId,
shardPlacement,
shardAllowedOnNodeUDF))
{
unassignedPlacementList =
lappend(unassignedPlacementList, shardPlacement);
}
}
bool emitWarning = false;
/*
* For the shardgroup placements that could not be assigned to their
* current node, assign them to any other node.
*/
ShardPlacement *unassignedShardPlacement = NULL;
foreach_ptr(unassignedShardPlacement, unassignedPlacementList)
{
bool separated = false;
WorkerNode *activeWorkerNode = NULL;
foreach_ptr(activeWorkerNode, activeWorkerNodeList)
{
if (TryAssignPlacementGroupToNodeGroup(context,
activeWorkerNode->groupId,
unassignedShardPlacement,
shardAllowedOnNodeUDF))
{
separated = true;
break;
}
}
if (!separated)
{
emitWarning = true;
}
}
if (emitWarning)
{
ereport(WARNING, (errmsg("could not separate all shard placements "
"that need a separate node")));
}
}
/*
* TryAssignPlacementGroupToNodeGroup is an helper to
* TryAssignPlacementGroupsToNodeGroups that tries to assign given
* shard placement to given node and returns true if it succeeds.
*/
static bool
TryAssignPlacementGroupToNodeGroup(RebalancerPlacementSeparationContext *context,
int32 candidateNodeGroupId,
ShardPlacement *shardPlacement,
FmgrInfo shardAllowedOnNodeUDF)
{
HTAB *nodePlacementGroupHash = context->nodePlacementGroupHash;
bool found = false;
NodeToPlacementGroupHashEntry *nodePlacementGroupHashEntry =
hash_search(nodePlacementGroupHash, &candidateNodeGroupId, HASH_FIND, &found);
if (!found)
{
ereport(ERROR, (errmsg("no such node is found")));
}
ShardgroupPlacement *shardgroupPlacement =
GetShardgroupPlacementForPlacement(shardPlacement->shardId,
shardPlacement->placementId);
if (nodePlacementGroupHashEntry->assignedPlacementGroup)
{
return ShardgroupPlacementsSame(shardgroupPlacement,
nodePlacementGroupHashEntry->
assignedPlacementGroup);
}
if (nodePlacementGroupHashEntry->hasPlacementsThatCannotBeMovedAway)
{
return false;
}
if (!nodePlacementGroupHashEntry->shouldHaveShards)
{
return false;
}
WorkerNode *workerNode = PrimaryNodeForGroup(candidateNodeGroupId, NULL);
Datum allowed = FunctionCall2(&shardAllowedOnNodeUDF, shardPlacement->shardId,
workerNode->nodeId);
if (!DatumGetBool(allowed))
{
return false;
}
nodePlacementGroupHashEntry->assignedPlacementGroup = shardgroupPlacement;
return true;
}
/*
* RebalancerPlacementSeparationContextPlacementIsAllowedOnWorker returns true
* if shard placement with given shardId & placementId is allowed to be stored
* on given worker node.
*/
bool
RebalancerPlacementSeparationContextPlacementIsAllowedOnWorker(
RebalancerPlacementSeparationContext *context,
uint64 shardId,
uint64 placementId,
WorkerNode *workerNode)
{
HTAB *nodePlacementGroupHash = context->nodePlacementGroupHash;
bool found = false;
NodeToPlacementGroupHashEntry *nodePlacementGroupHashEntry =
hash_search(nodePlacementGroupHash, &(workerNode->groupId), HASH_FIND, &found);
if (!found)
{
ereport(ERROR, (errmsg("no such node is found")));
}
ShardInterval *shardInterval = LoadShardInterval(shardId);
if (!shardInterval->needsSeparateNode)
{
/*
* It doesn't need a separate node, but is the node used to separate
* a shardgroup placement? If so, we cannot store it on this node.
*/
return nodePlacementGroupHashEntry->shouldHaveShards &&
nodePlacementGroupHashEntry->assignedPlacementGroup == NULL;
}
/*
* Given shard placement needs a separate node.
* Check if given worker node is the one that is assigned to separate it.
*/
if (nodePlacementGroupHashEntry->assignedPlacementGroup == NULL)
{
/* the node is not supposed to separate a placement group */
return false;
}
ShardgroupPlacement *placementGroup =
GetShardgroupPlacementForPlacement(shardId, placementId);
return ShardgroupPlacementsSame(nodePlacementGroupHashEntry->assignedPlacementGroup,
placementGroup);
}
/*
* LoadAllShardgroupPlacements loads all ShardgroupPlacements that belong
* to distributed tables in the cluster.
*/
static List *
LoadAllShardgroupPlacements(void)
{
List *shardgroupPlacementList = NIL;
List *relationIdList = NonColocatedDistRelationIdList();
Oid relationId = InvalidOid;
foreach_oid(relationId, relationIdList)
{
ArrayType *excludedShardArray = construct_empty_array(INT4OID);
List *shardPlacementList = FullShardPlacementList(relationId, excludedShardArray);
ShardPlacement *shardPlacement = NULL;
foreach_ptr(shardPlacement, shardPlacementList)
{
ShardgroupPlacement *shardgroupPlacement =
GetShardgroupPlacementForPlacement(shardPlacement->shardId,
shardPlacement->placementId);
shardgroupPlacementList = lappend(shardgroupPlacementList,
shardgroupPlacement);
}
}
return shardgroupPlacementList;
}
/*
* ShardPlacementListToShardgroupPlacementSet returns a hash set that contains
* all ShardgroupPlacements that are represented by given list of ShardPlacements.
*/
static HTAB *
ShardPlacementListToShardgroupPlacementSet(List *shardPlacementList)
{
HTAB *shardgroupPlacementSet = CreateSimpleHashSet(ShardgroupPlacement);
ShardPlacement *shardPlacement = NULL;
foreach_ptr(shardPlacement, shardPlacementList)
{
ShardgroupPlacement *findShardgroupPlacement =
GetShardgroupPlacementForPlacement(shardPlacement->shardId,
shardPlacement->placementId);
hash_search(shardgroupPlacementSet, findShardgroupPlacement, HASH_ENTER, NULL);
}
return shardgroupPlacementSet;
}

View File

@ -59,6 +59,7 @@
#include "distributed/multi_progress.h"
#include "distributed/multi_server_executor.h"
#include "distributed/pg_dist_rebalance_strategy.h"
#include "distributed/rebalancer_placement_separation.h"
#include "distributed/reference_table_utils.h"
#include "distributed/remote_commands.h"
#include "distributed/resource_lock.h"
@ -147,6 +148,8 @@ typedef struct RebalanceContext
FmgrInfo shardCostUDF;
FmgrInfo nodeCapacityUDF;
FmgrInfo shardAllowedOnNodeUDF;
RebalancerPlacementSeparationContext *shardgroupPlacementSeparationContext;
} RebalanceContext;
/* WorkerHashKey contains hostname and port to be used as a key in a hash */
@ -255,7 +258,8 @@ static bool FindAndMoveShardCost(float4 utilizationLowerBound,
float4 utilizationUpperBound,
float4 improvementThreshold,
RebalanceState *state);
static NodeFillState * FindAllowedTargetFillState(RebalanceState *state, uint64 shardId);
static NodeFillState * FindAllowedTargetFillState(RebalanceState *state, uint64 shardId,
uint64 placementId);
static void MoveShardCost(NodeFillState *sourceFillState, NodeFillState *targetFillState,
ShardCost *shardCost, RebalanceState *state);
static int CompareNodeFillStateAsc(const void *void1, const void *void2);
@ -264,10 +268,10 @@ static int CompareShardCostAsc(const void *void1, const void *void2);
static int CompareShardCostDesc(const void *void1, const void *void2);
static int CompareDisallowedPlacementAsc(const void *void1, const void *void2);
static int CompareDisallowedPlacementDesc(const void *void1, const void *void2);
static bool ShardAllowedOnNode(uint64 shardId, WorkerNode *workerNode, void *context);
static bool ShardAllowedOnNode(uint64 shardId, uint64 placementId, WorkerNode *workerNode,
void *context);
static float4 NodeCapacity(WorkerNode *workerNode, void *context);
static ShardCost GetShardCost(uint64 shardId, void *context);
static List * NonColocatedDistRelationIdList(void);
static ShardCost GetShardCost(uint64 shardId, uint64 placementId, void *context);
static void RebalanceTableShards(RebalanceOptions *options, Oid shardReplicationModeOid);
static int64 RebalanceTableShardsBackground(RebalanceOptions *options, Oid
shardReplicationModeOid);
@ -435,7 +439,7 @@ BigIntArrayDatumContains(Datum *array, int arrayLength, uint64 toFind)
* FullShardPlacementList returns a List containing all the shard placements of
* a specific table (excluding the excludedShardArray)
*/
static List *
List *
FullShardPlacementList(Oid relationId, ArrayType *excludedShardArray)
{
List *shardPlacementList = NIL;
@ -467,6 +471,9 @@ FullShardPlacementList(Oid relationId, ArrayType *excludedShardArray)
ShardPlacement *placement = CitusMakeNode(ShardPlacement);
placement->shardId = groupPlacement->shardId;
placement->shardLength = groupPlacement->shardLength;
placement->groupId = groupPlacement->groupId;
placement->colocationGroupId = citusTableCacheEntry->colocationId;
placement->partitionMethod = citusTableCacheEntry->partitionMethod;
placement->nodeId = worker->nodeId;
placement->nodeName = pstrdup(worker->workerName);
placement->nodePort = worker->workerPort;
@ -590,6 +597,12 @@ GetRebalanceSteps(RebalanceOptions *options)
options->threshold = options->rebalanceStrategy->minimumThreshold;
}
context.shardgroupPlacementSeparationContext =
PrepareRebalancerPlacementSeparationContext(
activeWorkerList,
FlattenNestedList(activeShardPlacementListList),
context.shardAllowedOnNodeUDF);
return RebalancePlacementUpdates(activeWorkerList,
activeShardPlacementListList,
options->threshold,
@ -604,7 +617,8 @@ GetRebalanceSteps(RebalanceOptions *options)
* ShardAllowedOnNode determines if shard is allowed on a specific worker node.
*/
static bool
ShardAllowedOnNode(uint64 shardId, WorkerNode *workerNode, void *voidContext)
ShardAllowedOnNode(uint64 shardId, uint64 placementId, WorkerNode *workerNode,
void *voidContext)
{
if (!workerNode->shouldHaveShards)
{
@ -612,6 +626,14 @@ ShardAllowedOnNode(uint64 shardId, WorkerNode *workerNode, void *voidContext)
}
RebalanceContext *context = voidContext;
if (!RebalancerPlacementSeparationContextPlacementIsAllowedOnWorker(
context->shardgroupPlacementSeparationContext,
shardId, placementId, workerNode))
{
return false;
}
Datum allowed = FunctionCall2(&context->shardAllowedOnNodeUDF, shardId,
workerNode->nodeId);
return DatumGetBool(allowed);
@ -645,10 +667,11 @@ NodeCapacity(WorkerNode *workerNode, void *voidContext)
* to be.
*/
static ShardCost
GetShardCost(uint64 shardId, void *voidContext)
GetShardCost(uint64 shardId, uint64 placementId, void *voidContext)
{
ShardCost shardCost = { 0 };
shardCost.shardId = shardId;
shardCost.placementId = placementId;
RebalanceContext *context = voidContext;
Datum shardCostDatum = FunctionCall1(&context->shardCostUDF, UInt64GetDatum(shardId));
shardCost.cost = DatumGetFloat4(shardCostDatum);
@ -794,9 +817,9 @@ AcquirePlacementColocationLock(Oid relationId, int lockMode,
ereport(ERROR, (errmsg("could not acquire the lock required to %s %s",
operationName,
generate_qualified_relation_name(relationId)),
errdetail("It means that either a concurrent shard move "
"or colocated distributed table creation is "
"happening."),
errdetail("It means that either a concurrent shard move, "
"colocated distributed table creation or "
"shard property change is happening."),
errhint("Make sure that the concurrent operation has "
"finished and re-run the command")));
}
@ -1828,7 +1851,7 @@ AddToWorkerShardIdSet(HTAB *shardsByWorker, char *workerName, int workerPort,
* NonColocatedDistRelationIdList returns a list of distributed table oids, one
* for each existing colocation group.
*/
static List *
List *
NonColocatedDistRelationIdList(void)
{
List *relationIdList = NIL;
@ -2562,7 +2585,8 @@ InitRebalanceState(List *workerNodeList, List *shardPlacementList,
Assert(fillState != NULL);
*shardCost = functions->shardCost(placement->shardId, functions->context);
*shardCost = functions->shardCost(placement->shardId, placement->placementId,
functions->context);
fillState->totalCost += shardCost->cost;
fillState->utilization = CalculateUtilization(fillState->totalCost,
@ -2574,8 +2598,8 @@ InitRebalanceState(List *workerNodeList, List *shardPlacementList,
state->totalCost += shardCost->cost;
if (!functions->shardAllowedOnNode(placement->shardId, fillState->node,
functions->context))
if (!functions->shardAllowedOnNode(placement->shardId, placement->placementId,
fillState->node, functions->context))
{
DisallowedPlacement *disallowed = palloc0(sizeof(DisallowedPlacement));
disallowed->shardCost = shardCost;
@ -2735,7 +2759,8 @@ MoveShardsAwayFromDisallowedNodes(RebalanceState *state)
foreach_ptr(disallowedPlacement, state->disallowedPlacementList)
{
NodeFillState *targetFillState = FindAllowedTargetFillState(
state, disallowedPlacement->shardCost->shardId);
state, disallowedPlacement->shardCost->shardId,
disallowedPlacement->shardCost->placementId);
if (targetFillState == NULL)
{
ereport(WARNING, (errmsg(
@ -2784,7 +2809,7 @@ CompareDisallowedPlacementDesc(const void *a, const void *b)
* where the shard can be moved to.
*/
static NodeFillState *
FindAllowedTargetFillState(RebalanceState *state, uint64 shardId)
FindAllowedTargetFillState(RebalanceState *state, uint64 shardId, uint64 placementId)
{
NodeFillState *targetFillState = NULL;
foreach_ptr(targetFillState, state->fillStateListAsc)
@ -2795,6 +2820,7 @@ FindAllowedTargetFillState(RebalanceState *state, uint64 shardId)
targetFillState->node);
if (!hasShard && state->functions->shardAllowedOnNode(
shardId,
placementId,
targetFillState->node,
state->functions->context))
{
@ -2969,6 +2995,7 @@ FindAndMoveShardCost(float4 utilizationLowerBound,
/* Skip shards that already are not allowed on the node */
if (!state->functions->shardAllowedOnNode(shardCost->shardId,
shardCost->placementId,
targetFillState->node,
state->functions->context))
{
@ -3165,7 +3192,7 @@ ReplicationPlacementUpdates(List *workerNodeList, List *activeShardPlacementList
{
WorkerNode *workerNode = list_nth(workerNodeList, workerNodeIndex);
if (!NodeCanHaveDistTablePlacements(workerNode))
if (!NodeCanBeUsedForNonSeparatedPlacements(workerNode))
{
/* never replicate placements to nodes that should not have placements */
continue;

View File

@ -95,21 +95,25 @@ static void CreateReplicaIdentitiesForDummyShards(HTAB *mapOfPlacementToDummySha
static void CreateObjectOnPlacement(List *objectCreationCommandList,
WorkerNode *workerNode);
static List * CreateSplitIntervalsForShardGroup(List *sourceColocatedShardList,
List *splitPointsForShard);
List *splitPointsForShard,
List *needsSeparateNodeForShardList);
static void CreateSplitIntervalsForShard(ShardInterval *sourceShard,
List *splitPointsForShard,
List *needsSeparateNodeForShardList,
List **shardSplitChildrenIntervalList);
static void BlockingShardSplit(SplitOperation splitOperation,
uint64 splitWorkflowId,
List *sourceColocatedShardIntervalList,
List *shardSplitPointsList,
List *workersForPlacementList,
List *needsSeparateNodeForShardList,
DistributionColumnMap *distributionColumnOverrides);
static void NonBlockingShardSplit(SplitOperation splitOperation,
uint64 splitWorkflowId,
List *sourceColocatedShardIntervalList,
List *shardSplitPointsList,
List *workersForPlacementList,
List *needsSeparateNodeForShardList,
DistributionColumnMap *distributionColumnOverrides,
uint32 targetColocationId);
static void DoSplitCopy(WorkerNode *sourceShardNode,
@ -427,6 +431,7 @@ GetWorkerNodesFromWorkerIds(List *nodeIdsForPlacementList)
* 'shardInterval' : Source shard interval to be split.
* 'shardSplitPointsList' : Split Points list for the source 'shardInterval'.
* 'nodeIdsForPlacementList' : Placement list corresponding to split children.
* 'needsSeparateNodeForShardList' : Whether each split children needs a separate node.
* 'distributionColumnOverrides' : Maps relation IDs to distribution columns.
* If not specified, the distribution column is read
* from the metadata.
@ -441,6 +446,7 @@ SplitShard(SplitMode splitMode,
uint64 shardIdToSplit,
List *shardSplitPointsList,
List *nodeIdsForPlacementList,
List *needsSeparateNodeForShardList,
DistributionColumnMap *distributionColumnOverrides,
List *colocatedShardIntervalList,
uint32 targetColocationId)
@ -514,6 +520,7 @@ SplitShard(SplitMode splitMode,
sourceColocatedShardIntervalList,
shardSplitPointsList,
workersForPlacementList,
needsSeparateNodeForShardList,
distributionColumnOverrides);
}
else
@ -526,6 +533,7 @@ SplitShard(SplitMode splitMode,
sourceColocatedShardIntervalList,
shardSplitPointsList,
workersForPlacementList,
needsSeparateNodeForShardList,
distributionColumnOverrides,
targetColocationId);
@ -547,6 +555,7 @@ SplitShard(SplitMode splitMode,
* sourceColocatedShardIntervalList : Source shard group to be split.
* shardSplitPointsList : Split Points list for the source 'shardInterval'.
* workersForPlacementList : Placement list corresponding to split children.
* needsSeparateNodeForShardList : Whether each split children needs a separate node.
*/
static void
BlockingShardSplit(SplitOperation splitOperation,
@ -554,6 +563,7 @@ BlockingShardSplit(SplitOperation splitOperation,
List *sourceColocatedShardIntervalList,
List *shardSplitPointsList,
List *workersForPlacementList,
List *needsSeparateNodeForShardList,
DistributionColumnMap *distributionColumnOverrides)
{
const char *operationName = SplitOperationAPIName[splitOperation];
@ -563,7 +573,8 @@ BlockingShardSplit(SplitOperation splitOperation,
/* First create shard interval metadata for split children */
List *shardGroupSplitIntervalListList = CreateSplitIntervalsForShardGroup(
sourceColocatedShardIntervalList,
shardSplitPointsList);
shardSplitPointsList,
needsSeparateNodeForShardList);
/* Only single placement allowed (already validated RelationReplicationFactor = 1) */
ShardInterval *firstShard = linitial(sourceColocatedShardIntervalList);
@ -1021,10 +1032,12 @@ CreateObjectOnPlacement(List *objectCreationCommandList,
* [ S1_1(-2147483648, 0), S1_2(1, 2147483647) ], // Split Interval List for S1.
* [ S2_1(-2147483648, 0), S2_2(1, 2147483647) ] // Split Interval List for S2.
* ]
* 'needsSeparateNodeForShardList': Whether each split children needs a separate node
*/
static List *
CreateSplitIntervalsForShardGroup(List *sourceColocatedShardIntervalList,
List *splitPointsForShard)
List *splitPointsForShard,
List *needsSeparateNodeForShardList)
{
List *shardGroupSplitIntervalListList = NIL;
@ -1033,6 +1046,7 @@ CreateSplitIntervalsForShardGroup(List *sourceColocatedShardIntervalList,
{
List *shardSplitIntervalList = NIL;
CreateSplitIntervalsForShard(shardToSplitInterval, splitPointsForShard,
needsSeparateNodeForShardList,
&shardSplitIntervalList);
shardGroupSplitIntervalListList = lappend(shardGroupSplitIntervalListList,
@ -1051,6 +1065,7 @@ CreateSplitIntervalsForShardGroup(List *sourceColocatedShardIntervalList,
static void
CreateSplitIntervalsForShard(ShardInterval *sourceShard,
List *splitPointsForShard,
List *needsSeparateNodeForShardList,
List **shardSplitChildrenIntervalList)
{
/* For 'N' split points, we will have N+1 shard intervals created. */
@ -1075,7 +1090,8 @@ CreateSplitIntervalsForShard(ShardInterval *sourceShard,
ShardInterval *splitChildShardInterval = CopyShardInterval(sourceShard);
splitChildShardInterval->shardIndex = -1;
splitChildShardInterval->shardId = GetNextShardIdForSplitChild();
splitChildShardInterval->needsSeparateNode =
list_nth_int(needsSeparateNodeForShardList, index);
splitChildShardInterval->minValueExists = true;
splitChildShardInterval->minValue = currentSplitChildMinValue;
splitChildShardInterval->maxValueExists = true;
@ -1177,7 +1193,8 @@ InsertSplitChildrenShardMetadata(List *shardGroupSplitIntervalListList,
shardInterval->shardId,
shardInterval->storageType,
IntegerToText(DatumGetInt32(shardInterval->minValue)),
IntegerToText(DatumGetInt32(shardInterval->maxValue)));
IntegerToText(DatumGetInt32(shardInterval->maxValue)),
shardInterval->needsSeparateNode);
InsertShardPlacementRow(
shardInterval->shardId,
@ -1373,6 +1390,7 @@ AcquireNonblockingSplitLock(Oid relationId)
* sourceColocatedShardIntervalList : Source shard group to be split.
* shardSplitPointsList : Split Points list for the source 'shardInterval'.
* workersForPlacementList : Placement list corresponding to split children.
* needsSeparateNodeForShardList : Whether each split children needs a separate node.
* distributionColumnList : Maps relation IDs to distribution columns.
* If not specified, the distribution column is read
* from the metadata.
@ -1385,6 +1403,7 @@ NonBlockingShardSplit(SplitOperation splitOperation,
List *sourceColocatedShardIntervalList,
List *shardSplitPointsList,
List *workersForPlacementList,
List *needsSeparateNodeForShardList,
DistributionColumnMap *distributionColumnOverrides,
uint32 targetColocationId)
{
@ -1398,7 +1417,8 @@ NonBlockingShardSplit(SplitOperation splitOperation,
/* First create shard interval metadata for split children */
List *shardGroupSplitIntervalListList = CreateSplitIntervalsForShardGroup(
sourceColocatedShardIntervalList,
shardSplitPointsList);
shardSplitPointsList,
needsSeparateNodeForShardList);
ShardInterval *firstShard = linitial(sourceColocatedShardIntervalList);

View File

@ -121,7 +121,6 @@ static void EnsureShardCanBeCopied(int64 shardId, const char *sourceNodeName,
int32 sourceNodePort, const char *targetNodeName,
int32 targetNodePort);
static List * RecreateTableDDLCommandList(Oid relationId);
static void EnsureTableListOwner(List *tableIdList);
static void ErrorIfReplicatingDistributedTableWithFKeys(List *tableIdList);
static void DropShardPlacementsFromMetadata(List *shardList,
@ -153,7 +152,6 @@ static bool TransferAlreadyCompleted(List *colocatedShardList,
char *sourceNodeName, uint32 sourceNodePort,
char *targetNodeName, uint32 targetNodePort,
ShardTransferType transferType);
static void LockColocatedRelationsForMove(List *colocatedTableList);
static void ErrorIfForeignTableForShardTransfer(List *colocatedTableList,
ShardTransferType transferType);
static List * RecreateShardDDLCommandList(ShardInterval *shardInterval,
@ -679,7 +677,7 @@ IsShardListOnNode(List *colocatedShardList, char *targetNodeName, uint32 targetN
* LockColocatedRelationsForMove takes a list of relations, locks all of them
* using ShareUpdateExclusiveLock
*/
static void
void
LockColocatedRelationsForMove(List *colocatedTableList)
{
Oid colocatedTableId = InvalidOid;
@ -1292,7 +1290,7 @@ LookupShardTransferMode(Oid shardReplicationModeOid)
* EnsureTableListOwner ensures current user owns given tables. Superusers
* are regarded as owners.
*/
static void
void
EnsureTableListOwner(List *tableIdList)
{
Oid tableId = InvalidOid;

View File

@ -166,7 +166,14 @@ master_create_empty_shard(PG_FUNCTION_ARGS)
uint64 shardId = GetNextShardId();
/* if enough live groups, add an extra candidate node as backup */
List *workerNodeList = DistributedTablePlacementNodeList(NoLock);
List *workerNodeList = NewDistributedTablePlacementNodeList(NoLock);
if (workerNodeList == NIL)
{
ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("no worker nodes are available for placing shards"),
errhint("Add more worker nodes.")));
}
if (list_length(workerNodeList) > ShardReplicationFactor)
{
@ -193,7 +200,9 @@ master_create_empty_shard(PG_FUNCTION_ARGS)
candidateNodeIndex++;
}
InsertShardRow(relationId, shardId, storageType, nullMinValue, nullMaxValue);
bool needsSeparateNode = false;
InsertShardRow(relationId, shardId, storageType, nullMinValue, nullMaxValue,
needsSeparateNode);
CreateAppendDistributedShardPlacements(relationId, shardId, candidateNodeList,
ShardReplicationFactor);

View File

@ -273,30 +273,36 @@ ErrorIfCoordinatorNotAddedAsWorkerNode()
/*
* DistributedTablePlacementNodeList returns a list of all active, primary
* NewDistributedTablePlacementNodeList returns a list of all active, primary
* worker nodes that can store new data, i.e shouldstoreshards is 'true'
* and that is not used to isolate a shardgroup placement.
*/
List *
DistributedTablePlacementNodeList(LOCKMODE lockMode)
NewDistributedTablePlacementNodeList(LOCKMODE lockMode)
{
EnsureModificationsCanRun();
return FilterActiveNodeListFunc(lockMode, NodeCanHaveDistTablePlacements);
return FilterActiveNodeListFunc(lockMode, NodeCanBeUsedForNonSeparatedPlacements);
}
/*
* NodeCanHaveDistTablePlacements returns true if the given node can have
* shards of a distributed table.
* NodeCanBeUsedForNonSeparatedPlacements returns true if given node can be
* used to store shard placements that don't need separate nodes.
*/
bool
NodeCanHaveDistTablePlacements(WorkerNode *node)
NodeCanBeUsedForNonSeparatedPlacements(WorkerNode *node)
{
if (!NodeIsPrimary(node))
{
return false;
}
return node->shouldHaveShards;
if (!node->shouldHaveShards)
{
return false;
}
return NodeGroupGetSeparatedShardgroupPlacement(node->groupId) == NULL;
}

View File

@ -4522,6 +4522,8 @@ GenerateSyntheticShardIntervalArray(int partitionCount)
shardInterval->shardId = INVALID_SHARD_ID;
shardInterval->valueTypeId = INT4OID;
shardInterval->needsSeparateNode = false;
shardIntervalArray[shardIndex] = shardInterval;
}

View File

@ -30,7 +30,6 @@ REVOKE ALL ON FUNCTION citus_internal.start_management_transaction FROM PUBLIC;
#include "udfs/citus_internal_add_object_metadata/12.2-1.sql"
#include "udfs/citus_internal_add_partition_metadata/12.2-1.sql"
#include "udfs/citus_internal_add_placement_metadata/12.2-1.sql"
#include "udfs/citus_internal_add_shard_metadata/12.2-1.sql"
#include "udfs/citus_internal_add_tenant_schema/12.2-1.sql"
#include "udfs/citus_internal_adjust_local_clock_to_remote/12.2-1.sql"
#include "udfs/citus_internal_delete_colocation_metadata/12.2-1.sql"
@ -52,3 +51,14 @@ DROP VIEW IF EXISTS pg_catalog.citus_lock_waits;
#include "udfs/citus_internal_update_placement_metadata/12.2-1.sql"
#include "udfs/citus_internal_update_relation_colocation/12.2-1.sql"
#include "udfs/repl_origin_helper/12.2-1.sql"
ALTER TABLE pg_dist_shard ADD COLUMN needsseparatenode boolean NOT NULL DEFAULT false;
DROP FUNCTION pg_catalog.citus_internal_add_shard_metadata(regclass, bigint, "char", text, text);
#include "udfs/citus_internal_add_shard_metadata/12.2-1.sql"
#include "udfs/citus_internal_shard_property_set/12.2-1.sql"
#include "udfs/citus_shard_property_set/12.2-1.sql"
DROP VIEW citus_shards;
#include "udfs/citus_shards/12.2-1.sql"

View File

@ -27,7 +27,6 @@ DROP FUNCTION citus_internal.add_colocation_metadata(int, int, int, regtype, oid
DROP FUNCTION citus_internal.add_object_metadata(text, text[], text[], integer, integer, boolean);
DROP FUNCTION citus_internal.add_partition_metadata(regclass, "char", text, integer, "char");
DROP FUNCTION citus_internal.add_placement_metadata(bigint, bigint, integer, bigint);
DROP FUNCTION citus_internal.add_shard_metadata(regclass, bigint, "char", text, text);
DROP FUNCTION citus_internal.add_tenant_schema(oid, integer);
DROP FUNCTION citus_internal.adjust_local_clock_to_remote(pg_catalog.cluster_clock);
DROP FUNCTION citus_internal.delete_colocation_metadata(int);
@ -51,3 +50,15 @@ DROP FUNCTION citus_internal.update_relation_colocation(oid, int);
DROP FUNCTION citus_internal.start_replication_origin_tracking();
DROP FUNCTION citus_internal.stop_replication_origin_tracking();
DROP FUNCTION citus_internal.is_replication_origin_tracking_active();
DROP VIEW pg_catalog.citus_shards;
#include "../udfs/citus_shards/12.0-1.sql"
DROP FUNCTION citus_internal.add_shard_metadata(regclass, bigint, "char", text, text, boolean);
DROP FUNCTION pg_catalog.citus_internal_add_shard_metadata(regclass, bigint, "char", text, text, boolean);
#include "../udfs/citus_internal_add_shard_metadata/10.2-1.sql"
DROP FUNCTION pg_catalog.citus_shard_property_set(shard_id bigint, anti_affinity boolean);
DROP FUNCTION citus_internal.shard_property_set(shard_id bigint, needs_separate_node boolean);
ALTER TABLE pg_dist_shard DROP COLUMN needsseparatenode;

View File

@ -1,21 +1,23 @@
CREATE OR REPLACE FUNCTION citus_internal.add_shard_metadata(
relation_id regclass, shard_id bigint,
storage_type "char", shard_min_value text,
shard_max_value text
shard_max_value text,
needs_separate_node boolean default false
)
RETURNS void
LANGUAGE C
AS 'MODULE_PATHNAME', $$citus_internal_add_shard_metadata$$;
COMMENT ON FUNCTION citus_internal.add_shard_metadata(regclass, bigint, "char", text, text) IS
COMMENT ON FUNCTION citus_internal.add_shard_metadata(regclass, bigint, "char", text, text, boolean) IS
'Inserts into pg_dist_shard with user checks';
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_add_shard_metadata(
relation_id regclass, shard_id bigint,
storage_type "char", shard_min_value text,
shard_max_value text
shard_max_value text,
needs_separate_node boolean default false
)
RETURNS void
LANGUAGE C
AS 'MODULE_PATHNAME';
COMMENT ON FUNCTION pg_catalog.citus_internal_add_shard_metadata(regclass, bigint, "char", text, text) IS
COMMENT ON FUNCTION pg_catalog.citus_internal_add_shard_metadata(regclass, bigint, "char", text, text, boolean) IS
'Inserts into pg_dist_shard with user checks';

View File

@ -1,21 +1,23 @@
CREATE OR REPLACE FUNCTION citus_internal.add_shard_metadata(
relation_id regclass, shard_id bigint,
storage_type "char", shard_min_value text,
shard_max_value text
shard_max_value text,
needs_separate_node boolean default false
)
RETURNS void
LANGUAGE C
AS 'MODULE_PATHNAME', $$citus_internal_add_shard_metadata$$;
COMMENT ON FUNCTION citus_internal.add_shard_metadata(regclass, bigint, "char", text, text) IS
COMMENT ON FUNCTION citus_internal.add_shard_metadata(regclass, bigint, "char", text, text, boolean) IS
'Inserts into pg_dist_shard with user checks';
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_add_shard_metadata(
relation_id regclass, shard_id bigint,
storage_type "char", shard_min_value text,
shard_max_value text
shard_max_value text,
needs_separate_node boolean default false
)
RETURNS void
LANGUAGE C
AS 'MODULE_PATHNAME';
COMMENT ON FUNCTION pg_catalog.citus_internal_add_shard_metadata(regclass, bigint, "char", text, text) IS
COMMENT ON FUNCTION pg_catalog.citus_internal_add_shard_metadata(regclass, bigint, "char", text, text, boolean) IS
'Inserts into pg_dist_shard with user checks';

View File

@ -0,0 +1,6 @@
CREATE OR REPLACE FUNCTION citus_internal.shard_property_set(
shard_id bigint,
needs_separate_node boolean)
RETURNS void
LANGUAGE C VOLATILE
AS 'MODULE_PATHNAME', $$citus_internal_shard_property_set$$;

View File

@ -0,0 +1,6 @@
CREATE OR REPLACE FUNCTION citus_internal.shard_property_set(
shard_id bigint,
needs_separate_node boolean)
RETURNS void
LANGUAGE C VOLATILE
AS 'MODULE_PATHNAME', $$citus_internal_shard_property_set$$;

View File

@ -0,0 +1,6 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_shard_property_set(shard_id bigint, anti_affinity boolean default null)
RETURNS void
LANGUAGE C VOLATILE
AS 'MODULE_PATHNAME', $$citus_shard_property_set$$;
COMMENT ON FUNCTION pg_catalog.citus_shard_property_set(bigint, boolean) IS
'Allows setting shard properties for all the shards within the shard group that given shard belongs to.';

View File

@ -0,0 +1,6 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_shard_property_set(shard_id bigint, anti_affinity boolean default null)
RETURNS void
LANGUAGE C VOLATILE
AS 'MODULE_PATHNAME', $$citus_shard_property_set$$;
COMMENT ON FUNCTION pg_catalog.citus_shard_property_set(bigint, boolean) IS
'Allows setting shard properties for all the shards within the shard group that given shard belongs to.';

View File

@ -0,0 +1,71 @@
CREATE OR REPLACE VIEW citus.citus_shards AS
SELECT
pg_dist_shard.logicalrelid AS table_name,
pg_dist_shard.shardid,
shard_name(pg_dist_shard.logicalrelid, pg_dist_shard.shardid) as shard_name,
CASE WHEN colocationid IN (SELECT colocationid FROM pg_dist_schema) THEN 'schema'
WHEN partkey IS NOT NULL THEN 'distributed'
WHEN repmodel = 't' THEN 'reference'
WHEN colocationid = 0 THEN 'local'
ELSE 'distributed' END AS citus_table_type,
colocationid AS colocation_id,
pg_dist_node.nodename,
pg_dist_node.nodeport,
size as shard_size,
CASE
WHEN NOT pg_dist_shard.needsseparatenode THEN false
ELSE
-- has_separate_node = true if the node doesn't have any other shard
-- placements except the ones that belong to the same shard group.
NOT EXISTS (
SELECT 1
FROM pg_dist_shard pds1
JOIN pg_dist_placement pdp1 USING (shardid)
JOIN pg_dist_partition pdp2 USING (logicalrelid)
WHERE
-- get the distributed table placements that are placed on the same node as this placement
pdp1.groupid = pg_dist_placement.groupid AND
(pdp2.partkey IS NOT NULL OR (pdp2.repmodel != 't' AND pdp2.colocationid != 0)) AND
-- filter out all the placements that belong to the same shard group
NOT (
pdp2.colocationid = pg_dist_partition.colocationid AND
((pds1.shardminvalue IS NULL AND pg_dist_shard.shardminvalue IS NULL) OR (pds1.shardminvalue = pg_dist_shard.shardminvalue))
)
)
END AS has_separate_node
FROM
pg_dist_shard
JOIN
pg_dist_placement
ON
pg_dist_shard.shardid = pg_dist_placement.shardid
JOIN
pg_dist_node
ON
pg_dist_placement.groupid = pg_dist_node.groupid
JOIN
pg_dist_partition
ON
pg_dist_partition.logicalrelid = pg_dist_shard.logicalrelid
LEFT JOIN
(SELECT shard_id, max(size) as size from citus_shard_sizes() GROUP BY shard_id) as shard_sizes
ON
pg_dist_shard.shardid = shard_sizes.shard_id
WHERE
pg_dist_placement.shardstate = 1
AND
-- filter out tables owned by extensions
pg_dist_partition.logicalrelid NOT IN (
SELECT
objid
FROM
pg_depend
WHERE
classid = 'pg_class'::regclass AND refclassid = 'pg_extension'::regclass AND deptype = 'e'
)
ORDER BY
pg_dist_shard.logicalrelid::text, shardid
;
ALTER VIEW citus.citus_shards SET SCHEMA pg_catalog;
GRANT SELECT ON pg_catalog.citus_shards TO public;

View File

@ -11,7 +11,28 @@ SELECT
colocationid AS colocation_id,
pg_dist_node.nodename,
pg_dist_node.nodeport,
size as shard_size
size as shard_size,
CASE
WHEN NOT pg_dist_shard.needsseparatenode THEN false
ELSE
-- has_separate_node = true if the node doesn't have any other shard
-- placements except the ones that belong to the same shard group.
NOT EXISTS (
SELECT 1
FROM pg_dist_shard pds1
JOIN pg_dist_placement pdp1 USING (shardid)
JOIN pg_dist_partition pdp2 USING (logicalrelid)
WHERE
-- get the distributed table placements that are placed on the same node as this placement
pdp1.groupid = pg_dist_placement.groupid AND
(pdp2.partkey IS NOT NULL OR (pdp2.repmodel != 't' AND pdp2.colocationid != 0)) AND
-- filter out all the placements that belong to the same shard group
NOT (
pdp2.colocationid = pg_dist_partition.colocationid AND
((pds1.shardminvalue IS NULL AND pg_dist_shard.shardminvalue IS NULL) OR (pds1.shardminvalue = pg_dist_shard.shardminvalue))
)
)
END AS has_separate_node
FROM
pg_dist_shard
JOIN

View File

@ -230,8 +230,9 @@ create_monolithic_shard_row(PG_FUNCTION_ARGS)
text *minInfoText = cstring_to_text(minInfo->data);
text *maxInfoText = cstring_to_text(maxInfo->data);
bool needsSeparateNode = false;
InsertShardRow(distributedTableId, newShardId, SHARD_STORAGE_TABLE, minInfoText,
maxInfoText);
maxInfoText, needsSeparateNode);
PG_RETURN_INT64(newShardId);
}

View File

@ -45,9 +45,10 @@ static uint64 JsonFieldValueUInt64Default(Datum jsonDocument, const char *key,
uint64 defaultValue);
static char * JsonFieldValueString(Datum jsonDocument, const char *key);
static ArrayType * PlacementUpdateListToJsonArray(List *placementUpdateList);
static bool ShardAllowedOnNode(uint64 shardId, WorkerNode *workerNode, void *context);
static bool ShardAllowedOnNode(uint64 shardId, uint64 placementId, WorkerNode *workerNode,
void *context);
static float NodeCapacity(WorkerNode *workerNode, void *context);
static ShardCost GetShardCost(uint64 shardId, void *context);
static ShardCost GetShardCost(uint64 shardId, uint64 placementId, void *context);
PG_FUNCTION_INFO_V1(shard_placement_rebalance_array);
@ -192,7 +193,8 @@ shard_placement_rebalance_array(PG_FUNCTION_ARGS)
* a worker when running the shard rebalancer unit tests.
*/
static bool
ShardAllowedOnNode(uint64 shardId, WorkerNode *workerNode, void *voidContext)
ShardAllowedOnNode(uint64 shardId, uint64 placementId, WorkerNode *workerNode,
void *voidContext)
{
RebalancePlacementContext *context = voidContext;
WorkerTestInfo *workerTestInfo = NULL;
@ -243,12 +245,13 @@ NodeCapacity(WorkerNode *workerNode, void *voidContext)
* the shard rebalancer unit tests.
*/
static ShardCost
GetShardCost(uint64 shardId, void *voidContext)
GetShardCost(uint64 shardId, uint64 placementId, void *voidContext)
{
RebalancePlacementContext *context = voidContext;
ShardCost shardCost;
memset_struct_0(shardCost);
shardCost.shardId = shardId;
shardCost.placementId = placementId;
ShardPlacementTestInfo *shardPlacementTestInfo = NULL;
foreach_ptr(shardPlacementTestInfo, context->shardPlacementTestInfoList)

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

@ -270,6 +270,23 @@ GenerateListFromElement(void *listElement, int listLength)
}
/*
* GenerateListFromIntElement returns a new list with length of listLength
* such that all the elements are identical with input listElement integer.
*/
List *
GenerateListFromIntElement(int listElement, int listLength)
{
List *list = NIL;
for (int i = 0; i < listLength; i++)
{
list = lappend_int(list, listElement);
}
return list;
}
/*
* list_filter_oid filters a list of oid-s based on a keepElement
* function
@ -289,3 +306,21 @@ list_filter_oid(List *list, bool (*keepElement)(Oid element))
return result;
}
/*
* FlattenNestedList takes a list of lists and returns a flattened list.
*/
List *
FlattenNestedList(List *nestedList)
{
List *flattenedList = NIL;
List *subList = NULL;
foreach_ptr(subList, nestedList)
{
flattenedList = list_concat(flattenedList, subList);
}
return flattenedList;
}

View File

@ -174,6 +174,8 @@ extern List * ListTake(List *pointerList, int size);
extern void * safe_list_nth(const List *list, int index);
extern List * GeneratePositiveIntSequenceList(int upTo);
extern List * GenerateListFromElement(void *listElement, int listLength);
extern List * GenerateListFromIntElement(int listElement, int listLength);
extern List * list_filter_oid(List *list, bool (*keepElement)(Oid element));
extern List * FlattenNestedList(List *nestedList);
#endif /* CITUS_LISTUTILS_H */

View File

@ -144,6 +144,7 @@ extern char * TenantSchemaInsertCommand(Oid schemaId, uint32 colocationId);
extern char * TenantSchemaDeleteCommand(char *schemaName);
extern char * UpdateNoneDistTableMetadataCommand(Oid relationId, char replicationModel,
uint32 colocationId, bool autoConverted);
extern char * ShardgroupSetPropertyCommand(uint64 shardId, bool *needsSeparateNodePtr);
extern char * AddPlacementMetadataCommand(uint64 shardId, uint64 placementId,
uint64 shardLength, int32 groupId);
extern char * DeletePlacementMetadataCommand(uint64 placementId);

View File

@ -68,6 +68,7 @@ typedef struct ShardInterval
Datum maxValue; /* a shard's typed max value datum */
uint64 shardId;
int shardIndex;
bool needsSeparateNode;
} ShardInterval;
@ -104,6 +105,14 @@ typedef struct ShardPlacement
} ShardPlacement;
typedef struct
{
uint32 colocatationId;
int shardIntervalIndex;
int32 nodeGroupId;
} ShardgroupPlacement;
typedef enum CascadeToColocatedOption
{
CASCADE_TO_COLOCATED_UNSPECIFIED,
@ -323,7 +332,13 @@ extern int ShardIntervalCount(Oid relationId);
extern List * LoadShardList(Oid relationId);
extern ShardInterval * CopyShardInterval(ShardInterval *srcInterval);
extern uint64 ShardLength(uint64 shardId);
extern ShardgroupPlacement * NodeGroupGetSeparatedShardgroupPlacement(int32 groupId);
extern bool ShardgroupPlacementsSame(const ShardgroupPlacement *leftGroup,
const ShardgroupPlacement *rightGroup);
extern bool NodeGroupHasShardPlacements(int32 groupId);
extern bool NodeGroupHasDistributedTableShardPlacements(int32 groupId);
extern ShardgroupPlacement * GetShardgroupPlacementForPlacement(uint64 shardId,
uint64 placementId);
extern bool IsActiveShardPlacement(ShardPlacement *ShardPlacement);
extern bool IsRemoteShardPlacement(ShardPlacement *shardPlacement);
extern bool IsPlacementOnWorkerNode(ShardPlacement *placement, WorkerNode *workerNode);
@ -350,7 +365,8 @@ extern List * RemoveCoordinatorPlacementIfNotSingleNode(List *placementList);
/* Function declarations to modify shard and shard placement data */
extern void InsertShardRow(Oid relationId, uint64 shardId, char storageType,
text *shardMinValue, text *shardMaxValue);
text *shardMinValue, text *shardMaxValue,
bool needsSeparateNode);
extern void DeleteShardRow(uint64 shardId);
extern ShardPlacement * InsertShardPlacementRowGlobally(uint64 shardId,
uint64 placementId,
@ -434,6 +450,7 @@ extern List * SendShardStatisticsQueriesInParallel(List *citusTableIds,
extern bool GetNodeDiskSpaceStatsForConnection(MultiConnection *connection,
uint64 *availableBytes,
uint64 *totalBytes);
extern void ShardgroupSetProperty(uint64 shardId, bool *needsSeparateNodePtr);
extern void ExecuteQueryViaSPI(char *query, int SPIOK);
extern void ExecuteAndLogQueryViaSPI(char *query, int SPIOK, int logLevel);
extern void EnsureSequenceTypeSupported(Oid seqOid, Oid attributeTypeId, Oid

View File

@ -30,6 +30,10 @@ typedef struct FormData_pg_dist_shard
text shardminvalue; /* partition key's minimum value in shard */
text shardmaxvalue; /* partition key's maximum value in shard */
#endif
bool needsSeparateNode; /*
* Whether the placements of this shard need to be
* separated from others.
*/
} FormData_pg_dist_shard;
/* ----------------
@ -43,13 +47,14 @@ typedef FormData_pg_dist_shard *Form_pg_dist_shard;
* compiler constants for pg_dist_shards
* ----------------
*/
#define Natts_pg_dist_shard 6
#define Natts_pg_dist_shard 7
#define Anum_pg_dist_shard_logicalrelid 1
#define Anum_pg_dist_shard_shardid 2
#define Anum_pg_dist_shard_shardstorage 3
#define Anum_pg_dist_shard_shardalias_DROPPED 4
#define Anum_pg_dist_shard_shardminvalue 5
#define Anum_pg_dist_shard_shardmaxvalue 6
#define Anum_pg_dist_shard_needsseparatenode 7
/*
* Valid values for shard storage types include foreign table, (standard) table

View File

@ -0,0 +1,37 @@
/*-------------------------------------------------------------------------
*
* rebalancer_placement_separation.h
* Routines to determine which worker node should be used to separate
* a colocated set of shard placements that need separate nodes.
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#ifndef PLACEMENT_ISOLATION_H
#define PLACEMENT_ISOLATION_H
#include "postgres.h"
#include "nodes/pg_list.h"
#include "utils/hsearch.h"
#include "distributed/metadata_utility.h"
struct RebalancerPlacementSeparationContext;
typedef struct RebalancerPlacementSeparationContext RebalancerPlacementSeparationContext;
/* *INDENT-OFF* */
extern RebalancerPlacementSeparationContext * PrepareRebalancerPlacementSeparationContext(
List *activeWorkerNodeList,
List *activeShardPlacementList,
FmgrInfo shardAllowedOnNodeUDF);
extern bool RebalancerPlacementSeparationContextPlacementIsAllowedOnWorker(
RebalancerPlacementSeparationContext *context,
uint64 shardId,
uint64 placementId,
WorkerNode *workerNode);
/* *INDENT-ON* */
#endif /* PLACEMENT_ISOLATION_H */

View File

@ -164,6 +164,7 @@ typedef struct NodeFillState
typedef struct ShardCost
{
uint64 shardId;
uint64 placementId;
/*
* cost is the cost of the shard. This doesn't have a unit.
@ -182,9 +183,10 @@ typedef struct DisallowedPlacement
typedef struct RebalancePlanFunctions
{
bool (*shardAllowedOnNode)(uint64 shardId, WorkerNode *workerNode, void *context);
bool (*shardAllowedOnNode)(uint64 shardId, uint64 placementId, WorkerNode *workerNode,
void *context);
float4 (*nodeCapacity)(WorkerNode *workerNode, void *context);
ShardCost (*shardCost)(uint64 shardId, void *context);
ShardCost (*shardCost)(uint64 shardId, uint64 placementId, void *context);
void *context;
} RebalancePlanFunctions;
@ -222,4 +224,7 @@ extern void SetupRebalanceMonitor(List *placementUpdateList,
uint64 initialProgressState,
PlacementUpdateStatus initialStatus);
extern List * NonColocatedDistRelationIdList(void);
extern List * FullShardPlacementList(Oid relationId, ArrayType *excludedShardArray);
#endif /* SHARD_REBALANCER_H */

View File

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

View File

@ -30,6 +30,7 @@ extern void TransferShards(int64 shardId,
extern uint64 ShardListSizeInBytes(List *colocatedShardList,
char *workerNodeName, uint32 workerNodePort);
extern void ErrorIfMoveUnsupportedTableType(Oid relationId);
extern void EnsureTableListOwner(List *tableIdList);
extern void CopyShardsToNode(WorkerNode *sourceNode, WorkerNode *targetNode,
List *shardIntervalList, char *snapshotName);
extern void VerifyTablesHaveReplicaIdentity(List *colocatedTableList);
@ -41,3 +42,4 @@ extern void UpdatePlacementUpdateStatusForShardIntervalList(List *shardIntervalL
extern void InsertDeferredDropCleanupRecordsForShards(List *shardIntervalList);
extern void InsertCleanupRecordsForShardPlacementsOnNode(List *shardIntervalList,
int32 groupId);
extern void LockColocatedRelationsForMove(List *colocatedTableList);

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 NodeCanBeUsedForNonSeparatedPlacements(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

@ -157,6 +157,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, ["multi_test_helpers", "multi_cluster_management", "multi_table_ddl"]
),

View File

@ -21,16 +21,16 @@ SELECT create_distributed_table('"t with space"', 'i');
INSERT INTO t1 SELECT generate_series(1, 100);
INSERT INTO "t with space" SELECT generate_series(1, 1000);
SELECT * FROM citus_shards;
table_name | shardid | shard_name | citus_table_type | colocation_id | nodename | nodeport | shard_size
table_name | shardid | shard_name | citus_table_type | colocation_id | nodename | nodeport | shard_size | has_separate_node
---------------------------------------------------------------------
"t with space" | 99456904 | citus_shards."t with space_99456904" | distributed | 456900 | localhost | 57637 | 40960
"t with space" | 99456905 | citus_shards."t with space_99456905" | distributed | 456900 | localhost | 57638 | 40960
"t with space" | 99456906 | citus_shards."t with space_99456906" | distributed | 456900 | localhost | 57637 | 40960
"t with space" | 99456907 | citus_shards."t with space_99456907" | distributed | 456900 | localhost | 57638 | 40960
t1 | 99456900 | citus_shards.t1_99456900 | distributed | 456900 | localhost | 57637 | 8192
t1 | 99456901 | citus_shards.t1_99456901 | distributed | 456900 | localhost | 57638 | 8192
t1 | 99456902 | citus_shards.t1_99456902 | distributed | 456900 | localhost | 57637 | 8192
t1 | 99456903 | citus_shards.t1_99456903 | distributed | 456900 | localhost | 57638 | 8192
"t with space" | 99456904 | citus_shards."t with space_99456904" | distributed | 456900 | localhost | 57637 | 40960 | f
"t with space" | 99456905 | citus_shards."t with space_99456905" | distributed | 456900 | localhost | 57638 | 40960 | f
"t with space" | 99456906 | citus_shards."t with space_99456906" | distributed | 456900 | localhost | 57637 | 40960 | f
"t with space" | 99456907 | citus_shards."t with space_99456907" | distributed | 456900 | localhost | 57638 | 40960 | f
t1 | 99456900 | citus_shards.t1_99456900 | distributed | 456900 | localhost | 57637 | 8192 | f
t1 | 99456901 | citus_shards.t1_99456901 | distributed | 456900 | localhost | 57638 | 8192 | f
t1 | 99456902 | citus_shards.t1_99456902 | distributed | 456900 | localhost | 57637 | 8192 | f
t1 | 99456903 | citus_shards.t1_99456903 | distributed | 456900 | localhost | 57638 | 8192 | f
(8 rows)
SET client_min_messages TO WARNING;

View File

@ -189,10 +189,10 @@ SELECT create_distributed_table_concurrently('table_1', 'id');
(1 row)
SELECT * FROM pg_dist_shard WHERE logicalrelid = 'table_1'::regclass;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode
---------------------------------------------------------------------
table_1 | 222247 | t | -2147483648 | -1
table_1 | 222248 | t | 0 | 2147483647
table_1 | 222247 | t | -2147483648 | -1 | f
table_1 | 222248 | t | 0 | 2147483647 | f
(2 rows)
DROP SCHEMA create_dist_tbl_con CASCADE;

File diff suppressed because it is too large Load Diff

View File

@ -108,3 +108,75 @@ ERROR: table "table_to_distribute" is already distributed
step s2-commit:
COMMIT;
starting permutation: s1-create_distributed_table s1-begin s2-begin s1_set-shard-property s2-create_distributed_table_colocated s1-rollback s2-commit
step s1-create_distributed_table:
SELECT create_distributed_table('table_to_distribute', 'id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1_set-shard-property:
SELECT citus_shard_property_set(shardid, anti_affinity=>'true')
FROM pg_dist_shard WHERE logicalrelid = 'table_to_distribute'::regclass
ORDER BY shardid LIMIT 1;
citus_shard_property_set
---------------------------------------------------------------------
(1 row)
step s2-create_distributed_table_colocated:
SELECT create_distributed_table('table_to_colocate', 'id', colocate_with=>'table_to_distribute');
ERROR: could not acquire the lock required to colocate distributed table public.table_to_distribute
step s1-rollback:
ROLLBACK;
step s2-commit:
COMMIT;
starting permutation: s1-create_distributed_table s1-begin s2-begin s2-create_distributed_table_colocated s1_set-shard-property s1-rollback s2-commit
step s1-create_distributed_table:
SELECT create_distributed_table('table_to_distribute', 'id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-create_distributed_table_colocated:
SELECT create_distributed_table('table_to_colocate', 'id', colocate_with=>'table_to_distribute');
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1_set-shard-property:
SELECT citus_shard_property_set(shardid, anti_affinity=>'true')
FROM pg_dist_shard WHERE logicalrelid = 'table_to_distribute'::regclass
ORDER BY shardid LIMIT 1;
ERROR: could not acquire the lock required to set a property for a shard of public.table_to_distribute
step s1-rollback:
ROLLBACK;
step s2-commit:
COMMIT;

View File

@ -57,12 +57,12 @@ step s2-print-status:
-- sanity check on total elements in the table
SELECT COUNT(*) FROM table_1;
logicalrelid|shardid|shardstorage|shardminvalue|shardmaxvalue
logicalrelid|shardid|shardstorage|shardminvalue|shardmaxvalue|needsseparatenode
---------------------------------------------------------------------
table_1 |1400294|t | -2147483648| -1073741825
table_1 |1400295|t | -1073741824| -1
table_1 |1400296|t | 0| 1073741823
table_1 |1400297|t | 1073741824| 2147483647
table_1 |1400306|t | -2147483648| -1073741825|f
table_1 |1400307|t | -1073741824| -1|f
table_1 |1400308|t | 0| 1073741823|f
table_1 |1400309|t | 1073741824| 2147483647|f
(4 rows)
count
@ -125,12 +125,12 @@ step s2-print-status:
-- sanity check on total elements in the table
SELECT COUNT(*) FROM table_1;
logicalrelid|shardid|shardstorage|shardminvalue|shardmaxvalue
logicalrelid|shardid|shardstorage|shardminvalue|shardmaxvalue|needsseparatenode
---------------------------------------------------------------------
table_1 |1400299|t | -2147483648| -1073741825
table_1 |1400300|t | -1073741824| -1
table_1 |1400301|t | 0| 1073741823
table_1 |1400302|t | 1073741824| 2147483647
table_1 |1400311|t | -2147483648| -1073741825|f
table_1 |1400312|t | -1073741824| -1|f
table_1 |1400313|t | 0| 1073741823|f
table_1 |1400314|t | 1073741824| 2147483647|f
(4 rows)
count
@ -193,12 +193,12 @@ step s2-print-status:
-- sanity check on total elements in the table
SELECT COUNT(*) FROM table_1;
logicalrelid|shardid|shardstorage|shardminvalue|shardmaxvalue
logicalrelid|shardid|shardstorage|shardminvalue|shardmaxvalue|needsseparatenode
---------------------------------------------------------------------
table_1 |1400304|t | -2147483648| -1073741825
table_1 |1400305|t | -1073741824| -1
table_1 |1400306|t | 0| 1073741823
table_1 |1400307|t | 1073741824| 2147483647
table_1 |1400316|t | -2147483648| -1073741825|f
table_1 |1400317|t | -1073741824| -1|f
table_1 |1400318|t | 0| 1073741823|f
table_1 |1400319|t | 1073741824| 2147483647|f
(4 rows)
count
@ -261,12 +261,12 @@ step s2-print-status:
-- sanity check on total elements in the table
SELECT COUNT(*) FROM table_1;
logicalrelid|shardid|shardstorage|shardminvalue|shardmaxvalue
logicalrelid|shardid|shardstorage|shardminvalue|shardmaxvalue|needsseparatenode
---------------------------------------------------------------------
table_1 |1400309|t | -2147483648| -1073741825
table_1 |1400310|t | -1073741824| -1
table_1 |1400311|t | 0| 1073741823
table_1 |1400312|t | 1073741824| 2147483647
table_1 |1400321|t | -2147483648| -1073741825|f
table_1 |1400322|t | -1073741824| -1|f
table_1 |1400323|t | 0| 1073741823|f
table_1 |1400324|t | 1073741824| 2147483647|f
(4 rows)
count

View File

@ -470,7 +470,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('super_user_table'::regclass, 1420000::bigint, 't'::"char", '-2147483648'::text, '-1610612737'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
ERROR: must be owner of table super_user_table
ROLLBACK;
-- the user is only allowed to add a shard for add a table which is in pg_dist_partition
@ -485,7 +485,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '-2147483648'::text, '-1610612737'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
ERROR: The relation "test_2" does not have a valid entry in pg_dist_partition.
ROLLBACK;
-- ok, now add the table to the pg_dist_partition
@ -544,7 +544,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, -1, 't'::"char", '-2147483648'::text, '-1610612737'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
ERROR: Invalid shard id: -1
ROLLBACK;
-- invalid storage types are not allowed
@ -559,7 +559,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, 1420000, 'X'::"char", '-2147483648'::text, '-1610612737'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
ERROR: Invalid shard storage type: X
ROLLBACK;
-- NULL shard ranges are not allowed for hash distributed tables
@ -574,7 +574,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, 1420000, 't'::"char", NULL, '-1610612737'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
ERROR: Shards of has distributed table "test_2" cannot have NULL shard ranges
ROLLBACK;
-- non-integer shard ranges are not allowed
@ -589,7 +589,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", 'non-int'::text, '-1610612737'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
ERROR: invalid input syntax for type integer: "non-int"
ROLLBACK;
-- shardMinValue should be smaller than shardMaxValue
@ -604,7 +604,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '-1610612737'::text, '-2147483648'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
ERROR: shardMinValue=-1610612737 is greater than shardMaxValue=-2147483648 for table "test_2", which is not allowed
ROLLBACK;
-- we do not allow overlapping shards for the same table
@ -621,7 +621,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '10'::text, '20'::text),
('test_2'::regclass, 1420001::bigint, 't'::"char", '20'::text, '30'::text),
('test_2'::regclass, 1420002::bigint, 't'::"char", '10'::text, '50'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
ERROR: Shard intervals overlap for table "test_2": 1420001 and 1420000
ROLLBACK;
-- Now let's check valid pg_dist_object updates
@ -780,7 +780,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '10'::text, '20'::text),
('test_2'::regclass, 1420001::bigint, 't'::"char", '20'::text, '30'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
ERROR: Metadata syncing is only allowed for hash, reference and local tables: X
ROLLBACK;
-- we do not allow NULL shardMinMax values
@ -797,7 +797,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '10'::text, '20'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
add_shard_metadata
---------------------------------------------------------------------
@ -807,7 +807,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
UPDATE pg_dist_shard SET shardminvalue = NULL WHERE shardid = 1420000;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, 1420001::bigint, 't'::"char", '20'::text, '30'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
ERROR: Shards of has distributed table "test_2" cannot have NULL shard ranges
ROLLBACK;
\c - metadata_sync_helper_role - :worker_1_port
@ -830,7 +830,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
('test_2'::regclass, 1420004::bigint, 't'::"char", '51'::text, '60'::text),
('test_2'::regclass, 1420005::bigint, 't'::"char", '61'::text, '70'::text),
('test_3'::regclass, 1420008::bigint, 't'::"char", '11'::text, '20'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
add_shard_metadata
---------------------------------------------------------------------
@ -871,7 +871,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
('test_3'::regclass, 1420011::bigint, 't'::"char", '41'::text, '50'::text),
('test_3'::regclass, 1420012::bigint, 't'::"char", '51'::text, '60'::text),
('test_3'::regclass, 1420013::bigint, 't'::"char", '61'::text, '70'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
add_shard_metadata
---------------------------------------------------------------------
@ -894,7 +894,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_ref'::regclass, 1420003::bigint, 't'::"char", '-1610612737'::text, NULL))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
ERROR: Shards of reference or local table "test_ref" should have NULL shard ranges
ROLLBACK;
-- reference tables cannot have multiple shards
@ -910,7 +910,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_ref'::regclass, 1420006::bigint, 't'::"char", NULL, NULL),
('test_ref'::regclass, 1420007::bigint, 't'::"char", NULL, NULL))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
ERROR: relation "test_ref" has already at least one shard, adding more is not allowed
ROLLBACK;
-- finally, add a shard for reference tables
@ -925,7 +925,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_ref'::regclass, 1420006::bigint, 't'::"char", NULL, NULL))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
add_shard_metadata
---------------------------------------------------------------------
@ -946,7 +946,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('super_user_table'::regclass, 1420007::bigint, 't'::"char", '11'::text, '20'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
add_shard_metadata
---------------------------------------------------------------------

View File

@ -68,7 +68,7 @@ ERROR: 0 is outside the valid range for parameter "citus.shard_replication_fact
-- use a replication factor higher than shard count
SET citus.shard_replication_factor TO 3;
SELECT create_distributed_table('table_to_distribute', 'name', 'hash');
ERROR: replication_factor (3) exceeds number of worker nodes (2)
ERROR: replication_factor (3) exceeds number of available worker nodes (2)
HINT: Add more worker nodes or try again with a lower replication factor.
RESET citus.shard_replication_factor;
-- finally, create shards and inspect metadata

View File

@ -1422,12 +1422,13 @@ ALTER EXTENSION citus UPDATE TO '12.2-1';
SELECT * FROM multi_extension.print_extension_changes();
previous_object | current_object
---------------------------------------------------------------------
function citus_internal_add_shard_metadata(regclass,bigint,"char",text,text) void |
| function citus_internal.acquire_citus_advisory_object_class_lock(integer,cstring) void
| function citus_internal.add_colocation_metadata(integer,integer,integer,regtype,oid) void
| function citus_internal.add_object_metadata(text,text[],text[],integer,integer,boolean) void
| function citus_internal.add_partition_metadata(regclass,"char",text,integer,"char") void
| function citus_internal.add_placement_metadata(bigint,bigint,integer,bigint) void
| function citus_internal.add_shard_metadata(regclass,bigint,"char",text,text) void
| function citus_internal.add_shard_metadata(regclass,bigint,"char",text,text,boolean) void
| function citus_internal.add_tenant_schema(oid,integer) void
| function citus_internal.adjust_local_clock_to_remote(cluster_clock) void
| function citus_internal.commit_management_command_2pc() void
@ -1443,6 +1444,7 @@ SELECT * FROM multi_extension.print_extension_changes();
| function citus_internal.local_blocked_processes() SETOF record
| function citus_internal.mark_node_not_synced(integer,integer) void
| function citus_internal.mark_object_distributed(oid,text,oid,text) void
| function citus_internal.shard_property_set(bigint,boolean) void
| function citus_internal.start_management_transaction(xid8) void
| function citus_internal.start_replication_origin_tracking() void
| function citus_internal.stop_replication_origin_tracking() void
@ -1450,7 +1452,9 @@ SELECT * FROM multi_extension.print_extension_changes();
| function citus_internal.update_none_dist_table_metadata(oid,"char",bigint,boolean) void
| function citus_internal.update_placement_metadata(bigint,integer,integer) void
| function citus_internal.update_relation_colocation(oid,integer) void
(28 rows)
| function citus_internal_add_shard_metadata(regclass,bigint,"char",text,text,boolean) void
| function citus_shard_property_set(bigint,boolean) void
(32 rows)
DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff;
-- show running version

View File

@ -696,9 +696,9 @@ NOTICE: issuing SELECT citus_internal.add_partition_metadata ('fix_idx_names.p2
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing SELECT citus_internal.add_partition_metadata ('fix_idx_names.p2'::regclass, 'h', 'dist_col', 1370001, 's')
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('fix_idx_names.p2'::regclass, 915002, 't'::"char", '-2147483648', '2147483647')) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
NOTICE: issuing WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('fix_idx_names.p2'::regclass, 915002, 't'::"char", '-2147483648', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data;
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('fix_idx_names.p2'::regclass, 915002, 't'::"char", '-2147483648', '2147483647')) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
NOTICE: issuing WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('fix_idx_names.p2'::regclass, 915002, 't'::"char", '-2147483648', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data;
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (xxxxxx, xxxxxx, xxxxxx, xxxxxx)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx

View File

@ -45,12 +45,12 @@ select * from pg_dist_shard
where logicalrelid='lineitem'::regclass or
logicalrelid='orders'::regclass
order by shardid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode
---------------------------------------------------------------------
lineitem | 360000 | t | -2147483648 | -1
lineitem | 360001 | t | 0 | 2147483647
orders | 360002 | t | -2147483648 | -1
orders | 360003 | t | 0 | 2147483647
lineitem | 360000 | t | -2147483648 | -1 | f
lineitem | 360001 | t | 0 | 2147483647 | f
orders | 360002 | t | -2147483648 | -1 | f
orders | 360003 | t | 0 | 2147483647 | f
(4 rows)
set client_min_messages to debug3;

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

View File

@ -194,8 +194,8 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['public', 'single_shard_tbl']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal.add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310008, 0, 2, 100008)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.single_shard_tbl'::regclass, 1310008, 't'::"char", NULL, NULL)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737', false), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825', false), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913', false), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1', false), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911', false), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823', false), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735', false), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('public.single_shard_tbl'::regclass, 1310008, 't'::"char", NULL, NULL, false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data;
(61 rows)
-- Drop single shard table
@ -256,7 +256,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal.add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['public', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal.add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737', false), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825', false), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913', false), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1', false), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911', false), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823', false), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735', false), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data;
(52 rows)
-- Show that schema changes are included in the activate node snapshot
@ -318,7 +318,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal.add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['mx_testing_schema', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal.add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737', false), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825', false), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913', false), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1', false), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911', false), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823', false), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735', false), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data;
(54 rows)
-- Show that append distributed tables are not included in the activate node snapshot
@ -386,7 +386,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal.add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['mx_testing_schema', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal.add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737', false), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825', false), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913', false), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1', false), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911', false), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823', false), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735', false), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data;
(54 rows)
-- Show that range distributed tables are not included in the activate node snapshot
@ -447,7 +447,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal.add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['mx_testing_schema', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal.add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737', false), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825', false), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913', false), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1', false), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911', false), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823', false), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735', false), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data;
(54 rows)
-- Test start_metadata_sync_to_node and citus_activate_node UDFs
@ -537,16 +537,16 @@ SELECT * FROM pg_dist_partition WHERE logicalrelid::text LIKE 'mx_testing_schema
(1 row)
SELECT * FROM pg_dist_shard WHERE logicalrelid::text LIKE 'mx_testing_schema%' ORDER BY shardid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode
---------------------------------------------------------------------
mx_testing_schema.mx_test_table | 1310000 | t | -2147483648 | -1610612737
mx_testing_schema.mx_test_table | 1310001 | t | -1610612736 | -1073741825
mx_testing_schema.mx_test_table | 1310002 | t | -1073741824 | -536870913
mx_testing_schema.mx_test_table | 1310003 | t | -536870912 | -1
mx_testing_schema.mx_test_table | 1310004 | t | 0 | 536870911
mx_testing_schema.mx_test_table | 1310005 | t | 536870912 | 1073741823
mx_testing_schema.mx_test_table | 1310006 | t | 1073741824 | 1610612735
mx_testing_schema.mx_test_table | 1310007 | t | 1610612736 | 2147483647
mx_testing_schema.mx_test_table | 1310000 | t | -2147483648 | -1610612737 | f
mx_testing_schema.mx_test_table | 1310001 | t | -1610612736 | -1073741825 | f
mx_testing_schema.mx_test_table | 1310002 | t | -1073741824 | -536870913 | f
mx_testing_schema.mx_test_table | 1310003 | t | -536870912 | -1 | f
mx_testing_schema.mx_test_table | 1310004 | t | 0 | 536870911 | f
mx_testing_schema.mx_test_table | 1310005 | t | 536870912 | 1073741823 | f
mx_testing_schema.mx_test_table | 1310006 | t | 1073741824 | 1610612735 | f
mx_testing_schema.mx_test_table | 1310007 | t | 1610612736 | 2147483647 | f
(8 rows)
SELECT * FROM pg_dist_shard_placement WHERE shardid IN (SELECT shardid FROM pg_dist_shard WHERE logicalrelid::text LIKE 'mx_testing_schema%') ORDER BY shardid, nodename, nodeport;
@ -676,16 +676,16 @@ SELECT * FROM pg_dist_partition WHERE logicalrelid::text LIKE 'mx_testing_schema
(1 row)
SELECT * FROM pg_dist_shard WHERE logicalrelid::text LIKE 'mx_testing_schema%' ORDER BY shardid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode
---------------------------------------------------------------------
mx_testing_schema.mx_test_table | 1310000 | t | -2147483648 | -1610612737
mx_testing_schema.mx_test_table | 1310001 | t | -1610612736 | -1073741825
mx_testing_schema.mx_test_table | 1310002 | t | -1073741824 | -536870913
mx_testing_schema.mx_test_table | 1310003 | t | -536870912 | -1
mx_testing_schema.mx_test_table | 1310004 | t | 0 | 536870911
mx_testing_schema.mx_test_table | 1310005 | t | 536870912 | 1073741823
mx_testing_schema.mx_test_table | 1310006 | t | 1073741824 | 1610612735
mx_testing_schema.mx_test_table | 1310007 | t | 1610612736 | 2147483647
mx_testing_schema.mx_test_table | 1310000 | t | -2147483648 | -1610612737 | f
mx_testing_schema.mx_test_table | 1310001 | t | -1610612736 | -1073741825 | f
mx_testing_schema.mx_test_table | 1310002 | t | -1073741824 | -536870913 | f
mx_testing_schema.mx_test_table | 1310003 | t | -536870912 | -1 | f
mx_testing_schema.mx_test_table | 1310004 | t | 0 | 536870911 | f
mx_testing_schema.mx_test_table | 1310005 | t | 536870912 | 1073741823 | f
mx_testing_schema.mx_test_table | 1310006 | t | 1073741824 | 1610612735 | f
mx_testing_schema.mx_test_table | 1310007 | t | 1610612736 | 2147483647 | f
(8 rows)
SELECT * FROM pg_dist_shard_placement WHERE shardid IN (SELECT shardid FROM pg_dist_shard WHERE logicalrelid::text LIKE 'mx_testing_schema%') ORDER BY shardid, nodename, nodeport;
@ -1021,7 +1021,7 @@ SELECT * FROM pg_dist_partition WHERE logicalrelid::text LIKE 'mx_test_schema%';
(0 rows)
SELECT * FROM pg_dist_shard WHERE logicalrelid::text LIKE 'mx_test_schema%';
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode
---------------------------------------------------------------------
(0 rows)
@ -1681,7 +1681,7 @@ SELECT "Column", "Type", "Definition" FROM index_attrs WHERE
relid = 'mx_ref_index'::regclass;
ERROR: relation "mx_ref_index" does not exist
SELECT * FROM pg_dist_shard WHERE shardid=:ref_table_shardid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode
---------------------------------------------------------------------
(0 rows)
@ -2056,12 +2056,12 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310075, 0, 0, 100077), (1310075, 0, 1, 100078), (1310075, 0, 5, 100079)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310076, 0, 1, 100080), (1310077, 0, 5, 100081), (1310078, 0, 1, 100082), (1310079, 0, 5, 100083)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310085, 0, 1, 100091), (1310086, 0, 5, 100092), (1310087, 0, 1, 100093), (1310088, 0, 5, 100094)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_test_schema_1.mx_table_1'::regclass, 1310022, 't'::"char", '-2147483648', '-1288490190'), ('mx_test_schema_1.mx_table_1'::regclass, 1310023, 't'::"char", '-1288490189', '-429496731'), ('mx_test_schema_1.mx_table_1'::regclass, 1310024, 't'::"char", '-429496730', '429496728'), ('mx_test_schema_1.mx_table_1'::regclass, 1310025, 't'::"char", '429496729', '1288490187'), ('mx_test_schema_1.mx_table_1'::regclass, 1310026, 't'::"char", '1288490188', '2147483647')) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_test_schema_2.mx_table_2'::regclass, 1310027, 't'::"char", '-2147483648', '-1288490190'), ('mx_test_schema_2.mx_table_2'::regclass, 1310028, 't'::"char", '-1288490189', '-429496731'), ('mx_test_schema_2.mx_table_2'::regclass, 1310029, 't'::"char", '-429496730', '429496728'), ('mx_test_schema_2.mx_table_2'::regclass, 1310030, 't'::"char", '429496729', '1288490187'), ('mx_test_schema_2.mx_table_2'::regclass, 1310031, 't'::"char", '1288490188', '2147483647')) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.dist_table_1'::regclass, 1310076, 't'::"char", '-2147483648', '-1073741825'), ('public.dist_table_1'::regclass, 1310077, 't'::"char", '-1073741824', '-1'), ('public.dist_table_1'::regclass, 1310078, 't'::"char", '0', '1073741823'), ('public.dist_table_1'::regclass, 1310079, 't'::"char", '1073741824', '2147483647')) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_ref'::regclass, 1310075, 't'::"char", NULL, NULL)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.test_table'::regclass, 1310085, 't'::"char", '-2147483648', '-1073741825'), ('public.test_table'::regclass, 1310086, 't'::"char", '-1073741824', '-1'), ('public.test_table'::regclass, 1310087, 't'::"char", '0', '1073741823'), ('public.test_table'::regclass, 1310088, 't'::"char", '1073741824', '2147483647')) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('mx_test_schema_1.mx_table_1'::regclass, 1310022, 't'::"char", '-2147483648', '-1288490190', false), ('mx_test_schema_1.mx_table_1'::regclass, 1310023, 't'::"char", '-1288490189', '-429496731', false), ('mx_test_schema_1.mx_table_1'::regclass, 1310024, 't'::"char", '-429496730', '429496728', false), ('mx_test_schema_1.mx_table_1'::regclass, 1310025, 't'::"char", '429496729', '1288490187', false), ('mx_test_schema_1.mx_table_1'::regclass, 1310026, 't'::"char", '1288490188', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('mx_test_schema_2.mx_table_2'::regclass, 1310027, 't'::"char", '-2147483648', '-1288490190', false), ('mx_test_schema_2.mx_table_2'::regclass, 1310028, 't'::"char", '-1288490189', '-429496731', false), ('mx_test_schema_2.mx_table_2'::regclass, 1310029, 't'::"char", '-429496730', '429496728', false), ('mx_test_schema_2.mx_table_2'::regclass, 1310030, 't'::"char", '429496729', '1288490187', false), ('mx_test_schema_2.mx_table_2'::regclass, 1310031, 't'::"char", '1288490188', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737', false), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825', false), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913', false), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1', false), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911', false), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823', false), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735', false), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('public.dist_table_1'::regclass, 1310076, 't'::"char", '-2147483648', '-1073741825', false), ('public.dist_table_1'::regclass, 1310077, 't'::"char", '-1073741824', '-1', false), ('public.dist_table_1'::regclass, 1310078, 't'::"char", '0', '1073741823', false), ('public.dist_table_1'::regclass, 1310079, 't'::"char", '1073741824', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('public.mx_ref'::regclass, 1310075, 't'::"char", NULL, NULL, false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('public.test_table'::regclass, 1310085, 't'::"char", '-2147483648', '-1073741825', false), ('public.test_table'::regclass, 1310086, 't'::"char", '-1073741824', '-1', false), ('public.test_table'::regclass, 1310087, 't'::"char", '0', '1073741823', false), ('public.test_table'::regclass, 1310088, 't'::"char", '1073741824', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data;
(118 rows)
-- shouldn't work since test_table is MX

View File

@ -194,8 +194,8 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['public', 'single_shard_tbl']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal.add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310008, 0, 2, 100008)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.single_shard_tbl'::regclass, 1310008, 't'::"char", NULL, NULL)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737', false), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825', false), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913', false), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1', false), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911', false), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823', false), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735', false), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('public.single_shard_tbl'::regclass, 1310008, 't'::"char", NULL, NULL, false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data;
(61 rows)
-- Drop single shard table
@ -256,7 +256,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal.add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['public', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal.add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737', false), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825', false), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913', false), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1', false), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911', false), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823', false), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735', false), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data;
(52 rows)
-- Show that schema changes are included in the activate node snapshot
@ -318,7 +318,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal.add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['mx_testing_schema', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal.add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737', false), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825', false), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913', false), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1', false), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911', false), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823', false), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735', false), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data;
(54 rows)
-- Show that append distributed tables are not included in the activate node snapshot
@ -386,7 +386,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal.add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['mx_testing_schema', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal.add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737', false), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825', false), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913', false), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1', false), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911', false), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823', false), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735', false), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data;
(54 rows)
-- Show that range distributed tables are not included in the activate node snapshot
@ -447,7 +447,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal.add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['mx_testing_schema', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal.add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737', false), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825', false), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913', false), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1', false), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911', false), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823', false), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735', false), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data;
(54 rows)
-- Test start_metadata_sync_to_node and citus_activate_node UDFs
@ -537,16 +537,16 @@ SELECT * FROM pg_dist_partition WHERE logicalrelid::text LIKE 'mx_testing_schema
(1 row)
SELECT * FROM pg_dist_shard WHERE logicalrelid::text LIKE 'mx_testing_schema%' ORDER BY shardid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode
---------------------------------------------------------------------
mx_testing_schema.mx_test_table | 1310000 | t | -2147483648 | -1610612737
mx_testing_schema.mx_test_table | 1310001 | t | -1610612736 | -1073741825
mx_testing_schema.mx_test_table | 1310002 | t | -1073741824 | -536870913
mx_testing_schema.mx_test_table | 1310003 | t | -536870912 | -1
mx_testing_schema.mx_test_table | 1310004 | t | 0 | 536870911
mx_testing_schema.mx_test_table | 1310005 | t | 536870912 | 1073741823
mx_testing_schema.mx_test_table | 1310006 | t | 1073741824 | 1610612735
mx_testing_schema.mx_test_table | 1310007 | t | 1610612736 | 2147483647
mx_testing_schema.mx_test_table | 1310000 | t | -2147483648 | -1610612737 | f
mx_testing_schema.mx_test_table | 1310001 | t | -1610612736 | -1073741825 | f
mx_testing_schema.mx_test_table | 1310002 | t | -1073741824 | -536870913 | f
mx_testing_schema.mx_test_table | 1310003 | t | -536870912 | -1 | f
mx_testing_schema.mx_test_table | 1310004 | t | 0 | 536870911 | f
mx_testing_schema.mx_test_table | 1310005 | t | 536870912 | 1073741823 | f
mx_testing_schema.mx_test_table | 1310006 | t | 1073741824 | 1610612735 | f
mx_testing_schema.mx_test_table | 1310007 | t | 1610612736 | 2147483647 | f
(8 rows)
SELECT * FROM pg_dist_shard_placement WHERE shardid IN (SELECT shardid FROM pg_dist_shard WHERE logicalrelid::text LIKE 'mx_testing_schema%') ORDER BY shardid, nodename, nodeport;
@ -676,16 +676,16 @@ SELECT * FROM pg_dist_partition WHERE logicalrelid::text LIKE 'mx_testing_schema
(1 row)
SELECT * FROM pg_dist_shard WHERE logicalrelid::text LIKE 'mx_testing_schema%' ORDER BY shardid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode
---------------------------------------------------------------------
mx_testing_schema.mx_test_table | 1310000 | t | -2147483648 | -1610612737
mx_testing_schema.mx_test_table | 1310001 | t | -1610612736 | -1073741825
mx_testing_schema.mx_test_table | 1310002 | t | -1073741824 | -536870913
mx_testing_schema.mx_test_table | 1310003 | t | -536870912 | -1
mx_testing_schema.mx_test_table | 1310004 | t | 0 | 536870911
mx_testing_schema.mx_test_table | 1310005 | t | 536870912 | 1073741823
mx_testing_schema.mx_test_table | 1310006 | t | 1073741824 | 1610612735
mx_testing_schema.mx_test_table | 1310007 | t | 1610612736 | 2147483647
mx_testing_schema.mx_test_table | 1310000 | t | -2147483648 | -1610612737 | f
mx_testing_schema.mx_test_table | 1310001 | t | -1610612736 | -1073741825 | f
mx_testing_schema.mx_test_table | 1310002 | t | -1073741824 | -536870913 | f
mx_testing_schema.mx_test_table | 1310003 | t | -536870912 | -1 | f
mx_testing_schema.mx_test_table | 1310004 | t | 0 | 536870911 | f
mx_testing_schema.mx_test_table | 1310005 | t | 536870912 | 1073741823 | f
mx_testing_schema.mx_test_table | 1310006 | t | 1073741824 | 1610612735 | f
mx_testing_schema.mx_test_table | 1310007 | t | 1610612736 | 2147483647 | f
(8 rows)
SELECT * FROM pg_dist_shard_placement WHERE shardid IN (SELECT shardid FROM pg_dist_shard WHERE logicalrelid::text LIKE 'mx_testing_schema%') ORDER BY shardid, nodename, nodeport;
@ -1021,7 +1021,7 @@ SELECT * FROM pg_dist_partition WHERE logicalrelid::text LIKE 'mx_test_schema%';
(0 rows)
SELECT * FROM pg_dist_shard WHERE logicalrelid::text LIKE 'mx_test_schema%';
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode
---------------------------------------------------------------------
(0 rows)
@ -1681,7 +1681,7 @@ SELECT "Column", "Type", "Definition" FROM index_attrs WHERE
relid = 'mx_ref_index'::regclass;
ERROR: relation "mx_ref_index" does not exist
SELECT * FROM pg_dist_shard WHERE shardid=:ref_table_shardid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode
---------------------------------------------------------------------
(0 rows)
@ -2056,12 +2056,12 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310075, 0, 0, 100077), (1310075, 0, 1, 100078), (1310075, 0, 5, 100079)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310076, 0, 1, 100080), (1310077, 0, 5, 100081), (1310078, 0, 1, 100082), (1310079, 0, 5, 100083)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310085, 0, 1, 100091), (1310086, 0, 5, 100092), (1310087, 0, 1, 100093), (1310088, 0, 5, 100094)) SELECT citus_internal.add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_test_schema_1.mx_table_1'::regclass, 1310022, 't'::"char", '-2147483648', '-1288490190'), ('mx_test_schema_1.mx_table_1'::regclass, 1310023, 't'::"char", '-1288490189', '-429496731'), ('mx_test_schema_1.mx_table_1'::regclass, 1310024, 't'::"char", '-429496730', '429496728'), ('mx_test_schema_1.mx_table_1'::regclass, 1310025, 't'::"char", '429496729', '1288490187'), ('mx_test_schema_1.mx_table_1'::regclass, 1310026, 't'::"char", '1288490188', '2147483647')) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_test_schema_2.mx_table_2'::regclass, 1310027, 't'::"char", '-2147483648', '-1288490190'), ('mx_test_schema_2.mx_table_2'::regclass, 1310028, 't'::"char", '-1288490189', '-429496731'), ('mx_test_schema_2.mx_table_2'::regclass, 1310029, 't'::"char", '-429496730', '429496728'), ('mx_test_schema_2.mx_table_2'::regclass, 1310030, 't'::"char", '429496729', '1288490187'), ('mx_test_schema_2.mx_table_2'::regclass, 1310031, 't'::"char", '1288490188', '2147483647')) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.dist_table_1'::regclass, 1310076, 't'::"char", '-2147483648', '-1073741825'), ('public.dist_table_1'::regclass, 1310077, 't'::"char", '-1073741824', '-1'), ('public.dist_table_1'::regclass, 1310078, 't'::"char", '0', '1073741823'), ('public.dist_table_1'::regclass, 1310079, 't'::"char", '1073741824', '2147483647')) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_ref'::regclass, 1310075, 't'::"char", NULL, NULL)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.test_table'::regclass, 1310085, 't'::"char", '-2147483648', '-1073741825'), ('public.test_table'::regclass, 1310086, 't'::"char", '-1073741824', '-1'), ('public.test_table'::regclass, 1310087, 't'::"char", '0', '1073741823'), ('public.test_table'::regclass, 1310088, 't'::"char", '1073741824', '2147483647')) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('mx_test_schema_1.mx_table_1'::regclass, 1310022, 't'::"char", '-2147483648', '-1288490190', false), ('mx_test_schema_1.mx_table_1'::regclass, 1310023, 't'::"char", '-1288490189', '-429496731', false), ('mx_test_schema_1.mx_table_1'::regclass, 1310024, 't'::"char", '-429496730', '429496728', false), ('mx_test_schema_1.mx_table_1'::regclass, 1310025, 't'::"char", '429496729', '1288490187', false), ('mx_test_schema_1.mx_table_1'::regclass, 1310026, 't'::"char", '1288490188', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('mx_test_schema_2.mx_table_2'::regclass, 1310027, 't'::"char", '-2147483648', '-1288490190', false), ('mx_test_schema_2.mx_table_2'::regclass, 1310028, 't'::"char", '-1288490189', '-429496731', false), ('mx_test_schema_2.mx_table_2'::regclass, 1310029, 't'::"char", '-429496730', '429496728', false), ('mx_test_schema_2.mx_table_2'::regclass, 1310030, 't'::"char", '429496729', '1288490187', false), ('mx_test_schema_2.mx_table_2'::regclass, 1310031, 't'::"char", '1288490188', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737', false), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825', false), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913', false), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1', false), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911', false), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823', false), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735', false), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('public.dist_table_1'::regclass, 1310076, 't'::"char", '-2147483648', '-1073741825', false), ('public.dist_table_1'::regclass, 1310077, 't'::"char", '-1073741824', '-1', false), ('public.dist_table_1'::regclass, 1310078, 't'::"char", '0', '1073741823', false), ('public.dist_table_1'::regclass, 1310079, 't'::"char", '1073741824', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('public.mx_ref'::regclass, 1310075, 't'::"char", NULL, NULL, false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) AS (VALUES ('public.test_table'::regclass, 1310085, 't'::"char", '-2147483648', '-1073741825', false), ('public.test_table'::regclass, 1310086, 't'::"char", '-1073741824', '-1', false), ('public.test_table'::regclass, 1310087, 't'::"char", '0', '1073741823', false), ('public.test_table'::regclass, 1310088, 't'::"char", '1073741824', '2147483647', false)) SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, needsseparatenode) FROM shard_data;
(118 rows)
-- shouldn't work since test_table is MX

View File

@ -12,10 +12,10 @@ SELECT create_distributed_table('source_table_xyz', 'key', 'range');
CALL public.create_range_partitioned_shards('source_table_xyz', '{"(0,a)","(25,z)"}','{"(24,a)","(49,z)"}');
SELECT * FROM pg_dist_shard WHERE logicalrelid='source_table_xyz'::regclass::oid ORDER BY shardid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode
---------------------------------------------------------------------
source_table_xyz | 4213581 | t | (0,a) | (24,a)
source_table_xyz | 4213582 | t | (25,z) | (49,z)
source_table_xyz | 4213581 | t | (0,a) | (24,a) | f
source_table_xyz | 4213582 | t | (25,z) | (49,z) | f
(2 rows)
SELECT shardid, nodename, nodeport FROM pg_dist_shard_placement WHERE EXISTS(SELECT shardid FROM pg_dist_shard WHERE shardid=pg_dist_shard_placement.shardid AND logicalrelid='source_table_xyz'::regclass::oid) ORDER BY 1, 2, 3;

View File

@ -65,7 +65,7 @@ SELECT * FROM pg_dist_partition;
(0 rows)
SELECT * FROM pg_dist_shard;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode
---------------------------------------------------------------------
(0 rows)

View File

@ -168,12 +168,12 @@ SELECT count(*) FROM orders_streaming WHERE o_orderkey = 103;
SELECT * FROM pg_dist_shard
WHERE logicalrelid = 'lineitem_streaming'::regclass OR logicalrelid = 'orders_streaming'::regclass
ORDER BY shardminvalue::BIGINT, logicalrelid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode
---------------------------------------------------------------------
lineitem_streaming | 1230000 | t | -2147483648 | -1
orders_streaming | 1230002 | t | -2147483648 | -1
lineitem_streaming | 1230001 | t | 0 | 2147483647
orders_streaming | 1230003 | t | 0 | 2147483647
lineitem_streaming | 1230000 | t | -2147483648 | -1 | f
orders_streaming | 1230002 | t | -2147483648 | -1 | f
lineitem_streaming | 1230001 | t | 0 | 2147483647 | f
orders_streaming | 1230003 | t | 0 | 2147483647 | f
(4 rows)
-- check without cascade option
@ -387,32 +387,32 @@ SELECT count(*) FROM orders_streaming WHERE o_orderkey = 103;
SELECT * FROM pg_dist_shard
WHERE logicalrelid = 'lineitem_streaming'::regclass OR logicalrelid = 'orders_streaming'::regclass
ORDER BY shardminvalue::BIGINT, logicalrelid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode
---------------------------------------------------------------------
lineitem_streaming | 1230040 | t | -2147483648 | -2147483648
orders_streaming | 1230042 | t | -2147483648 | -2147483648
lineitem_streaming | 1230041 | t | -2147483647 | -136164586
orders_streaming | 1230043 | t | -2147483647 | -136164586
lineitem_streaming | 1230035 | t | -136164585 | -136164585
orders_streaming | 1230038 | t | -136164585 | -136164585
lineitem_streaming | 1230036 | t | -136164584 | -85071815
orders_streaming | 1230039 | t | -136164584 | -85071815
lineitem_streaming | 1230011 | t | -85071814 | -85071814
orders_streaming | 1230014 | t | -85071814 | -85071814
lineitem_streaming | 1230012 | t | -85071813 | -1
orders_streaming | 1230015 | t | -85071813 | -1
lineitem_streaming | 1230004 | t | 0 | 108199380
orders_streaming | 1230007 | t | 0 | 108199380
lineitem_streaming | 1230005 | t | 108199381 | 108199381
orders_streaming | 1230008 | t | 108199381 | 108199381
lineitem_streaming | 1230028 | t | 108199382 | 412880111
orders_streaming | 1230031 | t | 108199382 | 412880111
lineitem_streaming | 1230029 | t | 412880112 | 412880112
orders_streaming | 1230032 | t | 412880112 | 412880112
lineitem_streaming | 1230044 | t | 412880113 | 2147483646
orders_streaming | 1230046 | t | 412880113 | 2147483646
lineitem_streaming | 1230045 | t | 2147483647 | 2147483647
orders_streaming | 1230047 | t | 2147483647 | 2147483647
lineitem_streaming | 1230040 | t | -2147483648 | -2147483648 | f
orders_streaming | 1230042 | t | -2147483648 | -2147483648 | f
lineitem_streaming | 1230041 | t | -2147483647 | -136164586 | f
orders_streaming | 1230043 | t | -2147483647 | -136164586 | f
lineitem_streaming | 1230035 | t | -136164585 | -136164585 | f
orders_streaming | 1230038 | t | -136164585 | -136164585 | f
lineitem_streaming | 1230036 | t | -136164584 | -85071815 | f
orders_streaming | 1230039 | t | -136164584 | -85071815 | f
lineitem_streaming | 1230011 | t | -85071814 | -85071814 | f
orders_streaming | 1230014 | t | -85071814 | -85071814 | f
lineitem_streaming | 1230012 | t | -85071813 | -1 | f
orders_streaming | 1230015 | t | -85071813 | -1 | f
lineitem_streaming | 1230004 | t | 0 | 108199380 | f
orders_streaming | 1230007 | t | 0 | 108199380 | f
lineitem_streaming | 1230005 | t | 108199381 | 108199381 | f
orders_streaming | 1230008 | t | 108199381 | 108199381 | f
lineitem_streaming | 1230028 | t | 108199382 | 412880111 | f
orders_streaming | 1230031 | t | 108199382 | 412880111 | f
lineitem_streaming | 1230029 | t | 412880112 | 412880112 | f
orders_streaming | 1230032 | t | 412880112 | 412880112 | f
lineitem_streaming | 1230044 | t | 412880113 | 2147483646 | f
orders_streaming | 1230046 | t | 412880113 | 2147483646 | f
lineitem_streaming | 1230045 | t | 2147483647 | 2147483647 | f
orders_streaming | 1230047 | t | 2147483647 | 2147483647 | f
(24 rows)
SELECT * FROM pg_dist_shard_placement WHERE shardid BETWEEN 1230000 AND 1399999 ORDER BY nodeport, shardid;
@ -541,32 +541,32 @@ SET search_path to "Tenant Isolation";
SELECT * FROM pg_dist_shard
WHERE logicalrelid = 'lineitem_streaming'::regclass OR logicalrelid = 'orders_streaming'::regclass
ORDER BY shardminvalue::BIGINT, logicalrelid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode
---------------------------------------------------------------------
lineitem_streaming | 1230040 | t | -2147483648 | -2147483648
orders_streaming | 1230042 | t | -2147483648 | -2147483648
lineitem_streaming | 1230041 | t | -2147483647 | -136164586
orders_streaming | 1230043 | t | -2147483647 | -136164586
lineitem_streaming | 1230035 | t | -136164585 | -136164585
orders_streaming | 1230038 | t | -136164585 | -136164585
lineitem_streaming | 1230036 | t | -136164584 | -85071815
orders_streaming | 1230039 | t | -136164584 | -85071815
lineitem_streaming | 1230011 | t | -85071814 | -85071814
orders_streaming | 1230014 | t | -85071814 | -85071814
lineitem_streaming | 1230012 | t | -85071813 | -1
orders_streaming | 1230015 | t | -85071813 | -1
lineitem_streaming | 1230004 | t | 0 | 108199380
orders_streaming | 1230007 | t | 0 | 108199380
lineitem_streaming | 1230005 | t | 108199381 | 108199381
orders_streaming | 1230008 | t | 108199381 | 108199381
lineitem_streaming | 1230028 | t | 108199382 | 412880111
orders_streaming | 1230031 | t | 108199382 | 412880111
lineitem_streaming | 1230029 | t | 412880112 | 412880112
orders_streaming | 1230032 | t | 412880112 | 412880112
lineitem_streaming | 1230044 | t | 412880113 | 2147483646
orders_streaming | 1230046 | t | 412880113 | 2147483646
lineitem_streaming | 1230045 | t | 2147483647 | 2147483647
orders_streaming | 1230047 | t | 2147483647 | 2147483647
lineitem_streaming | 1230040 | t | -2147483648 | -2147483648 | f
orders_streaming | 1230042 | t | -2147483648 | -2147483648 | f
lineitem_streaming | 1230041 | t | -2147483647 | -136164586 | f
orders_streaming | 1230043 | t | -2147483647 | -136164586 | f
lineitem_streaming | 1230035 | t | -136164585 | -136164585 | f
orders_streaming | 1230038 | t | -136164585 | -136164585 | f
lineitem_streaming | 1230036 | t | -136164584 | -85071815 | f
orders_streaming | 1230039 | t | -136164584 | -85071815 | f
lineitem_streaming | 1230011 | t | -85071814 | -85071814 | f
orders_streaming | 1230014 | t | -85071814 | -85071814 | f
lineitem_streaming | 1230012 | t | -85071813 | -1 | f
orders_streaming | 1230015 | t | -85071813 | -1 | f
lineitem_streaming | 1230004 | t | 0 | 108199380 | f
orders_streaming | 1230007 | t | 0 | 108199380 | f
lineitem_streaming | 1230005 | t | 108199381 | 108199381 | f
orders_streaming | 1230008 | t | 108199381 | 108199381 | f
lineitem_streaming | 1230028 | t | 108199382 | 412880111 | f
orders_streaming | 1230031 | t | 108199382 | 412880111 | f
lineitem_streaming | 1230029 | t | 412880112 | 412880112 | f
orders_streaming | 1230032 | t | 412880112 | 412880112 | f
lineitem_streaming | 1230044 | t | 412880113 | 2147483646 | f
orders_streaming | 1230046 | t | 412880113 | 2147483646 | f
lineitem_streaming | 1230045 | t | 2147483647 | 2147483647 | f
orders_streaming | 1230047 | t | 2147483647 | 2147483647 | f
(24 rows)
-- return to master node
@ -684,32 +684,32 @@ ERROR: cannot isolate tenant because tenant isolation is only support for hash
SELECT * FROM pg_dist_shard
WHERE logicalrelid = 'lineitem_streaming'::regclass OR logicalrelid = 'orders_streaming'::regclass
ORDER BY shardminvalue::BIGINT, logicalrelid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode
---------------------------------------------------------------------
lineitem_streaming | 1230040 | t | -2147483648 | -2147483648
orders_streaming | 1230042 | t | -2147483648 | -2147483648
lineitem_streaming | 1230041 | t | -2147483647 | -136164586
orders_streaming | 1230043 | t | -2147483647 | -136164586
lineitem_streaming | 1230035 | t | -136164585 | -136164585
orders_streaming | 1230038 | t | -136164585 | -136164585
lineitem_streaming | 1230036 | t | -136164584 | -85071815
orders_streaming | 1230039 | t | -136164584 | -85071815
lineitem_streaming | 1230011 | t | -85071814 | -85071814
orders_streaming | 1230014 | t | -85071814 | -85071814
lineitem_streaming | 1230012 | t | -85071813 | -1
orders_streaming | 1230015 | t | -85071813 | -1
lineitem_streaming | 1230004 | t | 0 | 108199380
orders_streaming | 1230007 | t | 0 | 108199380
lineitem_streaming | 1230005 | t | 108199381 | 108199381
orders_streaming | 1230008 | t | 108199381 | 108199381
lineitem_streaming | 1230028 | t | 108199382 | 412880111
orders_streaming | 1230031 | t | 108199382 | 412880111
lineitem_streaming | 1230029 | t | 412880112 | 412880112
orders_streaming | 1230032 | t | 412880112 | 412880112
lineitem_streaming | 1230044 | t | 412880113 | 2147483646
orders_streaming | 1230046 | t | 412880113 | 2147483646
lineitem_streaming | 1230045 | t | 2147483647 | 2147483647
orders_streaming | 1230047 | t | 2147483647 | 2147483647
lineitem_streaming | 1230040 | t | -2147483648 | -2147483648 | f
orders_streaming | 1230042 | t | -2147483648 | -2147483648 | f
lineitem_streaming | 1230041 | t | -2147483647 | -136164586 | f
orders_streaming | 1230043 | t | -2147483647 | -136164586 | f
lineitem_streaming | 1230035 | t | -136164585 | -136164585 | f
orders_streaming | 1230038 | t | -136164585 | -136164585 | f
lineitem_streaming | 1230036 | t | -136164584 | -85071815 | f
orders_streaming | 1230039 | t | -136164584 | -85071815 | f
lineitem_streaming | 1230011 | t | -85071814 | -85071814 | f
orders_streaming | 1230014 | t | -85071814 | -85071814 | f
lineitem_streaming | 1230012 | t | -85071813 | -1 | f
orders_streaming | 1230015 | t | -85071813 | -1 | f
lineitem_streaming | 1230004 | t | 0 | 108199380 | f
orders_streaming | 1230007 | t | 0 | 108199380 | f
lineitem_streaming | 1230005 | t | 108199381 | 108199381 | f
orders_streaming | 1230008 | t | 108199381 | 108199381 | f
lineitem_streaming | 1230028 | t | 108199382 | 412880111 | f
orders_streaming | 1230031 | t | 108199382 | 412880111 | f
lineitem_streaming | 1230029 | t | 412880112 | 412880112 | f
orders_streaming | 1230032 | t | 412880112 | 412880112 | f
lineitem_streaming | 1230044 | t | 412880113 | 2147483646 | f
orders_streaming | 1230046 | t | 412880113 | 2147483646 | f
lineitem_streaming | 1230045 | t | 2147483647 | 2147483647 | f
orders_streaming | 1230047 | t | 2147483647 | 2147483647 | f
(24 rows)
\c - postgres - :master_port

View File

@ -168,12 +168,12 @@ SELECT count(*) FROM orders_streaming WHERE o_orderkey = 103;
SELECT * FROM pg_dist_shard
WHERE logicalrelid = 'lineitem_streaming'::regclass OR logicalrelid = 'orders_streaming'::regclass
ORDER BY shardminvalue::BIGINT, logicalrelid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode
---------------------------------------------------------------------
lineitem_streaming | 1230000 | t | -2147483648 | -1
orders_streaming | 1230002 | t | -2147483648 | -1
lineitem_streaming | 1230001 | t | 0 | 2147483647
orders_streaming | 1230003 | t | 0 | 2147483647
lineitem_streaming | 1230000 | t | -2147483648 | -1 | f
orders_streaming | 1230002 | t | -2147483648 | -1 | f
lineitem_streaming | 1230001 | t | 0 | 2147483647 | f
orders_streaming | 1230003 | t | 0 | 2147483647 | f
(4 rows)
-- check without cascade option
@ -399,32 +399,32 @@ SELECT count(*) FROM orders_streaming WHERE o_orderkey = 103;
SELECT * FROM pg_dist_shard
WHERE logicalrelid = 'lineitem_streaming'::regclass OR logicalrelid = 'orders_streaming'::regclass
ORDER BY shardminvalue::BIGINT, logicalrelid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode
---------------------------------------------------------------------
lineitem_streaming | 1230040 | t | -2147483648 | -2147483648
orders_streaming | 1230042 | t | -2147483648 | -2147483648
lineitem_streaming | 1230041 | t | -2147483647 | -136164586
orders_streaming | 1230043 | t | -2147483647 | -136164586
lineitem_streaming | 1230035 | t | -136164585 | -136164585
orders_streaming | 1230038 | t | -136164585 | -136164585
lineitem_streaming | 1230036 | t | -136164584 | -85071815
orders_streaming | 1230039 | t | -136164584 | -85071815
lineitem_streaming | 1230011 | t | -85071814 | -85071814
orders_streaming | 1230014 | t | -85071814 | -85071814
lineitem_streaming | 1230012 | t | -85071813 | -1
orders_streaming | 1230015 | t | -85071813 | -1
lineitem_streaming | 1230004 | t | 0 | 108199380
orders_streaming | 1230007 | t | 0 | 108199380
lineitem_streaming | 1230005 | t | 108199381 | 108199381
orders_streaming | 1230008 | t | 108199381 | 108199381
lineitem_streaming | 1230028 | t | 108199382 | 412880111
orders_streaming | 1230031 | t | 108199382 | 412880111
lineitem_streaming | 1230029 | t | 412880112 | 412880112
orders_streaming | 1230032 | t | 412880112 | 412880112
lineitem_streaming | 1230044 | t | 412880113 | 2147483646
orders_streaming | 1230046 | t | 412880113 | 2147483646
lineitem_streaming | 1230045 | t | 2147483647 | 2147483647
orders_streaming | 1230047 | t | 2147483647 | 2147483647
lineitem_streaming | 1230040 | t | -2147483648 | -2147483648 | f
orders_streaming | 1230042 | t | -2147483648 | -2147483648 | f
lineitem_streaming | 1230041 | t | -2147483647 | -136164586 | f
orders_streaming | 1230043 | t | -2147483647 | -136164586 | f
lineitem_streaming | 1230035 | t | -136164585 | -136164585 | f
orders_streaming | 1230038 | t | -136164585 | -136164585 | f
lineitem_streaming | 1230036 | t | -136164584 | -85071815 | f
orders_streaming | 1230039 | t | -136164584 | -85071815 | f
lineitem_streaming | 1230011 | t | -85071814 | -85071814 | f
orders_streaming | 1230014 | t | -85071814 | -85071814 | f
lineitem_streaming | 1230012 | t | -85071813 | -1 | f
orders_streaming | 1230015 | t | -85071813 | -1 | f
lineitem_streaming | 1230004 | t | 0 | 108199380 | f
orders_streaming | 1230007 | t | 0 | 108199380 | f
lineitem_streaming | 1230005 | t | 108199381 | 108199381 | f
orders_streaming | 1230008 | t | 108199381 | 108199381 | f
lineitem_streaming | 1230028 | t | 108199382 | 412880111 | f
orders_streaming | 1230031 | t | 108199382 | 412880111 | f
lineitem_streaming | 1230029 | t | 412880112 | 412880112 | f
orders_streaming | 1230032 | t | 412880112 | 412880112 | f
lineitem_streaming | 1230044 | t | 412880113 | 2147483646 | f
orders_streaming | 1230046 | t | 412880113 | 2147483646 | f
lineitem_streaming | 1230045 | t | 2147483647 | 2147483647 | f
orders_streaming | 1230047 | t | 2147483647 | 2147483647 | f
(24 rows)
SELECT * FROM pg_dist_shard_placement WHERE shardid BETWEEN 1230000 AND 1399999 ORDER BY nodeport, shardid;
@ -553,32 +553,32 @@ SET search_path to "Tenant Isolation";
SELECT * FROM pg_dist_shard
WHERE logicalrelid = 'lineitem_streaming'::regclass OR logicalrelid = 'orders_streaming'::regclass
ORDER BY shardminvalue::BIGINT, logicalrelid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode
---------------------------------------------------------------------
lineitem_streaming | 1230040 | t | -2147483648 | -2147483648
orders_streaming | 1230042 | t | -2147483648 | -2147483648
lineitem_streaming | 1230041 | t | -2147483647 | -136164586
orders_streaming | 1230043 | t | -2147483647 | -136164586
lineitem_streaming | 1230035 | t | -136164585 | -136164585
orders_streaming | 1230038 | t | -136164585 | -136164585
lineitem_streaming | 1230036 | t | -136164584 | -85071815
orders_streaming | 1230039 | t | -136164584 | -85071815
lineitem_streaming | 1230011 | t | -85071814 | -85071814
orders_streaming | 1230014 | t | -85071814 | -85071814
lineitem_streaming | 1230012 | t | -85071813 | -1
orders_streaming | 1230015 | t | -85071813 | -1
lineitem_streaming | 1230004 | t | 0 | 108199380
orders_streaming | 1230007 | t | 0 | 108199380
lineitem_streaming | 1230005 | t | 108199381 | 108199381
orders_streaming | 1230008 | t | 108199381 | 108199381
lineitem_streaming | 1230028 | t | 108199382 | 412880111
orders_streaming | 1230031 | t | 108199382 | 412880111
lineitem_streaming | 1230029 | t | 412880112 | 412880112
orders_streaming | 1230032 | t | 412880112 | 412880112
lineitem_streaming | 1230044 | t | 412880113 | 2147483646
orders_streaming | 1230046 | t | 412880113 | 2147483646
lineitem_streaming | 1230045 | t | 2147483647 | 2147483647
orders_streaming | 1230047 | t | 2147483647 | 2147483647
lineitem_streaming | 1230040 | t | -2147483648 | -2147483648 | f
orders_streaming | 1230042 | t | -2147483648 | -2147483648 | f
lineitem_streaming | 1230041 | t | -2147483647 | -136164586 | f
orders_streaming | 1230043 | t | -2147483647 | -136164586 | f
lineitem_streaming | 1230035 | t | -136164585 | -136164585 | f
orders_streaming | 1230038 | t | -136164585 | -136164585 | f
lineitem_streaming | 1230036 | t | -136164584 | -85071815 | f
orders_streaming | 1230039 | t | -136164584 | -85071815 | f
lineitem_streaming | 1230011 | t | -85071814 | -85071814 | f
orders_streaming | 1230014 | t | -85071814 | -85071814 | f
lineitem_streaming | 1230012 | t | -85071813 | -1 | f
orders_streaming | 1230015 | t | -85071813 | -1 | f
lineitem_streaming | 1230004 | t | 0 | 108199380 | f
orders_streaming | 1230007 | t | 0 | 108199380 | f
lineitem_streaming | 1230005 | t | 108199381 | 108199381 | f
orders_streaming | 1230008 | t | 108199381 | 108199381 | f
lineitem_streaming | 1230028 | t | 108199382 | 412880111 | f
orders_streaming | 1230031 | t | 108199382 | 412880111 | f
lineitem_streaming | 1230029 | t | 412880112 | 412880112 | f
orders_streaming | 1230032 | t | 412880112 | 412880112 | f
lineitem_streaming | 1230044 | t | 412880113 | 2147483646 | f
orders_streaming | 1230046 | t | 412880113 | 2147483646 | f
lineitem_streaming | 1230045 | t | 2147483647 | 2147483647 | f
orders_streaming | 1230047 | t | 2147483647 | 2147483647 | f
(24 rows)
-- return to master node
@ -728,32 +728,32 @@ ERROR: cannot isolate tenant because tenant isolation is only support for hash
SELECT * FROM pg_dist_shard
WHERE logicalrelid = 'lineitem_streaming'::regclass OR logicalrelid = 'orders_streaming'::regclass
ORDER BY shardminvalue::BIGINT, logicalrelid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | needsseparatenode
---------------------------------------------------------------------
lineitem_streaming | 1230040 | t | -2147483648 | -2147483648
orders_streaming | 1230042 | t | -2147483648 | -2147483648
lineitem_streaming | 1230041 | t | -2147483647 | -136164586
orders_streaming | 1230043 | t | -2147483647 | -136164586
lineitem_streaming | 1230035 | t | -136164585 | -136164585
orders_streaming | 1230038 | t | -136164585 | -136164585
lineitem_streaming | 1230036 | t | -136164584 | -85071815
orders_streaming | 1230039 | t | -136164584 | -85071815
lineitem_streaming | 1230011 | t | -85071814 | -85071814
orders_streaming | 1230014 | t | -85071814 | -85071814
lineitem_streaming | 1230012 | t | -85071813 | -1
orders_streaming | 1230015 | t | -85071813 | -1
lineitem_streaming | 1230004 | t | 0 | 108199380
orders_streaming | 1230007 | t | 0 | 108199380
lineitem_streaming | 1230005 | t | 108199381 | 108199381
orders_streaming | 1230008 | t | 108199381 | 108199381
lineitem_streaming | 1230028 | t | 108199382 | 412880111
orders_streaming | 1230031 | t | 108199382 | 412880111
lineitem_streaming | 1230029 | t | 412880112 | 412880112
orders_streaming | 1230032 | t | 412880112 | 412880112
lineitem_streaming | 1230044 | t | 412880113 | 2147483646
orders_streaming | 1230046 | t | 412880113 | 2147483646
lineitem_streaming | 1230045 | t | 2147483647 | 2147483647
orders_streaming | 1230047 | t | 2147483647 | 2147483647
lineitem_streaming | 1230040 | t | -2147483648 | -2147483648 | f
orders_streaming | 1230042 | t | -2147483648 | -2147483648 | f
lineitem_streaming | 1230041 | t | -2147483647 | -136164586 | f
orders_streaming | 1230043 | t | -2147483647 | -136164586 | f
lineitem_streaming | 1230035 | t | -136164585 | -136164585 | f
orders_streaming | 1230038 | t | -136164585 | -136164585 | f
lineitem_streaming | 1230036 | t | -136164584 | -85071815 | f
orders_streaming | 1230039 | t | -136164584 | -85071815 | f
lineitem_streaming | 1230011 | t | -85071814 | -85071814 | f
orders_streaming | 1230014 | t | -85071814 | -85071814 | f
lineitem_streaming | 1230012 | t | -85071813 | -1 | f
orders_streaming | 1230015 | t | -85071813 | -1 | f
lineitem_streaming | 1230004 | t | 0 | 108199380 | f
orders_streaming | 1230007 | t | 0 | 108199380 | f
lineitem_streaming | 1230005 | t | 108199381 | 108199381 | f
orders_streaming | 1230008 | t | 108199381 | 108199381 | f
lineitem_streaming | 1230028 | t | 108199382 | 412880111 | f
orders_streaming | 1230031 | t | 108199382 | 412880111 | f
lineitem_streaming | 1230029 | t | 412880112 | 412880112 | f
orders_streaming | 1230032 | t | 412880112 | 412880112 | f
lineitem_streaming | 1230044 | t | 412880113 | 2147483646 | f
orders_streaming | 1230046 | t | 412880113 | 2147483646 | f
lineitem_streaming | 1230045 | t | 2147483647 | 2147483647 | f
orders_streaming | 1230047 | t | 2147483647 | 2147483647 | f
(24 rows)
\c - postgres - :master_port

View File

@ -626,3 +626,90 @@ BEGIN
JOIN pg_dist_node USING (nodeid);
END;
$func$ LANGUAGE plpgsql;
-- Takes a table name and returns an array of colocated shards
-- --by enumerating them based on shardminvalue-- for each shard
-- of given distributed table (including colocated shards).
CREATE OR REPLACE FUNCTION get_enumerated_shard_groups(
qualified_table_name text)
RETURNS TABLE (
shardids bigint[],
shardgroupindex bigint
)
AS $func$
BEGIN
RETURN QUERY
SELECT array_agg(shardid ORDER BY shardid) AS shardids,
ROW_NUMBER() OVER (ORDER BY shardminvalue) AS shardgroupindex
FROM pg_dist_shard
JOIN pg_dist_partition USING(logicalrelid)
WHERE colocationid = (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = qualified_table_name::regclass)
GROUP BY shardminvalue;
END;
$func$ LANGUAGE plpgsql;
-- Takes a table name and returns a json object for each shard group that
-- contains a shard whose placements need separate nodes.
--
-- This does not only return the shards of input relation but also considers
-- all colocated relations.
--
-- An example output is as follows:
--
-- [
-- {"10": [{"dist_1": true},{"dist_2": false}]},
-- {"15": [{"dist_1": false},{"dist_3": true}]}
-- ]
--
-- It only returned shard groups 10 and 15 because they are the only shard groups
-- that contain at least one shard whose placements need an isolation.
--
-- (Innermost) Boolean values represent needsseparatenode value for given
-- shard. For example,
--
-- {"15": [{"dist_1": false},{"dist_3": true}]}
--
-- means that the placements of dist_3 within shard group 15 needs
-- to be isolated.
CREATE OR REPLACE FUNCTION get_colocated_shards_needisolatednode(
qualified_table_name text)
RETURNS SETOF jsonb AS $func$
BEGIN
RETURN QUERY
SELECT
COALESCE(
jsonb_agg(jsonb_build_object(shardgroupindex, needsseparatenodejson) ORDER BY shardgroupindex),
'{}'::jsonb
) AS result
FROM (
SELECT shardgroupindex,
jsonb_agg(jsonb_build_object(logicalrelid, needsseparatenode) ORDER BY logicalrelid::text) AS needsseparatenodejson
FROM (
SELECT shardgroupindex,
logicalrelid,
needsseparatenode
FROM public.get_enumerated_shard_groups(qualified_table_name) AS shardgroups
JOIN pg_dist_shard
ON shardid = ANY(shardids)
) q1
GROUP BY shardgroupindex
) q2
WHERE needsseparatenodejson::text LIKE '%true%';
END;
$func$ LANGUAGE plpgsql;
-- Returns true if all placement groups within given shard group are isolated.
CREATE OR REPLACE FUNCTION verify_placements_in_shard_group_isolated(
qualified_table_name text,
shard_group_index bigint)
RETURNS boolean
AS $func$
DECLARE
v_result boolean;
BEGIN
SELECT bool_and(has_separate_node) INTO v_result
FROM citus_shards
JOIN (
SELECT shardids FROM public.get_enumerated_shard_groups(qualified_table_name) WHERE shardgroupindex = shard_group_index
) q
ON (shardid = ANY(q.shardids));
RETURN v_result;
END;
$func$ LANGUAGE plpgsql;

View File

@ -0,0 +1,21 @@
-- upgrade_columnar_before renames public to citus_schema and recreates public
-- schema. But this file depends on get_colocated_shards_needisolatednode()
-- function and get_colocated_shards_needisolatednode() depends on another
-- function --get_enumerated_shard_groups()-- that is presumably created earlier
-- within the original public schema, so we temporarily rename citus_schema to
-- public here; and revert those changes at the end of this file.
ALTER SCHEMA public RENAME TO old_public;
ALTER SCHEMA citus_schema RENAME TO public;
SELECT result FROM run_command_on_all_nodes($$
SELECT * FROM public.get_colocated_shards_needisolatednode('upgrade_isolate_placement_before.table_with_isolated_placements')
$$)
ORDER BY nodeid;
result
---------------------------------------------------------------------
[{"5": [{"upgrade_isolate_placement_before.table_with_isolated_placements": true}]}]
[{"5": [{"upgrade_isolate_placement_before.table_with_isolated_placements": true}]}]
[{"5": [{"upgrade_isolate_placement_before.table_with_isolated_placements": true}]}]
(3 rows)
ALTER SCHEMA public RENAME TO citus_schema;
ALTER SCHEMA old_public RENAME TO public;

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_property_set(:shardgroup_5_shardid, anti_affinity=>true);
citus_shard_property_set
---------------------------------------------------------------------
(1 row)
SELECT result FROM run_command_on_all_nodes($$
SELECT * FROM public.get_colocated_shards_needisolatednode('upgrade_isolate_placement_before.table_with_isolated_placements')
$$)
ORDER BY nodeid;
result
---------------------------------------------------------------------
[{"5": [{"upgrade_isolate_placement_before.table_with_isolated_placements": true}]}]
[{"5": [{"upgrade_isolate_placement_before.table_with_isolated_placements": true}]}]
[{"5": [{"upgrade_isolate_placement_before.table_with_isolated_placements": true}]}]
(3 rows)

View File

@ -61,7 +61,7 @@ ORDER BY 1;
function citus_internal.add_object_metadata(text,text[],text[],integer,integer,boolean)
function citus_internal.add_partition_metadata(regclass,"char",text,integer,"char")
function citus_internal.add_placement_metadata(bigint,bigint,integer,bigint)
function citus_internal.add_shard_metadata(regclass,bigint,"char",text,text)
function citus_internal.add_shard_metadata(regclass,bigint,"char",text,text,boolean)
function citus_internal.add_tenant_schema(oid,integer)
function citus_internal.adjust_local_clock_to_remote(cluster_clock)
function citus_internal.commit_management_command_2pc()
@ -84,6 +84,7 @@ ORDER BY 1;
function citus_internal.refresh_isolation_tester_prepared_statement()
function citus_internal.replace_isolation_tester_func()
function citus_internal.restore_isolation_tester_func()
function citus_internal.shard_property_set(bigint,boolean)
function citus_internal.start_management_transaction(xid8)
function citus_internal.start_replication_origin_tracking()
function citus_internal.stop_replication_origin_tracking()
@ -96,7 +97,7 @@ ORDER BY 1;
function citus_internal_add_partition_metadata(regclass,"char",text,integer,"char")
function citus_internal_add_placement_metadata(bigint,bigint,integer,bigint)
function citus_internal_add_placement_metadata(bigint,integer,bigint,integer,bigint)
function citus_internal_add_shard_metadata(regclass,bigint,"char",text,text)
function citus_internal_add_shard_metadata(regclass,bigint,"char",text,text,boolean)
function citus_internal_add_tenant_schema(oid,integer)
function citus_internal_adjust_local_clock_to_remote(cluster_clock)
function citus_internal_delete_colocation_metadata(integer)
@ -157,6 +158,7 @@ ORDER BY 1;
function citus_shard_cost_1(bigint)
function citus_shard_cost_by_disk_size(bigint)
function citus_shard_indexes_on_worker()
function citus_shard_property_set(bigint,boolean)
function citus_shard_sizes()
function citus_shards_on_worker()
function citus_split_shard_by_split_points(bigint,text[],integer[],citus.shard_transfer_mode)
@ -371,5 +373,5 @@ ORDER BY 1;
view citus_stat_tenants_local
view pg_dist_shard_placement
view time_partitions
(361 rows)
(363 rows)

View File

@ -43,6 +43,7 @@ test: comment_on_role
# don't parallelize single_shard_table_udfs to make sure colocation ids are sequential
test: single_shard_table_udfs
test: schema_based_sharding
test: isolate_placement
test: citus_schema_distribute_undistribute
test: multi_test_catalog_views

View File

@ -1,11 +1,13 @@
setup
{
CREATE TABLE table_to_distribute(id int);
CREATE TABLE table_to_colocate(id int);
}
teardown
{
DROP TABLE table_to_distribute CASCADE;
DROP TABLE table_to_colocate CASCADE;
}
session "s1"
@ -20,6 +22,13 @@ step "s1-create_distributed_table"
SELECT create_distributed_table('table_to_distribute', 'id');
}
step "s1_set-shard-property"
{
SELECT citus_shard_property_set(shardid, anti_affinity=>'true')
FROM pg_dist_shard WHERE logicalrelid = 'table_to_distribute'::regclass
ORDER BY shardid LIMIT 1;
}
step "s1-copy_to_local_table"
{
COPY table_to_distribute FROM PROGRAM 'echo 0 && echo 1 && echo 2 && echo 3 && echo 4 && echo 5 && echo 6 && echo 7 && echo 8';
@ -30,6 +39,11 @@ step "s1-commit"
COMMIT;
}
step "s1-rollback"
{
ROLLBACK;
}
session "s2"
step "s2-begin"
@ -42,6 +56,11 @@ step "s2-create_distributed_table"
SELECT create_distributed_table('table_to_distribute', 'id');
}
step "s2-create_distributed_table_colocated"
{
SELECT create_distributed_table('table_to_colocate', 'id', colocate_with=>'table_to_distribute');
}
step "s2-copy_to_local_table"
{
COPY table_to_distribute FROM PROGRAM 'echo 0 && echo 1 && echo 2 && echo 3 && echo 4 && echo 5 && echo 6 && echo 7 && echo 8';
@ -61,3 +80,7 @@ permutation "s1-begin" "s2-begin" "s2-copy_to_local_table" "s1-create_distribute
//concurrent create_distributed_table on non-empty table
permutation "s1-copy_to_local_table" "s1-begin" "s2-begin" "s1-create_distributed_table" "s2-create_distributed_table" "s1-commit" "s2-commit"
//concurrent create_distributed_table vs citus_shard_property_set
permutation "s1-create_distributed_table" "s1-begin" "s2-begin" "s1_set-shard-property" "s2-create_distributed_table_colocated" "s1-rollback" "s2-commit"
permutation "s1-create_distributed_table" "s1-begin" "s2-begin" "s2-create_distributed_table_colocated" "s1_set-shard-property" "s1-rollback" "s2-commit"

File diff suppressed because it is too large Load Diff

View File

@ -276,7 +276,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('super_user_table'::regclass, 1420000::bigint, 't'::"char", '-2147483648'::text, '-1610612737'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
ROLLBACK;
-- the user is only allowed to add a shard for add a table which is in pg_dist_partition
@ -286,7 +286,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '-2147483648'::text, '-1610612737'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
ROLLBACK;
-- ok, now add the table to the pg_dist_partition
@ -312,7 +312,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, -1, 't'::"char", '-2147483648'::text, '-1610612737'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
ROLLBACK;
-- invalid storage types are not allowed
@ -322,7 +322,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, 1420000, 'X'::"char", '-2147483648'::text, '-1610612737'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
ROLLBACK;
-- NULL shard ranges are not allowed for hash distributed tables
@ -332,7 +332,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, 1420000, 't'::"char", NULL, '-1610612737'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
ROLLBACK;
-- non-integer shard ranges are not allowed
@ -342,7 +342,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", 'non-int'::text, '-1610612737'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
ROLLBACK;
-- shardMinValue should be smaller than shardMaxValue
@ -352,7 +352,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '-1610612737'::text, '-2147483648'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
ROLLBACK;
-- we do not allow overlapping shards for the same table
@ -364,7 +364,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '10'::text, '20'::text),
('test_2'::regclass, 1420001::bigint, 't'::"char", '20'::text, '30'::text),
('test_2'::regclass, 1420002::bigint, 't'::"char", '10'::text, '50'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
ROLLBACK;
-- Now let's check valid pg_dist_object updates
@ -482,7 +482,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '10'::text, '20'::text),
('test_2'::regclass, 1420001::bigint, 't'::"char", '20'::text, '30'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
ROLLBACK;
-- we do not allow NULL shardMinMax values
@ -494,12 +494,12 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '10'::text, '20'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
-- manually ingest NULL values, otherwise not likely unless metadata is corrupted
UPDATE pg_dist_shard SET shardminvalue = NULL WHERE shardid = 1420000;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, 1420001::bigint, 't'::"char", '20'::text, '30'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
ROLLBACK;
\c - metadata_sync_helper_role - :worker_1_port
@ -518,7 +518,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
('test_2'::regclass, 1420004::bigint, 't'::"char", '51'::text, '60'::text),
('test_2'::regclass, 1420005::bigint, 't'::"char", '61'::text, '70'::text),
('test_3'::regclass, 1420008::bigint, 't'::"char", '11'::text, '20'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
COMMIT;
-- we cannot mark these two tables colocated because they are not colocated
@ -539,7 +539,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
('test_3'::regclass, 1420011::bigint, 't'::"char", '41'::text, '50'::text),
('test_3'::regclass, 1420012::bigint, 't'::"char", '51'::text, '60'::text),
('test_3'::regclass, 1420013::bigint, 't'::"char", '61'::text, '70'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
COMMIT;
-- shardMin/MaxValues should be NULL for reference tables
@ -549,7 +549,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_ref'::regclass, 1420003::bigint, 't'::"char", '-1610612737'::text, NULL))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
ROLLBACK;
-- reference tables cannot have multiple shards
@ -560,7 +560,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_ref'::regclass, 1420006::bigint, 't'::"char", NULL, NULL),
('test_ref'::regclass, 1420007::bigint, 't'::"char", NULL, NULL))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
ROLLBACK;
-- finally, add a shard for reference tables
@ -570,7 +570,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_ref'::regclass, 1420006::bigint, 't'::"char", NULL, NULL))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
COMMIT;
\c - postgres - :worker_1_port
@ -583,7 +583,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('super_user_table'::regclass, 1420007::bigint, 't'::"char", '11'::text, '20'::text))
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
SELECT citus_internal.add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue, false) FROM shard_data;
COMMIT;
\c - metadata_sync_helper_role - :worker_1_port

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

View File

@ -652,3 +652,94 @@ BEGIN
JOIN pg_dist_node USING (nodeid);
END;
$func$ LANGUAGE plpgsql;
-- Takes a table name and returns an array of colocated shards
-- --by enumerating them based on shardminvalue-- for each shard
-- of given distributed table (including colocated shards).
CREATE OR REPLACE FUNCTION get_enumerated_shard_groups(
qualified_table_name text)
RETURNS TABLE (
shardids bigint[],
shardgroupindex bigint
)
AS $func$
BEGIN
RETURN QUERY
SELECT array_agg(shardid ORDER BY shardid) AS shardids,
ROW_NUMBER() OVER (ORDER BY shardminvalue) AS shardgroupindex
FROM pg_dist_shard
JOIN pg_dist_partition USING(logicalrelid)
WHERE colocationid = (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = qualified_table_name::regclass)
GROUP BY shardminvalue;
END;
$func$ LANGUAGE plpgsql;
-- Takes a table name and returns a json object for each shard group that
-- contains a shard whose placements need separate nodes.
--
-- This does not only return the shards of input relation but also considers
-- all colocated relations.
--
-- An example output is as follows:
--
-- [
-- {"10": [{"dist_1": true},{"dist_2": false}]},
-- {"15": [{"dist_1": false},{"dist_3": true}]}
-- ]
--
-- It only returned shard groups 10 and 15 because they are the only shard groups
-- that contain at least one shard whose placements need an isolation.
--
-- (Innermost) Boolean values represent needsseparatenode value for given
-- shard. For example,
--
-- {"15": [{"dist_1": false},{"dist_3": true}]}
--
-- means that the placements of dist_3 within shard group 15 needs
-- to be isolated.
CREATE OR REPLACE FUNCTION get_colocated_shards_needisolatednode(
qualified_table_name text)
RETURNS SETOF jsonb AS $func$
BEGIN
RETURN QUERY
SELECT
COALESCE(
jsonb_agg(jsonb_build_object(shardgroupindex, needsseparatenodejson) ORDER BY shardgroupindex),
'{}'::jsonb
) AS result
FROM (
SELECT shardgroupindex,
jsonb_agg(jsonb_build_object(logicalrelid, needsseparatenode) ORDER BY logicalrelid::text) AS needsseparatenodejson
FROM (
SELECT shardgroupindex,
logicalrelid,
needsseparatenode
FROM public.get_enumerated_shard_groups(qualified_table_name) AS shardgroups
JOIN pg_dist_shard
ON shardid = ANY(shardids)
) q1
GROUP BY shardgroupindex
) q2
WHERE needsseparatenodejson::text LIKE '%true%';
END;
$func$ LANGUAGE plpgsql;
-- Returns true if all placement groups within given shard group are isolated.
CREATE OR REPLACE FUNCTION verify_placements_in_shard_group_isolated(
qualified_table_name text,
shard_group_index bigint)
RETURNS boolean
AS $func$
DECLARE
v_result boolean;
BEGIN
SELECT bool_and(has_separate_node) INTO v_result
FROM citus_shards
JOIN (
SELECT shardids FROM public.get_enumerated_shard_groups(qualified_table_name) WHERE shardgroupindex = shard_group_index
) q
ON (shardid = ANY(q.shardids));
RETURN v_result;
END;
$func$ LANGUAGE plpgsql;

View File

@ -0,0 +1,16 @@
-- upgrade_columnar_before renames public to citus_schema and recreates public
-- schema. But this file depends on get_colocated_shards_needisolatednode()
-- function and get_colocated_shards_needisolatednode() depends on another
-- function --get_enumerated_shard_groups()-- that is presumably created earlier
-- within the original public schema, so we temporarily rename citus_schema to
-- public here; and revert those changes at the end of this file.
ALTER SCHEMA public RENAME TO old_public;
ALTER SCHEMA citus_schema RENAME TO public;
SELECT result FROM run_command_on_all_nodes($$
SELECT * FROM public.get_colocated_shards_needisolatednode('upgrade_isolate_placement_before.table_with_isolated_placements')
$$)
ORDER BY nodeid;
ALTER SCHEMA public RENAME TO citus_schema;
ALTER SCHEMA old_public RENAME TO public;

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_property_set(:shardgroup_5_shardid, anti_affinity=>true);
SELECT result FROM run_command_on_all_nodes($$
SELECT * FROM public.get_colocated_shards_needisolatednode('upgrade_isolate_placement_before.table_with_isolated_placements')
$$)
ORDER BY nodeid;