Introduce Non-Blocking Shard Split Workflow

pull/6133/head
Sameer Awasekar 2022-08-04 16:32:38 +02:00 committed by Marco Slot
parent b67abdd28c
commit e236711eea
56 changed files with 8964 additions and 154 deletions

View File

@ -18,7 +18,7 @@ generated_downgrade_sql_files += $(patsubst %,$(citus_abs_srcdir)/build/sql/%,$(
DATA_built = $(generated_sql_files)
# directories with source files
SUBDIRS = . commands connection ddl deparser executor metadata operations planner progress relay safeclib test transaction utils worker
SUBDIRS = . commands connection ddl deparser executor metadata operations planner progress relay safeclib shardsplit test transaction utils worker
# enterprise modules
SUBDIRS += replication

View File

@ -377,6 +377,14 @@ GetConnParams(ConnectionHashKey *key, char ***keywords, char ***values,
authParamsIdx++;
}
if (key->replicationConnParam)
{
connKeywords[authParamsIdx] = MemoryContextStrdup(context, "replication");
connValues[authParamsIdx] = MemoryContextStrdup(context, "database");
authParamsIdx++;
}
PQconninfoFree(optionArray);
/* final step: add terminal NULL, required by libpq */

View File

@ -290,6 +290,15 @@ StartNodeUserDatabaseConnection(uint32 flags, const char *hostname, int32 port,
strlcpy(key.database, CurrentDatabaseName(), NAMEDATALEN);
}
if (flags & REQUIRE_REPLICATION_CONNECTION_PARAM)
{
key.replicationConnParam = true;
}
else
{
key.replicationConnParam = false;
}
if (CurrentCoordinatedTransactionState == COORD_TRANS_NONE)
{
CurrentCoordinatedTransactionState = COORD_TRANS_IDLE;
@ -597,6 +606,7 @@ ConnectionAvailableToNode(char *hostName, int nodePort, const char *userName,
key.port = nodePort;
strlcpy(key.user, userName, NAMEDATALEN);
strlcpy(key.database, database, NAMEDATALEN);
key.replicationConnParam = false;
ConnectionHashEntry *entry =
(ConnectionHashEntry *) hash_search(ConnectionHash, &key, HASH_FIND, &found);
@ -666,6 +676,7 @@ CloseConnection(MultiConnection *connection)
strlcpy(key.hostname, connection->hostname, MAX_NODE_LENGTH);
key.port = connection->port;
key.replicationConnParam = connection->requiresReplication;
strlcpy(key.user, connection->user, NAMEDATALEN);
strlcpy(key.database, connection->database, NAMEDATALEN);
@ -1210,6 +1221,7 @@ ConnectionHashHash(const void *key, Size keysize)
hash = hash_combine(hash, hash_uint32(entry->port));
hash = hash_combine(hash, string_hash(entry->user, NAMEDATALEN));
hash = hash_combine(hash, string_hash(entry->database, NAMEDATALEN));
hash = hash_combine(hash, hash_uint32(entry->replicationConnParam));
return hash;
}
@ -1223,6 +1235,7 @@ ConnectionHashCompare(const void *a, const void *b, Size keysize)
if (strncmp(ca->hostname, cb->hostname, MAX_NODE_LENGTH) != 0 ||
ca->port != cb->port ||
ca->replicationConnParam != cb->replicationConnParam ||
strncmp(ca->user, cb->user, NAMEDATALEN) != 0 ||
strncmp(ca->database, cb->database, NAMEDATALEN) != 0)
{
@ -1250,6 +1263,7 @@ StartConnectionEstablishment(MultiConnection *connection, ConnectionHashKey *key
connection->port = key->port;
strlcpy(connection->database, key->database, NAMEDATALEN);
strlcpy(connection->user, key->user, NAMEDATALEN);
connection->requiresReplication = key->replicationConnParam;
connection->pgConn = PQconnectStartParams((const char **) entry->keywords,
(const char **) entry->values,
@ -1286,6 +1300,7 @@ WarmUpConnParamsHash(void)
key.port = workerNode->workerPort;
strlcpy(key.database, CurrentDatabaseName(), NAMEDATALEN);
strlcpy(key.user, CurrentUserName(), NAMEDATALEN);
key.replicationConnParam = false;
FindOrCreateConnParamsEntry(&key);
}
}
@ -1460,6 +1475,7 @@ ShouldShutdownConnection(MultiConnection *connection, const int cachedConnection
connection->forceCloseAtTransactionEnd ||
PQstatus(connection->pgConn) != CONNECTION_OK ||
!RemoteTransactionIdle(connection) ||
connection->requiresReplication ||
(MaxCachedConnectionLifetime >= 0 &&
MillisecondsToTimeout(connection->connectionEstablishmentStart,
MaxCachedConnectionLifetime) <= 0);

View File

@ -248,7 +248,6 @@ static void RegisterWorkerNodeCacheCallbacks(void);
static void RegisterLocalGroupIdCacheCallbacks(void);
static void RegisterAuthinfoCacheCallbacks(void);
static void RegisterCitusTableCacheEntryReleaseCallbacks(void);
static uint32 WorkerNodeHashCode(const void *key, Size keySize);
static void ResetCitusTableCacheEntry(CitusTableCacheEntry *cacheEntry);
static void RemoveStaleShardIdCacheEntries(CitusTableCacheEntry *tableEntry);
static void CreateDistTableCache(void);
@ -3966,26 +3965,6 @@ RegisterAuthinfoCacheCallbacks(void)
}
/*
* WorkerNodeHashCode computes the hash code for a worker node from the node's
* host name and port number. Nodes that only differ by their rack locations
* hash to the same value.
*/
static uint32
WorkerNodeHashCode(const void *key, Size keySize)
{
const WorkerNode *worker = (const WorkerNode *) key;
const char *workerName = worker->workerName;
const uint32 *workerPort = &(worker->workerPort);
/* standard hash function outlined in Effective Java, Item 8 */
uint32 result = 17;
result = 37 * result + string_hash(workerName, WORKER_LENGTH);
result = 37 * result + tag_hash(workerPort, sizeof(uint32));
return result;
}
/*
* ResetCitusTableCacheEntry frees any out-of-band memory used by a cache entry,
* but does not free the entry itself.

View File

@ -1437,6 +1437,27 @@ ActiveShardPlacement(uint64 shardId, bool missingOk)
}
/*
* ActiveShardPlacementWorkerNode returns the worker node of the first placement of
* a shard.
*/
WorkerNode *
ActiveShardPlacementWorkerNode(uint64 shardId)
{
bool missingOK = false;
List *sourcePlacementList = ActiveShardPlacementList(shardId);
Assert(sourcePlacementList->length == 1);
ShardPlacement *sourceShardPlacement = linitial(sourcePlacementList);
WorkerNode *sourceShardToCopyNode = FindNodeWithNodeId(sourceShardPlacement->nodeId,
missingOK);
return sourceShardToCopyNode;
}
/*
* BuildShardPlacementList finds shard placements for the given shardId from
* system catalogs, converts these placements to their in-memory

View File

@ -81,18 +81,18 @@ LookupSplitMode(Oid shardTransferModeOid)
{
shardSplitMode = BLOCKING_SPLIT;
}
/* TODO(saawaek): Handle this appropriately based on replica identity */
else if (strncmp(enumLabel, "auto", NAMEDATALEN) == 0 ||
strncmp(enumLabel, "force_logical", NAMEDATALEN) == 0)
{
ereport(ERROR, (errmsg(
"Shard Tranfer mode: '%s' is not supported. Please use 'block_writes' instead.",
enumLabel)));
shardSplitMode = NON_BLOCKING_SPLIT;
}
else
{
/* We will not get here as postgres will validate the enum value. */
ereport(ERROR, (errmsg(
"Invalid shard tranfer mode: '%s'. Expected split mode is 'block_writes'.",
"Invalid shard tranfer mode: '%s'. Expected split mode is 'block_writes/auto/force_logical'.",
enumLabel)));
}

View File

@ -10,6 +10,7 @@
*/
#include "postgres.h"
#include "miscadmin.h"
#include "common/hashfn.h"
#include "nodes/pg_list.h"
#include "utils/array.h"
@ -34,8 +35,11 @@
#include "distributed/pg_dist_shard.h"
#include "distributed/metadata_sync.h"
#include "distributed/multi_physical_planner.h"
#include "commands/dbcommands.h"
#include "distributed/shardsplit_logical_replication.h"
#include "distributed/deparse_shard_query.h"
#include "distributed/shard_rebalancer.h"
#include "postmaster/postmaster.h"
/*
* Entry for map that tracks ShardInterval -> Placement Node
@ -61,8 +65,18 @@ static void CreateAndCopySplitShardsForShardGroup(
static void CreateSplitShardsForShardGroup(HTAB *mapOfShardToPlacementCreatedByWorkflow,
List *shardGroupSplitIntervalListList,
List *workersForPlacementList);
static void CreateDummyShardsForShardGroup(HTAB *mapOfDummyShardToPlacement,
List *sourceColocatedShardIntervalList,
List *shardGroupSplitIntervalListList,
WorkerNode *sourceWorkerNode,
List *workersForPlacementList);
static HTAB * CreateWorkerForPlacementSet(List *workersForPlacementList);
static void CreateAuxiliaryStructuresForShardGroup(List *shardGroupSplitIntervalListList,
List *workersForPlacementList);
List *workersForPlacementList,
bool includeReplicaIdentity);
static void CreateReplicaIdentities(HTAB *mapOfDummyShardToPlacement,
List *shardGroupSplitIntervalListList,
List *workersForPlacementList);
static void CreateObjectOnPlacement(List *objectCreationCommandList,
WorkerNode *workerNode);
static List * CreateSplitIntervalsForShardGroup(List *sourceColocatedShardList,
@ -74,13 +88,20 @@ static void BlockingShardSplit(SplitOperation splitOperation,
ShardInterval *shardIntervalToSplit,
List *shardSplitPointsList,
List *workersForPlacementList);
static void NonBlockingShardSplit(SplitOperation splitOperation,
ShardInterval *shardIntervalToSplit,
List *shardSplitPointsList,
List *workersForPlacementList);
static void DoSplitCopy(WorkerNode *sourceShardNode,
List *sourceColocatedShardIntervalList,
List *shardGroupSplitIntervalListList,
List *workersForPlacementList);
List *workersForPlacementList,
char *snapShotName);
static StringInfo CreateSplitCopyCommand(ShardInterval *sourceShardSplitInterval,
List *splitChildrenShardIntervalList,
List *workersForPlacementList);
static Task * CreateSplitCopyTask(StringInfo splitCopyUdfCommand, char *snapshotName, int
taskId, uint64 jobId);
static void InsertSplitChildrenShardMetadata(List *shardGroupSplitIntervalListList,
List *workersForPlacementList);
static void CreatePartitioningHierarchy(List *shardGroupSplitIntervalListList,
@ -90,6 +111,25 @@ static void CreateForeignKeyConstraints(List *shardGroupSplitIntervalListList,
static void TryDropSplitShardsOnFailure(HTAB *mapOfShardToPlacementCreatedByWorkflow);
static HTAB * CreateEmptyMapForShardsCreatedByWorkflow();
static Task * CreateTaskForDDLCommandList(List *ddlCommandList, WorkerNode *workerNode);
static StringInfo CreateSplitShardReplicationSetupUDF(
List *sourceColocatedShardIntervalList, List *shardGroupSplitIntervalListList,
List *destinationWorkerNodesList);
static char * CreateTemplateReplicationSlotAndReturnSnapshot(ShardInterval *shardInterval,
WorkerNode *sourceWorkerNode,
MultiConnection **
templateSlotConnection);
static List * ParseReplicationSlotInfoFromResult(PGresult *result);
static List * ExecuteSplitShardReplicationSetupUDF(WorkerNode *sourceWorkerNode,
List *sourceColocatedShardIntervalList,
List *shardGroupSplitIntervalListList,
List *destinationWorkerNodesList);
static void AddDummyShardEntryInMap(HTAB *mapOfDummyShards, uint32 targetNodeId,
ShardInterval *shardInterval);
static void DropDummyShards(HTAB *mapOfDummyShardToPlacement);
static void DropDummyShard(MultiConnection *connection, ShardInterval *shardInterval);
static uint64 GetNextShardIdForSplitChild(void);
/* Customize error message strings based on operation type */
static const char *const SplitOperationName[] =
@ -382,8 +422,11 @@ SplitShard(SplitMode splitMode,
}
else
{
/* we only support blocking shard split in this code path for now. */
ereport(ERROR, (errmsg("Invalid split mode value %d.", splitMode)));
NonBlockingShardSplit(
splitOperation,
shardIntervalToSplit,
shardSplitPointsList,
workersForPlacementList);
}
}
@ -611,7 +654,8 @@ CreateTaskForDDLCommandList(List *ddlCommandList, WorkerNode *workerNode)
*/
static void
CreateAuxiliaryStructuresForShardGroup(List *shardGroupSplitIntervalListList,
List *workersForPlacementList)
List *workersForPlacementList, bool
includeReplicaIdentity)
{
List *shardIntervalList = NIL;
List *ddlTaskExecList = NIL;
@ -634,7 +678,7 @@ CreateAuxiliaryStructuresForShardGroup(List *shardGroupSplitIntervalListList,
List *ddlCommandList = GetPostLoadTableCreationCommands(
shardInterval->relationId,
true /* includeIndexes */,
true /* includeReplicaIdentity */);
includeReplicaIdentity);
ddlCommandList = WorkerApplyShardDDLCommandList(
ddlCommandList,
shardInterval->shardId);
@ -676,12 +720,15 @@ CreateAndCopySplitShardsForShardGroup(HTAB *mapOfShardToPlacementCreatedByWorkfl
shardGroupSplitIntervalListList,
workersForPlacementList);
/* For Blocking split, copy isn't snapshotted */
char *snapshotName = NULL;
DoSplitCopy(sourceShardNode, sourceColocatedShardIntervalList,
shardGroupSplitIntervalListList, workersForPlacementList);
shardGroupSplitIntervalListList, workersForPlacementList, snapshotName);
/* Create auxiliary structures (indexes, stats, replicaindentities, triggers) */
CreateAuxiliaryStructuresForShardGroup(shardGroupSplitIntervalListList,
workersForPlacementList);
workersForPlacementList,
true /* includeReplicaIdentity*/);
}
@ -694,7 +741,8 @@ CreateAndCopySplitShardsForShardGroup(HTAB *mapOfShardToPlacementCreatedByWorkfl
*/
static void
DoSplitCopy(WorkerNode *sourceShardNode, List *sourceColocatedShardIntervalList,
List *shardGroupSplitIntervalListList, List *destinationWorkerNodesList)
List *shardGroupSplitIntervalListList, List *destinationWorkerNodesList,
char *snapShotName)
{
ShardInterval *sourceShardIntervalToCopy = NULL;
List *splitShardIntervalList = NIL;
@ -716,15 +764,13 @@ DoSplitCopy(WorkerNode *sourceShardNode, List *sourceColocatedShardIntervalList,
splitShardIntervalList,
destinationWorkerNodesList);
Task *splitCopyTask = CreateBasicTask(
INVALID_JOB_ID,
taskId,
READ_TASK,
splitCopyUdfCommand->data);
/* Create copy task. Snapshot name is required for nonblocking splits */
Task *splitCopyTask = CreateSplitCopyTask(splitCopyUdfCommand, snapShotName,
taskId,
sourceShardIntervalToCopy->shardId);
ShardPlacement *taskPlacement = CitusMakeNode(ShardPlacement);
SetPlacementNodeMetadata(taskPlacement, sourceShardNode);
splitCopyTask->taskPlacementList = list_make1(taskPlacement);
splitCopyTaskList = lappend(splitCopyTaskList, splitCopyTask);
@ -801,6 +847,48 @@ CreateSplitCopyCommand(ShardInterval *sourceShardSplitInterval,
}
/*
* CreateSplitCopyTask creates a task for copying data.
* In the case of Non-blocking split, snapshotted copy task is created with given 'snapshotName'.
* 'snapshotName' is NULL for Blocking split.
*/
static Task *
CreateSplitCopyTask(StringInfo splitCopyUdfCommand, char *snapshotName, int taskId, uint64
jobId)
{
List *ddlCommandList = NIL;
StringInfo beginTransaction = makeStringInfo();
appendStringInfo(beginTransaction,
"BEGIN TRANSACTION ISOLATION LEVEL REPEATABLE READ;");
ddlCommandList = lappend(ddlCommandList, beginTransaction->data);
/* Set snapshot for non-blocking shard split. */
if (snapshotName != NULL)
{
StringInfo snapShotString = makeStringInfo();
appendStringInfo(snapShotString, "SET TRANSACTION SNAPSHOT %s;",
quote_literal_cstr(
snapshotName));
ddlCommandList = lappend(ddlCommandList, snapShotString->data);
}
ddlCommandList = lappend(ddlCommandList, splitCopyUdfCommand->data);
StringInfo commitCommand = makeStringInfo();
appendStringInfo(commitCommand, "COMMIT;");
ddlCommandList = lappend(ddlCommandList, commitCommand->data);
Task *splitCopyTask = CitusMakeNode(Task);
splitCopyTask->jobId = jobId;
splitCopyTask->taskId = taskId;
splitCopyTask->taskType = READ_TASK;
splitCopyTask->replicationModel = REPLICATION_MODEL_INVALID;
SetTaskQueryStringList(splitCopyTask, ddlCommandList);
return splitCopyTask;
}
/*
* Create an object on a worker node.
*/
@ -868,7 +956,7 @@ CreateSplitIntervalsForShard(ShardInterval *sourceShard,
{
ShardInterval *splitChildShardInterval = CopyShardInterval(sourceShard);
splitChildShardInterval->shardIndex = -1;
splitChildShardInterval->shardId = GetNextShardId();
splitChildShardInterval->shardId = GetNextShardIdForSplitChild();
splitChildShardInterval->minValueExists = true;
splitChildShardInterval->minValue = currentSplitChildMinValue;
@ -1149,3 +1237,778 @@ TryDropSplitShardsOnFailure(HTAB *mapOfShardToPlacementCreatedByWorkflow)
NULL /* pgResult */);
}
}
/*
* SplitShard API to split a given shard (or shard group) in non-blocking fashion
* based on specified split points to a set of destination nodes.
* splitOperation : Customer operation that triggered split.
* shardIntervalToSplit : Source shard interval to be split.
* shardSplitPointsList : Split Points list for the source 'shardInterval'.
* workersForPlacementList : Placement list corresponding to split children.
*/
static void
NonBlockingShardSplit(SplitOperation splitOperation,
ShardInterval *shardIntervalToSplit,
List *shardSplitPointsList,
List *workersForPlacementList)
{
char *superUser = CitusExtensionOwnerName();
char *databaseName = get_database_name(MyDatabaseId);
List *sourceColocatedShardIntervalList = ColocatedShardIntervalList(
shardIntervalToSplit);
/* First create shard interval metadata for split children */
List *shardGroupSplitIntervalListList = CreateSplitIntervalsForShardGroup(
sourceColocatedShardIntervalList,
shardSplitPointsList);
WorkerNode *sourceShardToCopyNode = ActiveShardPlacementWorkerNode(
shardIntervalToSplit->shardId);
/* Create hashmap to group shards for publication-subscription management */
HTAB *shardSplitHashMapForPublication = CreateShardSplitInfoMapForPublication(
sourceColocatedShardIntervalList,
shardGroupSplitIntervalListList,
workersForPlacementList);
DropAllShardSplitLeftOvers(sourceShardToCopyNode, shardSplitHashMapForPublication);
int connectionFlags = FORCE_NEW_CONNECTION;
MultiConnection *sourceConnection = GetNodeUserDatabaseConnection(connectionFlags,
sourceShardToCopyNode
->
workerName,
sourceShardToCopyNode
->
workerPort,
superUser,
databaseName);
ClaimConnectionExclusively(sourceConnection);
HTAB *mapOfShardToPlacementCreatedByWorkflow =
CreateEmptyMapForShardsCreatedByWorkflow();
HTAB *mapOfDummyShardToPlacement = SetupHashMapForShardInfo();
/* Non-Blocking shard split workflow starts here */
PG_TRY();
{
/* 1) Physically create split children. */
CreateSplitShardsForShardGroup(mapOfShardToPlacementCreatedByWorkflow,
shardGroupSplitIntervalListList,
workersForPlacementList);
/*
* 2) Create dummy shards due to PG logical replication constraints.
* Refer to the comment section of 'CreateDummyShardsForShardGroup' for indepth
* information.
*/
CreateDummyShardsForShardGroup(
mapOfDummyShardToPlacement,
sourceColocatedShardIntervalList,
shardGroupSplitIntervalListList,
sourceShardToCopyNode,
workersForPlacementList);
CreateReplicaIdentities(mapOfDummyShardToPlacement,
shardGroupSplitIntervalListList, workersForPlacementList);
/* 3) Create Publications. */
CreateShardSplitPublications(sourceConnection, shardSplitHashMapForPublication);
/*
* 4) Create template replication Slot. It returns a snapshot. The snapshot remains
* valid till the lifetime of the session that creates it. The connection is closed
* at the end of the workflow.
*/
MultiConnection *templateSlotConnection = NULL;
char *snapShotName = CreateTemplateReplicationSlotAndReturnSnapshot(
shardIntervalToSplit, sourceShardToCopyNode, &templateSlotConnection);
/* 5) Do snapshotted Copy */
DoSplitCopy(sourceShardToCopyNode, sourceColocatedShardIntervalList,
shardGroupSplitIntervalListList, workersForPlacementList,
snapShotName);
/* 6) Execute 'worker_split_shard_replication_setup UDF */
List *replicationSlotInfoList = ExecuteSplitShardReplicationSetupUDF(
sourceShardToCopyNode,
sourceColocatedShardIntervalList,
shardGroupSplitIntervalListList,
workersForPlacementList);
/*
* Subscriber flow starts from here.
* Populate 'ShardSplitSubscriberMetadata' for subscription management.
*/
List *shardSplitSubscribersMetadataList =
PopulateShardSplitSubscriptionsMetadataList(
shardSplitHashMapForPublication, replicationSlotInfoList);
/* Create connections to the target nodes */
List *targetNodeConnectionList = CreateTargetNodeConnectionsForShardSplit(
shardSplitSubscribersMetadataList,
connectionFlags,
superUser, databaseName);
/* 7) Create copies of template replication slot */
char *templateSlotName = ShardSplitTemplateReplicationSlotName(
shardIntervalToSplit->shardId);
CreateReplicationSlots(sourceConnection, templateSlotName,
shardSplitSubscribersMetadataList);
/* 8) Create subscriptions on target nodes */
CreateShardSplitSubscriptions(targetNodeConnectionList,
shardSplitSubscribersMetadataList,
sourceShardToCopyNode,
superUser,
databaseName);
/* Used for testing */
ConflictOnlyWithIsolationTesting();
/* 9) Wait for subscriptions to be ready */
WaitForShardSplitRelationSubscriptionsBecomeReady(
shardSplitSubscribersMetadataList);
/* 10) Wait for subscribers to catchup till source LSN */
XLogRecPtr sourcePosition = GetRemoteLogPosition(sourceConnection);
WaitForShardSplitRelationSubscriptionsToBeCaughtUp(sourcePosition,
shardSplitSubscribersMetadataList);
/* 11) Create Auxilary structures */
CreateAuxiliaryStructuresForShardGroup(shardGroupSplitIntervalListList,
workersForPlacementList,
false /* includeReplicaIdentity*/);
/* 12) Wait for subscribers to catchup till source LSN */
sourcePosition = GetRemoteLogPosition(sourceConnection);
WaitForShardSplitRelationSubscriptionsToBeCaughtUp(sourcePosition,
shardSplitSubscribersMetadataList);
/* 13) Block writes on source shards */
BlockWritesToShardList(sourceColocatedShardIntervalList);
/* 14) Wait for subscribers to catchup till source LSN */
sourcePosition = GetRemoteLogPosition(sourceConnection);
WaitForShardSplitRelationSubscriptionsToBeCaughtUp(sourcePosition,
shardSplitSubscribersMetadataList);
/* 15) Drop Subscribers */
DropShardSplitSubsriptions(shardSplitSubscribersMetadataList);
/* 16) Drop Publications */
DropShardSplitPublications(sourceConnection, shardSplitHashMapForPublication);
/* 17) Drop replication slots
* Drop template and subscriber replication slots
*/
DropShardReplicationSlot(sourceConnection, ShardSplitTemplateReplicationSlotName(
shardIntervalToSplit->shardId));
DropShardSplitReplicationSlots(sourceConnection, replicationSlotInfoList);
/*
* 18) Drop old shards and delete related metadata. Have to do that before
* creating the new shard metadata, because there's cross-checks
* preventing inconsistent metadata (like overlapping shards).
*/
DropShardList(sourceColocatedShardIntervalList);
/* 19) Insert new shard and placement metdata */
InsertSplitChildrenShardMetadata(shardGroupSplitIntervalListList,
workersForPlacementList);
CreatePartitioningHierarchy(shardGroupSplitIntervalListList,
workersForPlacementList);
/*
* 20) Create foreign keys if exists after the metadata changes happening in
* DropShardList() and InsertSplitChildrenShardMetadata() because the foreign
* key creation depends on the new metadata.
*/
CreateForeignKeyConstraints(shardGroupSplitIntervalListList,
workersForPlacementList);
/*
* 21) Drop dummy shards.
*/
DropDummyShards(mapOfDummyShardToPlacement);
/* 22) Close source connection */
CloseConnection(sourceConnection);
/* 23) Close all subscriber connections */
CloseShardSplitSubscriberConnections(shardSplitSubscribersMetadataList);
/* 24) Close connection of template replication slot */
CloseConnection(templateSlotConnection);
}
PG_CATCH();
{
/* Do a best effort cleanup of shards created on workers in the above block */
TryDropSplitShardsOnFailure(mapOfShardToPlacementCreatedByWorkflow);
DropAllShardSplitLeftOvers(sourceShardToCopyNode,
shardSplitHashMapForPublication);
DropDummyShards(mapOfDummyShardToPlacement);
PG_RE_THROW();
}
PG_END_TRY();
}
/*
* Given we are using PG logical replication infrastructure there are some constraints
* that need to met around matching table names in source and target nodes:
* The restrictions in context of split are:
* Constraint 1: Dummy source shard(s) from shard group must exist on all destination nodes.
* Constraint 2: Dummy target shards from shard group must exist on source node.
* Example :
* Shard1[1-200] is co-located with Shard2[1-200] in Worker0.
* We are splitting 2-way to worker0 (same node) and worker1 (different node).
*
* Non-Dummy shards (expected from Split):
* In Worker0 --> Shard1_1 and Shard2_1.
* In Worker1 --> Shard1_2 and Shard2_2.
*
* Dummy shards:
* From constraint 1, we need to create: Dummy Shard1 and Shard2 in Worker0. Dummy Shard1 and Shard2 in Worker1
* Note 1 : Given there is an overlap of source and destination in Worker0, Shard1 and Shard2 need not be created.
* Be very careful here, dropping Shard1, Shard2 with customer data to create dummy Shard1, Shard2 on worker0 is catastrophic.
*
* From constraint 2, we need to create: Dummy Shard1_1, Shard2_1, Shard1_2 and Shard2_2 in Worker0.
* Note 2 : Given there is an overlap of source and destination in Worker0, Shard1_1 and Shard2_1 need not be created.
*/
static void
CreateDummyShardsForShardGroup(HTAB *mapOfDummyShardToPlacement,
List *sourceColocatedShardIntervalList,
List *shardGroupSplitIntervalListList,
WorkerNode *sourceWorkerNode,
List *workersForPlacementList)
{
/*
* Statisfy Constraint 1: Create dummy source shard(s) on all destination nodes.
* If source node is also in desintation, skip dummy shard creation(see Note 1 from function description).
* We are guarenteed to have a single active placement for source shard. This is enforced earlier by ErrorIfCannotSplitShardExtended.
*/
/* List 'workersForPlacementList' can have duplicates. We need all unique destination nodes. */
HTAB *workersForPlacementSet = CreateWorkerForPlacementSet(workersForPlacementList);
HASH_SEQ_STATUS status;
hash_seq_init(&status, workersForPlacementSet);
WorkerNode *workerPlacementNode = NULL;
while ((workerPlacementNode = (WorkerNode *) hash_seq_search(&status)) != NULL)
{
if (workerPlacementNode->nodeId == sourceWorkerNode->nodeId)
{
continue;
}
ShardInterval *shardInterval = NULL;
foreach_ptr(shardInterval, sourceColocatedShardIntervalList)
{
/* Populate list of commands necessary to create shard interval on destination */
List *splitShardCreationCommandList = GetPreLoadTableCreationCommands(
shardInterval->relationId,
false, /* includeSequenceDefaults */
NULL /* auto add columnar options for cstore tables */);
splitShardCreationCommandList = WorkerApplyShardDDLCommandList(
splitShardCreationCommandList,
shardInterval->shardId);
/* Create dummy source shard on the specified placement list */
CreateObjectOnPlacement(splitShardCreationCommandList, workerPlacementNode);
/* Add dummy source shard entry created for placement node in map */
AddDummyShardEntryInMap(mapOfDummyShardToPlacement,
workerPlacementNode->nodeId,
shardInterval);
}
}
/*
* Statisfy Constraint 2: Create dummy target shards from shard group on source node.
* If the target shard was created on source node as placement, skip it (See Note 2 from function description).
*/
List *shardIntervalList = NULL;
foreach_ptr(shardIntervalList, shardGroupSplitIntervalListList)
{
ShardInterval *shardInterval = NULL;
workerPlacementNode = NULL;
forboth_ptr(shardInterval, shardIntervalList, workerPlacementNode,
workersForPlacementList)
{
if (workerPlacementNode->nodeId == sourceWorkerNode->nodeId)
{
continue;
}
List *splitShardCreationCommandList = GetPreLoadTableCreationCommands(
shardInterval->relationId,
false, /* includeSequenceDefaults */
NULL /* auto add columnar options for cstore tables */);
splitShardCreationCommandList = WorkerApplyShardDDLCommandList(
splitShardCreationCommandList,
shardInterval->shardId);
/* Create dummy split child shard on source worker node */
CreateObjectOnPlacement(splitShardCreationCommandList, sourceWorkerNode);
/* Add dummy split child shard entry created on source node */
AddDummyShardEntryInMap(mapOfDummyShardToPlacement, sourceWorkerNode->nodeId,
shardInterval);
}
}
}
/*
* CreateWorkerForPlacementSet returns a set with unique worker nodes.
*/
static HTAB *
CreateWorkerForPlacementSet(List *workersForPlacementList)
{
HASHCTL info = { 0 };
info.keysize = sizeof(WorkerNode);
info.hash = WorkerNodeHashCode;
info.match = WorkerNodeCompare;
/* we don't have value field as it's a set */
info.entrysize = info.keysize;
uint32 hashFlags = (HASH_ELEM | HASH_FUNCTION | HASH_CONTEXT);
HTAB *workerForPlacementSet = hash_create("worker placement set", 32, &info,
hashFlags);
WorkerNode *workerForPlacement = NULL;
foreach_ptr(workerForPlacement, workersForPlacementList)
{
void *hashKey = (void *) workerForPlacement;
hash_search(workerForPlacementSet, hashKey, HASH_ENTER, NULL);
}
return workerForPlacementSet;
}
/*
* CreateTemplateReplicationSlotAndReturnSnapshot creates a replication slot
* and returns its snapshot. This slot acts as a 'Template' for creating
* replication slot copies used for logical replication.
*
* The snapshot remains valid till the lifetime of the session that creates it.
*/
char *
CreateTemplateReplicationSlotAndReturnSnapshot(ShardInterval *shardInterval,
WorkerNode *sourceWorkerNode,
MultiConnection **templateSlotConnection)
{
/*Create Template replication slot */
int connectionFlags = FORCE_NEW_CONNECTION;
connectionFlags |= REQUIRE_REPLICATION_CONNECTION_PARAM;
MultiConnection *sourceConnection = GetNodeUserDatabaseConnection(connectionFlags,
sourceWorkerNode->
workerName,
sourceWorkerNode->
workerPort,
CitusExtensionOwnerName(),
get_database_name(
MyDatabaseId));
ClaimConnectionExclusively(sourceConnection);
/*
* Try to drop leftover template replication slot if any from previous operation
* and create new one.
*/
char *snapShotName = CreateTemplateReplicationSlot(shardInterval,
sourceConnection);
*templateSlotConnection = sourceConnection;
return snapShotName;
}
/*
* ExecuteSplitShardReplicationSetupUDF executes
* 'worker_split_shard_replication_setup' UDF on source shard node
* and returns list of ReplicationSlotInfo.
*/
static List *
ExecuteSplitShardReplicationSetupUDF(WorkerNode *sourceWorkerNode,
List *sourceColocatedShardIntervalList,
List *shardGroupSplitIntervalListList,
List *destinationWorkerNodesList)
{
StringInfo splitShardReplicationUDF = CreateSplitShardReplicationSetupUDF(
sourceColocatedShardIntervalList,
shardGroupSplitIntervalListList,
destinationWorkerNodesList);
/* Force a new connection to execute the UDF */
int connectionFlags = 0;
MultiConnection *sourceConnection = GetNodeUserDatabaseConnection(connectionFlags,
sourceWorkerNode->
workerName,
sourceWorkerNode->
workerPort,
CitusExtensionOwnerName(),
get_database_name(
MyDatabaseId));
ClaimConnectionExclusively(sourceConnection);
PGresult *result = NULL;
int queryResult = ExecuteOptionalRemoteCommand(sourceConnection,
splitShardReplicationUDF->data,
&result);
/*
* Result should contain atleast one tuple. The information returned is
* set of tuples where each tuple is formatted as:
* <targetNodeId, tableOwnerName, replication_slot_name>.
*/
if (queryResult != RESPONSE_OKAY || !IsResponseOK(result) || PQntuples(result) < 1 ||
PQnfields(result) != 3)
{
PQclear(result);
ForgetResults(sourceConnection);
CloseConnection(sourceConnection);
ereport(ERROR, (errcode(ERRCODE_CONNECTION_FAILURE),
errmsg(
"Failed to run worker_split_shard_replication_setup UDF. It should successfully execute "
" for splitting a shard in a non-blocking way. Please retry.")));
}
/* Get replication slot information */
List *replicationSlotInfoList = ParseReplicationSlotInfoFromResult(result);
PQclear(result);
ForgetResults(sourceConnection);
CloseConnection(sourceConnection);
return replicationSlotInfoList;
}
/*
* CreateSplitShardReplicationSetupUDF creates and returns
* parameterized 'worker_split_shard_replication_setup' UDF command.
*
* 'sourceShardSplitIntervalList' : Source shard interval to split.
* 'shardGroupSplitIntervalListList' : List of shard intervals for split children..
* 'destinationWorkerNodesList' : List of workers for split children placement.
*
* For example consider below input values:
* sourceColocatedShardIntervalList : [sourceShardInterval]
* shardGroupSplitIntervalListList : [<childFirstShardInterval, childSecondShardInterval>]
* destinationWorkerNodesList : [worker1, worker2]
*
* SELECT * FROM worker_split_shard_replication_setup(
* Array[
* ROW(sourceShardId, childFirstShardId, childFirstMinRange, childFirstMaxRange, worker1)::citus.split_shard_info,
* ROW(sourceShardId, childSecondShardId, childSecondMinRange, childSecondMaxRange, worker2)::citus.split_shard_info
* ]);
*/
StringInfo
CreateSplitShardReplicationSetupUDF(List *sourceColocatedShardIntervalList,
List *shardGroupSplitIntervalListList,
List *destinationWorkerNodesList)
{
StringInfo splitChildrenRows = makeStringInfo();
ShardInterval *sourceShardIntervalToCopy = NULL;
List *splitChildShardIntervalList = NULL;
bool addComma = false;
forboth_ptr(sourceShardIntervalToCopy, sourceColocatedShardIntervalList,
splitChildShardIntervalList, shardGroupSplitIntervalListList)
{
int64 sourceShardId = sourceShardIntervalToCopy->shardId;
Oid relationId = sourceShardIntervalToCopy->relationId;
Var *partitionColumn = DistPartitionKey(relationId);
bool missingOK = false;
char *partitionColumnName =
get_attname(relationId, partitionColumn->varattno, missingOK);
ShardInterval *splitChildShardInterval = NULL;
WorkerNode *destinationWorkerNode = NULL;
forboth_ptr(splitChildShardInterval, splitChildShardIntervalList,
destinationWorkerNode, destinationWorkerNodesList)
{
if (addComma)
{
appendStringInfo(splitChildrenRows, ",");
}
StringInfo minValueString = makeStringInfo();
appendStringInfo(minValueString, "%d", DatumGetInt32(
splitChildShardInterval->minValue));
StringInfo maxValueString = makeStringInfo();
appendStringInfo(maxValueString, "%d", DatumGetInt32(
splitChildShardInterval->maxValue));
appendStringInfo(splitChildrenRows,
"ROW(%lu, %s, %lu, %s, %s, %u)::pg_catalog.split_shard_info",
sourceShardId,
quote_literal_cstr(partitionColumnName),
splitChildShardInterval->shardId,
quote_literal_cstr(minValueString->data),
quote_literal_cstr(maxValueString->data),
destinationWorkerNode->nodeId);
addComma = true;
}
}
StringInfo splitShardReplicationUDF = makeStringInfo();
appendStringInfo(splitShardReplicationUDF,
"SELECT * FROM pg_catalog.worker_split_shard_replication_setup(ARRAY[%s]);",
splitChildrenRows->data);
return splitShardReplicationUDF;
}
/*
* ParseReplicationSlotInfoFromResult parses custom datatype 'replication_slot_info'.
* 'replication_slot_info' is a tuple with below format:
* <targetNodeId, tableOwnerName, replicationSlotName>
*/
static List *
ParseReplicationSlotInfoFromResult(PGresult *result)
{
int64 rowCount = PQntuples(result);
List *replicationSlotInfoList = NIL;
for (int64 rowIndex = 0; rowIndex < rowCount; rowIndex++)
{
ReplicationSlotInfo *replicationSlotInfo = (ReplicationSlotInfo *) palloc0(
sizeof(ReplicationSlotInfo));
char *targeNodeIdString = PQgetvalue(result, rowIndex, 0 /* nodeId column*/);
replicationSlotInfo->targetNodeId = strtoul(targeNodeIdString, NULL, 10);
/* We're using the pstrdup to copy the data into the current memory context */
replicationSlotInfo->tableOwnerName = pstrdup(PQgetvalue(result, rowIndex,
1 /* table owner name column */));
/* Replication slot name */
replicationSlotInfo->slotName = pstrdup(PQgetvalue(result, rowIndex,
2 /* slot name column */));
replicationSlotInfoList = lappend(replicationSlotInfoList, replicationSlotInfo);
}
return replicationSlotInfoList;
}
/*
* AddDummyShardEntryInMap adds shard entry into hash map to keep track
* of dummy shards that are created. These shards are cleanedup after split completes.
*
* This is a cautious measure to keep track of dummy shards created for constraints
* of logical replication. We cautiously delete only the dummy shards added in the DummyShardHashMap.
*/
static void
AddDummyShardEntryInMap(HTAB *mapOfDummyShardToPlacement, uint32 targetNodeId,
ShardInterval *shardInterval)
{
NodeShardMappingKey key;
key.nodeId = targetNodeId;
key.tableOwnerId = TableOwnerOid(shardInterval->relationId);
bool found = false;
NodeShardMappingEntry *nodeMappingEntry =
(NodeShardMappingEntry *) hash_search(mapOfDummyShardToPlacement, &key,
HASH_ENTER,
&found);
if (!found)
{
nodeMappingEntry->shardSplitInfoList = NIL;
}
nodeMappingEntry->shardSplitInfoList =
lappend(nodeMappingEntry->shardSplitInfoList, (ShardInterval *) shardInterval);
}
/*
* DropDummyShards traverses the dummy shard map and drops shard at given node.
* It fails if the shard cannot be dropped.
*/
static void
DropDummyShards(HTAB *mapOfDummyShardToPlacement)
{
HASH_SEQ_STATUS status;
hash_seq_init(&status, mapOfDummyShardToPlacement);
NodeShardMappingEntry *entry = NULL;
while ((entry = (NodeShardMappingEntry *) hash_seq_search(&status)) != NULL)
{
uint32 nodeId = entry->key.nodeId;
WorkerNode *shardToBeDroppedNode = FindNodeWithNodeId(nodeId,
false /* missingOk */);
int connectionFlags = FOR_DDL;
connectionFlags |= OUTSIDE_TRANSACTION;
MultiConnection *connection = GetNodeUserDatabaseConnection(
connectionFlags,
shardToBeDroppedNode->workerName,
shardToBeDroppedNode->workerPort,
CurrentUserName(),
NULL /* databaseName */);
List *dummyShardIntervalList = entry->shardSplitInfoList;
ShardInterval *shardInterval = NULL;
foreach_ptr(shardInterval, dummyShardIntervalList)
{
DropDummyShard(connection, shardInterval);
}
CloseConnection(connection);
}
}
/*
* DropDummyShard drops a given shard on the node connection.
* It fails if the shard cannot be dropped.
*/
static void
DropDummyShard(MultiConnection *connection, ShardInterval *shardInterval)
{
char *qualifiedShardName = ConstructQualifiedShardName(shardInterval);
StringInfo dropShardQuery = makeStringInfo();
/* Caller enforces that foreign tables cannot be split (use DROP_REGULAR_TABLE_COMMAND) */
appendStringInfo(dropShardQuery, DROP_REGULAR_TABLE_COMMAND,
qualifiedShardName);
/*
* Since the dummy shard is expected to be present on the given node,
* fail if it cannot be dropped during cleanup.
*/
ExecuteCriticalRemoteCommand(
connection,
dropShardQuery->data);
}
/*
* CreateReplicaIdentities creates replica indentities for split children and dummy shards.
*/
static void
CreateReplicaIdentities(HTAB *mapOfDummyShardToPlacement,
List *shardGroupSplitIntervalListList,
List *workersForPlacementList)
{
/*
* Create Replica Identities for actual child shards.
*/
List *shardIntervalList = NIL;
foreach_ptr(shardIntervalList, shardGroupSplitIntervalListList)
{
ShardInterval *shardInterval = NULL;
WorkerNode *workerPlacementNode = NULL;
/*
* Iterate on split shard interval list for given shard and create tasks
* for every single split shard in a shard group.
*/
forboth_ptr(shardInterval, shardIntervalList, workerPlacementNode,
workersForPlacementList)
{
List *shardList = NIL;
shardList = lappend(shardList, shardInterval);
CreateReplicaIdentity(shardList, workerPlacementNode->workerName,
workerPlacementNode->workerPort);
}
}
/* Create Replica Identities for dummy shards */
HASH_SEQ_STATUS status;
hash_seq_init(&status, mapOfDummyShardToPlacement);
NodeShardMappingEntry *entry = NULL;
while ((entry = (NodeShardMappingEntry *) hash_seq_search(&status)) != NULL)
{
uint32 nodeId = entry->key.nodeId;
WorkerNode *shardToBeDroppedNode = FindNodeWithNodeId(nodeId,
false /* missingOk */);
List *dummyShardIntervalList = entry->shardSplitInfoList;
CreateReplicaIdentity(dummyShardIntervalList, shardToBeDroppedNode->workerName,
shardToBeDroppedNode->workerPort);
}
}
/*
* GetNextShardIdForSplitChild returns shard id to be used for split child.
* The function connects to the local node through a new connection and gets the next
* sequence. This prevents self deadlock when 'CREATE_REPLICATION_SLOT' is executed
* as a part of nonblocking split workflow.
*/
static uint64
GetNextShardIdForSplitChild()
{
uint64 shardId = 0;
/*
* In regression tests, we would like to generate shard IDs consistently
* even if the tests run in parallel. Instead of the sequence, we can use
* the next_shard_id GUC to specify which shard ID the current session should
* generate next. The GUC is automatically increased by 1 every time a new
* shard ID is generated.
*/
if (NextShardId > 0)
{
shardId = NextShardId;
NextShardId += 1;
return shardId;
}
StringInfo nextValueCommand = makeStringInfo();
appendStringInfo(nextValueCommand, "SELECT nextval(%s);", quote_literal_cstr(
"pg_catalog.pg_dist_shardid_seq"));
int connectionFlag = FORCE_NEW_CONNECTION;
MultiConnection *connection = GetNodeUserDatabaseConnection(connectionFlag,
LocalHostName,
PostPortNumber,
CitusExtensionOwnerName(),
get_database_name(
MyDatabaseId));
PGresult *result = NULL;
int queryResult = ExecuteOptionalRemoteCommand(connection, nextValueCommand->data,
&result);
if (queryResult != RESPONSE_OKAY || !IsResponseOK(result) || PQntuples(result) != 1 ||
PQnfields(result) != 1)
{
PQclear(result);
ForgetResults(connection);
CloseConnection(connection);
ereport(ERROR, (errcode(ERRCODE_CONNECTION_FAILURE),
errmsg(
"Could not generate next shard id while executing shard splits.")));
}
shardId = SafeStringToUint64(PQgetvalue(result, 0, 0 /* nodeId column*/));
CloseConnection(connection);
return shardId;
}

View File

@ -32,6 +32,11 @@
#include "utils/guc.h"
#include "utils/hsearch.h"
#include "utils/memutils.h"
#if PG_VERSION_NUM < PG_VERSION_13
#include "utils/hashutils.h"
#else
#include "common/hashfn.h"
#endif
/* Config variables managed via guc.c */
@ -361,6 +366,26 @@ WorkerNodeCompare(const void *lhsKey, const void *rhsKey, Size keySize)
}
/*
* WorkerNodeHashCode computes the hash code for a worker node from the node's
* host name and port number. Nodes that only differ by their rack locations
* hash to the same value.
*/
uint32
WorkerNodeHashCode(const void *key, Size keySize)
{
const WorkerNode *worker = (const WorkerNode *) key;
const char *workerName = worker->workerName;
const uint32 *workerPort = &(worker->workerPort);
/* standard hash function outlined in Effective Java, Item 8 */
uint32 result = 17;
result = 37 * result + string_hash(workerName, WORKER_LENGTH);
result = 37 * result + tag_hash(workerPort, sizeof(uint32));
return result;
}
/*
* NodeNamePortCompare implements the common logic for comparing two nodes
* with their given nodeNames and ports.

View File

@ -0,0 +1,457 @@
/*-------------------------------------------------------------------------
*
* worker_split_shard_replication_setup.c
* This file contains functions to setup information about list of shards
* that are being split.
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "miscadmin.h"
#include "postmaster/postmaster.h"
#include "common/hashfn.h"
#include "distributed/distribution_column.h"
#include "distributed/shardinterval_utils.h"
#include "distributed/shard_utils.h"
#include "distributed/shardsplit_shared_memory.h"
#include "distributed/connection_management.h"
#include "distributed/citus_safe_lib.h"
#include "distributed/listutils.h"
#include "distributed/remote_commands.h"
#include "distributed/tuplestore.h"
#include "distributed/shardsplit_logical_replication.h"
#include "utils/builtins.h"
#include "utils/lsyscache.h"
#include "commands/dbcommands.h"
/* declarations for dynamic loading */
PG_FUNCTION_INFO_V1(worker_split_shard_replication_setup);
static HTAB *ShardInfoHashMap = NULL;
/* Function declarations */
static void ParseShardSplitInfoFromDatum(Datum shardSplitInfoDatum,
uint64 *sourceShardId,
char **partitionColumnName,
uint64 *childShardId,
int32 *minValue,
int32 *maxValue,
int32 *nodeId);
static ShardSplitInfo * CreateShardSplitInfo(uint64 sourceShardIdToSplit,
char *partitionColumnName,
uint64 desSplitChildShardId,
int32 minValue,
int32 maxValue,
int32 nodeId);
static void AddShardSplitInfoEntryForNodeInMap(ShardSplitInfo *shardSplitInfo);
static void PopulateShardSplitInfoInSM(ShardSplitInfoSMHeader *shardSplitInfoSMHeader);
static void ReturnReplicationSlotInfo(Tuplestorestate *tupleStore,
TupleDesc tupleDescriptor);
/*
* worker_split_shard_replication_setup UDF creates in-memory data structures
* to store the meta information about the shard undergoing split and new split
* children along with their placements. This info is required during the catch up
* phase of logical replication.
* This meta information is stored in a shared memory segment and accessed
* by logical decoding plugin.
*
* Split information is given by user as an Array of custom data type 'pg_catalog.split_shard_info'.
* (worker_split_shard_replication_setup(pg_catalog.split_shard_info[]))
*
* Fields of custom data type 'pg_catalog.split_shard_info':
* source_shard_id - id of the shard that is undergoing a split
*
* distribution_column - Distribution column name
*
* child_shard_id - id of shard that stores a specific range of values
* belonging to sourceShardId(parent)
*
* shard_min_value - Lower bound(inclusive) of hash value which childShard stores
*
* shard_max_value - Upper bound(inclusive) of hash value which childShard stores
*
* node_id - Node where the childShardId is located
*
* The function parses the data and builds routing map with key for each distinct
* <nodeId, tableOwner> pair. Multiple shards can be placed on the same destination node.
* Source and destination nodes can be same too.
*
* There is a 1-1 mapping between a (table owner, node) and replication slot. One replication
* slot takes care of replicating changes for all shards belonging to the same owner on a particular node.
*
* During the replication phase, WAL senders will attach to the shared memory
* populated by current UDF. It routes the tuple from the source shard to the appropriate destination
* shard for which the respective slot is responsible.
*/
Datum
worker_split_shard_replication_setup(PG_FUNCTION_ARGS)
{
if (PG_ARGISNULL(0))
{
ereport(ERROR, (errmsg("split_shard_info array cannot be NULL")));
}
ArrayType *shardInfoArrayObject = PG_GETARG_ARRAYTYPE_P(0);
if (array_contains_nulls(shardInfoArrayObject))
{
ereport(ERROR, (errmsg("Unexpectedly shard info array contains a null value")));
}
/* SetupMap */
ShardInfoHashMap = SetupHashMapForShardInfo();
int shardSplitInfoCount = 0;
ArrayIterator shardInfo_iterator = array_create_iterator(shardInfoArrayObject, 0,
NULL);
Datum shardInfoDatum = 0;
bool isnull = false;
while (array_iterate(shardInfo_iterator, &shardInfoDatum, &isnull))
{
uint64 sourceShardId = 0;
char *partitionColumnName = NULL;
uint64 childShardId = 0;
int32 minValue = 0;
int32 maxValue = 0;
int32 nodeId = 0;
ParseShardSplitInfoFromDatum(shardInfoDatum, &sourceShardId,
&partitionColumnName, &childShardId,
&minValue, &maxValue, &nodeId);
ShardSplitInfo *shardSplitInfo = CreateShardSplitInfo(
sourceShardId,
partitionColumnName,
childShardId,
minValue,
maxValue,
nodeId);
AddShardSplitInfoEntryForNodeInMap(shardSplitInfo);
shardSplitInfoCount++;
}
dsm_handle dsmHandle;
ShardSplitInfoSMHeader *splitShardInfoSMHeader =
CreateSharedMemoryForShardSplitInfo(shardSplitInfoCount, &dsmHandle);
PopulateShardSplitInfoInSM(splitShardInfoSMHeader);
/* store handle in statically allocated shared memory*/
StoreShardSplitSharedMemoryHandle(dsmHandle);
TupleDesc tupleDescriptor = NULL;
Tuplestorestate *tupleStore = SetupTuplestore(fcinfo, &tupleDescriptor);
ReturnReplicationSlotInfo(tupleStore, tupleDescriptor);
PG_RETURN_VOID();
}
/*
* SetupHashMapForShardInfo initializes a hash map to store shard split
* information by grouping them node id wise. The key of the hash table
* is 'nodeId' and value is a list of ShardSplitInfo that are placed on
* this particular node.
*/
HTAB *
SetupHashMapForShardInfo()
{
HASHCTL info;
memset(&info, 0, sizeof(info));
info.keysize = sizeof(NodeShardMappingKey);
info.entrysize = sizeof(NodeShardMappingEntry);
info.hash = NodeShardMappingHash;
info.match = NodeShardMappingHashCompare;
info.hcxt = CurrentMemoryContext;
int hashFlags = (HASH_ELEM | HASH_CONTEXT | HASH_FUNCTION | HASH_COMPARE);
HTAB *shardInfoMap = hash_create("ShardInfoMap", 128, &info, hashFlags);
return shardInfoMap;
}
/*
* CreateShardSplitInfo function constructs ShardSplitInfo data structure
* with appropriate OIs' for source and destination relation.
*
* sourceShardIdToSplit - Existing shardId which has a valid entry in cache and catalogue
* partitionColumnName - Name of column to use for partitioning
* desSplitChildShardId - New split child shard which doesn't have an entry in metacache yet
* minValue - Minimum hash value for desSplitChildShardId
* maxValue - Maximum hash value for desSplitChildShardId
* nodeId - NodeId where
* However we can use shard ID and construct qualified shardName.
*/
ShardSplitInfo *
CreateShardSplitInfo(uint64 sourceShardIdToSplit,
char *partitionColumnName,
uint64 desSplitChildShardId,
int32 minValue,
int32 maxValue,
int32 nodeId)
{
ShardInterval *shardIntervalToSplit = LoadShardInterval(sourceShardIdToSplit);
/* If metadata is not synced, we cannot proceed further as split work flow assumes
* metadata to be synced on worker node hosting source shard to split.
*/
if (shardIntervalToSplit == NULL)
{
ereport(ERROR,
errmsg(
"Could not find metadata corresponding to source shard id: %ld. "
"Split workflow assumes metadata to be synced across "
"worker nodes hosting source shards.", sourceShardIdToSplit));
}
/* Oid of distributed table */
Oid citusTableOid = shardIntervalToSplit->relationId;
Oid sourceShardToSplitOid = GetTableLocalShardOid(citusTableOid,
sourceShardIdToSplit);
/* Oid of dummy table at the source */
Oid destSplitChildShardOid = GetTableLocalShardOid(citusTableOid,
desSplitChildShardId);
if (citusTableOid == InvalidOid ||
sourceShardToSplitOid == InvalidOid ||
destSplitChildShardOid == InvalidOid)
{
ereport(ERROR, (errcode(ERRCODE_SYNTAX_ERROR),
errmsg("Invalid citusTableOid:%u, "
"sourceShardToSplitOid:%u, "
"destSplitChildShardOid:%u ",
citusTableOid,
sourceShardToSplitOid,
destSplitChildShardOid)));
}
/* determine the partition column in the tuple descriptor */
Var *partitionColumn = BuildDistributionKeyFromColumnName(sourceShardToSplitOid,
partitionColumnName,
AccessShareLock);
if (partitionColumn == NULL)
{
ereport(ERROR, (errcode(ERRCODE_SYNTAX_ERROR),
errmsg("Invalid Partition Column")));
}
int partitionColumnIndex = partitionColumn->varattno - 1;
ShardSplitInfo *shardSplitInfo = palloc0(sizeof(ShardSplitInfo));
shardSplitInfo->distributedTableOid = citusTableOid;
shardSplitInfo->partitionColumnIndex = partitionColumnIndex;
shardSplitInfo->sourceShardOid = sourceShardToSplitOid;
shardSplitInfo->splitChildShardOid = destSplitChildShardOid;
shardSplitInfo->shardMinValue = minValue;
shardSplitInfo->shardMaxValue = maxValue;
shardSplitInfo->nodeId = nodeId;
shardSplitInfo->sourceShardId = sourceShardIdToSplit;
shardSplitInfo->splitChildShardId = desSplitChildShardId;
return shardSplitInfo;
}
/*
* AddShardSplitInfoEntryForNodeInMap function add's ShardSplitInfo entry
* to the hash map. The key is nodeId on which the new shard is to be placed.
*/
static void
AddShardSplitInfoEntryForNodeInMap(ShardSplitInfo *shardSplitInfo)
{
NodeShardMappingKey key;
key.nodeId = shardSplitInfo->nodeId;
key.tableOwnerId = TableOwnerOid(shardSplitInfo->distributedTableOid);
bool found = false;
NodeShardMappingEntry *nodeMappingEntry =
(NodeShardMappingEntry *) hash_search(ShardInfoHashMap, &key, HASH_ENTER,
&found);
if (!found)
{
nodeMappingEntry->shardSplitInfoList = NIL;
}
nodeMappingEntry->shardSplitInfoList =
lappend(nodeMappingEntry->shardSplitInfoList, (ShardSplitInfo *) shardSplitInfo);
}
/*
* PopulateShardSplitInfoInSM function copies information from the hash map
* into shared memory segment. This information is consumed by the WAL sender
* process during logical replication.
*
* shardSplitInfoSMHeader - Shared memory header
*/
static void
PopulateShardSplitInfoInSM(ShardSplitInfoSMHeader *shardSplitInfoSMHeader)
{
HASH_SEQ_STATUS status;
hash_seq_init(&status, ShardInfoHashMap);
NodeShardMappingEntry *entry = NULL;
int splitInfoIndex = 0;
while ((entry = (NodeShardMappingEntry *) hash_seq_search(&status)) != NULL)
{
uint32_t nodeId = entry->key.nodeId;
uint32_t tableOwnerId = entry->key.tableOwnerId;
char *derivedSlotName =
EncodeReplicationSlot(nodeId, tableOwnerId);
List *shardSplitInfoList = entry->shardSplitInfoList;
ShardSplitInfo *splitShardInfo = NULL;
foreach_ptr(splitShardInfo, shardSplitInfoList)
{
shardSplitInfoSMHeader->splitInfoArray[splitInfoIndex] = *splitShardInfo;
strcpy_s(shardSplitInfoSMHeader->splitInfoArray[splitInfoIndex].slotName,
NAMEDATALEN,
derivedSlotName);
splitInfoIndex++;
}
}
}
/*
* NodeShardMappingHash returns hash value by combining hash of node id
* and tableowner Id.
*/
uint32
NodeShardMappingHash(const void *key, Size keysize)
{
NodeShardMappingKey *entry = (NodeShardMappingKey *) key;
uint32 hash = hash_uint32(entry->nodeId);
hash = hash_combine(hash, hash_uint32(entry->tableOwnerId));
return hash;
}
/*
* Comparator function for hash keys
*/
int
NodeShardMappingHashCompare(const void *left, const void *right, Size keysize)
{
NodeShardMappingKey *leftKey = (NodeShardMappingKey *) left;
NodeShardMappingKey *rightKey = (NodeShardMappingKey *) right;
if (leftKey->nodeId != rightKey->nodeId ||
leftKey->tableOwnerId != rightKey->tableOwnerId)
{
return 1;
}
else
{
return 0;
}
}
/*
* ParseShardSplitInfoFromDatum deserializes individual fields of 'pg_catalog.split_shard_info'
* datatype.
*/
static void
ParseShardSplitInfoFromDatum(Datum shardSplitInfoDatum,
uint64 *sourceShardId,
char **partitionColumnName,
uint64 *childShardId,
int32 *minValue,
int32 *maxValue,
int32 *nodeId)
{
HeapTupleHeader dataTuple = DatumGetHeapTupleHeader(shardSplitInfoDatum);
bool isnull = false;
Datum sourceShardIdDatum = GetAttributeByName(dataTuple, "source_shard_id", &isnull);
if (isnull)
{
ereport(ERROR, (errmsg("source_shard_id for split_shard_info can't be null")));
}
*sourceShardId = DatumGetUInt64(sourceShardIdDatum);
Datum partitionColumnDatum = GetAttributeByName(dataTuple, "distribution_column",
&isnull);
if (isnull)
{
ereport(ERROR, (errmsg(
"distribution_column for split_shard_info can't be null")));
}
*partitionColumnName = TextDatumGetCString(partitionColumnDatum);
Datum childShardIdDatum = GetAttributeByName(dataTuple, "child_shard_id", &isnull);
if (isnull)
{
ereport(ERROR, (errmsg("child_shard_id for split_shard_info can't be null")));
}
*childShardId = DatumGetUInt64(childShardIdDatum);
Datum minValueDatum = GetAttributeByName(dataTuple, "shard_min_value", &isnull);
if (isnull)
{
ereport(ERROR, (errmsg("shard_min_value for split_shard_info can't be null")));
}
char *shardMinValueString = text_to_cstring(DatumGetTextP(minValueDatum));
*minValue = SafeStringToInt32(shardMinValueString);
Datum maxValueDatum = GetAttributeByName(dataTuple, "shard_max_value", &isnull);
if (isnull)
{
ereport(ERROR, (errmsg("shard_max_value for split_shard_info can't be null")));
}
char *shardMaxValueString = text_to_cstring(DatumGetTextP(maxValueDatum));
*maxValue = SafeStringToInt32(shardMaxValueString);
Datum nodeIdDatum = GetAttributeByName(dataTuple, "node_id", &isnull);
if (isnull)
{
ereport(ERROR, (errmsg("node_id for split_shard_info can't be null")));
}
*nodeId = DatumGetInt32(nodeIdDatum);
}
/*
* ReturnReplicationSlotInfo writes 'pg_catalog.replication_slot_info'
* records to tuplestore.
* This information is used by the coordinator to create replication slots as a
* part of non-blocking split workflow.
*/
static void
ReturnReplicationSlotInfo(Tuplestorestate *tupleStore, TupleDesc
tupleDescriptor)
{
HASH_SEQ_STATUS status;
hash_seq_init(&status, ShardInfoHashMap);
NodeShardMappingEntry *entry = NULL;
while ((entry = (NodeShardMappingEntry *) hash_seq_search(&status)) != NULL)
{
Datum values[3];
bool nulls[3];
memset(values, 0, sizeof(values));
memset(nulls, false, sizeof(nulls));
values[0] = Int32GetDatum(entry->key.nodeId);
char *tableOwnerName = GetUserNameFromId(entry->key.tableOwnerId, false);
values[1] = CStringGetTextDatum(tableOwnerName);
char *slotName = EncodeReplicationSlot(entry->key.nodeId,
entry->key.tableOwnerId);
values[2] = CStringGetTextDatum(slotName);
tuplestore_putvalues(tupleStore, tupleDescriptor, values, nulls);
}
}

View File

@ -88,8 +88,6 @@ static void CreateForeignConstraintsToReferenceTable(List *shardList,
MultiConnection *targetConnection);
static List * PrepareReplicationSubscriptionList(List *shardList);
static Bitmapset * TableOwnerIds(List *shardList);
static void CreateReplicaIdentity(List *shardList, char *nodeName, int32
nodePort);
static List * GetReplicaIdentityCommandListForShard(Oid relationId, uint64 shardId);
static List * GetIndexCommandListForShardBackingReplicaIdentity(Oid relationId,
uint64 shardId);
@ -115,7 +113,6 @@ static void CreatePartitioningHierarchy(List *shardList, char *targetNodeName,
int targetNodePort);
static void CreateColocatedForeignKeys(List *shardList, char *targetNodeName,
int targetNodePort);
static void ConflictOnlyWithIsolationTesting(void);
static void DropShardMovePublications(MultiConnection *connection,
Bitmapset *tableOwnerIds);
static void DropShardMoveSubscriptions(MultiConnection *connection,
@ -128,35 +125,26 @@ static void CreateShardMoveSubscriptions(MultiConnection *connection,
char *databaseName,
Bitmapset *tableOwnerIds);
static char * escape_param_str(const char *str);
static XLogRecPtr GetRemoteLogPosition(MultiConnection *connection);
static XLogRecPtr GetRemoteLSN(MultiConnection *connection, char *command);
static void WaitForRelationSubscriptionsBecomeReady(MultiConnection *targetConnection,
Bitmapset *tableOwnerIds);
static uint64 TotalRelationSizeForSubscription(MultiConnection *connection,
char *command);
static bool RelationSubscriptionsAreReady(MultiConnection *targetConnection,
Bitmapset *tableOwnerIds);
static void WaitForShardMoveSubscription(MultiConnection *targetConnection,
XLogRecPtr sourcePosition,
Bitmapset *tableOwnerIds);
Bitmapset *tableOwnerIds,
char *operationPrefix);
static void WaitForMiliseconds(long timeout);
static XLogRecPtr GetSubscriptionPosition(MultiConnection *connection,
Bitmapset *tableOwnerIds);
Bitmapset *tableOwnerIds,
char *operationPrefix);
static char * ShardMovePublicationName(Oid ownerId);
static char * ShardMoveSubscriptionName(Oid ownerId);
static void AcquireLogicalReplicationLock(void);
static void DropAllShardMoveLeftovers(void);
static void DropAllShardMoveSubscriptions(MultiConnection *connection);
static void DropAllShardMoveReplicationSlots(MultiConnection *connection);
static void DropAllShardMovePublications(MultiConnection *connection);
static void DropAllShardMoveUsers(MultiConnection *connection);
static char * ShardMoveSubscriptionNamesValueList(Bitmapset *tableOwnerIds);
static void DropShardMoveSubscription(MultiConnection *connection,
char *subscriptionName);
static void DropShardMoveReplicationSlot(MultiConnection *connection,
char *publicationName);
static void DropShardMovePublication(MultiConnection *connection, char *publicationName);
static void DropShardMoveUser(MultiConnection *connection, char *username);
static char * ShardSubscriptionNamesValueList(Bitmapset *tableOwnerIds,
char *operationPrefix);
/*
* LogicallyReplicateShards replicates a list of shards from one node to another
@ -229,14 +217,16 @@ LogicallyReplicateShards(List *shardList, char *sourceNodeName, int sourceNodePo
* subscription is not ready. There is no point of locking the shards before the
* subscriptions for each relation becomes ready, so wait for it.
*/
WaitForRelationSubscriptionsBecomeReady(targetConnection, tableOwnerIds);
WaitForRelationSubscriptionsBecomeReady(targetConnection, tableOwnerIds,
SHARD_MOVE_SUBSCRIPTION_PREFIX);
/*
* Wait until the subscription is caught up to changes that has happened
* after the initial COPY on the shards.
*/
XLogRecPtr sourcePosition = GetRemoteLogPosition(sourceConnection);
WaitForShardMoveSubscription(targetConnection, sourcePosition, tableOwnerIds);
WaitForShardSubscriptionToCatchUp(targetConnection, sourcePosition, tableOwnerIds,
SHARD_MOVE_SUBSCRIPTION_PREFIX);
/*
* Now lets create the post-load objects, such as the indexes, constraints
@ -246,7 +236,8 @@ LogicallyReplicateShards(List *shardList, char *sourceNodeName, int sourceNodePo
CreatePostLogicalReplicationDataLoadObjects(shardList, targetNodeName,
targetNodePort);
sourcePosition = GetRemoteLogPosition(sourceConnection);
WaitForShardMoveSubscription(targetConnection, sourcePosition, tableOwnerIds);
WaitForShardSubscriptionToCatchUp(targetConnection, sourcePosition, tableOwnerIds,
SHARD_MOVE_SUBSCRIPTION_PREFIX);
/*
* We're almost done, we'll block the writes to the shards that we're
@ -259,7 +250,8 @@ LogicallyReplicateShards(List *shardList, char *sourceNodeName, int sourceNodePo
BlockWritesToShardList(shardList);
sourcePosition = GetRemoteLogPosition(sourceConnection);
WaitForShardMoveSubscription(targetConnection, sourcePosition, tableOwnerIds);
WaitForShardSubscriptionToCatchUp(targetConnection, sourcePosition, tableOwnerIds,
SHARD_MOVE_SUBSCRIPTION_PREFIX);
/*
* We're creating the foreign constraints to reference tables after the
@ -461,7 +453,7 @@ TableOwnerIds(List *shardList)
* CreateReplicaIdentity gets a shardList and creates all the replica identities
* on the shards in the given node.
*/
static void
void
CreateReplicaIdentity(List *shardList, char *nodeName, int32 nodePort)
{
MemoryContext localContext = AllocSetContextCreate(CurrentMemoryContext,
@ -1053,7 +1045,7 @@ CreateForeignConstraintsToReferenceTable(List *shardList,
* Note that since the cost of calling this function is pretty low, we prefer
* to use it in non-assert builds as well not to diverge in the behaviour.
*/
static void
extern void
ConflictOnlyWithIsolationTesting()
{
LOCKTAG tag;
@ -1087,18 +1079,19 @@ DropShardMovePublications(MultiConnection *connection, Bitmapset *tableOwnerIds)
* If replication slot can not be dropped while dropping the subscriber, drop
* it here.
*/
DropShardMoveReplicationSlot(connection, ShardMoveSubscriptionName(ownerId));
DropShardMovePublication(connection, ShardMovePublicationName(ownerId));
DropShardReplicationSlot(connection, ShardSubscriptionName(ownerId,
SHARD_MOVE_SUBSCRIPTION_PREFIX));
DropShardPublication(connection, ShardMovePublicationName(ownerId));
}
}
/*
* DropShardMoveReplicationSlot drops the replication slot with the given name
* DropShardReplicationSlot drops the replication slot with the given name
* if it exists.
*/
static void
DropShardMoveReplicationSlot(MultiConnection *connection, char *replicationSlotName)
void
DropShardReplicationSlot(MultiConnection *connection, char *replicationSlotName)
{
ExecuteCriticalRemoteCommand(
connection,
@ -1111,11 +1104,11 @@ DropShardMoveReplicationSlot(MultiConnection *connection, char *replicationSlotN
/*
* DropShardMovePublication drops the publication with the given name if it
* DropShardPublication drops the publication with the given name if it
* exists.
*/
static void
DropShardMovePublication(MultiConnection *connection, char *publicationName)
void
DropShardPublication(MultiConnection *connection, char *publicationName)
{
ExecuteCriticalRemoteCommand(connection, psprintf(
"DROP PUBLICATION IF EXISTS %s",
@ -1135,7 +1128,7 @@ ShardMovePublicationName(Oid ownerId)
/*
* ShardMoveSubscriptionName returns the name of the subscription for the given
* ShardSubscriptionName returns the name of the subscription for the given
* owner. If we're running the isolation tester the function also appends the
* process id normal subscription name.
*
@ -1145,28 +1138,28 @@ ShardMovePublicationName(Oid ownerId)
* This PID is then extracted from the application_name to find out which PID on the
* coordinator is blocked by the blocked replication process.
*/
static char *
ShardMoveSubscriptionName(Oid ownerId)
char *
ShardSubscriptionName(Oid ownerId, char *operationPrefix)
{
if (RunningUnderIsolationTest)
{
return psprintf("%s%i_%i", SHARD_MOVE_SUBSCRIPTION_PREFIX, ownerId, MyProcPid);
return psprintf("%s%i_%i", operationPrefix, ownerId, MyProcPid);
}
else
{
return psprintf("%s%i", SHARD_MOVE_SUBSCRIPTION_PREFIX, ownerId);
return psprintf("%s%i", operationPrefix, ownerId);
}
}
/*
* ShardMoveSubscriptionRole returns the name of the role used by the
* ShardSubscriptionRole returns the name of the role used by the
* subscription that subscribes to the tables of the given owner.
*/
static char *
ShardMoveSubscriptionRole(Oid ownerId)
char *
ShardSubscriptionRole(Oid ownerId, char *operationPrefix)
{
return psprintf("%s%i", SHARD_MOVE_SUBSCRIPTION_ROLE_PREFIX, ownerId);
return psprintf("%s%i", operationPrefix, ownerId);
}
@ -1175,7 +1168,7 @@ ShardMoveSubscriptionRole(Oid ownerId)
* strings. This query is executed on the connection and the function then
* returns the results of the query in a List.
*/
static List *
List *
GetQueryResultStringList(MultiConnection *connection, char *query)
{
bool raiseInterrupts = true;
@ -1236,7 +1229,7 @@ DropAllShardMoveSubscriptions(MultiConnection *connection)
char *subscriptionName;
foreach_ptr(subscriptionName, subscriptionNameList)
{
DropShardMoveSubscription(connection, subscriptionName);
DropShardSubscription(connection, subscriptionName);
}
}
@ -1257,7 +1250,7 @@ DropAllShardMoveUsers(MultiConnection *connection)
char *username;
foreach_ptr(username, usernameList)
{
DropShardMoveUser(connection, username);
DropShardUser(connection, username);
}
}
@ -1278,7 +1271,7 @@ DropAllShardMoveReplicationSlots(MultiConnection *connection)
char *slotName;
foreach_ptr(slotName, slotNameList)
{
DropShardMoveReplicationSlot(connection, slotName);
DropShardReplicationSlot(connection, slotName);
}
}
@ -1299,7 +1292,7 @@ DropAllShardMovePublications(MultiConnection *connection)
char *publicationName;
foreach_ptr(publicationName, publicationNameList)
{
DropShardMovePublication(connection, publicationName);
DropShardPublication(connection, publicationName);
}
}
@ -1319,21 +1312,23 @@ DropShardMoveSubscriptions(MultiConnection *connection, Bitmapset *tableOwnerIds
int ownerId = -1;
while ((ownerId = bms_next_member(tableOwnerIds, ownerId)) >= 0)
{
DropShardMoveSubscription(connection, ShardMoveSubscriptionName(ownerId));
DropShardMoveUser(connection, ShardMoveSubscriptionRole(ownerId));
DropShardSubscription(connection, ShardSubscriptionName(ownerId,
SHARD_MOVE_SUBSCRIPTION_PREFIX));
DropShardUser(connection, ShardSubscriptionRole(ownerId,
SHARD_MOVE_SUBSCRIPTION_ROLE_PREFIX));
}
}
/*
* DropShardMoveSubscription drops subscription with the given name on the
* DropShardSubscription drops subscription with the given name on the
* subscriber node. Note that, it also drops the replication slot on the
* publisher node if it can drop the slot as well with the DROP SUBSCRIPTION
* command. Otherwise, only the subscription will be deleted with DROP
* SUBSCRIPTION via the connection.
*/
static void
DropShardMoveSubscription(MultiConnection *connection, char *subscriptionName)
void
DropShardSubscription(MultiConnection *connection, char *subscriptionName)
{
PGresult *result = NULL;
@ -1386,10 +1381,10 @@ DropShardMoveSubscription(MultiConnection *connection, char *subscriptionName)
/*
* DropShardMoveUser drops the user with the given name if it exists.
* DropShardUser drops the user with the given name if it exists.
*/
static void
DropShardMoveUser(MultiConnection *connection, char *username)
void
DropShardUser(MultiConnection *connection, char *username)
{
/*
* The DROP USER command should not propagate, so we temporarily disable
@ -1474,6 +1469,10 @@ CreateShardMoveSubscriptions(MultiConnection *connection, char *sourceNodeName,
/*
* The CREATE USER command should not propagate, so we temporarily
* disable DDL propagation.
*
* Subscription workers have SUPERUSER permissions. Hence we temporarily
* create a user with SUPERUSER permissions and then alter it to NOSUPERUSER.
* This prevents permission escalations.
*/
SendCommandListToWorkerOutsideTransaction(
connection->hostname, connection->port, connection->user,
@ -1481,7 +1480,7 @@ CreateShardMoveSubscriptions(MultiConnection *connection, char *sourceNodeName,
"SET LOCAL citus.enable_ddl_propagation TO OFF;",
psprintf(
"CREATE USER %s SUPERUSER IN ROLE %s",
ShardMoveSubscriptionRole(ownerId),
ShardSubscriptionRole(ownerId, SHARD_MOVE_SUBSCRIPTION_ROLE_PREFIX),
GetUserNameFromId(ownerId, false)
)));
@ -1493,7 +1492,8 @@ CreateShardMoveSubscriptions(MultiConnection *connection, char *sourceNodeName,
appendStringInfo(createSubscriptionCommand,
"CREATE SUBSCRIPTION %s CONNECTION %s PUBLICATION %s "
"WITH (citus_use_authinfo=true, enabled=false)",
quote_identifier(ShardMoveSubscriptionName(ownerId)),
quote_identifier(ShardSubscriptionName(ownerId,
SHARD_MOVE_SUBSCRIPTION_PREFIX)),
quote_literal_cstr(conninfo->data),
quote_identifier(ShardMovePublicationName(ownerId)));
@ -1502,8 +1502,10 @@ CreateShardMoveSubscriptions(MultiConnection *connection, char *sourceNodeName,
pfree(createSubscriptionCommand);
ExecuteCriticalRemoteCommand(connection, psprintf(
"ALTER SUBSCRIPTION %s OWNER TO %s",
ShardMoveSubscriptionName(ownerId),
ShardMoveSubscriptionRole(ownerId)
ShardSubscriptionName(ownerId,
SHARD_MOVE_SUBSCRIPTION_PREFIX),
ShardSubscriptionRole(ownerId,
SHARD_MOVE_SUBSCRIPTION_ROLE_PREFIX)
));
/*
@ -1516,12 +1518,13 @@ CreateShardMoveSubscriptions(MultiConnection *connection, char *sourceNodeName,
"SET LOCAL citus.enable_ddl_propagation TO OFF;",
psprintf(
"ALTER ROLE %s NOSUPERUSER",
ShardMoveSubscriptionRole(ownerId)
ShardSubscriptionRole(ownerId, SHARD_MOVE_SUBSCRIPTION_ROLE_PREFIX)
)));
ExecuteCriticalRemoteCommand(connection, psprintf(
"ALTER SUBSCRIPTION %s ENABLE",
ShardMoveSubscriptionName(ownerId)
ShardSubscriptionName(ownerId,
SHARD_MOVE_SUBSCRIPTION_PREFIX)
));
}
}
@ -1558,7 +1561,7 @@ escape_param_str(const char *str)
/*
* GetRemoteLogPosition gets the current WAL log position over the given connection.
*/
static XLogRecPtr
XLogRecPtr
GetRemoteLogPosition(MultiConnection *connection)
{
return GetRemoteLSN(connection, CURRENT_LOG_POSITION_COMMAND);
@ -1627,9 +1630,9 @@ GetRemoteLSN(MultiConnection *connection, char *command)
* on the target node doesn't change within LogicalReplicationErrorTimeout. The
* function also reports its progress in every logicalReplicationProgressReportTimeout.
*/
static void
void
WaitForRelationSubscriptionsBecomeReady(MultiConnection *targetConnection,
Bitmapset *tableOwnerIds)
Bitmapset *tableOwnerIds, char *operationPrefix)
{
uint64 previousTotalRelationSizeForSubscription = 0;
TimestampTz previousSizeChangeTime = GetCurrentTimestamp();
@ -1657,7 +1660,8 @@ WaitForRelationSubscriptionsBecomeReady(MultiConnection *targetConnection,
while (true)
{
/* we're done, all relations are ready */
if (RelationSubscriptionsAreReady(targetConnection, tableOwnerIds))
if (RelationSubscriptionsAreReady(targetConnection, tableOwnerIds,
operationPrefix))
{
ereport(LOG, (errmsg("The states of the relations belonging to the "
"subscriptions became READY on the "
@ -1667,7 +1671,8 @@ WaitForRelationSubscriptionsBecomeReady(MultiConnection *targetConnection,
break;
}
char *subscriptionValueList = ShardMoveSubscriptionNamesValueList(tableOwnerIds);
char *subscriptionValueList = ShardSubscriptionNamesValueList(tableOwnerIds,
operationPrefix);
/* Get the current total size of tables belonging to the subscriber */
uint64 currentTotalRelationSize =
@ -1821,12 +1826,12 @@ TotalRelationSizeForSubscription(MultiConnection *connection, char *command)
/*
* ShardMoveSubscriptionNamesValueList returns a SQL value list containing the
* ShardSubscriptionNamesValueList returns a SQL value list containing the
* subscription names for all of the given table owner ids. This value list can
* be used in a query by using the IN operator.
*/
static char *
ShardMoveSubscriptionNamesValueList(Bitmapset *tableOwnerIds)
ShardSubscriptionNamesValueList(Bitmapset *tableOwnerIds, char *operationPrefix)
{
StringInfo subscriptionValueList = makeStringInfo();
appendStringInfoString(subscriptionValueList, "(");
@ -1844,7 +1849,8 @@ ShardMoveSubscriptionNamesValueList(Bitmapset *tableOwnerIds)
first = false;
}
appendStringInfoString(subscriptionValueList,
quote_literal_cstr(ShardMoveSubscriptionName(ownerId)));
quote_literal_cstr(ShardSubscriptionName(ownerId,
operationPrefix)));
}
appendStringInfoString(subscriptionValueList, ")");
return subscriptionValueList->data;
@ -1857,11 +1863,12 @@ ShardMoveSubscriptionNamesValueList(Bitmapset *tableOwnerIds)
*/
static bool
RelationSubscriptionsAreReady(MultiConnection *targetConnection,
Bitmapset *tableOwnerIds)
Bitmapset *tableOwnerIds, char *operationPrefix)
{
bool raiseInterrupts = false;
char *subscriptionValueList = ShardMoveSubscriptionNamesValueList(tableOwnerIds);
char *subscriptionValueList = ShardSubscriptionNamesValueList(tableOwnerIds,
operationPrefix);
char *query = psprintf(
"SELECT count(*) FROM pg_subscription_rel, pg_stat_subscription "
"WHERE srsubid = subid AND srsubstate != 'r' AND subname IN %s",
@ -1906,14 +1913,15 @@ RelationSubscriptionsAreReady(MultiConnection *targetConnection,
/*
* WaitForShardMoveSubscription waits until the last LSN reported by the subscription.
* WaitForShardSubscriptionToCatchUp waits until the last LSN reported by the subscription.
*
* The function errors if the target LSN doesn't increase within LogicalReplicationErrorTimeout.
* The function also reports its progress in every logicalReplicationProgressReportTimeout.
*/
static void
WaitForShardMoveSubscription(MultiConnection *targetConnection, XLogRecPtr sourcePosition,
Bitmapset *tableOwnerIds)
void
WaitForShardSubscriptionToCatchUp(MultiConnection *targetConnection, XLogRecPtr
sourcePosition,
Bitmapset *tableOwnerIds, char *operationPrefix)
{
XLogRecPtr previousTargetPosition = 0;
TimestampTz previousLSNIncrementTime = GetCurrentTimestamp();
@ -1929,7 +1937,7 @@ WaitForShardMoveSubscription(MultiConnection *targetConnection, XLogRecPtr sourc
* a lot of memory.
*/
MemoryContext loopContext = AllocSetContextCreateInternal(CurrentMemoryContext,
"WaitForShardMoveSubscription",
"WaitForShardSubscriptionToCatchUp",
ALLOCSET_DEFAULT_MINSIZE,
ALLOCSET_DEFAULT_INITSIZE,
ALLOCSET_DEFAULT_MAXSIZE);
@ -1939,7 +1947,8 @@ WaitForShardMoveSubscription(MultiConnection *targetConnection, XLogRecPtr sourc
while (true)
{
XLogRecPtr targetPosition = GetSubscriptionPosition(targetConnection,
tableOwnerIds);
tableOwnerIds,
operationPrefix);
if (targetPosition >= sourcePosition)
{
ereport(LOG, (errmsg(
@ -2049,10 +2058,86 @@ WaitForMiliseconds(long timeout)
* replication.
*/
static XLogRecPtr
GetSubscriptionPosition(MultiConnection *connection, Bitmapset *tableOwnerIds)
GetSubscriptionPosition(MultiConnection *connection, Bitmapset *tableOwnerIds,
char *operationPrefix)
{
char *subscriptionValueList = ShardMoveSubscriptionNamesValueList(tableOwnerIds);
char *subscriptionValueList = ShardSubscriptionNamesValueList(tableOwnerIds,
operationPrefix);
return GetRemoteLSN(connection, psprintf(
"SELECT min(latest_end_lsn) FROM pg_stat_subscription "
"WHERE subname IN %s", subscriptionValueList));
}
/*
* CreateShardSplitSubscription creates the subscriptions used for shard split
* over the given connection. The subscription is created with 'copy_data'
* set to false and with the given replication slot name.
*/
void
CreateShardSplitSubscription(MultiConnection *connection, char *sourceNodeName,
int sourceNodePort, char *userName, char *databaseName,
char *publicationName, char *slotName,
Oid ownerId)
{
StringInfo createSubscriptionCommand = makeStringInfo();
StringInfo conninfo = makeStringInfo();
/*
* The CREATE USER command should not propagate, so we temporarily
* disable DDL propagation.
*/
SendCommandListToWorkerOutsideTransaction(
connection->hostname, connection->port, connection->user,
list_make2(
"SET LOCAL citus.enable_ddl_propagation TO OFF;",
psprintf(
"CREATE USER %s SUPERUSER IN ROLE %s",
ShardSubscriptionRole(ownerId, SHARD_SPLIT_SUBSCRIPTION_ROLE_PREFIX),
quote_identifier(GetUserNameFromId(ownerId, false))
)));
appendStringInfo(conninfo, "host='%s' port=%d user='%s' dbname='%s' "
"connect_timeout=20",
escape_param_str(sourceNodeName), sourceNodePort,
escape_param_str(userName), escape_param_str(databaseName));
appendStringInfo(createSubscriptionCommand,
"CREATE SUBSCRIPTION %s CONNECTION %s PUBLICATION %s "
"WITH (citus_use_authinfo=true, enabled=false, create_slot=false, copy_data=false, slot_name='%s')",
quote_identifier(ShardSubscriptionName(ownerId,
SHARD_SPLIT_SUBSCRIPTION_PREFIX)),
quote_literal_cstr(conninfo->data),
quote_identifier(publicationName),
escape_param_str(slotName));
ExecuteCriticalRemoteCommand(connection, createSubscriptionCommand->data);
pfree(createSubscriptionCommand->data);
pfree(createSubscriptionCommand);
ExecuteCriticalRemoteCommand(connection, psprintf(
"ALTER SUBSCRIPTION %s OWNER TO %s",
ShardSubscriptionName(ownerId,
SHARD_SPLIT_SUBSCRIPTION_PREFIX),
ShardSubscriptionRole(ownerId,
SHARD_SPLIT_SUBSCRIPTION_ROLE_PREFIX)
));
/*
* The ALTER ROLE command should not propagate, so we temporarily
* disable DDL propagation.
*/
SendCommandListToWorkerOutsideTransaction(
connection->hostname, connection->port, connection->user,
list_make2(
"SET LOCAL citus.enable_ddl_propagation TO OFF;",
psprintf(
"ALTER ROLE %s NOSUPERUSER",
ShardSubscriptionRole(ownerId, SHARD_SPLIT_SUBSCRIPTION_ROLE_PREFIX)
)));
ExecuteCriticalRemoteCommand(connection, psprintf(
"ALTER SUBSCRIPTION %s ENABLE",
ShardSubscriptionName(ownerId,
SHARD_SPLIT_SUBSCRIPTION_PREFIX)
));
}

View File

@ -0,0 +1,223 @@
/*-------------------------------------------------------------------------
*
* shardsplit_decoder.c
* Logical Replication output plugin
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "distributed/shardinterval_utils.h"
#include "distributed/shardsplit_shared_memory.h"
#include "distributed/listutils.h"
#include "replication/logical.h"
#include "utils/typcache.h"
extern void _PG_output_plugin_init(OutputPluginCallbacks *cb);
static LogicalDecodeChangeCB pgoutputChangeCB;
static HTAB *SourceToDestinationShardMap = NULL;
/* Plugin callback */
static void split_change_cb(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
Relation relation, ReorderBufferChange *change);
/* Helper methods */
static int32_t GetHashValueForIncomingTuple(Relation sourceShardRelation,
HeapTuple tuple,
int partitionColumIndex,
Oid distributedTableOid);
static Oid FindTargetRelationOid(Relation sourceShardRelation,
HeapTuple tuple,
char *currentSlotName);
/*
* Postgres uses 'pgoutput' as default plugin for logical replication.
* We want to reuse Postgres pgoutput's functionality as much as possible.
* Hence we load all the functions of this plugin and override as required.
*/
void
_PG_output_plugin_init(OutputPluginCallbacks *cb)
{
LogicalOutputPluginInit plugin_init =
(LogicalOutputPluginInit) (void *) load_external_function("pgoutput",
"_PG_output_plugin_init",
false, NULL);
if (plugin_init == NULL)
{
elog(ERROR, "output plugins have to declare the _PG_output_plugin_init symbol");
}
/* ask the output plugin to fill the callback struct */
plugin_init(cb);
/* actual pgoutput callback will be called with the appropriate destination shard */
pgoutputChangeCB = cb->change_cb;
cb->change_cb = split_change_cb;
}
/*
* split_change function emits the incoming tuple change
* to the appropriate destination shard.
*/
static void
split_change_cb(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
Relation relation, ReorderBufferChange *change)
{
if (!is_publishable_relation(relation))
{
return;
}
char *replicationSlotName = ctx->slot->data.name.data;
/*
* Initialize SourceToDestinationShardMap if not already initialized.
* This gets initialized during the replication of first message.
*/
if (SourceToDestinationShardMap == NULL)
{
SourceToDestinationShardMap = PopulateSourceToDestinationShardMapForSlot(
replicationSlotName, TopMemoryContext);
}
Oid targetRelationOid = InvalidOid;
switch (change->action)
{
case REORDER_BUFFER_CHANGE_INSERT:
{
HeapTuple newTuple = &(change->data.tp.newtuple->tuple);
targetRelationOid = FindTargetRelationOid(relation, newTuple,
replicationSlotName);
break;
}
/* updating non-partition column value */
case REORDER_BUFFER_CHANGE_UPDATE:
{
HeapTuple newTuple = &(change->data.tp.newtuple->tuple);
targetRelationOid = FindTargetRelationOid(relation, newTuple,
replicationSlotName);
break;
}
case REORDER_BUFFER_CHANGE_DELETE:
{
HeapTuple oldTuple = &(change->data.tp.oldtuple->tuple);
targetRelationOid = FindTargetRelationOid(relation, oldTuple,
replicationSlotName);
break;
}
/* Only INSERT/DELETE/UPDATE actions are visible in the replication path of split shard */
default:
ereport(ERROR, errmsg(
"Unexpected Action :%d. Expected action is INSERT/DELETE/UPDATE",
change->action));
}
/* Current replication slot is not responsible for handling the change */
if (targetRelationOid == InvalidOid)
{
return;
}
Relation targetRelation = RelationIdGetRelation(targetRelationOid);
pgoutputChangeCB(ctx, txn, targetRelation, change);
RelationClose(targetRelation);
}
/*
* FindTargetRelationOid returns the destination relation Oid for the incoming
* tuple.
* sourceShardRelation - Relation on which a commit has happened.
* tuple - changed tuple.
* currentSlotName - Name of replication slot that is processing this update.
*/
static Oid
FindTargetRelationOid(Relation sourceShardRelation,
HeapTuple tuple,
char *currentSlotName)
{
Oid targetRelationOid = InvalidOid;
Oid sourceShardRelationOid = sourceShardRelation->rd_id;
/* Get child shard list for source(parent) shard from hashmap*/
bool found = false;
SourceToDestinationShardMapEntry *entry =
(SourceToDestinationShardMapEntry *) hash_search(
SourceToDestinationShardMap, &sourceShardRelationOid, HASH_FIND, &found);
/*
* Source shard Oid might not exist in the hash map. This can happen
* in below cases:
* 1) The commit can belong to any other table that is not under going split.
* 2) The commit can be recursive in nature. When the source shard
* receives a commit(a), the WAL sender processes this commit message. This
* commit is applied to a child shard which is placed on the same node as a
* part of replication. This in turn creates one more commit(b) which is recursive in nature.
* Commit 'b' should be skipped as the source shard and destination for commit 'b'
* are same and the commit has already been applied.
*/
if (!found)
{
return InvalidOid;
}
ShardSplitInfo *shardSplitInfo = (ShardSplitInfo *) lfirst(list_head(
entry->
shardSplitInfoList));
int hashValue = GetHashValueForIncomingTuple(sourceShardRelation, tuple,
shardSplitInfo->partitionColumnIndex,
shardSplitInfo->distributedTableOid);
shardSplitInfo = NULL;
foreach_ptr(shardSplitInfo, entry->shardSplitInfoList)
{
if (shardSplitInfo->shardMinValue <= hashValue &&
shardSplitInfo->shardMaxValue >= hashValue)
{
targetRelationOid = shardSplitInfo->splitChildShardOid;
break;
}
}
return targetRelationOid;
}
/*
* GetHashValueForIncomingTuple returns the hash value of the partition
* column for the incoming tuple.
*/
static int32_t
GetHashValueForIncomingTuple(Relation sourceShardRelation,
HeapTuple tuple,
int partitionColumnIndex,
Oid distributedTableOid)
{
TupleDesc relationTupleDes = RelationGetDescr(sourceShardRelation);
Form_pg_attribute partitionColumn = TupleDescAttr(relationTupleDes,
partitionColumnIndex);
bool isNull = false;
Datum partitionColumnValue = heap_getattr(tuple,
partitionColumnIndex + 1,
relationTupleDes,
&isNull);
TypeCacheEntry *typeEntry = lookup_type_cache(partitionColumn->atttypid,
TYPECACHE_HASH_PROC_FINFO);
/* get hashed value of the distribution value */
Datum hashedValueDatum = FunctionCall1(&(typeEntry->hash_proc_finfo),
partitionColumnValue);
return DatumGetInt32(hashedValueDatum);
}

View File

@ -0,0 +1,781 @@
/*-------------------------------------------------------------------------
*
* shardsplit_logical_replication.c
*
* Function definitions for logically replicating shard to split children.
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "miscadmin.h"
#include "nodes/pg_list.h"
#include "distributed/colocation_utils.h"
#include "distributed/metadata_cache.h"
#include "distributed/multi_partitioning_utils.h"
#include "distributed/shardinterval_utils.h"
#include "distributed/connection_management.h"
#include "distributed/remote_commands.h"
#include "distributed/shard_split.h"
#include "distributed/shared_library_init.h"
#include "distributed/listutils.h"
#include "distributed/shardsplit_logical_replication.h"
#include "distributed/resource_lock.h"
#include "utils/builtins.h"
#include "commands/dbcommands.h"
static HTAB *ShardInfoHashMapForPublications = NULL;
/* function declarations */
static void AddPublishableShardEntryInMap(uint32 targetNodeId,
ShardInterval *shardInterval, bool
isChildShardInterval);
ShardSplitSubscriberMetadata * CreateShardSplitSubscriberMetadata(Oid tableOwnerId, uint32
nodeId,
List *
replicationSlotInfoList);
static void CreateShardSplitPublicationForNode(MultiConnection *connection,
List *shardList,
uint32_t publicationForTargetNodeId, Oid
tableOwner);
static char * ShardSplitPublicationName(uint32_t nodeId, Oid ownerId);
static void DropAllShardSplitSubscriptions(MultiConnection *cleanupConnection);
static void DropAllShardSplitPublications(MultiConnection *cleanupConnection);
static void DropAllShardSplitUsers(MultiConnection *cleanupConnection);
static void DropAllShardSplitReplicationSlots(MultiConnection *cleanupConnection);
/*
* CreateShardSplitInfoMapForPublication creates a hashmap that groups
* shards for creating publications and subscriptions.
*
* While creating publications and subscriptions, apart from table owners,
* placement of child shard matters too. To further understand this, please see
* the following example:
*
* Shard1(on Worker1) is to be split in Shard2 and Shard3 on Worker2 and Worker3 respectively.
* Lets assume the owner to be 'A'. The hashmap groups shard list in the following way.
*
* Map key
* ======= ------ ------
* <Worker2, 'A'> ------> |Shard2|-->|Shard1|
* ------ ------
*
* ------ ------
* <Worker3, 'A'> ------> |Shard3|-->|Shard1|
* ------ ------
* Shard1 is a dummy table that is to be created on Worker2 and Worker3.
* Based on the above placement, we would need to create two publications on the source node.
*/
HTAB *
CreateShardSplitInfoMapForPublication(List *sourceColocatedShardIntervalList,
List *shardGroupSplitIntervalListList,
List *destinationWorkerNodesList)
{
ShardInfoHashMapForPublications = SetupHashMapForShardInfo();
ShardInterval *sourceShardIntervalToCopy = NULL;
List *splitChildShardIntervalList = NULL;
forboth_ptr(sourceShardIntervalToCopy, sourceColocatedShardIntervalList,
splitChildShardIntervalList, shardGroupSplitIntervalListList)
{
/*
* Skipping partitioned table for logical replication.
* Since PG13, logical replication is supported for partitioned tables.
* However, we want to keep the behaviour consistent with shard moves.
*/
if (PartitionedTable(sourceShardIntervalToCopy->relationId))
{
continue;
}
ShardInterval *splitChildShardInterval = NULL;
WorkerNode *destinationWorkerNode = NULL;
forboth_ptr(splitChildShardInterval, splitChildShardIntervalList,
destinationWorkerNode, destinationWorkerNodesList)
{
uint32 destinationWorkerNodeId = destinationWorkerNode->nodeId;
/* Add child shard for publication.
* If a columnar shard is a part of publications, then writes on the shard fail.
* In the case of local split, adding child shards to the publication
* would prevent copying the initial data done through 'DoSplitCopy'.
* Hence we avoid adding columnar child shards to publication.
*/
if (!extern_IsColumnarTableAmTable(splitChildShardInterval->relationId))
{
AddPublishableShardEntryInMap(destinationWorkerNodeId,
splitChildShardInterval,
true /*isChildShardInterval*/);
}
/* Add parent shard if not already added */
AddPublishableShardEntryInMap(destinationWorkerNodeId,
sourceShardIntervalToCopy,
false /*isChildShardInterval*/);
}
}
return ShardInfoHashMapForPublications;
}
/*
* AddPublishableShardEntryInMap adds a shard interval in the list
* of shards to be published.
*/
static void
AddPublishableShardEntryInMap(uint32 targetNodeId, ShardInterval *shardInterval, bool
isChildShardInterval)
{
NodeShardMappingKey key;
key.nodeId = targetNodeId;
key.tableOwnerId = TableOwnerOid(shardInterval->relationId);
bool found = false;
NodeShardMappingEntry *nodeMappingEntry =
(NodeShardMappingEntry *) hash_search(ShardInfoHashMapForPublications, &key,
HASH_ENTER,
&found);
/* Create a new list for <nodeId, owner> pair */
if (!found)
{
nodeMappingEntry->shardSplitInfoList = NIL;
}
/* Add child shard interval */
if (isChildShardInterval)
{
nodeMappingEntry->shardSplitInfoList =
lappend(nodeMappingEntry->shardSplitInfoList,
(ShardInterval *) shardInterval);
/* We return from here as the child interval is only added once in the list */
return;
}
/* Check if parent is already added */
ShardInterval *existingShardInterval = NULL;
foreach_ptr(existingShardInterval, nodeMappingEntry->shardSplitInfoList)
{
if (existingShardInterval->shardId == shardInterval->shardId)
{
/* parent shard interval is already added hence return */
return;
}
}
/* Add parent shard Interval */
nodeMappingEntry->shardSplitInfoList =
lappend(nodeMappingEntry->shardSplitInfoList, (ShardInterval *) shardInterval);
}
/*
* CreateShardSplitPublications creates publications on the source node.
*
* sourceConnection - Connection of source node.
*
* shardInfoHashMapForPublication - ShardIntervals are grouped by <owner, nodeId> key.
* A publication is created for list of
* ShardIntervals mapped by key.
*/
void
CreateShardSplitPublications(MultiConnection *sourceConnection,
HTAB *shardInfoHashMapForPublication)
{
HASH_SEQ_STATUS status;
hash_seq_init(&status, shardInfoHashMapForPublication);
NodeShardMappingEntry *entry = NULL;
while ((entry = (NodeShardMappingEntry *) hash_seq_search(&status)) != NULL)
{
uint32 nodeId = entry->key.nodeId;
uint32 tableOwnerId = entry->key.tableOwnerId;
List *shardListForPublication = entry->shardSplitInfoList;
/* Create publication on shard list */
CreateShardSplitPublicationForNode(sourceConnection,
shardListForPublication,
nodeId,
tableOwnerId);
}
}
/*
* CreateShardSplitPublicationForNode creates a publication on source node
* for given shard list.
* We follow the 'SHARD_SPLIT_X_PREFIX' naming scheme for creating publications
* related to split operations.
*/
static void
CreateShardSplitPublicationForNode(MultiConnection *connection, List *shardList,
uint32_t publicationForTargetNodeId, Oid ownerId)
{
StringInfo createPublicationCommand = makeStringInfo();
bool prefixWithComma = false;
appendStringInfo(createPublicationCommand, "CREATE PUBLICATION %s FOR TABLE ",
ShardSplitPublicationName(publicationForTargetNodeId, ownerId));
ShardInterval *shard = NULL;
foreach_ptr(shard, shardList)
{
char *shardName = ConstructQualifiedShardName(shard);
if (prefixWithComma)
{
appendStringInfoString(createPublicationCommand, ",");
}
appendStringInfoString(createPublicationCommand, shardName);
prefixWithComma = true;
}
ExecuteCriticalRemoteCommand(connection, createPublicationCommand->data);
pfree(createPublicationCommand->data);
pfree(createPublicationCommand);
}
/*
* ShardSplitPublicationName returns publication name for Shard Split operations.
*/
static char *
ShardSplitPublicationName(uint32_t nodeId, Oid ownerId)
{
return psprintf("%s%u_%u", SHARD_SPLIT_PUBLICATION_PREFIX, nodeId, ownerId);
}
/*
* CreateTargetNodeConnectionsForShardSplit creates connections on target nodes.
* These connections are used for subscription managment. They are closed
* at the end of non-blocking split workflow.
*/
List *
CreateTargetNodeConnectionsForShardSplit(List *shardSplitSubscribersMetadataList, int
connectionFlags, char *user, char *databaseName)
{
List *targetNodeConnectionList = NIL;
ShardSplitSubscriberMetadata *shardSplitSubscriberMetadata = NULL;
foreach_ptr(shardSplitSubscriberMetadata, shardSplitSubscribersMetadataList)
{
/* slotinfo is expected to be already populated */
Assert(shardSplitSubscriberMetadata->slotInfo != NULL);
uint32 targetWorkerNodeId = shardSplitSubscriberMetadata->slotInfo->targetNodeId;
WorkerNode *targetWorkerNode = FindNodeWithNodeId(targetWorkerNodeId, false);
MultiConnection *targetConnection =
GetNodeUserDatabaseConnection(connectionFlags, targetWorkerNode->workerName,
targetWorkerNode->workerPort,
user,
databaseName);
ClaimConnectionExclusively(targetConnection);
targetNodeConnectionList = lappend(targetNodeConnectionList, targetConnection);
/* Cache the connections for each subscription */
shardSplitSubscriberMetadata->targetNodeConnection = targetConnection;
}
return targetNodeConnectionList;
}
/*
* PopulateShardSplitSubscriptionsMetadataList returns a list of 'ShardSplitSubscriberMetadata'
* structure.
*
* shardSplitInfoHashMap - Shards are grouped by <owner, node id> key.
* For each key, we create a metadata structure. This facilitates easy
* publication-subscription management.
*
* replicationSlotInfoList - List of replication slot info.
*/
List *
PopulateShardSplitSubscriptionsMetadataList(HTAB *shardSplitInfoHashMap,
List *replicationSlotInfoList)
{
HASH_SEQ_STATUS status;
hash_seq_init(&status, shardSplitInfoHashMap);
NodeShardMappingEntry *entry = NULL;
List *shardSplitSubscriptionMetadataList = NIL;
while ((entry = (NodeShardMappingEntry *) hash_seq_search(&status)) != NULL)
{
uint32 nodeId = entry->key.nodeId;
uint32 tableOwnerId = entry->key.tableOwnerId;
ShardSplitSubscriberMetadata *shardSplitSubscriberMetadata =
CreateShardSplitSubscriberMetadata(tableOwnerId, nodeId,
replicationSlotInfoList);
shardSplitSubscriptionMetadataList = lappend(shardSplitSubscriptionMetadataList,
shardSplitSubscriberMetadata);
}
return shardSplitSubscriptionMetadataList;
}
/*
* Creates a 'ShardSplitSubscriberMetadata' structure for given table owner, node id.
* It scans the list of 'ReplicationSlotInfo' to identify the corresponding slot
* to be used for given tableOwnerId and nodeId.
*/
ShardSplitSubscriberMetadata *
CreateShardSplitSubscriberMetadata(Oid tableOwnerId, uint32 nodeId,
List *replicationSlotInfoList)
{
ShardSplitSubscriberMetadata *shardSplitSubscriberMetadata = palloc0(
sizeof(ShardSplitSubscriberMetadata));
shardSplitSubscriberMetadata->tableOwnerId = tableOwnerId;
/*
* Each 'ReplicationSlotInfo' belongs to a unique combination of node id and owner.
* Traverse the slot list to identify the corresponding slot for given
* table owner and node.
*/
char *tableOwnerName = GetUserNameFromId(tableOwnerId, false);
ReplicationSlotInfo *replicationSlotInfo = NULL;
foreach_ptr(replicationSlotInfo, replicationSlotInfoList)
{
if (nodeId == replicationSlotInfo->targetNodeId &&
strcmp(tableOwnerName, replicationSlotInfo->tableOwnerName) == 0)
{
shardSplitSubscriberMetadata->slotInfo = replicationSlotInfo;
break;
}
}
return shardSplitSubscriberMetadata;
}
/*
* CreateShardSplitSubscriptions creates subscriptions for Shard Split operation.
* We follow Shard Split naming scheme for Publication-Subscription management.
*
* targetNodeConnectionList - List of connections to target nodes on which
* subscriptions have to be created.
*
* shardSplitSubscriberMetadataList - List of subscriber metadata.
*
* sourceWorkerNode - Source node.
*/
void
CreateShardSplitSubscriptions(List *targetNodeConnectionList,
List *shardSplitSubscriberMetadataList,
WorkerNode *sourceWorkerNode,
char *superUser,
char *databaseName)
{
MultiConnection *targetConnection = NULL;
ShardSplitSubscriberMetadata *shardSplitPubSubMetadata = NULL;
forboth_ptr(targetConnection, targetNodeConnectionList,
shardSplitPubSubMetadata, shardSplitSubscriberMetadataList)
{
uint32 publicationForNodeId = shardSplitPubSubMetadata->slotInfo->targetNodeId;
Oid ownerId = shardSplitPubSubMetadata->tableOwnerId;
CreateShardSplitSubscription(targetConnection,
sourceWorkerNode->workerName,
sourceWorkerNode->workerPort,
superUser,
databaseName,
ShardSplitPublicationName(publicationForNodeId,
ownerId),
shardSplitPubSubMetadata->slotInfo->slotName,
ownerId);
}
}
/*
* WaitForShardSplitRelationSubscriptionsBecomeReady waits for a list of subscriptions
* to be come ready. This method invokes 'WaitForRelationSubscriptionsBecomeReady' for each
* subscription.
*/
void
WaitForShardSplitRelationSubscriptionsBecomeReady(List *shardSplitPubSubMetadataList)
{
ShardSplitSubscriberMetadata *shardSplitPubSubMetadata = NULL;
foreach_ptr(shardSplitPubSubMetadata, shardSplitPubSubMetadataList)
{
Bitmapset *tableOwnerIds = bms_make_singleton(
shardSplitPubSubMetadata->tableOwnerId);
WaitForRelationSubscriptionsBecomeReady(
shardSplitPubSubMetadata->targetNodeConnection, tableOwnerIds,
SHARD_SPLIT_SUBSCRIPTION_PREFIX);
}
}
/*
* WaitForShardSplitRelationSubscriptionsToBeCaughtUp waits until subscriptions are caught up till
* the source LSN. This method invokes 'WaitForShardSubscriptionToCatchUp' for each subscription.
*/
void
WaitForShardSplitRelationSubscriptionsToBeCaughtUp(XLogRecPtr sourcePosition,
List *shardSplitPubSubMetadataList)
{
ShardSplitSubscriberMetadata *shardSplitPubSubMetadata = NULL;
foreach_ptr(shardSplitPubSubMetadata, shardSplitPubSubMetadataList)
{
Bitmapset *tableOwnerIds = bms_make_singleton(
shardSplitPubSubMetadata->tableOwnerId);
WaitForShardSubscriptionToCatchUp(shardSplitPubSubMetadata->targetNodeConnection,
sourcePosition,
tableOwnerIds,
SHARD_SPLIT_SUBSCRIPTION_PREFIX);
}
}
/*
* CreateTemplateReplicationSlot creates a replication slot that acts as a template
* slot for logically replicating split children in the 'catchup' phase of non-blocking split.
* It returns a snapshot name which is used to do snapshotted shard copy in the 'copy' phase
* of nonblocking split workflow.
*/
char *
CreateTemplateReplicationSlot(ShardInterval *shardIntervalToSplit,
MultiConnection *sourceConnection)
{
StringInfo createReplicationSlotCommand = makeStringInfo();
appendStringInfo(createReplicationSlotCommand,
"CREATE_REPLICATION_SLOT %s LOGICAL citus EXPORT_SNAPSHOT;",
ShardSplitTemplateReplicationSlotName(
shardIntervalToSplit->shardId));
PGresult *result = NULL;
int response = ExecuteOptionalRemoteCommand(sourceConnection,
createReplicationSlotCommand->data,
&result);
if (response != RESPONSE_OKAY || !IsResponseOK(result) || PQntuples(result) != 1)
{
ReportResultError(sourceConnection, result, ERROR);
}
/*'snapshot_name' is second column where index starts from zero.
* We're using the pstrdup to copy the data into the current memory context */
char *snapShotName = pstrdup(PQgetvalue(result, 0, 2 /* columIndex */));
return snapShotName;
}
/*
* ShardSplitTemplateReplicationSlotName returns name of template replication slot
* following the shard split naming scheme.
*/
char *
ShardSplitTemplateReplicationSlotName(uint64 shardId)
{
return psprintf("%s%lu", SHARD_SPLIT_TEMPLATE_REPLICATION_SLOT_PREFIX, shardId);
}
/*
* CreateReplicationSlots creates copies of template replication slot
* on the source node.
*
* sourceNodeConnection - Source node connection.
*
* templateSlotName - Template replication slot name whose copies have to be created.
* This slot holds a LSN from which the logical replication
* begins.
*
* shardSplitSubscriberMetadataList - List of 'ShardSplitSubscriberMetadata. '
*
* 'ShardSplitSubscriberMetadata' contains replication slot name that is used
* to create copies of template replication slot on source node. These slot names are returned by
* 'worker_split_shard_replication_setup' UDF and each slot is responsible for a specific
* split range. We try multiple attemtps to clean up these replicaton slot copies in the
* below order to be on safer side.
* 1. Clean up before starting shard split workflow.
* 2. Implicitly dropping slots while dropping subscriptions.
* 3. Explicitly dropping slots which would have skipped over from 2.
*/
void
CreateReplicationSlots(MultiConnection *sourceNodeConnection, char *templateSlotName,
List *shardSplitSubscriberMetadataList)
{
ShardSplitSubscriberMetadata *subscriberMetadata = NULL;
foreach_ptr(subscriberMetadata, shardSplitSubscriberMetadataList)
{
char *slotName = subscriberMetadata->slotInfo->slotName;
StringInfo createReplicationSlotCommand = makeStringInfo();
appendStringInfo(createReplicationSlotCommand,
"SELECT * FROM pg_catalog.pg_copy_logical_replication_slot (%s, %s)",
quote_literal_cstr(templateSlotName), quote_literal_cstr(
slotName));
ExecuteCriticalRemoteCommand(sourceNodeConnection,
createReplicationSlotCommand->data);
}
}
/*
* DropAllShardSplitLeftOvers drops shard split subscriptions, publications, roles
* and replication slots. These might have been left there after
* the coordinator crashed during a shard split. It's important to delete them
* for two reasons:
* 1. Starting new shard split might fail when they exist, because it cannot
* create them.
* 2. Leftover replication slots that are not consumed from anymore make it
* impossible for WAL to be dropped. This can cause out-of-disk issues.
*/
void
DropAllShardSplitLeftOvers(WorkerNode *sourceNode, HTAB *shardSplitHashMapForPubSub)
{
char *superUser = CitusExtensionOwnerName();
char *databaseName = get_database_name(MyDatabaseId);
/*
* We open new connections to all nodes. The reason for this is that
* operations on subscriptions and publications cannot be run in a
* transaction. By forcing a new connection we make sure no transaction is
* active on the connection.
*/
int connectionFlags = FORCE_NEW_CONNECTION;
HASH_SEQ_STATUS statusForSubscription;
hash_seq_init(&statusForSubscription, shardSplitHashMapForPubSub);
NodeShardMappingEntry *entry = NULL;
while ((entry = (NodeShardMappingEntry *) hash_seq_search(&statusForSubscription)) !=
NULL)
{
uint32_t nodeId = entry->key.nodeId;
WorkerNode *workerNode = FindNodeWithNodeId(nodeId, false /*missingOk*/);
MultiConnection *cleanupConnection = GetNodeUserDatabaseConnection(
connectionFlags, workerNode->workerName, workerNode->workerPort,
superUser, databaseName);
/* We need to claim the connection exclusively while dropping the subscription */
ClaimConnectionExclusively(cleanupConnection);
DropAllShardSplitSubscriptions(cleanupConnection);
DropAllShardSplitUsers(cleanupConnection);
/* Close connection after cleanup */
CloseConnection(cleanupConnection);
}
/*Drop all shard split publications at the source*/
MultiConnection *sourceNodeConnection = GetNodeUserDatabaseConnection(
connectionFlags, sourceNode->workerName, sourceNode->workerPort,
superUser, databaseName);
ClaimConnectionExclusively(sourceNodeConnection);
/*
* If replication slot could not be dropped while dropping the
* subscriber, drop it here.
*/
DropAllShardSplitReplicationSlots(sourceNodeConnection);
DropAllShardSplitPublications(sourceNodeConnection);
CloseConnection(sourceNodeConnection);
}
/*
* DropAllShardSplitSubscriptions drops all the existing subscriptions that
* match our shard split naming scheme on the node that the connection points
* to.
*/
void
DropAllShardSplitSubscriptions(MultiConnection *cleanupConnection)
{
char *query = psprintf(
"SELECT subname FROM pg_catalog.pg_subscription "
"WHERE subname LIKE %s || '%%'",
quote_literal_cstr(SHARD_SPLIT_SUBSCRIPTION_PREFIX));
List *subscriptionNameList = GetQueryResultStringList(cleanupConnection, query);
char *subscriptionName = NULL;
foreach_ptr(subscriptionName, subscriptionNameList)
{
DisableAndDropShardSplitSubscription(cleanupConnection, subscriptionName);
}
}
/*
* DropAllShardSplitPublications drops all the existing publications that
* match our shard split naming scheme on the node that the connection points
* to.
*/
static void
DropAllShardSplitPublications(MultiConnection *connection)
{
char *query = psprintf(
"SELECT pubname FROM pg_catalog.pg_publication "
"WHERE pubname LIKE %s || '%%'",
quote_literal_cstr(SHARD_SPLIT_PUBLICATION_PREFIX));
List *publicationNameList = GetQueryResultStringList(connection, query);
char *publicationName;
foreach_ptr(publicationName, publicationNameList)
{
DropShardPublication(connection, publicationName);
}
}
/*
* DropAllShardSplitUsers drops all the users that match our shard split naming
* scheme. The users are temporary created for shard splits.
*/
static void
DropAllShardSplitUsers(MultiConnection *connection)
{
char *query = psprintf(
"SELECT rolname FROM pg_catalog.pg_roles "
"WHERE rolname LIKE %s || '%%'",
quote_literal_cstr(SHARD_SPLIT_SUBSCRIPTION_ROLE_PREFIX));
List *usernameList = GetQueryResultStringList(connection, query);
char *username;
foreach_ptr(username, usernameList)
{
DropShardUser(connection, username);
}
}
/*
* DropAllShardSplitReplicationSlots drops all the existing replication slots
* that match shard split naming scheme on the node that the connection
* points to.
*/
static void
DropAllShardSplitReplicationSlots(MultiConnection *cleanupConnection)
{
char *query = psprintf(
"SELECT slot_name FROM pg_catalog.pg_replication_slots "
"WHERE slot_name LIKE %s || '%%'",
quote_literal_cstr(SHARD_SPLIT_REPLICATION_SLOT_PREFIX));
List *slotNameList = GetQueryResultStringList(cleanupConnection, query);
char *slotName;
foreach_ptr(slotName, slotNameList)
{
DropShardReplicationSlot(cleanupConnection, slotName);
}
}
/*
* DropShardSplitPublications drops the publication used for shard splits over the given
* connection, if it exists.
*/
void
DropShardSplitPublications(MultiConnection *sourceConnection,
HTAB *shardInfoHashMapForPublication)
{
HASH_SEQ_STATUS status;
hash_seq_init(&status, shardInfoHashMapForPublication);
NodeShardMappingEntry *entry = NULL;
while ((entry = (NodeShardMappingEntry *) hash_seq_search(&status)) != NULL)
{
uint32 nodeId = entry->key.nodeId;
uint32 tableOwnerId = entry->key.tableOwnerId;
DropShardPublication(sourceConnection, ShardSplitPublicationName(nodeId,
tableOwnerId));
}
}
/*
* DropShardSplitSubsriptions disables and drops subscriptions from the subscriber node that
* are used to split shards. Note that, it does not drop the replication slots on the publisher node.
* Replication slots will be dropped separately by calling DropShardSplitReplicationSlots.
*/
void
DropShardSplitSubsriptions(List *shardSplitSubscribersMetadataList)
{
ShardSplitSubscriberMetadata *subscriberMetadata = NULL;
foreach_ptr(subscriberMetadata, shardSplitSubscribersMetadataList)
{
uint32 tableOwnerId = subscriberMetadata->tableOwnerId;
MultiConnection *targetNodeConnection = subscriberMetadata->targetNodeConnection;
DisableAndDropShardSplitSubscription(targetNodeConnection, ShardSubscriptionName(
tableOwnerId,
SHARD_SPLIT_SUBSCRIPTION_PREFIX));
DropShardUser(targetNodeConnection, ShardSubscriptionRole(tableOwnerId,
SHARD_SPLIT_SUBSCRIPTION_ROLE_PREFIX));
}
}
/*
* DisableAndDropShardSplitSubscription disables the subscription, resets the slot name to 'none' and
* then drops subscription on the given connection. It does not drop the replication slot.
* The caller of this method should ensure to cleanup the replication slot.
*
* Directly executing 'DROP SUBSCRIPTION' attempts to drop the replication slot at the source node.
* When the subscription is local, direcly dropping the subscription can lead to a self deadlock.
* To avoid this, we first disable the subscription, reset the slot name and then drop the subscription.
*/
void
DisableAndDropShardSplitSubscription(MultiConnection *connection, char *subscriptionName)
{
StringInfo alterSubscriptionSlotCommand = makeStringInfo();
StringInfo alterSubscriptionDisableCommand = makeStringInfo();
appendStringInfo(alterSubscriptionDisableCommand,
"ALTER SUBSCRIPTION %s DISABLE",
quote_identifier(subscriptionName));
ExecuteCriticalRemoteCommand(connection,
alterSubscriptionDisableCommand->data);
appendStringInfo(alterSubscriptionSlotCommand,
"ALTER SUBSCRIPTION %s SET (slot_name = NONE)",
quote_identifier(subscriptionName));
ExecuteCriticalRemoteCommand(connection, alterSubscriptionSlotCommand->data);
ExecuteCriticalRemoteCommand(connection, psprintf(
"DROP SUBSCRIPTION %s",
quote_identifier(subscriptionName)));
}
/*
* DropShardSplitReplicationSlots drops replication slots on the source node.
*/
void
DropShardSplitReplicationSlots(MultiConnection *sourceConnection,
List *replicationSlotInfoList)
{
ReplicationSlotInfo *replicationSlotInfo = NULL;
foreach_ptr(replicationSlotInfo, replicationSlotInfoList)
{
DropShardReplicationSlot(sourceConnection, replicationSlotInfo->slotName);
}
}
/*
* CloseShardSplitSubscriberConnections closes connection of subscriber nodes.
* 'ShardSplitSubscriberMetadata' holds connection for a subscriber node. The method
* traverses the list and closes each connection.
*/
void
CloseShardSplitSubscriberConnections(List *shardSplitSubscriberMetadataList)
{
ShardSplitSubscriberMetadata *subscriberMetadata = NULL;
foreach_ptr(subscriberMetadata, shardSplitSubscriberMetadataList)
{
CloseConnection(subscriberMetadata->targetNodeConnection);
}
}

View File

@ -0,0 +1,387 @@
/*-------------------------------------------------------------------------
*
* shardsplit_shared_memory.c
* API's for creating and accessing shared memory segments to store
* shard split information. 'setup_shard_replication' UDF creates the
* shared memory, populates the contents and WAL sender processes are
* the consumers.
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "distributed/shardinterval_utils.h"
#include "distributed/shardsplit_shared_memory.h"
#include "distributed/citus_safe_lib.h"
#include "distributed/multi_logical_replication.h"
#include "storage/ipc.h"
#include "utils/memutils.h"
#include "common/hashfn.h"
const char *SharedMemoryNameForHandleManagement =
"Shared memory handle for shard split";
static shmem_startup_hook_type prev_shmem_startup_hook = NULL;
/* Function declarations */
static ShardSplitInfoSMHeader * AllocateSharedMemoryForShardSplitInfo(int
shardSplitInfoCount,
Size
shardSplitInfoSize,
dsm_handle *
dsmHandle);
static ShardSplitInfoSMHeader * GetShardSplitInfoSMHeaderFromDSMHandle(dsm_handle
dsmHandle);
static dsm_handle GetShardSplitSharedMemoryHandle(void);
static void ShardSplitShmemInit(void);
/*
* GetShardSplitInfoSMHeaderFromDSMHandle returns the header of the shared memory
* segment. It pins the mapping till lifetime of the backend process accessing it.
*/
static ShardSplitInfoSMHeader *
GetShardSplitInfoSMHeaderFromDSMHandle(dsm_handle dsmHandle)
{
dsm_segment *dsmSegment = dsm_find_mapping(dsmHandle);
if (dsmSegment == NULL)
{
dsmSegment = dsm_attach(dsmHandle);
}
if (dsmSegment == NULL)
{
ereport(ERROR,
(errmsg("could not attach to dynamic shared memory segment "
"corresponding to handle:%u", dsmHandle)));
}
/*
* Detatching segment associated with resource owner with 'dsm_pin_mapping' call before the
* resource owner releases, to avoid warning being logged and potential leaks.
*/
dsm_pin_mapping(dsmSegment);
ShardSplitInfoSMHeader *header = (ShardSplitInfoSMHeader *) dsm_segment_address(
dsmSegment);
if (header == NULL)
{
ereport(ERROR,
(errmsg("Could not get shared memory segment header "
"corresponding to handle for split workflow:%u", dsmHandle)));
}
return header;
}
/*
* GetShardSplitInfoSMHeader returns pointer to the header of shared memory segment.
*/
ShardSplitInfoSMHeader *
GetShardSplitInfoSMHeader()
{
dsm_handle dsmHandle = GetShardSplitSharedMemoryHandle();
ShardSplitInfoSMHeader *shardSplitInfoSMHeader =
GetShardSplitInfoSMHeaderFromDSMHandle(dsmHandle);
return shardSplitInfoSMHeader;
}
/*
* AllocateSharedMemoryForShardSplitInfo is used to allocate and store
* information about the shard undergoing a split. The function allocates dynamic
* shared memory segment consisting of a header and an array of ShardSplitInfo structure.
* The contents of this shared memory segment are consumed by WAL sender process
* during catch up phase of replication through logical decoding plugin.
*
* The shared memory segment exists till the catch up phase completes or the
* postmaster shutsdown.
*/
static ShardSplitInfoSMHeader *
AllocateSharedMemoryForShardSplitInfo(int shardSplitInfoCount, Size shardSplitInfoSize,
dsm_handle *dsmHandle)
{
if (shardSplitInfoCount <= 0 || shardSplitInfoSize <= 0)
{
ereport(ERROR,
(errmsg("shardSplitInfoCount and size of each step should be "
"positive values")));
}
Size totalSize = offsetof(ShardSplitInfoSMHeader, splitInfoArray) +
(shardSplitInfoCount * shardSplitInfoSize);
dsm_segment *dsmSegment = dsm_create(totalSize, DSM_CREATE_NULL_IF_MAXSEGMENTS);
if (dsmSegment == NULL)
{
ereport(ERROR,
(errmsg("could not create a dynamic shared memory segment to "
"store shard split info")));
}
*dsmHandle = dsm_segment_handle(dsmSegment);
/*
* Pin the segment till Postmaster shutsdown since we need this
* segment even after the session ends for replication catchup phase.
*/
dsm_pin_segment(dsmSegment);
ShardSplitInfoSMHeader *shardSplitInfoSMHeader =
GetShardSplitInfoSMHeaderFromDSMHandle(*dsmHandle);
shardSplitInfoSMHeader->count = shardSplitInfoCount;
return shardSplitInfoSMHeader;
}
/*
* CreateSharedMemoryForShardSplitInfo is a wrapper function which creates shared memory
* for storing shard split infomation. The function returns pointer to the header of
* shared memory segment.
*
* shardSplitInfoCount - number of 'ShardSplitInfo ' elements to be allocated
* dsmHandle - handle of the allocated shared memory segment
*/
ShardSplitInfoSMHeader *
CreateSharedMemoryForShardSplitInfo(int shardSplitInfoCount, dsm_handle *dsmHandle)
{
ShardSplitInfoSMHeader *shardSplitInfoSMHeader =
AllocateSharedMemoryForShardSplitInfo(shardSplitInfoCount,
sizeof(ShardSplitInfo),
dsmHandle);
return shardSplitInfoSMHeader;
}
/*
* ReleaseSharedMemoryOfShardSplitInfo releases(unpins) the dynamic shared memory segment
* allocated by 'worker_split_shard_replication_setup'. This shared memory was pinned
* to Postmaster process and is valid till Postmaster shutsdown or
* explicitly unpinned by calling 'dsm_unpin_segment'.
*/
void
ReleaseSharedMemoryOfShardSplitInfo()
{
/* Get handle of dynamic shared memory segment*/
dsm_handle dsmHandle = GetShardSplitSharedMemoryHandle();
/*
* Unpin the dynamic shared memory segment. 'dsm_pin_segment' was
* called previously by 'AllocateSharedMemoryForShardSplitInfo'.
*/
dsm_unpin_segment(dsmHandle);
/*
* As dynamic shared memory is unpinned, store an invalid handle in static
* shared memory used for handle management.
*/
StoreShardSplitSharedMemoryHandle(DSM_HANDLE_INVALID);
}
/*
* EncodeReplicationSlot returns an encoded replication slot name
* in the following format.
* Slot Name = citus_split_nodeId_tableOwnerOid
* Max supported length of replication slot name is 64 bytes.
*/
char *
EncodeReplicationSlot(uint32_t nodeId,
uint32_t tableOwnerId)
{
StringInfo slotName = makeStringInfo();
appendStringInfo(slotName, "%s%u_%u", SHARD_SPLIT_REPLICATION_SLOT_PREFIX, nodeId,
tableOwnerId);
if (slotName->len > NAMEDATALEN)
{
ereport(ERROR,
(errmsg(
"Replication Slot name:%s having length:%d is greater than maximum allowed length:%d",
slotName->data, slotName->len, NAMEDATALEN)));
}
return slotName->data;
}
/*
* InitializeShardSplitSMHandleManagement requests the necessary shared memory
* from Postgres and sets up the shared memory startup hook.
* This memory is used to store handle of other shared memories allocated during split workflow.
*/
void
InitializeShardSplitSMHandleManagement(void)
{
prev_shmem_startup_hook = shmem_startup_hook;
shmem_startup_hook = ShardSplitShmemInit;
}
static void
ShardSplitShmemInit(void)
{
bool alreadyInitialized = false;
ShardSplitShmemData *smData = ShmemInitStruct(SharedMemoryNameForHandleManagement,
sizeof(ShardSplitShmemData),
&alreadyInitialized);
if (!alreadyInitialized)
{
char *trancheName = "Split Shard Setup Tranche";
NamedLWLockTranche *namedLockTranche =
&smData->namedLockTranche;
/* start by zeroing out all the memory */
memset(smData, 0,
sizeof(ShardSplitShmemData));
namedLockTranche->trancheId = LWLockNewTrancheId();
LWLockRegisterTranche(namedLockTranche->trancheId, trancheName);
LWLockInitialize(&smData->lock,
namedLockTranche->trancheId);
smData->dsmHandle = DSM_HANDLE_INVALID;
}
if (prev_shmem_startup_hook != NULL)
{
prev_shmem_startup_hook();
}
}
/*
* StoreShardSplitSharedMemoryHandle stores a handle of shared memory
* allocated and populated by 'worker_split_shard_replication_setup' UDF.
* This handle is stored in a different statically allocated shared memory
* segment with name 'Shared memory handle for shard split'.
*/
void
StoreShardSplitSharedMemoryHandle(dsm_handle dsmHandle)
{
bool found = false;
ShardSplitShmemData *smData = ShmemInitStruct(SharedMemoryNameForHandleManagement,
sizeof(ShardSplitShmemData),
&found);
if (!found)
{
ereport(ERROR,
errmsg(
"Shared memory for handle management should have been initialized during boot"));
}
/*
* We only support non concurrent split. However, it is fine to take a
* lock and store the handle incase concurrent splits are introduced in future.
*/
LWLockAcquire(&smData->lock, LW_EXCLUSIVE);
/*
* In a normal situation, previously stored handle should have been invalidated
* before the current function is called.
* If this handle is still valid, it means cleanup of previous split shard
* workflow failed. Log a waring and continue the current shard split operation.
*/
if (smData->dsmHandle != DSM_HANDLE_INVALID)
{
ereport(WARNING,
errmsg(
"Previous split shard worflow was not successfully and could not complete the cleanup phase."
" Continuing with the current split shard workflow."));
}
/* Store the incoming handle */
smData->dsmHandle = dsmHandle;
LWLockRelease(&smData->lock);
}
/*
* GetShardSplitSharedMemoryHandle returns the handle of dynamic shared memory segment stored
* by 'worker_split_shard_replication_setup' UDF. This handle is requested by WAL sender processes
* during logical replication phase or during cleanup.
*/
dsm_handle
GetShardSplitSharedMemoryHandle(void)
{
bool found = false;
ShardSplitShmemData *smData = ShmemInitStruct(SharedMemoryNameForHandleManagement,
sizeof(ShardSplitShmemData),
&found);
if (!found)
{
ereport(ERROR,
errmsg(
"Shared memory for handle management should have been initialized during boot"));
}
LWLockAcquire(&smData->lock, LW_SHARED);
dsm_handle dsmHandle = smData->dsmHandle;
LWLockRelease(&smData->lock);
return dsmHandle;
}
/*
* PopulateSourceToDestinationShardMapForSlot populates 'SourceToDestinationShard' hash map for a given slot.
* Key of the map is Oid of source shard which is undergoing a split and value is a list of corresponding child shards.
* To populate the map, the function traverses 'ShardSplitInfo' array stored within shared memory segment.
*/
HTAB *
PopulateSourceToDestinationShardMapForSlot(char *slotName, MemoryContext cxt)
{
HASHCTL info;
memset(&info, 0, sizeof(info));
info.keysize = sizeof(Oid);
info.entrysize = sizeof(SourceToDestinationShardMapEntry);
info.hash = uint32_hash;
info.hcxt = cxt;
int hashFlags = (HASH_ELEM | HASH_CONTEXT | HASH_FUNCTION);
HTAB *sourceShardToDesShardMap = hash_create("SourceToDestinationShardMap", 128,
&info, hashFlags);
MemoryContext oldContext = MemoryContextSwitchTo(cxt);
ShardSplitInfoSMHeader *smHeader = GetShardSplitInfoSMHeader();
for (int index = 0; index < smHeader->count; index++)
{
if (strcmp(smHeader->splitInfoArray[index].slotName, slotName) == 0)
{
Oid sourceShardOid = smHeader->splitInfoArray[index].sourceShardOid;
bool found = false;
SourceToDestinationShardMapEntry *entry =
(SourceToDestinationShardMapEntry *) hash_search(
sourceShardToDesShardMap, &sourceShardOid, HASH_ENTER, &found);
if (!found)
{
entry->shardSplitInfoList = NIL;
entry->sourceShardKey = sourceShardOid;
}
ShardSplitInfo *shardSplitInfoForSlot = (ShardSplitInfo *) palloc0(
sizeof(ShardSplitInfo));
*shardSplitInfoForSlot = smHeader->splitInfoArray[index];
entry->shardSplitInfoList = lappend(entry->shardSplitInfoList,
(ShardSplitInfo *) shardSplitInfoForSlot);
}
}
MemoryContextSwitchTo(oldContext);
return sourceShardToDesShardMap;
}

View File

@ -75,6 +75,7 @@
#include "distributed/run_from_same_connection.h"
#include "distributed/shard_cleaner.h"
#include "distributed/shared_connection_stats.h"
#include "distributed/shardsplit_shared_memory.h"
#include "distributed/query_pushdown_planning.h"
#include "distributed/time_constants.h"
#include "distributed/query_stats.h"
@ -395,6 +396,9 @@ _PG_init(void)
InitializeSharedConnectionStats();
InitializeLocallyReservedSharedConnections();
/* initialize shard split shared memory handle management */
InitializeShardSplitSMHandleManagement();
/* enable modification of pg_catalog tables during pg_upgrade */
if (IsBinaryUpgrade)
{

View File

@ -70,3 +70,4 @@ DROP FUNCTION pg_catalog.get_all_active_transactions(OUT datid oid, OUT process_
#include "udfs/citus_split_shard_by_split_points/11.1-1.sql"
#include "udfs/worker_split_copy/11.1-1.sql"
#include "udfs/worker_copy_table_to_node/11.1-1.sql"
#include "udfs/worker_split_shard_replication_setup/11.1-1.sql"

View File

@ -77,6 +77,11 @@ DROP FUNCTION pg_catalog.worker_copy_table_to_node(
source_table regclass,
target_node_id integer);
DROP FUNCTION pg_catalog.worker_split_shard_replication_setup(
splitShardInfo pg_catalog.split_shard_info[]);
DROP TYPE pg_catalog.split_shard_info;
DROP TYPE pg_catalog.replication_slot_info;
DROP FUNCTION pg_catalog.get_all_active_transactions(OUT datid oid, OUT process_id int, OUT initiator_node_identifier int4,
OUT worker_query BOOL, OUT transaction_number int8, OUT transaction_stamp timestamptz,
OUT global_pid int8);

View File

@ -0,0 +1,25 @@
CREATE TYPE citus.split_shard_info AS (
source_shard_id bigint,
distribution_column text,
child_shard_id bigint,
shard_min_value text,
shard_max_value text,
node_id integer);
ALTER TYPE citus.split_shard_info SET SCHEMA pg_catalog;
COMMENT ON TYPE pg_catalog.split_shard_info
IS 'Stores split child shard information';
CREATE TYPE citus.replication_slot_info AS(node_id integer, slot_owner text, slot_name text);
ALTER TYPE citus.replication_slot_info SET SCHEMA pg_catalog;
COMMENT ON TYPE pg_catalog.replication_slot_info
IS 'Replication slot information to be used for subscriptions during non blocking shard split';
CREATE OR REPLACE FUNCTION pg_catalog.worker_split_shard_replication_setup(
splitShardInfo pg_catalog.split_shard_info[])
RETURNS setof pg_catalog.replication_slot_info
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$worker_split_shard_replication_setup$$;
COMMENT ON FUNCTION pg_catalog.worker_split_shard_replication_setup(splitShardInfo pg_catalog.split_shard_info[])
IS 'Replication setup for splitting a shard';
REVOKE ALL ON FUNCTION pg_catalog.worker_split_shard_replication_setup(pg_catalog.split_shard_info[]) FROM PUBLIC;

View File

@ -0,0 +1,25 @@
CREATE TYPE citus.split_shard_info AS (
source_shard_id bigint,
distribution_column text,
child_shard_id bigint,
shard_min_value text,
shard_max_value text,
node_id integer);
ALTER TYPE citus.split_shard_info SET SCHEMA pg_catalog;
COMMENT ON TYPE pg_catalog.split_shard_info
IS 'Stores split child shard information';
CREATE TYPE citus.replication_slot_info AS(node_id integer, slot_owner text, slot_name text);
ALTER TYPE citus.replication_slot_info SET SCHEMA pg_catalog;
COMMENT ON TYPE pg_catalog.replication_slot_info
IS 'Replication slot information to be used for subscriptions during non blocking shard split';
CREATE OR REPLACE FUNCTION pg_catalog.worker_split_shard_replication_setup(
splitShardInfo pg_catalog.split_shard_info[])
RETURNS setof pg_catalog.replication_slot_info
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$worker_split_shard_replication_setup$$;
COMMENT ON FUNCTION pg_catalog.worker_split_shard_replication_setup(splitShardInfo pg_catalog.split_shard_info[])
IS 'Replication setup for splitting a shard';
REVOKE ALL ON FUNCTION pg_catalog.worker_split_shard_replication_setup(pg_catalog.split_shard_info[]) FROM PUBLIC;

View File

@ -110,6 +110,53 @@ SafeStringToInt64(const char *str)
}
/*
* SafeStringToInt32 converts a string containing a number to a int32. When it
* fails it calls ereport.
*
* The different error cases are inspired by
* https://stackoverflow.com/a/26083517/2570866
*/
int32
SafeStringToInt32(const char *str)
{
char *endptr;
errno = 0;
long number = strtol(str, &endptr, 10);
if (str == endptr)
{
ereport(ERROR, (errmsg("Error parsing %s as int32, no digits found\n", str)));
}
else if ((errno == ERANGE && number == LONG_MIN) || number < INT32_MIN)
{
ereport(ERROR, (errmsg("Error parsing %s as int32, underflow occurred\n", str)));
}
else if ((errno == ERANGE && number == LONG_MAX) || number > INT32_MAX)
{
ereport(ERROR, (errmsg("Error parsing %s as int32, overflow occurred\n", str)));
}
else if (errno == EINVAL)
{
ereport(ERROR, (errmsg(
"Error parsing %s as int32, base contains unsupported value\n",
str)));
}
else if (errno != 0 && number == 0)
{
int err = errno;
ereport(ERROR, (errmsg("Error parsing %s as int32, errno %d\n", str, err)));
}
else if (errno == 0 && str && *endptr != '\0')
{
ereport(ERROR, (errmsg(
"Error parsing %s as int32, aditional characters remain after int32\n",
str)));
}
return number;
}
/*
* SafeStringToUint64 converts a string containing a number to a uint64. When it
* fails it calls ereport.

View File

@ -19,6 +19,7 @@
extern void ereport_constraint_handler(const char *message, void *pointer, errno_t error);
extern int64 SafeStringToInt64(const char *str);
extern int32 SafeStringToInt32(const char *str);
extern uint64 SafeStringToUint64(const char *str);
extern void SafeQsort(void *ptr, rsize_t count, rsize_t size,
int (*comp)(const void *, const void *));

View File

@ -97,7 +97,18 @@ enum MultiConnectionMode
* establishments may be suspended until a connection slot is available to
* the remote host.
*/
WAIT_FOR_CONNECTION = 1 << 7
WAIT_FOR_CONNECTION = 1 << 7,
/*
* Use the flag to start a connection for streaming replication.
* This flag constructs additional libpq connection parameters needed for streaming
* replication protocol. It adds 'replication=database' param which instructs
* the backend to go into logical replication walsender mode.
* https://www.postgresql.org/docs/current/protocol-replication.html
*
* This is need to run 'CREATE_REPLICATION_SLOT' command.
*/
REQUIRE_REPLICATION_CONNECTION_PARAM = 1 << 8
};
@ -187,6 +198,9 @@ typedef struct MultiConnection
/* number of bytes sent to PQputCopyData() since last flush */
uint64 copyBytesWrittenSinceLastFlush;
/* replication option */
bool requiresReplication;
MultiConnectionStructInitializationState initilizationState;
} MultiConnection;
@ -207,6 +221,7 @@ typedef struct ConnectionHashKey
int32 port;
char user[NAMEDATALEN];
char database[NAMEDATALEN];
bool replicationConnParam;
} ConnectionHashKey;
/* hash entry */

View File

@ -25,6 +25,7 @@
#include "distributed/connection_management.h"
#include "distributed/errormessage.h"
#include "distributed/relay_utility.h"
#include "distributed/worker_manager.h"
#include "utils/acl.h"
#include "utils/relcache.h"
@ -226,6 +227,7 @@ extern List * ActiveShardPlacementListOnGroup(uint64 shardId, int32 groupId);
extern List * ActiveShardPlacementList(uint64 shardId);
extern List * ShardPlacementListWithoutOrphanedPlacements(uint64 shardId);
extern ShardPlacement * ActiveShardPlacement(uint64 shardId, bool missingOk);
extern WorkerNode * ActiveShardPlacementWorkerNode(uint64 shardId);
extern List * BuildShardPlacementList(int64 shardId);
extern List * AllShardPlacementsOnNodeGroup(int32 groupId);
extern List * AllShardPlacementsWithShardPlacementState(ShardState shardState);

View File

@ -26,8 +26,44 @@ extern void LogicallyReplicateShards(List *shardList, char *sourceNodeName,
int sourceNodePort, char *targetNodeName,
int targetNodePort);
#define SHARD_MOVE_PUBLICATION_PREFIX "citus_shard_move_publication_"
#define SHARD_MOVE_SUBSCRIPTION_PREFIX "citus_shard_move_subscription_"
#define SHARD_MOVE_SUBSCRIPTION_ROLE_PREFIX "citus_shard_move_subscription_role_"
extern void ConflictOnlyWithIsolationTesting(void);
extern void CreateReplicaIdentity(List *shardList, char *nodeName, int32
nodePort);
extern XLogRecPtr GetRemoteLogPosition(MultiConnection *connection);
extern List * GetQueryResultStringList(MultiConnection *connection, char *query);
extern void DropShardSubscription(MultiConnection *connection,
char *subscriptionName);
extern void DropShardPublication(MultiConnection *connection, char *publicationName);
extern void DropShardUser(MultiConnection *connection, char *username);
extern void DropShardReplicationSlot(MultiConnection *connection,
char *publicationName);
extern char * ShardSubscriptionRole(Oid ownerId, char *operationPrefix);
extern char * ShardSubscriptionName(Oid ownerId, char *operationPrefix);
extern void CreateShardSplitSubscription(MultiConnection *connection,
char *sourceNodeName,
int sourceNodePort, char *userName,
char *databaseName,
char *publicationName, char *slotName,
Oid ownerId);
extern void WaitForRelationSubscriptionsBecomeReady(MultiConnection *targetConnection,
Bitmapset *tableOwnerIds,
char *operationPrefix);
extern void WaitForShardSubscriptionToCatchUp(MultiConnection *targetConnection,
XLogRecPtr sourcePosition,
Bitmapset *tableOwnerIds,
char *operationPrefix);
#define SHARD_MOVE_PUBLICATION_PREFIX "citus_shard_move_publication_"
#define SHARD_MOVE_SUBSCRIPTION_ROLE_PREFIX "citus_shard_move_subscription_role_"
#define SHARD_MOVE_SUBSCRIPTION_PREFIX "citus_shard_move_subscription_"
#define SHARD_SPLIT_PUBLICATION_PREFIX "citus_shard_split_publication_"
#define SHARD_SPLIT_SUBSCRIPTION_PREFIX "citus_shard_split_subscription_"
#define SHARD_SPLIT_SUBSCRIPTION_ROLE_PREFIX "citus_shard_split_subscription_role_"
#define SHARD_SPLIT_TEMPLATE_REPLICATION_SLOT_PREFIX "citus_shard_split_template_slot_"
#define SHARD_SPLIT_REPLICATION_SLOT_PREFIX "citus_shard_split_"
#endif /* MULTI_LOGICAL_REPLICATION_H_ */

View File

@ -15,7 +15,8 @@
/* Split Modes supported by Shard Split API */
typedef enum SplitMode
{
BLOCKING_SPLIT = 0
BLOCKING_SPLIT = 0,
NON_BLOCKING_SPLIT = 1
} SplitMode;
/*
@ -28,6 +29,7 @@ typedef enum SplitOperation
ISOLATE_TENANT_TO_NEW_SHARD
} SplitOperation;
/*
* SplitShard API to split a given shard (or shard group) using split mode and
* specified split points to a set of destination nodes.

View File

@ -0,0 +1,114 @@
/*-------------------------------------------------------------------------
*
* shardsplit_logical_replication.h
*
* Function declarations for logically replicating shard to split children.
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#ifndef SHARDSPLIT_LOGICAL_REPLICATION_H
#define SHARDSPLIT_LOGICAL_REPLICATION_H
#include "distributed/multi_logical_replication.h"
/*
* Invocation of 'worker_split_shard_replication_setup' UDF returns set of records
* of custom datatype 'replication_slot_info'. This information is parsed and stored in
* the below data structure. The information is used to create a subscriber on target node
* with corresponding slot name.
*/
typedef struct ReplicationSlotInfo
{
uint32 targetNodeId;
char *tableOwnerName;
char *slotName;
} ReplicationSlotInfo;
/*
* Stores information necesary for creating a subscriber on target node.
* Based on how a shard is split and mapped to target nodes, for each unique combination of
* <tableOwner, targetNodeId> there is a 'ShardSplitSubscriberMetadata'.
*/
typedef struct ShardSplitSubscriberMetadata
{
Oid tableOwnerId;
ReplicationSlotInfo *slotInfo;
/*
* Exclusively claimed connection for a subscription.The target node of subscription
* is pointed by ReplicationSlotInfo.
*/
MultiConnection *targetNodeConnection;
} ShardSplitSubscriberMetadata;
/* key for NodeShardMappingEntry */
typedef struct NodeShardMappingKey
{
uint32_t nodeId;
Oid tableOwnerId;
} NodeShardMappingKey;
/* Entry for hash map */
typedef struct NodeShardMappingEntry
{
NodeShardMappingKey key;
List *shardSplitInfoList;
} NodeShardMappingEntry;
extern uint32 NodeShardMappingHash(const void *key, Size keysize);
extern int NodeShardMappingHashCompare(const void *left, const void *right, Size keysize);
extern HTAB * SetupHashMapForShardInfo(void);
/* Functions for subscriber metadata management */
extern List * PopulateShardSplitSubscriptionsMetadataList(HTAB *shardSplitInfoHashMap,
List *replicationSlotInfoList);
extern HTAB * CreateShardSplitInfoMapForPublication(
List *sourceColocatedShardIntervalList,
List *shardGroupSplitIntervalListList,
List *destinationWorkerNodesList);
/* Functions for creating publications and subscriptions*/
extern void AlterShardSplitPublications(MultiConnection *sourceConnection,
HTAB *shardInfoHashMapForPublication);
extern void CreateShardSplitSubscriptions(List *targetNodeConnectionList,
List *shardSplitSubscriberMetadataList,
WorkerNode *sourceWorkerNode, char *superUser,
char *databaseName);
extern void CreateReplicationSlots(MultiConnection *sourceNodeConnection,
char *templateSlotName,
List *shardSplitSubscriberMetadataList);
extern List * CreateTargetNodeConnectionsForShardSplit(
List *shardSplitSubscribersMetadataList,
int
connectionFlags, char *user,
char *databaseName);
/* Functions to drop publisher-subscriber resources */
extern void CreateShardSplitPublications(MultiConnection *sourceConnection,
HTAB *shardInfoHashMapForPublication);
extern char * CreateTemplateReplicationSlot(ShardInterval *shardIntervalToSplit,
MultiConnection *
sourceConnection);
extern void DropAllShardSplitLeftOvers(WorkerNode *sourceNode,
HTAB *shardSplitMapOfPublications);
extern void DropShardSplitPublications(MultiConnection *sourceConnection,
HTAB *shardInfoHashMapForPublication);
extern void DropShardSplitSubsriptions(List *shardSplitSubscribersMetadataList);
extern void DropShardSplitReplicationSlots(MultiConnection *sourceConnection,
List *replicationSlotInfoList);
extern void DisableAndDropShardSplitSubscription(MultiConnection *connection,
char *subscriptionName);
/* Wrapper functions which wait for a subscriber to be ready and catchup */
extern void WaitForShardSplitRelationSubscriptionsBecomeReady(
List *shardSplitPubSubMetadataList);
extern void WaitForShardSplitRelationSubscriptionsToBeCaughtUp(XLogRecPtr sourcePosition,
List *
shardSplitPubSubMetadataList);
extern char * ShardSplitTemplateReplicationSlotName(uint64 shardId);
extern void CloseShardSplitSubscriberConnections(List *shardSplitSubscriberMetadataList);
#endif /* SHARDSPLIT_LOGICAL_REPLICATION_H */

View File

@ -0,0 +1,84 @@
/*-------------------------------------------------------------------------
*
* shardsplit_shared_memory.h
* API's for creating and accessing shared memory segments to store
* shard split information. 'worker_split_shard_replication_setup' UDF creates the
* shared memory and populates the contents. WAL sender processes are consumer
* of split information for appropriate tuple routing.
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#ifndef SHARDSPLIT_SHARED_MEMORY_H
#define SHARDSPLIT_SHARED_MEMORY_H
/*
* In-memory mapping of a split child shard.
*/
typedef struct ShardSplitInfo
{
Oid distributedTableOid; /* citus distributed table Oid */
int partitionColumnIndex; /* partition column index */
Oid sourceShardOid; /* parent shard Oid */
Oid splitChildShardOid; /* child shard Oid */
int32 shardMinValue; /* min hash value */
int32 shardMaxValue; /* max hash value */
uint32_t nodeId; /* node where child shard is to be placed */
uint64 sourceShardId; /* parent shardId */
uint64 splitChildShardId; /* child shardId*/
char slotName[NAMEDATALEN]; /* replication slot name belonging to this node */
} ShardSplitInfo;
/*
* Header of the shared memory segment where shard split information is stored.
*/
typedef struct ShardSplitInfoSMHeader
{
int count; /* number of elements in the shared memory */
ShardSplitInfo splitInfoArray[FLEXIBLE_ARRAY_MEMBER];
} ShardSplitInfoSMHeader;
/*
* Shard split information is populated and stored in shared memory in the form of one dimensional
* array by 'worker_split_shard_replication_setup'. Information belonging to same replication
* slot is grouped together and stored contiguously within this array.
* 'SourceToDestinationShardMap' maps list of child(destination) shards that should be processed by a replication
* slot corresponding to a parent(source) shard. When a parent shard receives a change, the decoder can use this map
* to traverse only the list of child shards corresponding the given parent.
*/
typedef struct SourceToDestinationShardMapEntry
{
Oid sourceShardKey;
List *shardSplitInfoList;
} SourceToDestinationShardMapEntry;
typedef struct ShardSplitShmemData
{
int trancheId;
NamedLWLockTranche namedLockTranche;
LWLock lock;
dsm_handle dsmHandle;
} ShardSplitShmemData;
/* Functions for creating and accessing shared memory used for dsm handle managment */
void InitializeShardSplitSMHandleManagement(void);
void StoreShardSplitSharedMemoryHandle(dsm_handle dsmHandle);
/* Functions for creating and accessing shared memory segments consisting shard split information */
extern ShardSplitInfoSMHeader * CreateSharedMemoryForShardSplitInfo(int
shardSplitInfoCount,
dsm_handle *dsmHandle);
extern void ReleaseSharedMemoryOfShardSplitInfo(void);
extern ShardSplitInfoSMHeader * GetShardSplitInfoSMHeader(void);
extern HTAB * PopulateSourceToDestinationShardMapForSlot(char *slotName, MemoryContext
cxt);
extern char * EncodeReplicationSlot(uint32_t nodeId, uint32_t tableOwnerId);
#endif /* SHARDSPLIT_SHARED_MEMORY_H */

View File

@ -110,6 +110,7 @@ extern List * PgDistTableMetadataSyncCommandList(void);
/* Function declarations for worker node utilities */
extern int CompareWorkerNodes(const void *leftElement, const void *rightElement);
extern uint32 WorkerNodeHashCode(const void *key, Size keySize);
extern int WorkerNodeCompare(const void *lhsKey, const void *rhsKey, Size keySize);
extern int NodeNamePortCompare(const char *workerLhsName, const char *workerRhsName,
int workerLhsPort, int workerRhsPort);

View File

@ -7,3 +7,6 @@ test: isolation_cluster_management
test: isolation_logical_replication_single_shard_commands
test: isolation_logical_replication_multi_shard_commands
test: isolation_non_blocking_shard_split
test: isolation_non_blocking_shard_split_with_index_as_replicaIdentity
test: isolation_non_blocking_shard_split_fkey

View File

@ -0,0 +1,9 @@
# Split Shard tests.
# Include tests from 'minimal_schedule' for setup.
test: multi_test_helpers multi_test_helpers_superuser columnar_test_helpers
test: multi_cluster_management
test: multi_test_catalog_views
test: tablespace
# Split tests go here.
test: citus_split_shard_by_split_points_negative
test: citus_split_shard_by_split_points

View File

@ -0,0 +1,124 @@
/*
Citus Shard Split Test.The test is model similar to 'shard_move_constraints'.
Here is a high level overview of test plan:
1. Create a table 'sensors' (ShardCount = 2) to be split. Add indexes and statistics on this table.
2. Create two other tables: 'reference_table' and 'colocated_dist_table', co-located with sensors.
3. Create Foreign key constraints between the two co-located distributed tables.
4. Load data into the three tables.
5. Move one of the shards for 'sensors' to test ShardMove -> Split.
6. Trigger Split on both shards of 'sensors'. This will also split co-located tables.
7. Move one of the split shard to test Split -> ShardMove.
8. Split an already split shard second time on a different schema.
*/
CREATE SCHEMA "citus_split_test_schema";
CREATE ROLE test_split_role WITH LOGIN;
GRANT USAGE, CREATE ON SCHEMA "citus_split_test_schema" TO test_split_role;
SET ROLE test_split_role;
SET search_path TO "citus_split_test_schema";
SET citus.next_shard_id TO 8981000;
SET citus.next_placement_id TO 8610000;
SET citus.shard_count TO 2;
SET citus.shard_replication_factor TO 1;
-- BEGIN: Create table to split, along with other co-located tables. Add indexes, statistics etc.
CREATE TABLE sensors(
measureid integer,
eventdatetime date,
measure_data jsonb,
meaure_quantity decimal(15, 2),
measure_status char(1),
measure_comment varchar(44),
PRIMARY KEY (measureid, eventdatetime, measure_data));
SELECT create_distributed_table('sensors', 'measureid', colocate_with:='none');
create_distributed_table
---------------------------------------------------------------------
(1 row)
INSERT INTO sensors SELECT i, '2020-01-05', '{}', 11011.10, 'A', 'I <3 Citus' FROM generate_series(0,1000)i;
-- END: Create table to split, along with other co-located tables. Add indexes, statistics etc.
-- BEGIN : Move one shard before we split it.
\c - postgres - :master_port
SET ROLE test_split_role;
SET search_path TO "citus_split_test_schema";
SET citus.next_shard_id TO 8981007;
SET citus.defer_drop_after_shard_move TO OFF;
SELECT citus_move_shard_placement(8981000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, shard_transfer_mode:='force_logical');
citus_move_shard_placement
---------------------------------------------------------------------
(1 row)
-- END : Move one shard before we split it.
-- BEGIN : Set node id variables
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
-- END : Set node id variables
-- BEGIN : Split two shards : One with move and One without move.
-- Perform 2 way split
SELECT * FROM citus_shards WHERE nodeport IN (:worker_1_port, :worker_2_port);
table_name | shardid | shard_name | citus_table_type | colocation_id | nodename | nodeport | shard_size
---------------------------------------------------------------------
sensors | 8981000 | citus_split_test_schema.sensors_8981000 | distributed | 1390009 | localhost | 57638 | 40960
sensors | 8981001 | citus_split_test_schema.sensors_8981001 | distributed | 1390009 | localhost | 57638 | 40960
(2 rows)
SELECT pg_catalog.citus_split_shard_by_split_points(
8981000,
ARRAY['-1073741824'],
ARRAY[:worker_2_node, :worker_2_node],
'force_logical');
WARNING: replication slot "citus_shard_split_template_slot_8981000" does not exist
CONTEXT: while executing command on localhost:xxxxx
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
SELECT * FROM citus_shards WHERE nodeport IN (:worker_1_port, :worker_2_port);
table_name | shardid | shard_name | citus_table_type | colocation_id | nodename | nodeport | shard_size
---------------------------------------------------------------------
sensors | 8981001 | citus_split_test_schema.sensors_8981001 | distributed | 1390009 | localhost | 57638 | 40960
sensors | 8981007 | citus_split_test_schema.sensors_8981007 | distributed | 1390009 | localhost | 57638 | 24576
sensors | 8981008 | citus_split_test_schema.sensors_8981008 | distributed | 1390009 | localhost | 57638 | 24576
(3 rows)
\c - - - :worker_2_port
SELECT slot_name FROM pg_replication_slots;
slot_name
---------------------------------------------------------------------
citus_shard_split_template_slot_8981000
citus_shard_split_18_20648
(2 rows)
\c - - - :master_port
SELECT pg_catalog.citus_split_shard_by_split_points(
8981001,
ARRAY['536870911', '1610612735'],
ARRAY[:worker_1_node, :worker_1_node, :worker_2_node],
'force_logical');
WARNING: replication slot "citus_shard_split_template_slot_8981001" does not exist
CONTEXT: while executing command on localhost:xxxxx
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
SELECT * FROM citus_shards WHERE nodeport IN (:worker_1_port, :worker_2_port);
table_name | shardid | shard_name | citus_table_type | colocation_id | nodename | nodeport | shard_size
---------------------------------------------------------------------
citus_split_test_schema.sensors | 102042 | citus_split_test_schema.sensors_102042 | distributed | 1390009 | localhost | 57637 | 8192
citus_split_test_schema.sensors | 102043 | citus_split_test_schema.sensors_102043 | distributed | 1390009 | localhost | 57637 | 16384
citus_split_test_schema.sensors | 102044 | citus_split_test_schema.sensors_102044 | distributed | 1390009 | localhost | 57638 | 16384
citus_split_test_schema.sensors | 8981007 | citus_split_test_schema.sensors_8981007 | distributed | 1390009 | localhost | 57638 | 24576
citus_split_test_schema.sensors | 8981008 | citus_split_test_schema.sensors_8981008 | distributed | 1390009 | localhost | 57638 | 24576
(5 rows)
\c - - - :worker_2_port
SELECT slot_name FROM pg_replication_slots;
slot_name
---------------------------------------------------------------------
citus_shard_split_template_slot_8981001
citus_shard_split_16_20648
citus_shard_split_18_20648
(3 rows)

View File

@ -0,0 +1,812 @@
CREATE SCHEMA "citus_split_test_schema_columnar_partitioned";
SET search_path TO "citus_split_test_schema_columnar_partitioned";
SET citus.next_shard_id TO 8970000;
SET citus.next_placement_id TO 8770000;
SET citus.shard_count TO 1;
SET citus.shard_replication_factor TO 1;
-- BEGIN: Create table to split, along with other co-located tables. Add indexes, statistics etc.
CREATE TABLE sensors(
measureid integer,
eventdatetime date,
measure_data jsonb,
PRIMARY KEY (measureid, eventdatetime, measure_data))
PARTITION BY RANGE(eventdatetime);
-- Table access method is specified on child tables
CREATE TABLE sensorscolumnar(
measureid integer,
eventdatetime date,
measure_data jsonb,
PRIMARY KEY (measureid, eventdatetime, measure_data))
PARTITION BY RANGE(eventdatetime);
-- Create Partitions of table 'sensors'.
CREATE TABLE sensors_old PARTITION OF sensors FOR VALUES FROM ('2000-01-01') TO ('2020-01-01');
CREATE TABLE sensors_2020_01_01 PARTITION OF sensors FOR VALUES FROM ('2020-01-01') TO ('2020-02-01');
CREATE TABLE sensors_news PARTITION OF sensors FOR VALUES FROM ('2020-05-01') TO ('2025-01-01');
CREATE TABLE sensorscolumnar_old PARTITION OF sensorscolumnar FOR VALUES FROM ('2000-01-01') TO ('2020-01-01') USING COLUMNAR;
CREATE TABLE sensorscolumnar_2020_01_01 PARTITION OF sensorscolumnar FOR VALUES FROM ('2020-01-01') TO ('2020-02-01') USING COLUMNAR;
CREATE TABLE sensorscolumnar_news PARTITION OF sensorscolumnar FOR VALUES FROM ('2020-05-01') TO ('2025-01-01') USING COLUMNAR;
-- Create index on parent and child partitions.
CREATE INDEX index_on_parent ON sensors(lower(measureid::text));
CREATE INDEX index_on_child ON sensors_2020_01_01(lower(measure_data::text));
CREATE INDEX index_on_parent_columnar ON sensorscolumnar(lower(measureid::text));
CREATE INDEX index_on_child_columnar ON sensorscolumnar_2020_01_01(lower(measure_data::text));
ALTER INDEX index_on_parent ALTER COLUMN 1 SET STATISTICS 1000;
ALTER INDEX index_on_child ALTER COLUMN 1 SET STATISTICS 1000;
ALTER INDEX index_on_parent_columnar ALTER COLUMN 1 SET STATISTICS 1000;
ALTER INDEX index_on_child_columnar ALTER COLUMN 1 SET STATISTICS 1000;
-- Create statistics on parent and child partitions.
CREATE STATISTICS s1 (dependencies) ON measureid, eventdatetime FROM sensors;
CREATE STATISTICS s2 (dependencies) ON measureid, eventdatetime FROM sensors_2020_01_01;
CREATE STATISTICS s1_c (dependencies) ON measureid, eventdatetime FROM sensorscolumnar;
CREATE STATISTICS s2_c (dependencies) ON measureid, eventdatetime FROM sensorscolumnar_2020_01_01;
CLUSTER sensors_2020_01_01 USING index_on_child;
SELECT create_distributed_table('sensors', 'measureid');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table('sensorscolumnar', 'measureid');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- create colocated distributed tables
CREATE TABLE colocated_dist_table (measureid integer PRIMARY KEY);
SELECT create_distributed_table('colocated_dist_table', 'measureid');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CLUSTER colocated_dist_table USING colocated_dist_table_pkey;
CREATE TABLE colocated_partitioned_table(
measureid integer,
eventdatetime date,
PRIMARY KEY (measureid, eventdatetime))
PARTITION BY RANGE(eventdatetime);
CREATE TABLE colocated_partitioned_table_2020_01_01 PARTITION OF colocated_partitioned_table FOR VALUES FROM ('2020-01-01') TO ('2020-02-01');
SELECT create_distributed_table('colocated_partitioned_table', 'measureid');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CLUSTER colocated_partitioned_table_2020_01_01 USING colocated_partitioned_table_2020_01_01_pkey;
-- create reference tables
CREATE TABLE reference_table (measureid integer PRIMARY KEY);
SELECT create_reference_table('reference_table');
create_reference_table
---------------------------------------------------------------------
(1 row)
SELECT shard.shardid, logicalrelid, shardminvalue, shardmaxvalue, nodename, nodeport
FROM pg_dist_shard AS shard
INNER JOIN pg_dist_placement placement ON shard.shardid = placement.shardid
INNER JOIN pg_dist_node node ON placement.groupid = node.groupid
INNER JOIN pg_catalog.pg_class cls ON shard.logicalrelid = cls.oid
INNER JOIN pg_catalog.pg_namespace ns ON cls.relnamespace = ns.oid
WHERE node.noderole = 'primary' AND ns.nspname = 'citus_split_test_schema_columnar_partitioned'
ORDER BY logicalrelid, shardminvalue::BIGINT, nodeport;
shardid | logicalrelid | shardminvalue | shardmaxvalue | nodename | nodeport
---------------------------------------------------------------------
8970000 | sensors | -2147483648 | 2147483647 | localhost | 57637
8970004 | sensorscolumnar | -2147483648 | 2147483647 | localhost | 57637
8970001 | sensors_old | -2147483648 | 2147483647 | localhost | 57637
8970002 | sensors_2020_01_01 | -2147483648 | 2147483647 | localhost | 57637
8970003 | sensors_news | -2147483648 | 2147483647 | localhost | 57637
8970005 | sensorscolumnar_old | -2147483648 | 2147483647 | localhost | 57637
8970006 | sensorscolumnar_2020_01_01 | -2147483648 | 2147483647 | localhost | 57637
8970007 | sensorscolumnar_news | -2147483648 | 2147483647 | localhost | 57637
8970008 | colocated_dist_table | -2147483648 | 2147483647 | localhost | 57637
8970009 | colocated_partitioned_table | -2147483648 | 2147483647 | localhost | 57637
8970010 | colocated_partitioned_table_2020_01_01 | -2147483648 | 2147483647 | localhost | 57637
8970011 | reference_table | | | localhost | 57637
8970011 | reference_table | | | localhost | 57638
(13 rows)
-- END: Create table to split, along with other co-located tables. Add indexes, statistics etc.
-- BEGIN: Create constraints for tables.
-- from parent to regular dist
ALTER TABLE sensors ADD CONSTRAINT fkey_from_parent_to_dist FOREIGN KEY (measureid) REFERENCES colocated_dist_table(measureid);
-- from parent to parent
ALTER TABLE sensors ADD CONSTRAINT fkey_from_parent_to_parent FOREIGN KEY (measureid, eventdatetime) REFERENCES colocated_partitioned_table(measureid, eventdatetime);
-- from parent to child
ALTER TABLE sensors ADD CONSTRAINT fkey_from_parent_to_child FOREIGN KEY (measureid, eventdatetime) REFERENCES colocated_partitioned_table_2020_01_01(measureid, eventdatetime);
ALTER TABLE sensors ADD CONSTRAINT fkey_from_parent_to_ref FOREIGN KEY (measureid) REFERENCES reference_table(measureid);
-- from child to regular dist
ALTER TABLE sensors_2020_01_01 ADD CONSTRAINT fkey_from_child_to_dist FOREIGN KEY (measureid) REFERENCES colocated_dist_table(measureid);
-- from child to parent
ALTER TABLE sensors_2020_01_01 ADD CONSTRAINT fkey_from_child_to_parent FOREIGN KEY (measureid,eventdatetime) REFERENCES colocated_partitioned_table(measureid,eventdatetime);
-- from child to child
ALTER TABLE sensors_2020_01_01 ADD CONSTRAINT fkey_from_child_to_child FOREIGN KEY (measureid,eventdatetime) REFERENCES colocated_partitioned_table_2020_01_01(measureid,eventdatetime);
ALTER TABLE sensors_2020_01_01 ADD CONSTRAINT fkey_from_child_to_ref FOREIGN KEY (measureid) REFERENCES reference_table(measureid);
-- No support for foreign keys, unique constraints, or exclusion constraints in columnar tables.
-- Please see: https://github.com/citusdata/citus/tree/main/src/backend/columnar/README.md
-- END: Create constraints for tables.
-- BEGIN: Load data into tables
INSERT INTO reference_table SELECT i FROM generate_series(0,1000)i;
INSERT INTO colocated_dist_table SELECT i FROM generate_series(0,1000)i;
INSERT INTO colocated_partitioned_table SELECT i, '2020-01-05' FROM generate_series(0,1000)i;
INSERT INTO sensors SELECT i, '2020-01-05', '{}' FROM generate_series(0,1000)i;
INSERT INTO sensorscolumnar SELECT i, '2020-01-05', '{}' FROM generate_series(0,1000)i;
-- END: Load data into tables
-- BEGIN: Show the current state on workers
\c - - - :worker_1_port
SET search_path TO "citus_split_test_schema_columnar_partitioned";
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like '%_89%'
ORDER BY 1, 2;
relname | Constraint | Definition
---------------------------------------------------------------------
sensors_2020_01_01_8970002 | fkey_from_child_to_child_8970002 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8970010(eventdatetime, measureid)
sensors_2020_01_01_8970002 | fkey_from_child_to_dist_8970002 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8970008(measureid)
sensors_2020_01_01_8970002 | fkey_from_child_to_parent_8970002 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8970009(eventdatetime, measureid)
sensors_2020_01_01_8970002 | fkey_from_child_to_ref_8970002 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
sensors_2020_01_01_8970002 | fkey_from_parent_to_child_8970000 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8970010(eventdatetime, measureid)
sensors_2020_01_01_8970002 | fkey_from_parent_to_dist_8970000 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8970008(measureid)
sensors_2020_01_01_8970002 | fkey_from_parent_to_parent_8970000 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8970009(eventdatetime, measureid)
sensors_2020_01_01_8970002 | fkey_from_parent_to_ref_8970000 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
sensors_2020_01_01_8970002 | sensors_2020_01_01_8970002_measureid_eventdatetime_fkey | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8970010(eventdatetime, measureid)
sensors_8970000 | fkey_from_parent_to_child_8970000 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8970010(eventdatetime, measureid)
sensors_8970000 | fkey_from_parent_to_dist_8970000 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8970008(measureid)
sensors_8970000 | fkey_from_parent_to_parent_8970000 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8970009(eventdatetime, measureid)
sensors_8970000 | fkey_from_parent_to_ref_8970000 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
sensors_8970000 | sensors_8970000_measureid_eventdatetime_fkey | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8970010(eventdatetime, measureid)
sensors_news_8970003 | fkey_from_parent_to_child_8970000 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8970010(eventdatetime, measureid)
sensors_news_8970003 | fkey_from_parent_to_dist_8970000 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8970008(measureid)
sensors_news_8970003 | fkey_from_parent_to_parent_8970000 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8970009(eventdatetime, measureid)
sensors_news_8970003 | fkey_from_parent_to_ref_8970000 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
sensors_old_8970001 | fkey_from_parent_to_child_8970000 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8970010(eventdatetime, measureid)
sensors_old_8970001 | fkey_from_parent_to_dist_8970000 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8970008(measureid)
sensors_old_8970001 | fkey_from_parent_to_parent_8970000 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8970009(eventdatetime, measureid)
sensors_old_8970001 | fkey_from_parent_to_ref_8970000 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
(22 rows)
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like '%_89%' ORDER BY 1,2;
tablename | indexdef
---------------------------------------------------------------------
colocated_dist_table_8970008 | CREATE UNIQUE INDEX colocated_dist_table_pkey_8970008 ON citus_split_test_schema_columnar_partitioned.colocated_dist_table_8970008 USING btree (measureid)
colocated_partitioned_table_2020_01_01_8970010 | CREATE UNIQUE INDEX colocated_partitioned_table_2020_01_01_pkey_8970010 ON citus_split_test_schema_columnar_partitioned.colocated_partitioned_table_2020_01_01_8970010 USING btree (measureid, eventdatetime)
colocated_partitioned_table_8970009 | CREATE UNIQUE INDEX colocated_partitioned_table_pkey_8970009 ON ONLY citus_split_test_schema_columnar_partitioned.colocated_partitioned_table_8970009 USING btree (measureid, eventdatetime)
reference_table_8970011 | CREATE UNIQUE INDEX reference_table_pkey_8970011 ON citus_split_test_schema_columnar_partitioned.reference_table_8970011 USING btree (measureid)
sensors_2020_01_01_8970002 | CREATE INDEX index_on_child_8970002 ON citus_split_test_schema_columnar_partitioned.sensors_2020_01_01_8970002 USING btree (lower((measure_data)::text))
sensors_2020_01_01_8970002 | CREATE INDEX sensors_2020_01_01_lower_idx_8970002 ON citus_split_test_schema_columnar_partitioned.sensors_2020_01_01_8970002 USING btree (lower((measureid)::text))
sensors_2020_01_01_8970002 | CREATE UNIQUE INDEX sensors_2020_01_01_pkey_8970002 ON citus_split_test_schema_columnar_partitioned.sensors_2020_01_01_8970002 USING btree (measureid, eventdatetime, measure_data)
sensors_8970000 | CREATE INDEX index_on_parent_8970000 ON ONLY citus_split_test_schema_columnar_partitioned.sensors_8970000 USING btree (lower((measureid)::text))
sensors_8970000 | CREATE UNIQUE INDEX sensors_pkey_8970000 ON ONLY citus_split_test_schema_columnar_partitioned.sensors_8970000 USING btree (measureid, eventdatetime, measure_data)
sensors_news_8970003 | CREATE INDEX sensors_news_lower_idx_8970003 ON citus_split_test_schema_columnar_partitioned.sensors_news_8970003 USING btree (lower((measureid)::text))
sensors_news_8970003 | CREATE UNIQUE INDEX sensors_news_pkey_8970003 ON citus_split_test_schema_columnar_partitioned.sensors_news_8970003 USING btree (measureid, eventdatetime, measure_data)
sensors_old_8970001 | CREATE INDEX sensors_old_lower_idx_8970001 ON citus_split_test_schema_columnar_partitioned.sensors_old_8970001 USING btree (lower((measureid)::text))
sensors_old_8970001 | CREATE UNIQUE INDEX sensors_old_pkey_8970001 ON citus_split_test_schema_columnar_partitioned.sensors_old_8970001 USING btree (measureid, eventdatetime, measure_data)
sensorscolumnar_2020_01_01_8970006 | CREATE INDEX index_on_child_columnar_8970006 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_2020_01_01_8970006 USING btree (lower((measure_data)::text))
sensorscolumnar_2020_01_01_8970006 | CREATE INDEX sensorscolumnar_2020_01_01_lower_idx_8970006 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_2020_01_01_8970006 USING btree (lower((measureid)::text))
sensorscolumnar_2020_01_01_8970006 | CREATE UNIQUE INDEX sensorscolumnar_2020_01_01_pkey_8970006 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_2020_01_01_8970006 USING btree (measureid, eventdatetime, measure_data)
sensorscolumnar_8970004 | CREATE INDEX index_on_parent_columnar_8970004 ON ONLY citus_split_test_schema_columnar_partitioned.sensorscolumnar_8970004 USING btree (lower((measureid)::text))
sensorscolumnar_8970004 | CREATE UNIQUE INDEX sensorscolumnar_pkey_8970004 ON ONLY citus_split_test_schema_columnar_partitioned.sensorscolumnar_8970004 USING btree (measureid, eventdatetime, measure_data)
sensorscolumnar_news_8970007 | CREATE INDEX sensorscolumnar_news_lower_idx_8970007 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_news_8970007 USING btree (lower((measureid)::text))
sensorscolumnar_news_8970007 | CREATE UNIQUE INDEX sensorscolumnar_news_pkey_8970007 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_news_8970007 USING btree (measureid, eventdatetime, measure_data)
sensorscolumnar_old_8970005 | CREATE INDEX sensorscolumnar_old_lower_idx_8970005 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_old_8970005 USING btree (lower((measureid)::text))
sensorscolumnar_old_8970005 | CREATE UNIQUE INDEX sensorscolumnar_old_pkey_8970005 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_old_8970005 USING btree (measureid, eventdatetime, measure_data)
(22 rows)
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema_columnar_partitioned')
)
ORDER BY stxname ASC;
stxname
---------------------------------------------------------------------
s1
s1_8970000
s1_c
s1_c_8970004
s2
s2_8970002
s2_c
s2_c_8970006
(8 rows)
\c - - - :worker_2_port
SET search_path TO "citus_split_test_schema_columnar_partitioned";
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like '%_89%'
ORDER BY 1, 2;
relname | Constraint | Definition
---------------------------------------------------------------------
(0 rows)
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like '%_89%' ORDER BY 1,2;
tablename | indexdef
---------------------------------------------------------------------
reference_table_8970011 | CREATE UNIQUE INDEX reference_table_pkey_8970011 ON citus_split_test_schema_columnar_partitioned.reference_table_8970011 USING btree (measureid)
(1 row)
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema_columnar_partitioned')
)
ORDER BY stxname ASC;
stxname
---------------------------------------------------------------------
s1
s1_c
s2
s2_c
(4 rows)
-- END: Show the current state on workers
-- BEGIN: Split a shard along its co-located shards
\c - - - :master_port
SET search_path TO "citus_split_test_schema_columnar_partitioned";
SET citus.next_shard_id TO 8999000;
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
SELECT pg_catalog.citus_split_shard_by_split_points(
8970000,
ARRAY['-2120000000'],
ARRAY[:worker_1_node, :worker_2_node],
'force_logical');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
-- END: Split a shard along its co-located shards
-- BEGIN: Validate Shard Info and Data
SELECT shard.shardid, logicalrelid, shardminvalue, shardmaxvalue, nodename, nodeport
FROM pg_dist_shard AS shard
INNER JOIN pg_dist_placement placement ON shard.shardid = placement.shardid
INNER JOIN pg_dist_node node ON placement.groupid = node.groupid
INNER JOIN pg_catalog.pg_class cls ON shard.logicalrelid = cls.oid
INNER JOIN pg_catalog.pg_namespace ns ON cls.relnamespace = ns.oid
WHERE node.noderole = 'primary' AND ns.nspname = 'citus_split_test_schema_columnar_partitioned'
ORDER BY logicalrelid, shardminvalue::BIGINT, nodeport;
shardid | logicalrelid | shardminvalue | shardmaxvalue | nodename | nodeport
---------------------------------------------------------------------
8999000 | sensors | -2147483648 | -2120000000 | localhost | 57637
8999001 | sensors | -2119999999 | 2147483647 | localhost | 57638
8999008 | sensorscolumnar | -2147483648 | -2120000000 | localhost | 57637
8999009 | sensorscolumnar | -2119999999 | 2147483647 | localhost | 57638
8999002 | sensors_old | -2147483648 | -2120000000 | localhost | 57637
8999003 | sensors_old | -2119999999 | 2147483647 | localhost | 57638
8999004 | sensors_2020_01_01 | -2147483648 | -2120000000 | localhost | 57637
8999005 | sensors_2020_01_01 | -2119999999 | 2147483647 | localhost | 57638
8999006 | sensors_news | -2147483648 | -2120000000 | localhost | 57637
8999007 | sensors_news | -2119999999 | 2147483647 | localhost | 57638
8999010 | sensorscolumnar_old | -2147483648 | -2120000000 | localhost | 57637
8999011 | sensorscolumnar_old | -2119999999 | 2147483647 | localhost | 57638
8999012 | sensorscolumnar_2020_01_01 | -2147483648 | -2120000000 | localhost | 57637
8999013 | sensorscolumnar_2020_01_01 | -2119999999 | 2147483647 | localhost | 57638
8999014 | sensorscolumnar_news | -2147483648 | -2120000000 | localhost | 57637
8999015 | sensorscolumnar_news | -2119999999 | 2147483647 | localhost | 57638
8999016 | colocated_dist_table | -2147483648 | -2120000000 | localhost | 57637
8999017 | colocated_dist_table | -2119999999 | 2147483647 | localhost | 57638
8999018 | colocated_partitioned_table | -2147483648 | -2120000000 | localhost | 57637
8999019 | colocated_partitioned_table | -2119999999 | 2147483647 | localhost | 57638
8999020 | colocated_partitioned_table_2020_01_01 | -2147483648 | -2120000000 | localhost | 57637
8999021 | colocated_partitioned_table_2020_01_01 | -2119999999 | 2147483647 | localhost | 57638
8970011 | reference_table | | | localhost | 57637
8970011 | reference_table | | | localhost | 57638
(24 rows)
SELECT count(*) FROM reference_table;
count
---------------------------------------------------------------------
1001
(1 row)
SELECT count(*) FROM colocated_partitioned_table;
count
---------------------------------------------------------------------
1001
(1 row)
SELECT count(*) FROM colocated_dist_table;
count
---------------------------------------------------------------------
1001
(1 row)
SELECT count(*) FROM sensors;
count
---------------------------------------------------------------------
1001
(1 row)
SELECT count(*) FROM sensorscolumnar;
count
---------------------------------------------------------------------
1001
(1 row)
-- END: Validate Shard Info and Data
-- BEGIN: Show the updated state on workers
\c - - - :worker_1_port
SET search_path TO "citus_split_test_schema_columnar_partitioned";
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like '%_89%'
ORDER BY 1, 2;
relname | Constraint | Definition
---------------------------------------------------------------------
sensors_2020_01_01_8999004 | fkey_from_child_to_child_8999004 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999020(eventdatetime, measureid)
sensors_2020_01_01_8999004 | fkey_from_child_to_dist_8999004 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8999016(measureid)
sensors_2020_01_01_8999004 | fkey_from_child_to_parent_8999004 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8999018(eventdatetime, measureid)
sensors_2020_01_01_8999004 | fkey_from_child_to_ref_8999004 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
sensors_2020_01_01_8999004 | fkey_from_parent_to_child_8999000 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999020(eventdatetime, measureid)
sensors_2020_01_01_8999004 | fkey_from_parent_to_dist_8999000 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8999016(measureid)
sensors_2020_01_01_8999004 | fkey_from_parent_to_parent_8999000 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8999018(eventdatetime, measureid)
sensors_2020_01_01_8999004 | fkey_from_parent_to_ref_8999000 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
sensors_2020_01_01_8999004 | sensors_2020_01_01_8999004_measureid_eventdatetime_fkey | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999020(eventdatetime, measureid)
sensors_8999000 | fkey_from_parent_to_child_8999000 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999020(eventdatetime, measureid)
sensors_8999000 | fkey_from_parent_to_dist_8999000 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8999016(measureid)
sensors_8999000 | fkey_from_parent_to_parent_8999000 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8999018(eventdatetime, measureid)
sensors_8999000 | fkey_from_parent_to_ref_8999000 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
sensors_8999000 | sensors_8999000_measureid_eventdatetime_fkey | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999020(eventdatetime, measureid)
sensors_news_8999006 | fkey_from_parent_to_child_8999000 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999020(eventdatetime, measureid)
sensors_news_8999006 | fkey_from_parent_to_dist_8999000 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8999016(measureid)
sensors_news_8999006 | fkey_from_parent_to_parent_8999000 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8999018(eventdatetime, measureid)
sensors_news_8999006 | fkey_from_parent_to_ref_8999000 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
sensors_old_8999002 | fkey_from_parent_to_child_8999000 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999020(eventdatetime, measureid)
sensors_old_8999002 | fkey_from_parent_to_dist_8999000 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8999016(measureid)
sensors_old_8999002 | fkey_from_parent_to_parent_8999000 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8999018(eventdatetime, measureid)
sensors_old_8999002 | fkey_from_parent_to_ref_8999000 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
(22 rows)
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like '%_89%' ORDER BY 1,2;
tablename | indexdef
---------------------------------------------------------------------
colocated_dist_table_8999016 | CREATE UNIQUE INDEX colocated_dist_table_pkey_8999016 ON citus_split_test_schema_columnar_partitioned.colocated_dist_table_8999016 USING btree (measureid)
colocated_partitioned_table_2020_01_01_8999020 | CREATE UNIQUE INDEX colocated_partitioned_table_2020_01_01_pkey_8999020 ON citus_split_test_schema_columnar_partitioned.colocated_partitioned_table_2020_01_01_8999020 USING btree (measureid, eventdatetime)
colocated_partitioned_table_8999018 | CREATE UNIQUE INDEX colocated_partitioned_table_pkey_8999018 ON ONLY citus_split_test_schema_columnar_partitioned.colocated_partitioned_table_8999018 USING btree (measureid, eventdatetime)
reference_table_8970011 | CREATE UNIQUE INDEX reference_table_pkey_8970011 ON citus_split_test_schema_columnar_partitioned.reference_table_8970011 USING btree (measureid)
sensors_2020_01_01_8999004 | CREATE INDEX index_on_child_8999004 ON citus_split_test_schema_columnar_partitioned.sensors_2020_01_01_8999004 USING btree (lower((measure_data)::text))
sensors_2020_01_01_8999004 | CREATE INDEX sensors_2020_01_01_lower_idx_8999004 ON citus_split_test_schema_columnar_partitioned.sensors_2020_01_01_8999004 USING btree (lower((measureid)::text))
sensors_2020_01_01_8999004 | CREATE UNIQUE INDEX sensors_2020_01_01_pkey_8999004 ON citus_split_test_schema_columnar_partitioned.sensors_2020_01_01_8999004 USING btree (measureid, eventdatetime, measure_data)
sensors_8999000 | CREATE INDEX index_on_parent_8999000 ON ONLY citus_split_test_schema_columnar_partitioned.sensors_8999000 USING btree (lower((measureid)::text))
sensors_8999000 | CREATE UNIQUE INDEX sensors_pkey_8999000 ON ONLY citus_split_test_schema_columnar_partitioned.sensors_8999000 USING btree (measureid, eventdatetime, measure_data)
sensors_news_8999006 | CREATE INDEX sensors_news_lower_idx_8999006 ON citus_split_test_schema_columnar_partitioned.sensors_news_8999006 USING btree (lower((measureid)::text))
sensors_news_8999006 | CREATE UNIQUE INDEX sensors_news_pkey_8999006 ON citus_split_test_schema_columnar_partitioned.sensors_news_8999006 USING btree (measureid, eventdatetime, measure_data)
sensors_old_8999002 | CREATE INDEX sensors_old_lower_idx_8999002 ON citus_split_test_schema_columnar_partitioned.sensors_old_8999002 USING btree (lower((measureid)::text))
sensors_old_8999002 | CREATE UNIQUE INDEX sensors_old_pkey_8999002 ON citus_split_test_schema_columnar_partitioned.sensors_old_8999002 USING btree (measureid, eventdatetime, measure_data)
sensorscolumnar_2020_01_01_8999012 | CREATE INDEX index_on_child_columnar_8999012 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_2020_01_01_8999012 USING btree (lower((measure_data)::text))
sensorscolumnar_2020_01_01_8999012 | CREATE INDEX sensorscolumnar_2020_01_01_lower_idx_8999012 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_2020_01_01_8999012 USING btree (lower((measureid)::text))
sensorscolumnar_2020_01_01_8999012 | CREATE UNIQUE INDEX sensorscolumnar_2020_01_01_pkey_8999012 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_2020_01_01_8999012 USING btree (measureid, eventdatetime, measure_data)
sensorscolumnar_8999008 | CREATE INDEX index_on_parent_columnar_8999008 ON ONLY citus_split_test_schema_columnar_partitioned.sensorscolumnar_8999008 USING btree (lower((measureid)::text))
sensorscolumnar_8999008 | CREATE UNIQUE INDEX sensorscolumnar_pkey_8999008 ON ONLY citus_split_test_schema_columnar_partitioned.sensorscolumnar_8999008 USING btree (measureid, eventdatetime, measure_data)
sensorscolumnar_news_8999014 | CREATE INDEX sensorscolumnar_news_lower_idx_8999014 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_news_8999014 USING btree (lower((measureid)::text))
sensorscolumnar_news_8999014 | CREATE UNIQUE INDEX sensorscolumnar_news_pkey_8999014 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_news_8999014 USING btree (measureid, eventdatetime, measure_data)
sensorscolumnar_old_8999010 | CREATE INDEX sensorscolumnar_old_lower_idx_8999010 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_old_8999010 USING btree (lower((measureid)::text))
sensorscolumnar_old_8999010 | CREATE UNIQUE INDEX sensorscolumnar_old_pkey_8999010 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_old_8999010 USING btree (measureid, eventdatetime, measure_data)
(22 rows)
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema_columnar_partitioned')
)
ORDER BY stxname ASC;
stxname
---------------------------------------------------------------------
s1
s1_8999000
s1_c
s1_c_8999008
s2
s2_8999004
s2_c
s2_c_8999012
(8 rows)
\c - - - :worker_2_port
SET search_path TO "citus_split_test_schema_columnar_partitioned";
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like '%_89%'
ORDER BY 1, 2;
relname | Constraint | Definition
---------------------------------------------------------------------
sensors_2020_01_01_8999005 | fkey_from_child_to_child_8999005 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999021(eventdatetime, measureid)
sensors_2020_01_01_8999005 | fkey_from_child_to_dist_8999005 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8999017(measureid)
sensors_2020_01_01_8999005 | fkey_from_child_to_parent_8999005 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8999019(eventdatetime, measureid)
sensors_2020_01_01_8999005 | fkey_from_child_to_ref_8999005 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
sensors_2020_01_01_8999005 | fkey_from_parent_to_child_8999001 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999021(eventdatetime, measureid)
sensors_2020_01_01_8999005 | fkey_from_parent_to_dist_8999001 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8999017(measureid)
sensors_2020_01_01_8999005 | fkey_from_parent_to_parent_8999001 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8999019(eventdatetime, measureid)
sensors_2020_01_01_8999005 | fkey_from_parent_to_ref_8999001 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
sensors_2020_01_01_8999005 | sensors_2020_01_01_8999005_measureid_eventdatetime_fkey | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999021(eventdatetime, measureid)
sensors_8999001 | fkey_from_parent_to_child_8999001 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999021(eventdatetime, measureid)
sensors_8999001 | fkey_from_parent_to_dist_8999001 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8999017(measureid)
sensors_8999001 | fkey_from_parent_to_parent_8999001 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8999019(eventdatetime, measureid)
sensors_8999001 | fkey_from_parent_to_ref_8999001 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
sensors_8999001 | sensors_8999001_measureid_eventdatetime_fkey | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999021(eventdatetime, measureid)
sensors_news_8999007 | fkey_from_parent_to_child_8999001 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999021(eventdatetime, measureid)
sensors_news_8999007 | fkey_from_parent_to_dist_8999001 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8999017(measureid)
sensors_news_8999007 | fkey_from_parent_to_parent_8999001 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8999019(eventdatetime, measureid)
sensors_news_8999007 | fkey_from_parent_to_ref_8999001 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
sensors_old_8999003 | fkey_from_parent_to_child_8999001 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999021(eventdatetime, measureid)
sensors_old_8999003 | fkey_from_parent_to_dist_8999001 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8999017(measureid)
sensors_old_8999003 | fkey_from_parent_to_parent_8999001 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8999019(eventdatetime, measureid)
sensors_old_8999003 | fkey_from_parent_to_ref_8999001 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
(22 rows)
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like '%_89%' ORDER BY 1,2;
tablename | indexdef
---------------------------------------------------------------------
colocated_dist_table_8999017 | CREATE UNIQUE INDEX colocated_dist_table_pkey_8999017 ON citus_split_test_schema_columnar_partitioned.colocated_dist_table_8999017 USING btree (measureid)
colocated_partitioned_table_2020_01_01_8999021 | CREATE UNIQUE INDEX colocated_partitioned_table_2020_01_01_pkey_8999021 ON citus_split_test_schema_columnar_partitioned.colocated_partitioned_table_2020_01_01_8999021 USING btree (measureid, eventdatetime)
colocated_partitioned_table_8999019 | CREATE UNIQUE INDEX colocated_partitioned_table_pkey_8999019 ON ONLY citus_split_test_schema_columnar_partitioned.colocated_partitioned_table_8999019 USING btree (measureid, eventdatetime)
reference_table_8970011 | CREATE UNIQUE INDEX reference_table_pkey_8970011 ON citus_split_test_schema_columnar_partitioned.reference_table_8970011 USING btree (measureid)
sensors_2020_01_01_8999005 | CREATE INDEX index_on_child_8999005 ON citus_split_test_schema_columnar_partitioned.sensors_2020_01_01_8999005 USING btree (lower((measure_data)::text))
sensors_2020_01_01_8999005 | CREATE INDEX sensors_2020_01_01_lower_idx_8999005 ON citus_split_test_schema_columnar_partitioned.sensors_2020_01_01_8999005 USING btree (lower((measureid)::text))
sensors_2020_01_01_8999005 | CREATE UNIQUE INDEX sensors_2020_01_01_pkey_8999005 ON citus_split_test_schema_columnar_partitioned.sensors_2020_01_01_8999005 USING btree (measureid, eventdatetime, measure_data)
sensors_8999001 | CREATE INDEX index_on_parent_8999001 ON ONLY citus_split_test_schema_columnar_partitioned.sensors_8999001 USING btree (lower((measureid)::text))
sensors_8999001 | CREATE UNIQUE INDEX sensors_pkey_8999001 ON ONLY citus_split_test_schema_columnar_partitioned.sensors_8999001 USING btree (measureid, eventdatetime, measure_data)
sensors_news_8999007 | CREATE INDEX sensors_news_lower_idx_8999007 ON citus_split_test_schema_columnar_partitioned.sensors_news_8999007 USING btree (lower((measureid)::text))
sensors_news_8999007 | CREATE UNIQUE INDEX sensors_news_pkey_8999007 ON citus_split_test_schema_columnar_partitioned.sensors_news_8999007 USING btree (measureid, eventdatetime, measure_data)
sensors_old_8999003 | CREATE INDEX sensors_old_lower_idx_8999003 ON citus_split_test_schema_columnar_partitioned.sensors_old_8999003 USING btree (lower((measureid)::text))
sensors_old_8999003 | CREATE UNIQUE INDEX sensors_old_pkey_8999003 ON citus_split_test_schema_columnar_partitioned.sensors_old_8999003 USING btree (measureid, eventdatetime, measure_data)
sensorscolumnar_2020_01_01_8999013 | CREATE INDEX index_on_child_columnar_8999013 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_2020_01_01_8999013 USING btree (lower((measure_data)::text))
sensorscolumnar_2020_01_01_8999013 | CREATE INDEX sensorscolumnar_2020_01_01_lower_idx_8999013 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_2020_01_01_8999013 USING btree (lower((measureid)::text))
sensorscolumnar_2020_01_01_8999013 | CREATE UNIQUE INDEX sensorscolumnar_2020_01_01_pkey_8999013 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_2020_01_01_8999013 USING btree (measureid, eventdatetime, measure_data)
sensorscolumnar_8999009 | CREATE INDEX index_on_parent_columnar_8999009 ON ONLY citus_split_test_schema_columnar_partitioned.sensorscolumnar_8999009 USING btree (lower((measureid)::text))
sensorscolumnar_8999009 | CREATE UNIQUE INDEX sensorscolumnar_pkey_8999009 ON ONLY citus_split_test_schema_columnar_partitioned.sensorscolumnar_8999009 USING btree (measureid, eventdatetime, measure_data)
sensorscolumnar_news_8999015 | CREATE INDEX sensorscolumnar_news_lower_idx_8999015 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_news_8999015 USING btree (lower((measureid)::text))
sensorscolumnar_news_8999015 | CREATE UNIQUE INDEX sensorscolumnar_news_pkey_8999015 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_news_8999015 USING btree (measureid, eventdatetime, measure_data)
sensorscolumnar_old_8999011 | CREATE INDEX sensorscolumnar_old_lower_idx_8999011 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_old_8999011 USING btree (lower((measureid)::text))
sensorscolumnar_old_8999011 | CREATE UNIQUE INDEX sensorscolumnar_old_pkey_8999011 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_old_8999011 USING btree (measureid, eventdatetime, measure_data)
(22 rows)
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema_columnar_partitioned')
)
ORDER BY stxname ASC;
stxname
---------------------------------------------------------------------
s1
s1_8999001
s1_c
s1_c_8999009
s2
s2_8999005
s2_c
s2_c_8999013
(8 rows)
-- END: Show the updated state on workers
-- BEGIN: Split a partition table directly
\c - - - :master_port
SET search_path TO "citus_split_test_schema_columnar_partitioned";
SET citus.next_shard_id TO 8999100;
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
SELECT pg_catalog.citus_split_shard_by_split_points(
8999002, -- sensors_old
ARRAY['-2127770000'],
ARRAY[:worker_1_node, :worker_2_node],
'auto');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
-- END: Split a partition table directly
-- BEGIN: Validate Shard Info and Data
SELECT shard.shardid, logicalrelid, shardminvalue, shardmaxvalue, nodename, nodeport
FROM pg_dist_shard AS shard
INNER JOIN pg_dist_placement placement ON shard.shardid = placement.shardid
INNER JOIN pg_dist_node node ON placement.groupid = node.groupid
INNER JOIN pg_catalog.pg_class cls ON shard.logicalrelid = cls.oid
INNER JOIN pg_catalog.pg_namespace ns ON cls.relnamespace = ns.oid
WHERE node.noderole = 'primary' AND ns.nspname = 'citus_split_test_schema_columnar_partitioned'
ORDER BY logicalrelid, shardminvalue::BIGINT, nodeport;
shardid | logicalrelid | shardminvalue | shardmaxvalue | nodename | nodeport
---------------------------------------------------------------------
8999100 | sensors | -2147483648 | -2127770000 | localhost | 57637
8999101 | sensors | -2127769999 | -2120000000 | localhost | 57638
8999001 | sensors | -2119999999 | 2147483647 | localhost | 57638
8999108 | sensorscolumnar | -2147483648 | -2127770000 | localhost | 57637
8999109 | sensorscolumnar | -2127769999 | -2120000000 | localhost | 57638
8999009 | sensorscolumnar | -2119999999 | 2147483647 | localhost | 57638
8999102 | sensors_old | -2147483648 | -2127770000 | localhost | 57637
8999103 | sensors_old | -2127769999 | -2120000000 | localhost | 57638
8999003 | sensors_old | -2119999999 | 2147483647 | localhost | 57638
8999104 | sensors_2020_01_01 | -2147483648 | -2127770000 | localhost | 57637
8999105 | sensors_2020_01_01 | -2127769999 | -2120000000 | localhost | 57638
8999005 | sensors_2020_01_01 | -2119999999 | 2147483647 | localhost | 57638
8999106 | sensors_news | -2147483648 | -2127770000 | localhost | 57637
8999107 | sensors_news | -2127769999 | -2120000000 | localhost | 57638
8999007 | sensors_news | -2119999999 | 2147483647 | localhost | 57638
8999110 | sensorscolumnar_old | -2147483648 | -2127770000 | localhost | 57637
8999111 | sensorscolumnar_old | -2127769999 | -2120000000 | localhost | 57638
8999011 | sensorscolumnar_old | -2119999999 | 2147483647 | localhost | 57638
8999112 | sensorscolumnar_2020_01_01 | -2147483648 | -2127770000 | localhost | 57637
8999113 | sensorscolumnar_2020_01_01 | -2127769999 | -2120000000 | localhost | 57638
8999013 | sensorscolumnar_2020_01_01 | -2119999999 | 2147483647 | localhost | 57638
8999114 | sensorscolumnar_news | -2147483648 | -2127770000 | localhost | 57637
8999115 | sensorscolumnar_news | -2127769999 | -2120000000 | localhost | 57638
8999015 | sensorscolumnar_news | -2119999999 | 2147483647 | localhost | 57638
8999116 | colocated_dist_table | -2147483648 | -2127770000 | localhost | 57637
8999117 | colocated_dist_table | -2127769999 | -2120000000 | localhost | 57638
8999017 | colocated_dist_table | -2119999999 | 2147483647 | localhost | 57638
8999118 | colocated_partitioned_table | -2147483648 | -2127770000 | localhost | 57637
8999119 | colocated_partitioned_table | -2127769999 | -2120000000 | localhost | 57638
8999019 | colocated_partitioned_table | -2119999999 | 2147483647 | localhost | 57638
8999120 | colocated_partitioned_table_2020_01_01 | -2147483648 | -2127770000 | localhost | 57637
8999121 | colocated_partitioned_table_2020_01_01 | -2127769999 | -2120000000 | localhost | 57638
8999021 | colocated_partitioned_table_2020_01_01 | -2119999999 | 2147483647 | localhost | 57638
8970011 | reference_table | | | localhost | 57637
8970011 | reference_table | | | localhost | 57638
(35 rows)
SELECT count(*) FROM reference_table;
count
---------------------------------------------------------------------
1001
(1 row)
SELECT count(*) FROM colocated_partitioned_table;
count
---------------------------------------------------------------------
1001
(1 row)
SELECT count(*) FROM colocated_dist_table;
count
---------------------------------------------------------------------
1001
(1 row)
SELECT count(*) FROM sensors;
count
---------------------------------------------------------------------
1001
(1 row)
SELECT count(*) FROM sensorscolumnar;
count
---------------------------------------------------------------------
1001
(1 row)
-- END: Validate Shard Info and Data
-- BEGIN: Show the updated state on workers
\c - - - :worker_1_port
SET search_path TO "citus_split_test_schema_columnar_partitioned";
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like '%_89%'
ORDER BY 1, 2;
relname | Constraint | Definition
---------------------------------------------------------------------
sensors_2020_01_01_8999104 | fkey_from_child_to_child_8999104 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999120(eventdatetime, measureid)
sensors_2020_01_01_8999104 | fkey_from_child_to_dist_8999104 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8999116(measureid)
sensors_2020_01_01_8999104 | fkey_from_child_to_parent_8999104 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8999118(eventdatetime, measureid)
sensors_2020_01_01_8999104 | fkey_from_child_to_ref_8999104 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
sensors_2020_01_01_8999104 | fkey_from_parent_to_child_8999100 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999120(eventdatetime, measureid)
sensors_2020_01_01_8999104 | fkey_from_parent_to_dist_8999100 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8999116(measureid)
sensors_2020_01_01_8999104 | fkey_from_parent_to_parent_8999100 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8999118(eventdatetime, measureid)
sensors_2020_01_01_8999104 | fkey_from_parent_to_ref_8999100 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
sensors_2020_01_01_8999104 | sensors_2020_01_01_8999104_measureid_eventdatetime_fkey | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999120(eventdatetime, measureid)
sensors_8999100 | fkey_from_parent_to_child_8999100 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999120(eventdatetime, measureid)
sensors_8999100 | fkey_from_parent_to_dist_8999100 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8999116(measureid)
sensors_8999100 | fkey_from_parent_to_parent_8999100 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8999118(eventdatetime, measureid)
sensors_8999100 | fkey_from_parent_to_ref_8999100 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
sensors_8999100 | sensors_8999100_measureid_eventdatetime_fkey | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999120(eventdatetime, measureid)
sensors_news_8999106 | fkey_from_parent_to_child_8999100 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999120(eventdatetime, measureid)
sensors_news_8999106 | fkey_from_parent_to_dist_8999100 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8999116(measureid)
sensors_news_8999106 | fkey_from_parent_to_parent_8999100 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8999118(eventdatetime, measureid)
sensors_news_8999106 | fkey_from_parent_to_ref_8999100 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
sensors_old_8999102 | fkey_from_parent_to_child_8999100 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999120(eventdatetime, measureid)
sensors_old_8999102 | fkey_from_parent_to_dist_8999100 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8999116(measureid)
sensors_old_8999102 | fkey_from_parent_to_parent_8999100 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8999118(eventdatetime, measureid)
sensors_old_8999102 | fkey_from_parent_to_ref_8999100 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
(22 rows)
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like '%_89%' ORDER BY 1,2;
tablename | indexdef
---------------------------------------------------------------------
colocated_dist_table_8999116 | CREATE UNIQUE INDEX colocated_dist_table_pkey_8999116 ON citus_split_test_schema_columnar_partitioned.colocated_dist_table_8999116 USING btree (measureid)
colocated_partitioned_table_2020_01_01_8999120 | CREATE UNIQUE INDEX colocated_partitioned_table_2020_01_01_pkey_8999120 ON citus_split_test_schema_columnar_partitioned.colocated_partitioned_table_2020_01_01_8999120 USING btree (measureid, eventdatetime)
colocated_partitioned_table_8999118 | CREATE UNIQUE INDEX colocated_partitioned_table_pkey_8999118 ON ONLY citus_split_test_schema_columnar_partitioned.colocated_partitioned_table_8999118 USING btree (measureid, eventdatetime)
reference_table_8970011 | CREATE UNIQUE INDEX reference_table_pkey_8970011 ON citus_split_test_schema_columnar_partitioned.reference_table_8970011 USING btree (measureid)
sensors_2020_01_01_8999104 | CREATE INDEX index_on_child_8999104 ON citus_split_test_schema_columnar_partitioned.sensors_2020_01_01_8999104 USING btree (lower((measure_data)::text))
sensors_2020_01_01_8999104 | CREATE INDEX sensors_2020_01_01_lower_idx_8999104 ON citus_split_test_schema_columnar_partitioned.sensors_2020_01_01_8999104 USING btree (lower((measureid)::text))
sensors_2020_01_01_8999104 | CREATE UNIQUE INDEX sensors_2020_01_01_pkey_8999104 ON citus_split_test_schema_columnar_partitioned.sensors_2020_01_01_8999104 USING btree (measureid, eventdatetime, measure_data)
sensors_8999100 | CREATE INDEX index_on_parent_8999100 ON ONLY citus_split_test_schema_columnar_partitioned.sensors_8999100 USING btree (lower((measureid)::text))
sensors_8999100 | CREATE UNIQUE INDEX sensors_pkey_8999100 ON ONLY citus_split_test_schema_columnar_partitioned.sensors_8999100 USING btree (measureid, eventdatetime, measure_data)
sensors_news_8999106 | CREATE INDEX sensors_news_lower_idx_8999106 ON citus_split_test_schema_columnar_partitioned.sensors_news_8999106 USING btree (lower((measureid)::text))
sensors_news_8999106 | CREATE UNIQUE INDEX sensors_news_pkey_8999106 ON citus_split_test_schema_columnar_partitioned.sensors_news_8999106 USING btree (measureid, eventdatetime, measure_data)
sensors_old_8999102 | CREATE INDEX sensors_old_lower_idx_8999102 ON citus_split_test_schema_columnar_partitioned.sensors_old_8999102 USING btree (lower((measureid)::text))
sensors_old_8999102 | CREATE UNIQUE INDEX sensors_old_pkey_8999102 ON citus_split_test_schema_columnar_partitioned.sensors_old_8999102 USING btree (measureid, eventdatetime, measure_data)
sensorscolumnar_2020_01_01_8999112 | CREATE INDEX index_on_child_columnar_8999112 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_2020_01_01_8999112 USING btree (lower((measure_data)::text))
sensorscolumnar_2020_01_01_8999112 | CREATE INDEX sensorscolumnar_2020_01_01_lower_idx_8999112 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_2020_01_01_8999112 USING btree (lower((measureid)::text))
sensorscolumnar_2020_01_01_8999112 | CREATE UNIQUE INDEX sensorscolumnar_2020_01_01_pkey_8999112 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_2020_01_01_8999112 USING btree (measureid, eventdatetime, measure_data)
sensorscolumnar_8999108 | CREATE INDEX index_on_parent_columnar_8999108 ON ONLY citus_split_test_schema_columnar_partitioned.sensorscolumnar_8999108 USING btree (lower((measureid)::text))
sensorscolumnar_8999108 | CREATE UNIQUE INDEX sensorscolumnar_pkey_8999108 ON ONLY citus_split_test_schema_columnar_partitioned.sensorscolumnar_8999108 USING btree (measureid, eventdatetime, measure_data)
sensorscolumnar_news_8999114 | CREATE INDEX sensorscolumnar_news_lower_idx_8999114 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_news_8999114 USING btree (lower((measureid)::text))
sensorscolumnar_news_8999114 | CREATE UNIQUE INDEX sensorscolumnar_news_pkey_8999114 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_news_8999114 USING btree (measureid, eventdatetime, measure_data)
sensorscolumnar_old_8999110 | CREATE INDEX sensorscolumnar_old_lower_idx_8999110 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_old_8999110 USING btree (lower((measureid)::text))
sensorscolumnar_old_8999110 | CREATE UNIQUE INDEX sensorscolumnar_old_pkey_8999110 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_old_8999110 USING btree (measureid, eventdatetime, measure_data)
(22 rows)
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema_columnar_partitioned')
)
ORDER BY stxname ASC;
stxname
---------------------------------------------------------------------
s1
s1_8999100
s1_c
s1_c_8999108
s2
s2_8999104
s2_c
s2_c_8999112
(8 rows)
\c - - - :worker_2_port
SET search_path TO "citus_split_test_schema_columnar_partitioned";
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like '%_89%'
ORDER BY 1, 2;
relname | Constraint | Definition
---------------------------------------------------------------------
sensors_2020_01_01_8999005 | fkey_from_child_to_child_8999005 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999021(eventdatetime, measureid)
sensors_2020_01_01_8999005 | fkey_from_child_to_dist_8999005 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8999017(measureid)
sensors_2020_01_01_8999005 | fkey_from_child_to_parent_8999005 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8999019(eventdatetime, measureid)
sensors_2020_01_01_8999005 | fkey_from_child_to_ref_8999005 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
sensors_2020_01_01_8999005 | fkey_from_parent_to_child_8999001 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999021(eventdatetime, measureid)
sensors_2020_01_01_8999005 | fkey_from_parent_to_dist_8999001 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8999017(measureid)
sensors_2020_01_01_8999005 | fkey_from_parent_to_parent_8999001 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8999019(eventdatetime, measureid)
sensors_2020_01_01_8999005 | fkey_from_parent_to_ref_8999001 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
sensors_2020_01_01_8999005 | sensors_2020_01_01_8999005_measureid_eventdatetime_fkey | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999021(eventdatetime, measureid)
sensors_2020_01_01_8999105 | fkey_from_child_to_child_8999105 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999121(eventdatetime, measureid)
sensors_2020_01_01_8999105 | fkey_from_child_to_dist_8999105 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8999117(measureid)
sensors_2020_01_01_8999105 | fkey_from_child_to_parent_8999105 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8999119(eventdatetime, measureid)
sensors_2020_01_01_8999105 | fkey_from_child_to_ref_8999105 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
sensors_2020_01_01_8999105 | fkey_from_parent_to_child_8999101 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999121(eventdatetime, measureid)
sensors_2020_01_01_8999105 | fkey_from_parent_to_dist_8999101 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8999117(measureid)
sensors_2020_01_01_8999105 | fkey_from_parent_to_parent_8999101 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8999119(eventdatetime, measureid)
sensors_2020_01_01_8999105 | fkey_from_parent_to_ref_8999101 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
sensors_2020_01_01_8999105 | sensors_2020_01_01_8999105_measureid_eventdatetime_fkey | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999121(eventdatetime, measureid)
sensors_8999001 | fkey_from_parent_to_child_8999001 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999021(eventdatetime, measureid)
sensors_8999001 | fkey_from_parent_to_dist_8999001 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8999017(measureid)
sensors_8999001 | fkey_from_parent_to_parent_8999001 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8999019(eventdatetime, measureid)
sensors_8999001 | fkey_from_parent_to_ref_8999001 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
sensors_8999001 | sensors_8999001_measureid_eventdatetime_fkey | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999021(eventdatetime, measureid)
sensors_8999101 | fkey_from_parent_to_child_8999101 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999121(eventdatetime, measureid)
sensors_8999101 | fkey_from_parent_to_dist_8999101 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8999117(measureid)
sensors_8999101 | fkey_from_parent_to_parent_8999101 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8999119(eventdatetime, measureid)
sensors_8999101 | fkey_from_parent_to_ref_8999101 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
sensors_8999101 | sensors_8999101_measureid_eventdatetime_fkey | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999121(eventdatetime, measureid)
sensors_news_8999007 | fkey_from_parent_to_child_8999001 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999021(eventdatetime, measureid)
sensors_news_8999007 | fkey_from_parent_to_dist_8999001 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8999017(measureid)
sensors_news_8999007 | fkey_from_parent_to_parent_8999001 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8999019(eventdatetime, measureid)
sensors_news_8999007 | fkey_from_parent_to_ref_8999001 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
sensors_news_8999107 | fkey_from_parent_to_child_8999101 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999121(eventdatetime, measureid)
sensors_news_8999107 | fkey_from_parent_to_dist_8999101 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8999117(measureid)
sensors_news_8999107 | fkey_from_parent_to_parent_8999101 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8999119(eventdatetime, measureid)
sensors_news_8999107 | fkey_from_parent_to_ref_8999101 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
sensors_old_8999003 | fkey_from_parent_to_child_8999001 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999021(eventdatetime, measureid)
sensors_old_8999003 | fkey_from_parent_to_dist_8999001 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8999017(measureid)
sensors_old_8999003 | fkey_from_parent_to_parent_8999001 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8999019(eventdatetime, measureid)
sensors_old_8999003 | fkey_from_parent_to_ref_8999001 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
sensors_old_8999103 | fkey_from_parent_to_child_8999101 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_2020_01_01_8999121(eventdatetime, measureid)
sensors_old_8999103 | fkey_from_parent_to_dist_8999101 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8999117(measureid)
sensors_old_8999103 | fkey_from_parent_to_parent_8999101 | FOREIGN KEY (eventdatetime, measureid) REFERENCES colocated_partitioned_table_8999119(eventdatetime, measureid)
sensors_old_8999103 | fkey_from_parent_to_ref_8999101 | FOREIGN KEY (measureid) REFERENCES reference_table_8970011(measureid)
(44 rows)
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like '%_89%' ORDER BY 1,2;
tablename | indexdef
---------------------------------------------------------------------
colocated_dist_table_8999017 | CREATE UNIQUE INDEX colocated_dist_table_pkey_8999017 ON citus_split_test_schema_columnar_partitioned.colocated_dist_table_8999017 USING btree (measureid)
colocated_dist_table_8999117 | CREATE UNIQUE INDEX colocated_dist_table_pkey_8999117 ON citus_split_test_schema_columnar_partitioned.colocated_dist_table_8999117 USING btree (measureid)
colocated_partitioned_table_2020_01_01_8999021 | CREATE UNIQUE INDEX colocated_partitioned_table_2020_01_01_pkey_8999021 ON citus_split_test_schema_columnar_partitioned.colocated_partitioned_table_2020_01_01_8999021 USING btree (measureid, eventdatetime)
colocated_partitioned_table_2020_01_01_8999121 | CREATE UNIQUE INDEX colocated_partitioned_table_2020_01_01_pkey_8999121 ON citus_split_test_schema_columnar_partitioned.colocated_partitioned_table_2020_01_01_8999121 USING btree (measureid, eventdatetime)
colocated_partitioned_table_8999019 | CREATE UNIQUE INDEX colocated_partitioned_table_pkey_8999019 ON ONLY citus_split_test_schema_columnar_partitioned.colocated_partitioned_table_8999019 USING btree (measureid, eventdatetime)
colocated_partitioned_table_8999119 | CREATE UNIQUE INDEX colocated_partitioned_table_pkey_8999119 ON ONLY citus_split_test_schema_columnar_partitioned.colocated_partitioned_table_8999119 USING btree (measureid, eventdatetime)
reference_table_8970011 | CREATE UNIQUE INDEX reference_table_pkey_8970011 ON citus_split_test_schema_columnar_partitioned.reference_table_8970011 USING btree (measureid)
sensors_2020_01_01_8999005 | CREATE INDEX index_on_child_8999005 ON citus_split_test_schema_columnar_partitioned.sensors_2020_01_01_8999005 USING btree (lower((measure_data)::text))
sensors_2020_01_01_8999005 | CREATE INDEX sensors_2020_01_01_lower_idx_8999005 ON citus_split_test_schema_columnar_partitioned.sensors_2020_01_01_8999005 USING btree (lower((measureid)::text))
sensors_2020_01_01_8999005 | CREATE UNIQUE INDEX sensors_2020_01_01_pkey_8999005 ON citus_split_test_schema_columnar_partitioned.sensors_2020_01_01_8999005 USING btree (measureid, eventdatetime, measure_data)
sensors_2020_01_01_8999105 | CREATE INDEX index_on_child_8999105 ON citus_split_test_schema_columnar_partitioned.sensors_2020_01_01_8999105 USING btree (lower((measure_data)::text))
sensors_2020_01_01_8999105 | CREATE INDEX sensors_2020_01_01_lower_idx_8999105 ON citus_split_test_schema_columnar_partitioned.sensors_2020_01_01_8999105 USING btree (lower((measureid)::text))
sensors_2020_01_01_8999105 | CREATE UNIQUE INDEX sensors_2020_01_01_pkey_8999105 ON citus_split_test_schema_columnar_partitioned.sensors_2020_01_01_8999105 USING btree (measureid, eventdatetime, measure_data)
sensors_8999001 | CREATE INDEX index_on_parent_8999001 ON ONLY citus_split_test_schema_columnar_partitioned.sensors_8999001 USING btree (lower((measureid)::text))
sensors_8999001 | CREATE UNIQUE INDEX sensors_pkey_8999001 ON ONLY citus_split_test_schema_columnar_partitioned.sensors_8999001 USING btree (measureid, eventdatetime, measure_data)
sensors_8999101 | CREATE INDEX index_on_parent_8999101 ON ONLY citus_split_test_schema_columnar_partitioned.sensors_8999101 USING btree (lower((measureid)::text))
sensors_8999101 | CREATE UNIQUE INDEX sensors_pkey_8999101 ON ONLY citus_split_test_schema_columnar_partitioned.sensors_8999101 USING btree (measureid, eventdatetime, measure_data)
sensors_news_8999007 | CREATE INDEX sensors_news_lower_idx_8999007 ON citus_split_test_schema_columnar_partitioned.sensors_news_8999007 USING btree (lower((measureid)::text))
sensors_news_8999007 | CREATE UNIQUE INDEX sensors_news_pkey_8999007 ON citus_split_test_schema_columnar_partitioned.sensors_news_8999007 USING btree (measureid, eventdatetime, measure_data)
sensors_news_8999107 | CREATE INDEX sensors_news_lower_idx_8999107 ON citus_split_test_schema_columnar_partitioned.sensors_news_8999107 USING btree (lower((measureid)::text))
sensors_news_8999107 | CREATE UNIQUE INDEX sensors_news_pkey_8999107 ON citus_split_test_schema_columnar_partitioned.sensors_news_8999107 USING btree (measureid, eventdatetime, measure_data)
sensors_old_8999003 | CREATE INDEX sensors_old_lower_idx_8999003 ON citus_split_test_schema_columnar_partitioned.sensors_old_8999003 USING btree (lower((measureid)::text))
sensors_old_8999003 | CREATE UNIQUE INDEX sensors_old_pkey_8999003 ON citus_split_test_schema_columnar_partitioned.sensors_old_8999003 USING btree (measureid, eventdatetime, measure_data)
sensors_old_8999103 | CREATE INDEX sensors_old_lower_idx_8999103 ON citus_split_test_schema_columnar_partitioned.sensors_old_8999103 USING btree (lower((measureid)::text))
sensors_old_8999103 | CREATE UNIQUE INDEX sensors_old_pkey_8999103 ON citus_split_test_schema_columnar_partitioned.sensors_old_8999103 USING btree (measureid, eventdatetime, measure_data)
sensorscolumnar_2020_01_01_8999013 | CREATE INDEX index_on_child_columnar_8999013 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_2020_01_01_8999013 USING btree (lower((measure_data)::text))
sensorscolumnar_2020_01_01_8999013 | CREATE INDEX sensorscolumnar_2020_01_01_lower_idx_8999013 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_2020_01_01_8999013 USING btree (lower((measureid)::text))
sensorscolumnar_2020_01_01_8999013 | CREATE UNIQUE INDEX sensorscolumnar_2020_01_01_pkey_8999013 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_2020_01_01_8999013 USING btree (measureid, eventdatetime, measure_data)
sensorscolumnar_2020_01_01_8999113 | CREATE INDEX index_on_child_columnar_8999113 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_2020_01_01_8999113 USING btree (lower((measure_data)::text))
sensorscolumnar_2020_01_01_8999113 | CREATE INDEX sensorscolumnar_2020_01_01_lower_idx_8999113 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_2020_01_01_8999113 USING btree (lower((measureid)::text))
sensorscolumnar_2020_01_01_8999113 | CREATE UNIQUE INDEX sensorscolumnar_2020_01_01_pkey_8999113 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_2020_01_01_8999113 USING btree (measureid, eventdatetime, measure_data)
sensorscolumnar_8999009 | CREATE INDEX index_on_parent_columnar_8999009 ON ONLY citus_split_test_schema_columnar_partitioned.sensorscolumnar_8999009 USING btree (lower((measureid)::text))
sensorscolumnar_8999009 | CREATE UNIQUE INDEX sensorscolumnar_pkey_8999009 ON ONLY citus_split_test_schema_columnar_partitioned.sensorscolumnar_8999009 USING btree (measureid, eventdatetime, measure_data)
sensorscolumnar_8999109 | CREATE INDEX index_on_parent_columnar_8999109 ON ONLY citus_split_test_schema_columnar_partitioned.sensorscolumnar_8999109 USING btree (lower((measureid)::text))
sensorscolumnar_8999109 | CREATE UNIQUE INDEX sensorscolumnar_pkey_8999109 ON ONLY citus_split_test_schema_columnar_partitioned.sensorscolumnar_8999109 USING btree (measureid, eventdatetime, measure_data)
sensorscolumnar_news_8999015 | CREATE INDEX sensorscolumnar_news_lower_idx_8999015 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_news_8999015 USING btree (lower((measureid)::text))
sensorscolumnar_news_8999015 | CREATE UNIQUE INDEX sensorscolumnar_news_pkey_8999015 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_news_8999015 USING btree (measureid, eventdatetime, measure_data)
sensorscolumnar_news_8999115 | CREATE INDEX sensorscolumnar_news_lower_idx_8999115 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_news_8999115 USING btree (lower((measureid)::text))
sensorscolumnar_news_8999115 | CREATE UNIQUE INDEX sensorscolumnar_news_pkey_8999115 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_news_8999115 USING btree (measureid, eventdatetime, measure_data)
sensorscolumnar_old_8999011 | CREATE INDEX sensorscolumnar_old_lower_idx_8999011 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_old_8999011 USING btree (lower((measureid)::text))
sensorscolumnar_old_8999011 | CREATE UNIQUE INDEX sensorscolumnar_old_pkey_8999011 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_old_8999011 USING btree (measureid, eventdatetime, measure_data)
sensorscolumnar_old_8999111 | CREATE INDEX sensorscolumnar_old_lower_idx_8999111 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_old_8999111 USING btree (lower((measureid)::text))
sensorscolumnar_old_8999111 | CREATE UNIQUE INDEX sensorscolumnar_old_pkey_8999111 ON citus_split_test_schema_columnar_partitioned.sensorscolumnar_old_8999111 USING btree (measureid, eventdatetime, measure_data)
(43 rows)
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema_columnar_partitioned')
)
ORDER BY stxname ASC;
stxname
---------------------------------------------------------------------
s1
s1_8999001
s1_8999101
s1_c
s1_c_8999009
s1_c_8999109
s2
s2_8999005
s2_8999105
s2_c
s2_c_8999013
s2_c_8999113
(12 rows)
-- END: Show the updated state on workers
--BEGIN : Cleanup
\c - postgres - :master_port
DROP SCHEMA "citus_split_test_schema_columnar_partitioned" CASCADE;
NOTICE: drop cascades to 5 other objects
DETAIL: drop cascades to table citus_split_test_schema_columnar_partitioned.sensors
drop cascades to table citus_split_test_schema_columnar_partitioned.sensorscolumnar
drop cascades to table citus_split_test_schema_columnar_partitioned.colocated_dist_table
drop cascades to table citus_split_test_schema_columnar_partitioned.colocated_partitioned_table
drop cascades to table citus_split_test_schema_columnar_partitioned.reference_table
--END : Cleanup

View File

@ -0,0 +1,129 @@
/*
Citus non-blocking shard split test.
The test ensures after a error free non-blocking split operation, following objects
created for logical replication are cleaned up:
1) At source node: Publications, replication slots, dummy shards.
2) At destination node: Subscriptions, dummy shards.
*/
CREATE SCHEMA "citus_split_test_schema";
SET search_path TO "citus_split_test_schema";
SET citus.next_shard_id TO 8981000;
SET citus.next_placement_id TO 8610000;
SET citus.shard_count TO 2;
SET citus.shard_replication_factor TO 1;
-- BEGIN: Create table to split, along with other co-located tables. Add indexes, statistics etc.
CREATE TABLE sensors(
measureid integer,
eventdatetime date,
measure_data jsonb,
meaure_quantity decimal(15, 2),
measure_status char(1),
measure_comment varchar(44),
PRIMARY KEY (measureid, eventdatetime, measure_data));
SELECT create_distributed_table('sensors', 'measureid', colocate_with:='none');
create_distributed_table
---------------------------------------------------------------------
(1 row)
INSERT INTO sensors SELECT i, '2020-01-05', '{}', 11011.10, 'A', 'I <3 Citus' FROM generate_series(0,1000)i;
-- BEGIN : Move one shard before we split it.
\c - postgres - :master_port
SET search_path TO "citus_split_test_schema";
SET citus.next_shard_id TO 8981007;
SET citus.defer_drop_after_shard_move TO OFF;
-- BEGIN : Set node id variables
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
-- END : Set node id variables
-- BEGIN : Split two shards
-- Perform 2 way split
SELECT pg_catalog.citus_split_shard_by_split_points(
8981000,
ARRAY['-1073741824'],
ARRAY[:worker_2_node, :worker_2_node],
'force_logical');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
\c - - - :worker_1_port
SET search_path TO "citus_split_test_schema";
SET citus.show_shards_for_app_name_prefixes = '*';
-- Dummy shards should be cleaned up. 8981007, 8981008 are dummy shards
-- created at source.
SELECT count(*) FROM pg_class where relname like '%sensors_8981007%';
count
---------------------------------------------------------------------
0
(1 row)
SELECT count(*) FROM pg_class where relname like '%sensors_8981008%';
count
---------------------------------------------------------------------
0
(1 row)
-- Replication slots should be cleanedup at source
SELECT slot_name FROM pg_replication_slots;
slot_name
---------------------------------------------------------------------
(0 rows)
-- Publications should be cleaned up on worker1
SELECT count(*) FROM pg_publication;
count
---------------------------------------------------------------------
0
(1 row)
\c - - - :worker_2_port
SET search_path TO "citus_split_test_schema";
-- All subscriptions should be cleaned up.
SELECT count(*) FROM pg_subscription;
count
---------------------------------------------------------------------
0
(1 row)
-- Trigger a 3-way local split.
\c - - - :master_port
SET search_path TO "citus_split_test_schema";
SELECT pg_catalog.citus_split_shard_by_split_points(
8981001,
ARRAY['536870911', '1610612735'],
ARRAY[:worker_2_node, :worker_2_node, :worker_2_node],
'force_logical');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
\c - - - :worker_2_port
SET search_path TO "citus_split_test_schema";
-- Replication slots should be cleaned up
SELECT slot_name FROM pg_replication_slots;
slot_name
---------------------------------------------------------------------
(0 rows)
-- Publications should be cleanedup
SELECT count(*) FROM pg_publication;
count
---------------------------------------------------------------------
0
(1 row)
-- All subscriptions should be cleaned up.
SELECT count(*) FROM pg_subscription;
count
---------------------------------------------------------------------
0
(1 row)
--BEGIN : Cleanup
\c - postgres - :master_port
DROP SCHEMA "citus_split_test_schema" CASCADE;
NOTICE: drop cascades to table citus_split_test_schema.sensors
--END : Cleanup

View File

@ -0,0 +1,459 @@
/*
Citus Shard Split Test.The test is model similar to 'shard_move_constraints'.
Here is a high level overview of test plan:
1. Create a table 'sensors' (ShardCount = 2) to be split. Add indexes and statistics on this table.
2. Create two other tables: 'reference_table' and 'colocated_dist_table', co-located with sensors.
3. Create Foreign key constraints between the two co-located distributed tables.
4. Load data into the three tables.
5. Move one of the shards for 'sensors' to test ShardMove -> Split.
6. Trigger Split on both shards of 'sensors'. This will also split co-located tables.
7. Move one of the split shard to test Split -> ShardMove.
8. Split an already split shard second time on a different schema.
*/
CREATE SCHEMA "citus_split_test_schema";
CREATE ROLE test_shard_split_role WITH LOGIN;
GRANT USAGE, CREATE ON SCHEMA "citus_split_test_schema" TO test_shard_split_role;
SET ROLE test_shard_split_role;
SET search_path TO "citus_split_test_schema";
SET citus.next_shard_id TO 8981000;
SET citus.next_placement_id TO 8610000;
SET citus.shard_count TO 2;
SET citus.shard_replication_factor TO 1;
-- BEGIN: Create table to split, along with other co-located tables. Add indexes, statistics etc.
CREATE TABLE sensors(
measureid integer,
eventdatetime date,
measure_data jsonb,
meaure_quantity decimal(15, 2),
measure_status char(1),
measure_comment varchar(44),
PRIMARY KEY (measureid, eventdatetime, measure_data));
CREATE INDEX index_on_sensors ON sensors(lower(measureid::text));
ALTER INDEX index_on_sensors ALTER COLUMN 1 SET STATISTICS 1000;
CREATE INDEX hash_index_on_sensors ON sensors USING HASH((measure_data->'IsFailed'));
CREATE INDEX index_with_include_on_sensors ON sensors ((measure_data->'IsFailed')) INCLUDE (measure_data, eventdatetime, measure_status);
CREATE STATISTICS stats_on_sensors (dependencies) ON measureid, eventdatetime FROM sensors;
SELECT create_distributed_table('sensors', 'measureid', colocate_with:='none');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- END: Create table to split, along with other co-located tables. Add indexes, statistics etc.
-- BEGIN: Create co-located distributed and reference tables.
CREATE TABLE reference_table (measureid integer PRIMARY KEY);
SELECT create_reference_table('reference_table');
create_reference_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE colocated_dist_table (measureid integer PRIMARY KEY);
CLUSTER colocated_dist_table USING colocated_dist_table_pkey;
SELECT create_distributed_table('colocated_dist_table', 'measureid', colocate_with:='sensors');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE table_with_index_rep_identity(key int NOT NULL);
CREATE UNIQUE INDEX uqx ON table_with_index_rep_identity(key);
ALTER TABLE table_with_index_rep_identity REPLICA IDENTITY USING INDEX uqx;
CLUSTER table_with_index_rep_identity USING uqx;
SELECT create_distributed_table('table_with_index_rep_identity', 'key', colocate_with:='sensors');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- END: Create co-located distributed and reference tables.
-- BEGIN : Create Foreign key constraints.
ALTER TABLE sensors ADD CONSTRAINT fkey_table_to_dist FOREIGN KEY (measureid) REFERENCES colocated_dist_table(measureid);
-- END : Create Foreign key constraints.
-- BEGIN : Load data into tables.
INSERT INTO reference_table SELECT i FROM generate_series(0,1000)i;
INSERT INTO colocated_dist_table SELECT i FROM generate_series(0,1000)i;
INSERT INTO sensors SELECT i, '2020-01-05', '{}', 11011.10, 'A', 'I <3 Citus' FROM generate_series(0,1000)i;
SELECT COUNT(*) FROM sensors;
count
---------------------------------------------------------------------
1001
(1 row)
SELECT COUNT(*) FROM reference_table;
count
---------------------------------------------------------------------
1001
(1 row)
SELECT COUNT(*) FROM colocated_dist_table;
count
---------------------------------------------------------------------
1001
(1 row)
-- END: Load data into tables.
-- BEGIN : Display current state.
SELECT shard.shardid, logicalrelid, shardminvalue, shardmaxvalue, nodename, nodeport
FROM pg_dist_shard AS shard
INNER JOIN pg_dist_placement placement ON shard.shardid = placement.shardid
INNER JOIN pg_dist_node node ON placement.groupid = node.groupid
INNER JOIN pg_catalog.pg_class cls ON shard.logicalrelid = cls.oid
WHERE node.noderole = 'primary' AND (logicalrelid = 'sensors'::regclass OR logicalrelid = 'colocated_dist_table'::regclass OR logicalrelid = 'table_with_index_rep_identity'::regclass)
ORDER BY logicalrelid, shardminvalue::BIGINT;
shardid | logicalrelid | shardminvalue | shardmaxvalue | nodename | nodeport
---------------------------------------------------------------------
8981000 | sensors | -2147483648 | -1 | localhost | 57637
8981001 | sensors | 0 | 2147483647 | localhost | 57638
8981003 | colocated_dist_table | -2147483648 | -1 | localhost | 57637
8981004 | colocated_dist_table | 0 | 2147483647 | localhost | 57638
8981005 | table_with_index_rep_identity | -2147483648 | -1 | localhost | 57637
8981006 | table_with_index_rep_identity | 0 | 2147483647 | localhost | 57638
(6 rows)
\c - - - :worker_1_port
SET search_path TO "citus_split_test_schema", public, pg_catalog;
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like 'sensors_%'
ORDER BY 1, 2;
relname | Constraint | Definition
---------------------------------------------------------------------
sensors_8981000 | fkey_table_to_dist_8981000 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8981003(measureid)
(1 row)
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'sensors_%' ORDER BY 1,2;
tablename | indexdef
---------------------------------------------------------------------
sensors_8981000 | CREATE INDEX hash_index_on_sensors_8981000 ON citus_split_test_schema.sensors_8981000 USING hash (((measure_data -> 'IsFailed'::text)))
sensors_8981000 | CREATE INDEX index_on_sensors_8981000 ON citus_split_test_schema.sensors_8981000 USING btree (lower((measureid)::text))
sensors_8981000 | CREATE INDEX index_with_include_on_sensors_8981000 ON citus_split_test_schema.sensors_8981000 USING btree (((measure_data -> 'IsFailed'::text))) INCLUDE (measure_data, eventdatetime, measure_status)
sensors_8981000 | CREATE UNIQUE INDEX sensors_pkey_8981000 ON citus_split_test_schema.sensors_8981000 USING btree (measureid, eventdatetime, measure_data)
(4 rows)
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'table_with_index_rep_identity_%' ORDER BY 1,2;
tablename | indexdef
---------------------------------------------------------------------
table_with_index_rep_identity_8981005 | CREATE UNIQUE INDEX uqx_8981005 ON citus_split_test_schema.table_with_index_rep_identity_8981005 USING btree (key)
(1 row)
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema')
)
ORDER BY stxname ASC;
stxname
---------------------------------------------------------------------
stats_on_sensors
stats_on_sensors_8981000
(2 rows)
\c - - - :worker_2_port
SET search_path TO "citus_split_test_schema", public, pg_catalog;
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like 'sensors_%'
ORDER BY 1, 2;
relname | Constraint | Definition
---------------------------------------------------------------------
sensors_8981001 | fkey_table_to_dist_8981001 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8981004(measureid)
(1 row)
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'sensors_%' ORDER BY 1,2;
tablename | indexdef
---------------------------------------------------------------------
sensors_8981001 | CREATE INDEX hash_index_on_sensors_8981001 ON citus_split_test_schema.sensors_8981001 USING hash (((measure_data -> 'IsFailed'::text)))
sensors_8981001 | CREATE INDEX index_on_sensors_8981001 ON citus_split_test_schema.sensors_8981001 USING btree (lower((measureid)::text))
sensors_8981001 | CREATE INDEX index_with_include_on_sensors_8981001 ON citus_split_test_schema.sensors_8981001 USING btree (((measure_data -> 'IsFailed'::text))) INCLUDE (measure_data, eventdatetime, measure_status)
sensors_8981001 | CREATE UNIQUE INDEX sensors_pkey_8981001 ON citus_split_test_schema.sensors_8981001 USING btree (measureid, eventdatetime, measure_data)
(4 rows)
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'table_with_index_rep_identity_%' ORDER BY 1,2;
tablename | indexdef
---------------------------------------------------------------------
table_with_index_rep_identity_8981006 | CREATE UNIQUE INDEX uqx_8981006 ON citus_split_test_schema.table_with_index_rep_identity_8981006 USING btree (key)
(1 row)
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema')
)
ORDER BY stxname ASC;
stxname
---------------------------------------------------------------------
stats_on_sensors
stats_on_sensors_8981001
(2 rows)
-- END : Display current state
-- BEGIN : Move one shard before we split it.
\c - postgres - :master_port
SET ROLE test_shard_split_role;
SET search_path TO "citus_split_test_schema";
SET citus.next_shard_id TO 8981007;
SET citus.defer_drop_after_shard_move TO OFF;
SELECT citus_move_shard_placement(8981000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, shard_transfer_mode:='force_logical');
citus_move_shard_placement
---------------------------------------------------------------------
(1 row)
-- END : Move one shard before we split it.
-- BEGIN : Set node id variables
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
-- END : Set node id variables
-- BEGIN : Split two shards : One with move and One without move.
-- Perform 2 way split
SELECT pg_catalog.citus_split_shard_by_split_points(
8981000,
ARRAY['-1073741824'],
ARRAY[:worker_1_node, :worker_2_node],
'force_logical');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
-- Perform 3 way split
SELECT pg_catalog.citus_split_shard_by_split_points(
8981001,
ARRAY['536870911', '1610612735'],
ARRAY[:worker_1_node, :worker_1_node, :worker_2_node],
'force_logical');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
-- END : Split two shards : One with move and One without move.
-- BEGIN : Move a shard post split.
SELECT citus_move_shard_placement(8981007, 'localhost', :worker_1_port, 'localhost', :worker_2_port, shard_transfer_mode:='block_writes');
citus_move_shard_placement
---------------------------------------------------------------------
(1 row)
-- END : Move a shard post split.
-- BEGIN : Display current state.
SELECT shard.shardid, logicalrelid, shardminvalue, shardmaxvalue, nodename, nodeport
FROM pg_dist_shard AS shard
INNER JOIN pg_dist_placement placement ON shard.shardid = placement.shardid
INNER JOIN pg_dist_node node ON placement.groupid = node.groupid
INNER JOIN pg_catalog.pg_class cls ON shard.logicalrelid = cls.oid
WHERE node.noderole = 'primary' AND (logicalrelid = 'sensors'::regclass OR logicalrelid = 'colocated_dist_table'::regclass OR logicalrelid = 'table_with_index_rep_identity'::regclass)
ORDER BY logicalrelid, shardminvalue::BIGINT;
shardid | logicalrelid | shardminvalue | shardmaxvalue | nodename | nodeport
---------------------------------------------------------------------
8981007 | sensors | -2147483648 | -1073741824 | localhost | 57638
8981008 | sensors | -1073741823 | -1 | localhost | 57638
8981013 | sensors | 0 | 536870911 | localhost | 57637
8981014 | sensors | 536870912 | 1610612735 | localhost | 57637
8981015 | sensors | 1610612736 | 2147483647 | localhost | 57638
8981009 | colocated_dist_table | -2147483648 | -1073741824 | localhost | 57638
8981010 | colocated_dist_table | -1073741823 | -1 | localhost | 57638
8981016 | colocated_dist_table | 0 | 536870911 | localhost | 57637
8981017 | colocated_dist_table | 536870912 | 1610612735 | localhost | 57637
8981018 | colocated_dist_table | 1610612736 | 2147483647 | localhost | 57638
8981011 | table_with_index_rep_identity | -2147483648 | -1073741824 | localhost | 57638
8981012 | table_with_index_rep_identity | -1073741823 | -1 | localhost | 57638
8981019 | table_with_index_rep_identity | 0 | 536870911 | localhost | 57637
8981020 | table_with_index_rep_identity | 536870912 | 1610612735 | localhost | 57637
8981021 | table_with_index_rep_identity | 1610612736 | 2147483647 | localhost | 57638
(15 rows)
\c - - - :worker_1_port
SET search_path TO "citus_split_test_schema", public, pg_catalog;
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like 'sensors_%'
ORDER BY 1, 2;
relname | Constraint | Definition
---------------------------------------------------------------------
sensors_8981013 | fkey_table_to_dist_8981013 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8981016(measureid)
sensors_8981014 | fkey_table_to_dist_8981014 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8981017(measureid)
(2 rows)
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'sensors_%' ORDER BY 1,2;
tablename | indexdef
---------------------------------------------------------------------
sensors_8981013 | CREATE INDEX hash_index_on_sensors_8981013 ON citus_split_test_schema.sensors_8981013 USING hash (((measure_data -> 'IsFailed'::text)))
sensors_8981013 | CREATE INDEX index_on_sensors_8981013 ON citus_split_test_schema.sensors_8981013 USING btree (lower((measureid)::text))
sensors_8981013 | CREATE INDEX index_with_include_on_sensors_8981013 ON citus_split_test_schema.sensors_8981013 USING btree (((measure_data -> 'IsFailed'::text))) INCLUDE (measure_data, eventdatetime, measure_status)
sensors_8981013 | CREATE UNIQUE INDEX sensors_pkey_8981013 ON citus_split_test_schema.sensors_8981013 USING btree (measureid, eventdatetime, measure_data)
sensors_8981014 | CREATE INDEX hash_index_on_sensors_8981014 ON citus_split_test_schema.sensors_8981014 USING hash (((measure_data -> 'IsFailed'::text)))
sensors_8981014 | CREATE INDEX index_on_sensors_8981014 ON citus_split_test_schema.sensors_8981014 USING btree (lower((measureid)::text))
sensors_8981014 | CREATE INDEX index_with_include_on_sensors_8981014 ON citus_split_test_schema.sensors_8981014 USING btree (((measure_data -> 'IsFailed'::text))) INCLUDE (measure_data, eventdatetime, measure_status)
sensors_8981014 | CREATE UNIQUE INDEX sensors_pkey_8981014 ON citus_split_test_schema.sensors_8981014 USING btree (measureid, eventdatetime, measure_data)
(8 rows)
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'table_with_index_rep_identity_%' ORDER BY 1,2;
tablename | indexdef
---------------------------------------------------------------------
table_with_index_rep_identity_8981019 | CREATE UNIQUE INDEX uqx_8981019 ON citus_split_test_schema.table_with_index_rep_identity_8981019 USING btree (key)
table_with_index_rep_identity_8981020 | CREATE UNIQUE INDEX uqx_8981020 ON citus_split_test_schema.table_with_index_rep_identity_8981020 USING btree (key)
(2 rows)
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema')
)
ORDER BY stxname ASC;
stxname
---------------------------------------------------------------------
stats_on_sensors
stats_on_sensors_8981013
stats_on_sensors_8981014
(3 rows)
\c - - - :worker_2_port
SET search_path TO "citus_split_test_schema", public, pg_catalog;
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like 'sensors_%'
ORDER BY 1, 2;
relname | Constraint | Definition
---------------------------------------------------------------------
sensors_8981007 | fkey_table_to_dist_8981007 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8981009(measureid)
sensors_8981008 | fkey_table_to_dist_8981008 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8981010(measureid)
sensors_8981015 | fkey_table_to_dist_8981015 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8981018(measureid)
(3 rows)
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'sensors_%' ORDER BY 1,2;
tablename | indexdef
---------------------------------------------------------------------
sensors_8981007 | CREATE INDEX hash_index_on_sensors_8981007 ON citus_split_test_schema.sensors_8981007 USING hash (((measure_data -> 'IsFailed'::text)))
sensors_8981007 | CREATE INDEX index_on_sensors_8981007 ON citus_split_test_schema.sensors_8981007 USING btree (lower((measureid)::text))
sensors_8981007 | CREATE INDEX index_with_include_on_sensors_8981007 ON citus_split_test_schema.sensors_8981007 USING btree (((measure_data -> 'IsFailed'::text))) INCLUDE (measure_data, eventdatetime, measure_status)
sensors_8981007 | CREATE UNIQUE INDEX sensors_pkey_8981007 ON citus_split_test_schema.sensors_8981007 USING btree (measureid, eventdatetime, measure_data)
sensors_8981008 | CREATE INDEX hash_index_on_sensors_8981008 ON citus_split_test_schema.sensors_8981008 USING hash (((measure_data -> 'IsFailed'::text)))
sensors_8981008 | CREATE INDEX index_on_sensors_8981008 ON citus_split_test_schema.sensors_8981008 USING btree (lower((measureid)::text))
sensors_8981008 | CREATE INDEX index_with_include_on_sensors_8981008 ON citus_split_test_schema.sensors_8981008 USING btree (((measure_data -> 'IsFailed'::text))) INCLUDE (measure_data, eventdatetime, measure_status)
sensors_8981008 | CREATE UNIQUE INDEX sensors_pkey_8981008 ON citus_split_test_schema.sensors_8981008 USING btree (measureid, eventdatetime, measure_data)
sensors_8981015 | CREATE INDEX hash_index_on_sensors_8981015 ON citus_split_test_schema.sensors_8981015 USING hash (((measure_data -> 'IsFailed'::text)))
sensors_8981015 | CREATE INDEX index_on_sensors_8981015 ON citus_split_test_schema.sensors_8981015 USING btree (lower((measureid)::text))
sensors_8981015 | CREATE INDEX index_with_include_on_sensors_8981015 ON citus_split_test_schema.sensors_8981015 USING btree (((measure_data -> 'IsFailed'::text))) INCLUDE (measure_data, eventdatetime, measure_status)
sensors_8981015 | CREATE UNIQUE INDEX sensors_pkey_8981015 ON citus_split_test_schema.sensors_8981015 USING btree (measureid, eventdatetime, measure_data)
(12 rows)
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'table_with_index_rep_identity_%' ORDER BY 1,2;
tablename | indexdef
---------------------------------------------------------------------
table_with_index_rep_identity_8981011 | CREATE UNIQUE INDEX uqx_8981011 ON citus_split_test_schema.table_with_index_rep_identity_8981011 USING btree (key)
table_with_index_rep_identity_8981012 | CREATE UNIQUE INDEX uqx_8981012 ON citus_split_test_schema.table_with_index_rep_identity_8981012 USING btree (key)
table_with_index_rep_identity_8981021 | CREATE UNIQUE INDEX uqx_8981021 ON citus_split_test_schema.table_with_index_rep_identity_8981021 USING btree (key)
(3 rows)
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema')
)
ORDER BY stxname ASC;
stxname
---------------------------------------------------------------------
stats_on_sensors
stats_on_sensors_8981007
stats_on_sensors_8981008
stats_on_sensors_8981015
(4 rows)
-- END : Display current state
-- BEGIN: Should be able to change/drop constraints
\c - postgres - :master_port
SET ROLE test_shard_split_role;
SET search_path TO "citus_split_test_schema";
ALTER INDEX index_on_sensors RENAME TO index_on_sensors_renamed;
ALTER INDEX index_on_sensors_renamed ALTER COLUMN 1 SET STATISTICS 200;
DROP STATISTICS stats_on_sensors;
DROP INDEX index_on_sensors_renamed;
ALTER TABLE sensors DROP CONSTRAINT fkey_table_to_dist;
-- END: Should be able to change/drop constraints
-- BEGIN: Split second time on another schema
SET search_path TO public;
SET citus.next_shard_id TO 8981031;
SELECT pg_catalog.citus_split_shard_by_split_points(
8981007,
ARRAY['-2100000000'],
ARRAY[:worker_1_node, :worker_2_node],
'force_logical');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
SET search_path TO "citus_split_test_schema";
SELECT shard.shardid, logicalrelid, shardminvalue, shardmaxvalue, nodename, nodeport
FROM pg_dist_shard AS shard
INNER JOIN pg_dist_placement placement ON shard.shardid = placement.shardid
INNER JOIN pg_dist_node node ON placement.groupid = node.groupid
INNER JOIN pg_catalog.pg_class cls ON shard.logicalrelid = cls.oid
WHERE node.noderole = 'primary' AND (logicalrelid = 'sensors'::regclass OR logicalrelid = 'colocated_dist_table'::regclass OR logicalrelid = 'table_with_index_rep_identity'::regclass)
ORDER BY logicalrelid, shardminvalue::BIGINT;
shardid | logicalrelid | shardminvalue | shardmaxvalue | nodename | nodeport
---------------------------------------------------------------------
8981031 | sensors | -2147483648 | -2100000000 | localhost | 57637
8981032 | sensors | -2099999999 | -1073741824 | localhost | 57638
8981008 | sensors | -1073741823 | -1 | localhost | 57638
8981013 | sensors | 0 | 536870911 | localhost | 57637
8981014 | sensors | 536870912 | 1610612735 | localhost | 57637
8981015 | sensors | 1610612736 | 2147483647 | localhost | 57638
8981033 | colocated_dist_table | -2147483648 | -2100000000 | localhost | 57637
8981034 | colocated_dist_table | -2099999999 | -1073741824 | localhost | 57638
8981010 | colocated_dist_table | -1073741823 | -1 | localhost | 57638
8981016 | colocated_dist_table | 0 | 536870911 | localhost | 57637
8981017 | colocated_dist_table | 536870912 | 1610612735 | localhost | 57637
8981018 | colocated_dist_table | 1610612736 | 2147483647 | localhost | 57638
8981035 | table_with_index_rep_identity | -2147483648 | -2100000000 | localhost | 57637
8981036 | table_with_index_rep_identity | -2099999999 | -1073741824 | localhost | 57638
8981012 | table_with_index_rep_identity | -1073741823 | -1 | localhost | 57638
8981019 | table_with_index_rep_identity | 0 | 536870911 | localhost | 57637
8981020 | table_with_index_rep_identity | 536870912 | 1610612735 | localhost | 57637
8981021 | table_with_index_rep_identity | 1610612736 | 2147483647 | localhost | 57638
(18 rows)
-- END: Split second time on another schema
-- BEGIN: Validate Data Count
SELECT COUNT(*) FROM sensors;
count
---------------------------------------------------------------------
1001
(1 row)
SELECT COUNT(*) FROM reference_table;
count
---------------------------------------------------------------------
1001
(1 row)
SELECT COUNT(*) FROM colocated_dist_table;
count
---------------------------------------------------------------------
1001
(1 row)
-- END: Validate Data Count
--BEGIN : Cleanup
\c - postgres - :master_port
DROP SCHEMA "citus_split_test_schema" CASCADE;
NOTICE: drop cascades to 4 other objects
DETAIL: drop cascades to table citus_split_test_schema.sensors
drop cascades to table citus_split_test_schema.reference_table
drop cascades to table citus_split_test_schema.colocated_dist_table
drop cascades to table citus_split_test_schema.table_with_index_rep_identity
--END : Cleanup

View File

@ -32,19 +32,7 @@ SELECT create_distributed_table('table_to_split','id');
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
-- UDF fails for any other shard_transfer_mode other than block_writes.
SELECT citus_split_shard_by_split_points(
49761302,
ARRAY['50'],
ARRAY[101, 201],
'auto');
ERROR: Shard Tranfer mode: 'auto' is not supported. Please use 'block_writes' instead.
SELECT citus_split_shard_by_split_points(
49761302,
ARRAY['50'],
ARRAY[101, 201],
'force_logical');
ERROR: Shard Tranfer mode: 'force_logical' is not supported. Please use 'block_writes' instead.
-- UDF fails for any other shard_transfer_mode other than block_writes/force_logical/auto.
SELECT citus_split_shard_by_split_points(
49761302,
ARRAY['50'],

View File

@ -0,0 +1,775 @@
unused step name: s2-select
Parsed test spec with 4 sessions
starting permutation: s1-load-cache s2-print-cluster s3-acquire-advisory-lock s1-begin s2-begin s1-non-blocking-shard-split s2-insert s2-end s2-print-cluster s3-release-advisory-lock s1-end s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-load-cache:
-- Indirect way to load cache.
TRUNCATE to_split_table;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 0
(1 row)
id|value
---------------------------------------------------------------------
(0 rows)
step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1-non-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500001,
ARRAY['-1073741824'],
ARRAY[2, 2],
'force_logical');
<waiting ...>
step s2-insert:
SELECT get_shard_id_for_distribution_column('to_split_table', 123456789);
INSERT INTO to_split_table VALUES (123456789, 1);
get_shard_id_for_distribution_column
---------------------------------------------------------------------
1500001
(1 row)
step s2-end:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 1
(1 row)
id|value
---------------------------------------------------------------------
123456789| 1
(1 row)
step s3-release-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s1-non-blocking-shard-split: <... completed>
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-end:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57638|1500002|t | 0
57638|1500003|t | 1
(2 rows)
id|value
---------------------------------------------------------------------
123456789| 1
(1 row)
starting permutation: s1-load-cache s2-insert s2-print-cluster s3-acquire-advisory-lock s1-begin s1-non-blocking-shard-split s2-update s3-release-advisory-lock s1-end s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-load-cache:
-- Indirect way to load cache.
TRUNCATE to_split_table;
step s2-insert:
SELECT get_shard_id_for_distribution_column('to_split_table', 123456789);
INSERT INTO to_split_table VALUES (123456789, 1);
get_shard_id_for_distribution_column
---------------------------------------------------------------------
1500001
(1 row)
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 1
(1 row)
id|value
---------------------------------------------------------------------
123456789| 1
(1 row)
step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
step s1-non-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500001,
ARRAY['-1073741824'],
ARRAY[2, 2],
'force_logical');
<waiting ...>
step s2-update:
UPDATE to_split_table SET value = 111 WHERE id = 123456789;
step s3-release-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s1-non-blocking-shard-split: <... completed>
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-end:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57638|1500002|t | 0
57638|1500003|t | 1
(2 rows)
id|value
---------------------------------------------------------------------
123456789| 111
(1 row)
starting permutation: s1-load-cache s2-insert s2-print-cluster s3-acquire-advisory-lock s1-begin s1-non-blocking-shard-split s2-delete s3-release-advisory-lock s1-end s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-load-cache:
-- Indirect way to load cache.
TRUNCATE to_split_table;
step s2-insert:
SELECT get_shard_id_for_distribution_column('to_split_table', 123456789);
INSERT INTO to_split_table VALUES (123456789, 1);
get_shard_id_for_distribution_column
---------------------------------------------------------------------
1500001
(1 row)
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 1
(1 row)
id|value
---------------------------------------------------------------------
123456789| 1
(1 row)
step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
step s1-non-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500001,
ARRAY['-1073741824'],
ARRAY[2, 2],
'force_logical');
<waiting ...>
step s2-delete:
DELETE FROM to_split_table WHERE id = 123456789;
step s3-release-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s1-non-blocking-shard-split: <... completed>
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-end:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57638|1500002|t | 0
57638|1500003|t | 0
(2 rows)
id|value
---------------------------------------------------------------------
(0 rows)
starting permutation: s1-load-cache s2-print-cluster s3-acquire-advisory-lock s1-begin s2-begin s1-non-blocking-shard-split s2-insert s2-end s2-print-cluster s4-begin s3-release-advisory-lock s4-insert s1-end s4-end s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-load-cache:
-- Indirect way to load cache.
TRUNCATE to_split_table;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 0
(1 row)
id|value
---------------------------------------------------------------------
(0 rows)
step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1-non-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500001,
ARRAY['-1073741824'],
ARRAY[2, 2],
'force_logical');
<waiting ...>
step s2-insert:
SELECT get_shard_id_for_distribution_column('to_split_table', 123456789);
INSERT INTO to_split_table VALUES (123456789, 1);
get_shard_id_for_distribution_column
---------------------------------------------------------------------
1500001
(1 row)
step s2-end:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 1
(1 row)
id|value
---------------------------------------------------------------------
123456789| 1
(1 row)
step s4-begin:
BEGIN;
step s3-release-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s1-non-blocking-shard-split: <... completed>
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s4-insert:
INSERT INTO to_split_table VALUES (900, 1);
<waiting ...>
step s1-end:
COMMIT;
step s4-insert: <... completed>
ERROR: could not find valid entry for shard xxxxx
step s4-end:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57638|1500002|t | 0
57638|1500003|t | 1
(2 rows)
id|value
---------------------------------------------------------------------
123456789| 1
(1 row)
starting permutation: s2-print-cluster s3-acquire-advisory-lock s1-begin s2-begin s1-non-blocking-shard-split s2-insert s2-end s2-print-cluster s3-release-advisory-lock s1-end s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 0
(1 row)
id|value
---------------------------------------------------------------------
(0 rows)
step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1-non-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500001,
ARRAY['-1073741824'],
ARRAY[2, 2],
'force_logical');
<waiting ...>
step s2-insert:
SELECT get_shard_id_for_distribution_column('to_split_table', 123456789);
INSERT INTO to_split_table VALUES (123456789, 1);
get_shard_id_for_distribution_column
---------------------------------------------------------------------
1500001
(1 row)
step s2-end:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 1
(1 row)
id|value
---------------------------------------------------------------------
123456789| 1
(1 row)
step s3-release-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s1-non-blocking-shard-split: <... completed>
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-end:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57638|1500002|t | 0
57638|1500003|t | 1
(2 rows)
id|value
---------------------------------------------------------------------
123456789| 1
(1 row)
starting permutation: s2-insert s2-print-cluster s3-acquire-advisory-lock s1-begin s1-non-blocking-shard-split s2-update s3-release-advisory-lock s1-end s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s2-insert:
SELECT get_shard_id_for_distribution_column('to_split_table', 123456789);
INSERT INTO to_split_table VALUES (123456789, 1);
get_shard_id_for_distribution_column
---------------------------------------------------------------------
1500001
(1 row)
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 1
(1 row)
id|value
---------------------------------------------------------------------
123456789| 1
(1 row)
step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
step s1-non-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500001,
ARRAY['-1073741824'],
ARRAY[2, 2],
'force_logical');
<waiting ...>
step s2-update:
UPDATE to_split_table SET value = 111 WHERE id = 123456789;
step s3-release-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s1-non-blocking-shard-split: <... completed>
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-end:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57638|1500002|t | 0
57638|1500003|t | 1
(2 rows)
id|value
---------------------------------------------------------------------
123456789| 111
(1 row)
starting permutation: s2-insert s2-print-cluster s3-acquire-advisory-lock s1-begin s1-non-blocking-shard-split s2-delete s3-release-advisory-lock s1-end s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s2-insert:
SELECT get_shard_id_for_distribution_column('to_split_table', 123456789);
INSERT INTO to_split_table VALUES (123456789, 1);
get_shard_id_for_distribution_column
---------------------------------------------------------------------
1500001
(1 row)
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 1
(1 row)
id|value
---------------------------------------------------------------------
123456789| 1
(1 row)
step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
step s1-non-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500001,
ARRAY['-1073741824'],
ARRAY[2, 2],
'force_logical');
<waiting ...>
step s2-delete:
DELETE FROM to_split_table WHERE id = 123456789;
step s3-release-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s1-non-blocking-shard-split: <... completed>
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-end:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57638|1500002|t | 0
57638|1500003|t | 0
(2 rows)
id|value
---------------------------------------------------------------------
(0 rows)

View File

@ -0,0 +1,250 @@
unused step name: s1-copy
unused step name: s1-ddl
Parsed test spec with 3 sessions
starting permutation: s2-add-fkey s3-acquire-advisory-lock s1-begin s2-begin s2-non-blocking-shard-split s1-insert s1-commit s3-release-advisory-lock s2-commit s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s2-add-fkey:
ALTER TABLE table_to_split ADD CONSTRAINT fkey_const FOREIGN KEY(value) REFERENCES reference_table(id);
step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-non-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500002,
ARRAY['-1073741824'],
ARRAY[1, 2],
'force_logical');
<waiting ...>
step s1-insert:
INSERT INTO reference_table VALUES (5, 10);
step s1-commit:
COMMIT;
step s3-release-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s2-non-blocking-shard-split: <... completed>
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s2-commit:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('table_to_split', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows of table_to_split
SELECT id, value FROM table_to_split ORDER BY id, value;
-- rows of reference table
SELECT * FROM reference_table;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500004|t | 0
57638|1500003|t | 0
57638|1500005|t | 0
(3 rows)
id|value
---------------------------------------------------------------------
(0 rows)
id|value
---------------------------------------------------------------------
5| 10
(1 row)
starting permutation: s2-add-fkey s3-acquire-advisory-lock s1-begin s2-begin s2-non-blocking-shard-split s1-update s1-commit s3-release-advisory-lock s2-commit s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s2-add-fkey:
ALTER TABLE table_to_split ADD CONSTRAINT fkey_const FOREIGN KEY(value) REFERENCES reference_table(id);
step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-non-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500002,
ARRAY['-1073741824'],
ARRAY[1, 2],
'force_logical');
<waiting ...>
step s1-update:
UPDATE reference_table SET value = 5 WHERE id = 5;
step s1-commit:
COMMIT;
step s3-release-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s2-non-blocking-shard-split: <... completed>
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s2-commit:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('table_to_split', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows of table_to_split
SELECT id, value FROM table_to_split ORDER BY id, value;
-- rows of reference table
SELECT * FROM reference_table;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500004|t | 0
57638|1500003|t | 0
57638|1500005|t | 0
(3 rows)
id|value
---------------------------------------------------------------------
(0 rows)
id|value
---------------------------------------------------------------------
(0 rows)
starting permutation: s2-add-fkey s3-acquire-advisory-lock s1-begin s2-begin s2-non-blocking-shard-split s1-delete s1-commit s3-release-advisory-lock s2-commit s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s2-add-fkey:
ALTER TABLE table_to_split ADD CONSTRAINT fkey_const FOREIGN KEY(value) REFERENCES reference_table(id);
step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-non-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500002,
ARRAY['-1073741824'],
ARRAY[1, 2],
'force_logical');
<waiting ...>
step s1-delete:
DELETE FROM reference_table WHERE id = 5;
step s1-commit:
COMMIT;
step s3-release-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s2-non-blocking-shard-split: <... completed>
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s2-commit:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('table_to_split', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows of table_to_split
SELECT id, value FROM table_to_split ORDER BY id, value;
-- rows of reference table
SELECT * FROM reference_table;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500004|t | 0
57638|1500003|t | 0
57638|1500005|t | 0
(3 rows)
id|value
---------------------------------------------------------------------
(0 rows)
id|value
---------------------------------------------------------------------
(0 rows)

View File

@ -0,0 +1,640 @@
unused step name: s2-select
Parsed test spec with 3 sessions
starting permutation: s1-load-cache s2-print-cluster s3-acquire-advisory-lock s1-begin s2-begin s1-non-blocking-shard-split s2-insert s2-end s2-print-cluster s3-release-advisory-lock s1-end s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-load-cache:
-- Indirect way to load cache.
TRUNCATE to_split_table;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 0
(1 row)
id|value
---------------------------------------------------------------------
(0 rows)
step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1-non-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500001,
ARRAY['-1073741824'],
ARRAY[2, 2],
'force_logical');
<waiting ...>
step s2-insert:
SELECT get_shard_id_for_distribution_column('to_split_table', 123456789);
INSERT INTO to_split_table VALUES (123456789, 1);
get_shard_id_for_distribution_column
---------------------------------------------------------------------
1500001
(1 row)
step s2-end:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 1
(1 row)
id|value
---------------------------------------------------------------------
123456789| 1
(1 row)
step s3-release-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s1-non-blocking-shard-split: <... completed>
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-end:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57638|1500002|t | 0
57638|1500003|t | 1
(2 rows)
id|value
---------------------------------------------------------------------
123456789| 1
(1 row)
starting permutation: s1-load-cache s2-insert s2-print-cluster s3-acquire-advisory-lock s1-begin s1-non-blocking-shard-split s2-update s3-release-advisory-lock s1-end s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-load-cache:
-- Indirect way to load cache.
TRUNCATE to_split_table;
step s2-insert:
SELECT get_shard_id_for_distribution_column('to_split_table', 123456789);
INSERT INTO to_split_table VALUES (123456789, 1);
get_shard_id_for_distribution_column
---------------------------------------------------------------------
1500001
(1 row)
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 1
(1 row)
id|value
---------------------------------------------------------------------
123456789| 1
(1 row)
step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
step s1-non-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500001,
ARRAY['-1073741824'],
ARRAY[2, 2],
'force_logical');
<waiting ...>
step s2-update:
UPDATE to_split_table SET value = 111 WHERE id = 123456789;
step s3-release-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s1-non-blocking-shard-split: <... completed>
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-end:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57638|1500002|t | 0
57638|1500003|t | 1
(2 rows)
id|value
---------------------------------------------------------------------
123456789| 111
(1 row)
starting permutation: s1-load-cache s2-insert s2-print-cluster s3-acquire-advisory-lock s1-begin s1-non-blocking-shard-split s2-delete s3-release-advisory-lock s1-end s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-load-cache:
-- Indirect way to load cache.
TRUNCATE to_split_table;
step s2-insert:
SELECT get_shard_id_for_distribution_column('to_split_table', 123456789);
INSERT INTO to_split_table VALUES (123456789, 1);
get_shard_id_for_distribution_column
---------------------------------------------------------------------
1500001
(1 row)
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 1
(1 row)
id|value
---------------------------------------------------------------------
123456789| 1
(1 row)
step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
step s1-non-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500001,
ARRAY['-1073741824'],
ARRAY[2, 2],
'force_logical');
<waiting ...>
step s2-delete:
DELETE FROM to_split_table WHERE id = 123456789;
step s3-release-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s1-non-blocking-shard-split: <... completed>
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-end:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57638|1500002|t | 0
57638|1500003|t | 0
(2 rows)
id|value
---------------------------------------------------------------------
(0 rows)
starting permutation: s2-print-cluster s3-acquire-advisory-lock s1-begin s2-begin s1-non-blocking-shard-split s2-insert s2-end s2-print-cluster s3-release-advisory-lock s1-end s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 0
(1 row)
id|value
---------------------------------------------------------------------
(0 rows)
step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1-non-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500001,
ARRAY['-1073741824'],
ARRAY[2, 2],
'force_logical');
<waiting ...>
step s2-insert:
SELECT get_shard_id_for_distribution_column('to_split_table', 123456789);
INSERT INTO to_split_table VALUES (123456789, 1);
get_shard_id_for_distribution_column
---------------------------------------------------------------------
1500001
(1 row)
step s2-end:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 1
(1 row)
id|value
---------------------------------------------------------------------
123456789| 1
(1 row)
step s3-release-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s1-non-blocking-shard-split: <... completed>
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-end:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57638|1500002|t | 0
57638|1500003|t | 1
(2 rows)
id|value
---------------------------------------------------------------------
123456789| 1
(1 row)
starting permutation: s2-insert s2-print-cluster s3-acquire-advisory-lock s1-begin s1-non-blocking-shard-split s2-update s3-release-advisory-lock s1-end s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s2-insert:
SELECT get_shard_id_for_distribution_column('to_split_table', 123456789);
INSERT INTO to_split_table VALUES (123456789, 1);
get_shard_id_for_distribution_column
---------------------------------------------------------------------
1500001
(1 row)
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 1
(1 row)
id|value
---------------------------------------------------------------------
123456789| 1
(1 row)
step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
step s1-non-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500001,
ARRAY['-1073741824'],
ARRAY[2, 2],
'force_logical');
<waiting ...>
step s2-update:
UPDATE to_split_table SET value = 111 WHERE id = 123456789;
step s3-release-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s1-non-blocking-shard-split: <... completed>
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-end:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57638|1500002|t | 0
57638|1500003|t | 1
(2 rows)
id|value
---------------------------------------------------------------------
123456789| 111
(1 row)
starting permutation: s2-insert s2-print-cluster s3-acquire-advisory-lock s1-begin s1-non-blocking-shard-split s2-delete s3-release-advisory-lock s1-end s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s2-insert:
SELECT get_shard_id_for_distribution_column('to_split_table', 123456789);
INSERT INTO to_split_table VALUES (123456789, 1);
get_shard_id_for_distribution_column
---------------------------------------------------------------------
1500001
(1 row)
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 1
(1 row)
id|value
---------------------------------------------------------------------
123456789| 1
(1 row)
step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
step s1-non-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500001,
ARRAY['-1073741824'],
ARRAY[2, 2],
'force_logical');
<waiting ...>
step s2-delete:
DELETE FROM to_split_table WHERE id = 123456789;
step s3-release-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s1-non-blocking-shard-split: <... completed>
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-end:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57638|1500002|t | 0
57638|1500003|t | 0
(2 rows)
id|value
---------------------------------------------------------------------
(0 rows)

View File

@ -1099,9 +1099,12 @@ SELECT * FROM multi_extension.print_extension_changes();
| function citus_split_shard_by_split_points(bigint,text[],integer[],citus.shard_transfer_mode) void
| function worker_copy_table_to_node(regclass,integer) void
| function worker_split_copy(bigint,split_copy_info[]) void
| function worker_split_shard_replication_setup(split_shard_info[]) SETOF replication_slot_info
| type replication_slot_info
| type split_copy_info
| type split_shard_info
| view citus_locks
(27 rows)
(30 rows)
DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff;
-- show running version

View File

@ -0,0 +1,233 @@
\c - - - :master_port
CREATE USER myuser;
CREATE USER admin_user;
GRANT USAGE, CREATE ON SCHEMA split_shard_replication_setup_schema, public to myuser;
GRANT USAGE, CREATE ON SCHEMA split_shard_replication_setup_schema, public to admin_user;
SET search_path TO split_shard_replication_setup_schema;
SET citus.shard_replication_factor TO 1;
SET citus.shard_count TO 1;
SET citus.next_shard_id TO 4;
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
\c - myuser - -
SET search_path TO split_shard_replication_setup_schema;
SET citus.shard_replication_factor TO 1;
SET citus.shard_count TO 1;
SET citus.next_shard_id TO 4;
CREATE TABLE table_first (id bigserial PRIMARY KEY, value char);
SELECT create_distributed_table('table_first','id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
\c - admin_user - -
SET search_path TO split_shard_replication_setup_schema;
SET citus.next_shard_id TO 7;
SET citus.shard_count TO 1;
CREATE TABLE table_second (id bigserial PRIMARY KEY, value char);
SELECT create_distributed_table('table_second', 'id', colocate_with => 'table_first');
create_distributed_table
---------------------------------------------------------------------
(1 row)
\c - myuser - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
CREATE TABLE table_first_5(id bigserial PRIMARY KEY, value char);
CREATE TABLE table_first_6(id bigserial PRIMARY KEY, value char);
\c - myuser - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
CREATE TABLE table_first_4(id bigserial PRIMARY KEY, value char);
CREATE TABLE table_first_5(id bigserial PRIMARY KEY, value char);
CREATE TABLE table_first_6(id bigserial PRIMARY KEY, value char);
\c - admin_user - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
CREATE TABLE table_second_8(id bigserial PRIMARY KEY, value char);
CREATE TABLE table_second_9(id bigserial PRIMARY KEY, value char);
\c - admin_user - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
CREATE TABLE table_second_7(id bigserial PRIMARY KEY, value char);
CREATE TABLE table_second_8(id bigserial PRIMARY KEY, value char);
CREATE TABLE table_second_9(id bigserial PRIMARY KEY, value char);
--- Test scenario one starts from here
--- 1. table_first and table_second are colocated tables.
--- 2. myuser is the owner table_first and admin_user is the owner of table_second.
--- 3. Shard table_first_4 and table_second_7 are colocated on worker1
--- 4. table_first_4 is split into table_first_5 and table_first_6 with target as worker2
--- 5. table_second_7 is split into table_second_8 and table_second_9 with target as worker2
--- 6. Create two publishers and two subscribers for respective table owners.
--- 7. Insert into table_first_4 and table_second_7 at source worker1
--- 8. Expect the results in child shards on worker2
-- Create publication at worker1
\c - postgres - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
CREATE PUBLICATION pub1 FOR TABLE table_first_4, table_first_5, table_first_6;
CREATE PUBLICATION pub2 FOR TABLE table_second_7, table_second_8, table_second_9;
SELECT count(*) FROM pg_catalog.worker_split_shard_replication_setup(ARRAY[
ROW(4, 'id', 5, '-2147483648', '-1', :worker_2_node)::pg_catalog.split_shard_info,
ROW(4, 'id', 6, '0', '2147483647', :worker_2_node)::pg_catalog.split_shard_info,
ROW(7, 'id', 8, '-2147483648', '-1', :worker_2_node)::pg_catalog.split_shard_info,
ROW(7, 'id', 9, '0', '2147483647', :worker_2_node)::pg_catalog.split_shard_info
]);
WARNING: Previous split shard worflow was not successfully and could not complete the cleanup phase. Continuing with the current split shard workflow.
count
---------------------------------------------------------------------
2
(1 row)
SELECT relowner AS table_owner_one FROM pg_class WHERE relname='table_first' \gset
SELECT relowner AS table_owner_two FROM pg_class WHERE relname='table_second' \gset
SELECT slot_name AS slot_for_first_owner FROM pg_create_logical_replication_slot(FORMAT('citus_shard_split_%s_%s', :worker_2_node, :table_owner_one), 'citus') \gset
SELECT slot_name AS slot_for_second_owner FROM pg_create_logical_replication_slot(FORMAT('citus_shard_split_%s_%s', :worker_2_node, :table_owner_two), 'citus') \gset
-- Create subscription at worker2 with copy_data to 'false'
\c - postgres - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
SET client_min_messages TO WARNING;
CREATE SUBSCRIPTION sub1
CONNECTION 'host=localhost port=xxxxx user=postgres dbname=regression'
PUBLICATION pub1
WITH (
create_slot=false,
enabled=true,
slot_name=:slot_for_first_owner,
copy_data=false);
\c - myuser - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
INSERT INTO table_first_4 VALUES(100, 'a');
INSERT INTO table_first_4 VALUES(400, 'a');
INSERT INTO table_first_4 VALUES(500, 'a');
SELECT wait_for_expected_rowcount_at_table('table_first_4', 3);
wait_for_expected_rowcount_at_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM table_first_4;
id | value
---------------------------------------------------------------------
100 | a
400 | a
500 | a
(3 rows)
\c - admin_user - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
INSERT INTO table_second_7 VALUES(100, 'a');
INSERT INTO table_second_7 VALUES(400, 'a');
SELECT wait_for_expected_rowcount_at_table('table_second_7', 2);
wait_for_expected_rowcount_at_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM table_second_7;
id | value
---------------------------------------------------------------------
100 | a
400 | a
(2 rows)
-- expect data in table_first_5/6
\c - myuser - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
SELECT * FROM table_first_4;
id | value
---------------------------------------------------------------------
(0 rows)
SELECT wait_for_expected_rowcount_at_table('table_first_5', 1);
wait_for_expected_rowcount_at_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM table_first_5;
id | value
---------------------------------------------------------------------
400 | a
(1 row)
SELECT wait_for_expected_rowcount_at_table('table_first_6', 2);
wait_for_expected_rowcount_at_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM table_first_6;
id | value
---------------------------------------------------------------------
100 | a
500 | a
(2 rows)
-- should have zero rows in all the below tables as the subscription is not yet created for admin_user
\c - admin_user - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
SELECT * FROM table_second_7;
id | value
---------------------------------------------------------------------
(0 rows)
SELECT * FROM table_second_8;
id | value
---------------------------------------------------------------------
(0 rows)
SELECT * FROM table_second_9;
id | value
---------------------------------------------------------------------
(0 rows)
\c - postgres - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
SET client_min_messages TO WARNING;
CREATE SUBSCRIPTION sub2
CONNECTION 'host=localhost port=xxxxx user=postgres dbname=regression'
PUBLICATION pub2
WITH (
create_slot=false,
enabled=true,
slot_name=:slot_for_second_owner,
copy_data=false);
-- expect data
\c - admin_user - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
SELECT * FROM table_second_7;
id | value
---------------------------------------------------------------------
(0 rows)
SELECT wait_for_expected_rowcount_at_table('table_second_8', 1);
wait_for_expected_rowcount_at_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM table_second_8;
id | value
---------------------------------------------------------------------
400 | a
(1 row)
SELECT wait_for_expected_rowcount_at_table('table_second_9', 1);
wait_for_expected_rowcount_at_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM table_second_9;
id | value
---------------------------------------------------------------------
100 | a
(1 row)
\c - postgres - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
DROP PUBLICATION pub1;
DROP PUBLICATION pub2;
\c - postgres - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
SET client_min_messages TO ERROR;
DROP SUBSCRIPTION sub1;
DROP SUBSCRIPTION sub2;

View File

@ -0,0 +1,245 @@
CREATE SCHEMA split_shard_replication_setup_schema;
SET search_path TO split_shard_replication_setup_schema;
SET citus.shard_replication_factor TO 1;
SET citus.shard_count TO 1;
SET citus.next_shard_id TO 1;
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
CREATE OR REPLACE FUNCTION wait_for_expected_rowcount_at_table(tableName text, expectedCount integer) RETURNS void AS $$
DECLARE
actualCount integer;
BEGIN
EXECUTE FORMAT('SELECT COUNT(*) FROM %s', tableName) INTO actualCount;
WHILE expectedCount != actualCount LOOP
EXECUTE FORMAT('SELECT COUNT(*) FROM %s', tableName) INTO actualCount;
END LOOP;
END$$ LANGUAGE plpgsql;
CREATE OR REPLACE FUNCTION wait_for_updated_rowcount_at_table(tableName text, expectedCount integer) RETURNS void AS $$
DECLARE
actualCount integer;
BEGIN
EXECUTE FORMAT($query$SELECT COUNT(*) FROM %s WHERE value='b'$query$, tableName) INTO actualCount;
WHILE expectedCount != actualCount LOOP
EXECUTE FORMAT($query$SELECT COUNT(*) FROM %s WHERE value='b'$query$, tableName) INTO actualCount;
END LOOP;
END$$ LANGUAGE plpgsql;
-- Create distributed table (non co-located)
CREATE TABLE table_to_split (id bigserial PRIMARY KEY, value char);
SELECT create_distributed_table('table_to_split','id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- Test scenario one starts from here
-- 1. table_to_split is a citus distributed table
-- 2. Shard table_to_split_1 is located on worker1.
-- 3. table_to_split_1 is split into table_to_split_2 and table_to_split_3.
-- table_to_split_2/3 are located on worker2
-- 4. execute UDF split_shard_replication_setup on worker1 with below
-- params:
-- worker_split_shard_replication_setup
-- (
-- ARRAY[
-- ROW(1 /*source shardId */, 2 /* new shardId */,-2147483648 /* minHashValue */, -1 /* maxHasValue */ , 18 /* nodeId where new shard is placed */ ),
-- ROW(1, 3 , 0 , 2147483647, 18 )
-- ]
-- );
-- 5. Create Replication slot with 'citus'
-- 6. Setup Pub/Sub
-- 7. Insert into table_to_split_1 at source worker1
-- 8. Expect the results in either table_to_split_2 or table_to_split_3 at worker2
\c - - - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
CREATE TABLE table_to_split_1(id bigserial PRIMARY KEY, value char);
CREATE TABLE table_to_split_2(id bigserial PRIMARY KEY, value char);
CREATE TABLE table_to_split_3(id bigserial PRIMARY KEY, value char);
-- Create dummy shard tables(table_to_split_2/3b) at worker1
-- This is needed for Pub/Sub framework to work.
\c - - - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
CREATE TABLE table_to_split_2(id bigserial PRIMARY KEY, value char);
CREATE TABLE table_to_split_3(id bigserial PRIMARY KEY, value char);
-- Create publication at worker1
CREATE PUBLICATION pub1 FOR TABLE table_to_split_1, table_to_split_2, table_to_split_3;
SELECT count(*) FROM pg_catalog.worker_split_shard_replication_setup(ARRAY[
ROW(1, 'id', 2, '-2147483648', '-1', :worker_2_node)::pg_catalog.split_shard_info,
ROW(1, 'id', 3, '0', '2147483647', :worker_2_node)::pg_catalog.split_shard_info
]);
count
---------------------------------------------------------------------
1
(1 row)
SELECT slot_name FROM pg_create_logical_replication_slot(FORMAT('citus_shard_split_%s_10', :worker_2_node), 'citus') \gset
-- Create subscription at worker2 with copy_data to 'false' and derived replication slot name
\c - - - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
CREATE SUBSCRIPTION sub1
CONNECTION 'host=localhost port=xxxxx user=postgres dbname=regression'
PUBLICATION pub1
WITH (
create_slot=false,
enabled=true,
slot_name=:slot_name,
copy_data=false);
-- No data is present at this moment in all the below tables at worker2
SELECT * FROM table_to_split_1;
id | value
---------------------------------------------------------------------
(0 rows)
SELECT * FROM table_to_split_2;
id | value
---------------------------------------------------------------------
(0 rows)
SELECT * FROM table_to_split_3;
id | value
---------------------------------------------------------------------
(0 rows)
-- Insert data in table_to_split_1 at worker1
\c - - - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
INSERT INTO table_to_split_1 values(100, 'a');
INSERT INTO table_to_split_1 values(400, 'a');
INSERT INTO table_to_split_1 values(500, 'a');
SELECT * FROM table_to_split_1;
id | value
---------------------------------------------------------------------
100 | a
400 | a
500 | a
(3 rows)
SELECT * FROM table_to_split_2;
id | value
---------------------------------------------------------------------
(0 rows)
SELECT * FROM table_to_split_3;
id | value
---------------------------------------------------------------------
(0 rows)
-- Expect data to be present in shard xxxxx and shard xxxxx based on the hash value.
\c - - - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
SELECT * FROM table_to_split_1; -- should alwasy have zero rows
id | value
---------------------------------------------------------------------
(0 rows)
SELECT wait_for_expected_rowcount_at_table('table_to_split_2', 1);
wait_for_expected_rowcount_at_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM table_to_split_2;
id | value
---------------------------------------------------------------------
400 | a
(1 row)
SELECT wait_for_expected_rowcount_at_table('table_to_split_3', 2);
wait_for_expected_rowcount_at_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM table_to_split_3;
id | value
---------------------------------------------------------------------
100 | a
500 | a
(2 rows)
-- UPDATE data of table_to_split_1 from worker1
\c - - - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
UPDATE table_to_split_1 SET value='b' WHERE id = 100;
UPDATE table_to_split_1 SET value='b' WHERE id = 400;
UPDATE table_to_split_1 SET value='b' WHERE id = 500;
\c - - - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
SELECT * FROM table_to_split_1;
id | value
---------------------------------------------------------------------
(0 rows)
-- Value should be updated in table_to_split_2;
SELECT wait_for_updated_rowcount_at_table('table_to_split_2', 1);
wait_for_updated_rowcount_at_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM table_to_split_2;
id | value
---------------------------------------------------------------------
400 | b
(1 row)
-- Value should be updated in table_to_split_3;
SELECT wait_for_updated_rowcount_at_table('table_to_split_3', 2);
wait_for_updated_rowcount_at_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM table_to_split_3;
id | value
---------------------------------------------------------------------
100 | b
500 | b
(2 rows)
\c - - - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
DELETE FROM table_to_split_1;
-- Child shard rows should be deleted
\c - - - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
SELECT wait_for_expected_rowcount_at_table('table_to_split_1', 0);
wait_for_expected_rowcount_at_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM table_to_split_1;
id | value
---------------------------------------------------------------------
(0 rows)
SELECT wait_for_expected_rowcount_at_table('table_to_split_2', 0);
wait_for_expected_rowcount_at_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM table_to_split_2;
id | value
---------------------------------------------------------------------
(0 rows)
SELECT wait_for_expected_rowcount_at_table('table_to_split_3', 0);
wait_for_expected_rowcount_at_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM table_to_split_3;
id | value
---------------------------------------------------------------------
(0 rows)
-- drop publication from worker1
\c - - - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
DROP PUBLICATION pub1;
\c - - - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
SET client_min_messages TO ERROR;
DROP SUBSCRIPTION sub1;

View File

@ -0,0 +1,107 @@
-- Test scenario (parent shard and child shards are located on same machine)
-- 1. table_to_split_1 is split into table_to_split_2 and table_to_split_3.
-- 2. table_to_split_1 is located on worker1.
-- 3. table_to_split_2 and table_to_split_3 are located on worker1
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
\c - - - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
SET client_min_messages TO ERROR;
-- Create publication at worker1
CREATE PUBLICATION pub1 for table table_to_split_1, table_to_split_2, table_to_split_3;
-- Worker1 is target for table_to_split_2 and table_to_split_3
SELECT count(*) FROM pg_catalog.worker_split_shard_replication_setup(ARRAY[
ROW(1, 'id', 2, '-2147483648', '-1', :worker_1_node)::pg_catalog.split_shard_info,
ROW(1, 'id', 3, '0', '2147483647', :worker_1_node)::pg_catalog.split_shard_info
]);
count
---------------------------------------------------------------------
1
(1 row)
SELECT slot_name AS local_slot FROM pg_create_logical_replication_slot(FORMAT('citus_shard_split_%s_10', :worker_1_node), 'citus') \gset
-- Create subscription at worker1 with copy_data to 'false' a
BEGIN;
CREATE SUBSCRIPTION local_subscription
CONNECTION 'host=localhost port=xxxxx user=postgres dbname=regression'
PUBLICATION pub1
WITH (
create_slot=false,
enabled=true,
slot_name=:local_slot,
copy_data=false);
COMMIT;
INSERT INTO table_to_split_1 VALUES(100, 'a');
INSERT INTO table_to_split_1 VALUES(400, 'a');
INSERT INTO table_to_split_1 VALUES(500, 'a');
-- expect data to present in table_to_split_2/3 on worker1
SELECT * FROM table_to_split_1;
id | value
---------------------------------------------------------------------
100 | a
400 | a
500 | a
(3 rows)
SELECT wait_for_expected_rowcount_at_table('table_to_split_2', 1);
wait_for_expected_rowcount_at_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM table_to_split_2;
id | value
---------------------------------------------------------------------
400 | a
(1 row)
SELECT wait_for_expected_rowcount_at_table('table_to_split_3', 2);
wait_for_expected_rowcount_at_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM table_to_split_3;
id | value
---------------------------------------------------------------------
100 | a
500 | a
(2 rows)
DELETE FROM table_to_split_1;
SELECT wait_for_expected_rowcount_at_table('table_to_split_1', 0);
wait_for_expected_rowcount_at_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM table_to_split_1;
id | value
---------------------------------------------------------------------
(0 rows)
SELECT wait_for_expected_rowcount_at_table('table_to_split_2', 0);
wait_for_expected_rowcount_at_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM table_to_split_2;
id | value
---------------------------------------------------------------------
(0 rows)
SELECT wait_for_expected_rowcount_at_table('table_to_split_3', 0);
wait_for_expected_rowcount_at_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM table_to_split_3;
id | value
---------------------------------------------------------------------
(0 rows)
-- clean up
DROP SUBSCRIPTION local_subscription;
DROP PUBLICATION pub1;

View File

@ -0,0 +1,157 @@
-- Test scenario (Parent and one child on same node. Other child on different node)
-- 1. table_to_split_1 is split into table_to_split_2 and table_to_split_3.
-- 2. table_to_split_1 is located on worker1.
-- 3. table_to_split_2 is located on worker1 and table_to_split_3 is located on worker2
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
\c - - - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
-- Create publication at worker1
CREATE PUBLICATION pub1 FOR TABLE table_to_split_1, table_to_split_2, table_to_split_3;
SELECT count(*) FROM pg_catalog.worker_split_shard_replication_setup(ARRAY[
ROW(1, 'id', 2, '-2147483648', '-1', :worker_1_node)::pg_catalog.split_shard_info,
ROW(1, 'id', 3, '0', '2147483647', :worker_2_node)::pg_catalog.split_shard_info
]);
WARNING: Previous split shard worflow was not successfully and could not complete the cleanup phase. Continuing with the current split shard workflow.
count
---------------------------------------------------------------------
2
(1 row)
SELECT slot_name AS slot_for_worker1 FROM pg_create_logical_replication_slot(FORMAT('citus_shard_split_%s_10', :worker_1_node), 'citus') \gset
SELECT slot_name AS slot_for_worker2 FROM pg_create_logical_replication_slot(FORMAT('citus_shard_split_%s_10', :worker_2_node), 'citus') \gset
-- Create subscription at worker1 with copy_data to 'false' and 'slot_for_worker1'
CREATE SUBSCRIPTION sub_worker1
CONNECTION 'host=localhost port=xxxxx user=postgres dbname=regression'
PUBLICATION pub1
WITH (
create_slot=false,
enabled=true,
slot_name=:slot_for_worker1,
copy_data=false);
\c - - - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
-- Create subscription at worker2 with copy_data to 'false' and 'slot_for_worker2'
CREATE SUBSCRIPTION sub_worker2
CONNECTION 'host=localhost port=xxxxx user=postgres dbname=regression'
PUBLICATION pub1
WITH (
create_slot=false,
enabled=true,
slot_name=:slot_for_worker2,
copy_data=false);
-- No data is present at this moment in all the below tables at worker2
SELECT * FROM table_to_split_1;
id | value
---------------------------------------------------------------------
(0 rows)
SELECT * FROM table_to_split_2;
id | value
---------------------------------------------------------------------
(0 rows)
SELECT * FROM table_to_split_3;
id | value
---------------------------------------------------------------------
(0 rows)
-- Insert data in table_to_split_1 at worker1
\c - - - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
INSERT INTO table_to_split_1 VALUES(100, 'a');
INSERT INTO table_to_split_1 VALUES(400, 'a');
INSERT INTO table_to_split_1 VALUES(500, 'a');
UPDATE table_to_split_1 SET value='b' WHERE id = 400;
SELECT * FROM table_to_split_1;
id | value
---------------------------------------------------------------------
100 | a
500 | a
400 | b
(3 rows)
-- expect data to present in table_to_split_2 on worker1 as its destination for value '400'
SELECT wait_for_expected_rowcount_at_table('table_to_split_2', 1);
wait_for_expected_rowcount_at_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM table_to_split_2;
id | value
---------------------------------------------------------------------
400 | b
(1 row)
SELECT * FROM table_to_split_3;
id | value
---------------------------------------------------------------------
(0 rows)
-- Expect data to be present only in table_to_split3 on worker2
\c - - - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
SELECT * FROM table_to_split_1;
id | value
---------------------------------------------------------------------
(0 rows)
SELECT * FROM table_to_split_2;
id | value
---------------------------------------------------------------------
(0 rows)
SELECT wait_for_expected_rowcount_at_table('table_to_split_3', 2);
wait_for_expected_rowcount_at_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM table_to_split_3;
id | value
---------------------------------------------------------------------
100 | a
500 | a
(2 rows)
-- delete all from table_to_split_1
\c - - - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
DELETE FROM table_to_split_1;
-- rows from table_to_split_2 should be deleted
SELECT wait_for_expected_rowcount_at_table('table_to_split_2', 0);
wait_for_expected_rowcount_at_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM table_to_split_2;
id | value
---------------------------------------------------------------------
(0 rows)
-- rows from table_to_split_3 should be deleted
\c - - - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
SELECT wait_for_expected_rowcount_at_table('table_to_split_3', 0);
wait_for_expected_rowcount_at_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM table_to_split_3;
id | value
---------------------------------------------------------------------
(0 rows)
\c - - - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
SET client_min_messages TO ERROR;
DROP SUBSCRIPTION sub_worker2;
-- drop publication from worker1
\c - - - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
SET client_min_messages TO ERROR;
DROP SUBSCRIPTION sub_worker1;
DROP PUBLICATION pub1;

View File

@ -0,0 +1,121 @@
-- File to create functions and helpers needed for split shard tests
-- Populates shared memory mapping for parent shard with id 1.
-- targetNode1, targetNode2 are the locations where child shard xxxxx and 3 are placed respectively
CREATE OR REPLACE FUNCTION split_shard_replication_setup_helper(targetNode1 integer, targetNode2 integer) RETURNS text AS $$
DECLARE
memoryId bigint := 0;
memoryIdText text;
begin
SELECT * into memoryId from worker_split_shard_replication_setup(ARRAY[ARRAY[1,2,-2147483648,-1, targetNode1], ARRAY[1,3,0,2147483647,targetNode2]]);
SELECT FORMAT('%s', memoryId) into memoryIdText;
return memoryIdText;
end
$$ LANGUAGE plpgsql;
-- Create replication slots for targetNode1 and targetNode2 incase of non-colocated shards
CREATE OR REPLACE FUNCTION create_replication_slot(targetNode1 integer, targetNode2 integer) RETURNS text AS $$
DECLARE
targetOneSlotName text;
targetTwoSlotName text;
sharedMemoryId text;
derivedSlotName text;
begin
SELECT * into sharedMemoryId from public.split_shard_replication_setup_helper(targetNode1, targetNode2);
SELECT FORMAT('citus_split_%s_%s_10', targetNode1, sharedMemoryId) into derivedSlotName;
SELECT slot_name into targetOneSlotName from pg_create_logical_replication_slot(derivedSlotName, 'decoding_plugin_for_shard_split');
-- if new child shards are placed on different nodes, create one more replication slot
if (targetNode1 != targetNode2) then
SELECT FORMAT('citus_split_%s_%s_10', targetNode2, sharedMemoryId) into derivedSlotName;
SELECT slot_name into targetTwoSlotName from pg_create_logical_replication_slot(derivedSlotName, 'decoding_plugin_for_shard_split');
INSERT INTO slotName_table values(targetTwoSlotName, targetNode2, 1);
end if;
INSERT INTO slotName_table values(targetOneSlotName, targetNode1, 2);
return targetOneSlotName;
end
$$ LANGUAGE plpgsql;
-- Populates shared memory mapping for colocated parent shards 4 and 7.
-- shard xxxxx has child shards 5 and 6. Shard 7 has child shards 8 and 9.
CREATE OR REPLACE FUNCTION split_shard_replication_setup_for_colocated_shards(targetNode1 integer, targetNode2 integer) RETURNS text AS $$
DECLARE
memoryId bigint := 0;
memoryIdText text;
begin
SELECT * into memoryId from worker_split_shard_replication_setup(
ARRAY[
ARRAY[4, 5, -2147483648,-1, targetNode1],
ARRAY[4, 6, 0 ,2147483647, targetNode2],
ARRAY[7, 8, -2147483648,-1, targetNode1],
ARRAY[7, 9, 0, 2147483647 , targetNode2]
]);
SELECT FORMAT('%s', memoryId) into memoryIdText;
return memoryIdText;
end
$$ LANGUAGE plpgsql;
-- Create replication slots for targetNode1 and targetNode2 incase of colocated shards
CREATE OR REPLACE FUNCTION create_replication_slot_for_colocated_shards(targetNode1 integer, targetNode2 integer) RETURNS text AS $$
DECLARE
targetOneSlotName text;
targetTwoSlotName text;
sharedMemoryId text;
derivedSlotNameOne text;
derivedSlotNameTwo text;
tableOwnerOne bigint;
tableOwnerTwo bigint;
begin
-- setup shared memory information
SELECT * into sharedMemoryId from public.split_shard_replication_setup_for_colocated_shards(targetNode1, targetNode2);
SELECT relowner into tableOwnerOne from pg_class where relname='table_first';
SELECT FORMAT('citus_split_%s_%s_%s', targetNode1, sharedMemoryId, tableOwnerOne) into derivedSlotNameOne;
SELECT slot_name into targetOneSlotName from pg_create_logical_replication_slot(derivedSlotNameOne, 'decoding_plugin_for_shard_split');
SELECT relowner into tableOwnerTwo from pg_class where relname='table_second';
SELECT FORMAT('citus_split_%s_%s_%s', targetNode2, sharedMemoryId, tableOwnerTwo) into derivedSlotNameTwo;
SELECT slot_name into targetTwoSlotName from pg_create_logical_replication_slot(derivedSlotNameTwo, 'decoding_plugin_for_shard_split');
INSERT INTO slotName_table values(targetOneSlotName, targetNode1, 1);
INSERT INTO slotName_table values(targetTwoSlotName, targetNode2, 2);
return targetOneSlotName;
end
$$ LANGUAGE plpgsql;
-- create subscription on target node with given 'subscriptionName'
CREATE OR REPLACE FUNCTION create_subscription(targetNodeId integer, subscriptionName text) RETURNS text AS $$
DECLARE
replicationSlotName text;
nodeportLocal int;
subname text;
begin
SELECT name into replicationSlotName from slotName_table where nodeId = targetNodeId;
EXECUTE FORMAT($sub$create subscription %s connection 'host=localhost port=xxxxx user=postgres dbname=regression' publication PUB1 with(create_slot=false, enabled=true, slot_name='%s', copy_data=false)$sub$, subscriptionName, replicationSlotName);
return replicationSlotName;
end
$$ LANGUAGE plpgsql;
-- create subscription on target node with given 'subscriptionName'
CREATE OR REPLACE FUNCTION create_subscription_for_owner_one(targetNodeId integer, subscriptionName text) RETURNS text AS $$
DECLARE
replicationSlotName text;
nodeportLocal int;
subname text;
begin
SELECT name into replicationSlotName from slotName_table where id = 1;
EXECUTE FORMAT($sub$create subscription %s connection 'host=localhost port=xxxxx user=postgres dbname=regression' publication PUB1 with(create_slot=false, enabled=true, slot_name='%s', copy_data=false)$sub$, subscriptionName, replicationSlotName);
return replicationSlotName;
end
$$ LANGUAGE plpgsql;
-- create subscription on target node with given 'subscriptionName'
CREATE OR REPLACE FUNCTION create_subscription_for_owner_two(targetNodeId integer, subscriptionName text) RETURNS text AS $$
DECLARE
replicationSlotName text;
nodeportLocal int;
subname text;
begin
SELECT name into replicationSlotName from slotName_table where id = 2;
EXECUTE FORMAT($sub$create subscription %s connection 'host=localhost port=xxxxx user=postgres dbname=regression' publication PUB2 with(create_slot=false, enabled=true, slot_name='%s', copy_data=false)$sub$, subscriptionName, replicationSlotName);
return replicationSlotName;
end
$$ LANGUAGE plpgsql;

View File

@ -229,6 +229,7 @@ ORDER BY 1;
function worker_record_sequence_dependency(regclass,regclass,name)
function worker_save_query_explain_analyze(text,jsonb)
function worker_split_copy(bigint,split_copy_info[])
function worker_split_shard_replication_setup(split_shard_info[])
schema citus
schema citus_internal
sequence pg_dist_colocationid_seq
@ -252,7 +253,9 @@ ORDER BY 1;
type citus.shard_transfer_mode
type citus_copy_format
type noderole
type replication_slot_info
type split_copy_info
type split_shard_info
view citus_dist_stat_activity
view citus_lock_waits
view citus_locks
@ -264,5 +267,5 @@ ORDER BY 1;
view citus_stat_statements
view pg_dist_shard_placement
view time_partitions
(256 rows)
(259 rows)

View File

@ -0,0 +1,158 @@
// Test scenario for nonblocking split and concurrent INSERT/UPDATE/DELETE
// session s1 - Executes non-blocking shard split
// session s2 - Does concurrent writes
// session s3 - Holds advisory locks
// session s4 - Tries to insert when the shards are Blocked for write
//
setup
{
SET citus.shard_count to 1;
SET citus.shard_replication_factor to 1;
SELECT setval('pg_dist_shardid_seq', 1500000);
CREATE TABLE to_split_table (id int PRIMARY KEY, value int);
SELECT create_distributed_table('to_split_table', 'id');
}
teardown
{
DROP TABLE to_split_table;
}
session "s1"
step "s1-begin"
{
BEGIN;
}
// cache all placements
step "s1-load-cache"
{
-- Indirect way to load cache.
TRUNCATE to_split_table;
}
step "s1-non-blocking-shard-split"
{
SELECT pg_catalog.citus_split_shard_by_split_points(
1500001,
ARRAY['-1073741824'],
ARRAY[2, 2],
'force_logical');
}
step "s1-end"
{
COMMIT;
}
session "s2"
step "s2-begin"
{
BEGIN;
}
step "s2-insert"
{
SELECT get_shard_id_for_distribution_column('to_split_table', 123456789);
INSERT INTO to_split_table VALUES (123456789, 1);
}
step "s2-update"
{
UPDATE to_split_table SET value = 111 WHERE id = 123456789;
}
step "s2-delete"
{
DELETE FROM to_split_table WHERE id = 123456789;
}
step "s2-select"
{
SELECT count(*) FROM to_split_table WHERE id = 123456789;
}
step "s2-end"
{
COMMIT;
}
step "s2-print-cluster"
{
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
}
session "s3"
// this advisory lock with (almost) random values are only used
// for testing purposes. For details, check Citus' logical replication
// source code
step "s3-acquire-advisory-lock"
{
SELECT pg_advisory_lock(44000, 55152);
}
step "s3-release-advisory-lock"
{
SELECT pg_advisory_unlock(44000, 55152);
}
session "s4"
step "s4-begin"
{
BEGIN;
}
step "s4-insert"
{
INSERT INTO to_split_table VALUES (900, 1);
}
step "s4-end"
{
COMMIT;
}
// Concurrent Insert:
// s3 holds advisory lock -> s1 starts non-blocking shard split and waits for advisory lock ->
// s2 inserts a row successfully demonstrating nonblocking split -> s3 releases the advisory lock
// -> s1 completes split -> result is reflected in new shards
permutation "s1-load-cache" "s2-print-cluster" "s3-acquire-advisory-lock" "s1-begin" "s2-begin" "s1-non-blocking-shard-split" "s2-insert" "s2-end" "s2-print-cluster" "s3-release-advisory-lock" "s1-end" "s2-print-cluster"
// Concurrent Update:
// s2 inserts a row to be updated later ->s3 holds advisory lock -> s1 starts non-blocking shard split and waits for advisory lock ->
// s2 udpates the row -> s3 releases the advisory lock
// -> s1 completes split -> result is reflected in new shards
permutation "s1-load-cache" "s2-insert" "s2-print-cluster" "s3-acquire-advisory-lock" "s1-begin" "s1-non-blocking-shard-split" "s2-update" "s3-release-advisory-lock" "s1-end" "s2-print-cluster"
// Concurrent Delete:
// s2 inserts a row to be deleted later ->s3 holds advisory lock -> s1 starts non-blocking shard split and waits for advisory lock ->
// s2 deletes the row -> s3 releases the advisory lock
// -> s1 completes split -> result is reflected in new shards
permutation "s1-load-cache" "s2-insert" "s2-print-cluster" "s3-acquire-advisory-lock" "s1-begin" "s1-non-blocking-shard-split" "s2-delete" "s3-release-advisory-lock" "s1-end" "s2-print-cluster"
// Demonstrating blocking Insert when the writes are blocked by nonblocking split workflow
// s3 holds advisory lock -> s1 starts non-blocking shard split and waits for advisory lock ->
// s2 inserts the row successfully-> s4 begins-> s3 releases the advisory lock thus s2 moves ahead to block writes
// -> s4 inserts(waiting as the writes are blocked) -> s1 commits -> s4 fails as shard meta data gets update
permutation "s1-load-cache" "s2-print-cluster" "s3-acquire-advisory-lock" "s1-begin" "s2-begin" "s1-non-blocking-shard-split" "s2-insert" "s2-end" "s2-print-cluster" "s4-begin" "s3-release-advisory-lock" "s4-insert" "s1-end" "s4-end" "s2-print-cluster"
// Same flow without loading cache
permutation "s2-print-cluster" "s3-acquire-advisory-lock" "s1-begin" "s2-begin" "s1-non-blocking-shard-split" "s2-insert" "s2-end" "s2-print-cluster" "s3-release-advisory-lock" "s1-end" "s2-print-cluster"
permutation "s2-insert" "s2-print-cluster" "s3-acquire-advisory-lock" "s1-begin" "s1-non-blocking-shard-split" "s2-update" "s3-release-advisory-lock" "s1-end" "s2-print-cluster"
permutation "s2-insert" "s2-print-cluster" "s3-acquire-advisory-lock" "s1-begin" "s1-non-blocking-shard-split" "s2-delete" "s3-release-advisory-lock" "s1-end" "s2-print-cluster"

View File

@ -0,0 +1,142 @@
// Test scenario for nonblocking split and concurrent INSERT/UPDATE/DELETE.
// Test uses Index as replica identity.
// session s1 - Does concurrent writes on reference table
// session s2 - Executes non-blocking shard split
// session s3 - Holds advisory locks
setup
{
SELECT setval('pg_dist_shardid_seq', 1500000);
SET citus.shard_count to 2;
SET citus.shard_replication_factor to 1;
CREATE TABLE reference_table (id int PRIMARY KEY, value int);
SELECT create_reference_table('reference_table');
CREATE TABLE table_to_split (id int, value int);
SELECT create_distributed_table('table_to_split', 'id');
}
teardown
{
DROP TABLE table_to_split CASCADE;
DROP TABLE reference_table CASCADE;
}
session "s1"
step "s1-begin"
{
BEGIN;
}
step "s1-insert"
{
INSERT INTO reference_table VALUES (5, 10);
}
step "s1-update"
{
UPDATE reference_table SET value = 5 WHERE id = 5;
}
step "s1-delete"
{
DELETE FROM reference_table WHERE id = 5;
}
step "s1-ddl"
{
CREATE INDEX reference_table_index ON reference_table(id);
}
step "s1-copy"
{
COPY reference_table FROM PROGRAM 'echo "1,1\n2,2\n3,3\n4,4\n5,5"' WITH CSV;
}
step "s1-commit"
{
COMMIT;
}
session "s2"
step "s2-begin"
{
BEGIN;
}
step "s2-non-blocking-shard-split"
{
SELECT pg_catalog.citus_split_shard_by_split_points(
1500002,
ARRAY['-1073741824'],
ARRAY[1, 2],
'force_logical');
}
step "s2-add-fkey"
{
ALTER TABLE table_to_split ADD CONSTRAINT fkey_const FOREIGN KEY(value) REFERENCES reference_table(id);
}
step "s2-commit"
{
COMMIT;
}
step "s2-print-cluster"
{
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('table_to_split', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows of table_to_split
SELECT id, value FROM table_to_split ORDER BY id, value;
-- rows of reference table
SELECT * FROM reference_table;
}
session "s3"
// this advisory lock with (almost) random values are only used
// for testing purposes. For details, check Citus' logical replication
// source code
step "s3-acquire-advisory-lock"
{
SELECT pg_advisory_lock(44000, 55152);
}
step "s3-release-advisory-lock"
{
SELECT pg_advisory_unlock(44000, 55152);
}
// Run nonblocking shard split while concurrently performing an DML on the
// reference table which the distributed table have a foreign key to.
// Modifications should not be blocked.
// Concurrent Insert:
// s2 add fkey constrain->s3 holds advisory lock -> s2 starts non-blocking shard split and waits for advisory lock ->
// s1 inserts a row in reference table successfully demonstrating nonblocking split -> s3 releases the advisory lock
// -> s2 completes split -> result is reflected in new shards
permutation "s2-add-fkey" "s3-acquire-advisory-lock" "s1-begin" "s2-begin" "s2-non-blocking-shard-split" "s1-insert" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"
// Concurrent Update:
// s2 add fkey constrain->s3 holds advisory lock -> s2 starts non-blocking shard split and waits for advisory lock ->
// s1 updates row of reference table -> s3 releases the advisory lock
// -> s2 completes split -> result is reflected in new shards
permutation "s2-add-fkey" "s3-acquire-advisory-lock" "s1-begin" "s2-begin" "s2-non-blocking-shard-split" "s1-update" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"
// Concurrent Delete:
// s2 add fkey constrain->s3 holds advisory lock -> s2 starts non-blocking shard split and waits for advisory lock ->
// s1 deletes row of reference table -> s3 releases the advisory lock
// -> s2 completes split -> result is reflected in new shards
permutation "s2-add-fkey" "s3-acquire-advisory-lock" "s1-begin" "s2-begin" "s2-non-blocking-shard-split" "s1-delete" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"

View File

@ -0,0 +1,138 @@
// Test scenario for nonblocking split and concurrent INSERT/UPDATE/DELETE.
// Test uses Index as replica identity.
// session s1 - Executes non-blocking shard split
// session s2 - Does concurrent writes
// session s3 - Holds advisory locks
setup
{
SET citus.shard_count to 1;
SET citus.shard_replication_factor to 1;
SELECT setval('pg_dist_shardid_seq', 1500000);
CREATE TABLE to_split_table (id int NOT NULL, value int);
CREATE UNIQUE INDEX split_table_index ON to_split_table(id);
ALTER TABLE to_split_table REPLICA IDENTITY USING INDEX split_table_index;
SELECT create_distributed_table('to_split_table', 'id');
}
teardown
{
DROP TABLE to_split_table CASCADE;
}
session "s1"
step "s1-begin"
{
BEGIN;
}
// cache all placements
step "s1-load-cache"
{
-- Indirect way to load cache.
TRUNCATE to_split_table;
}
step "s1-non-blocking-shard-split"
{
SELECT pg_catalog.citus_split_shard_by_split_points(
1500001,
ARRAY['-1073741824'],
ARRAY[2, 2],
'force_logical');
}
step "s1-end"
{
COMMIT;
}
session "s2"
step "s2-begin"
{
BEGIN;
}
step "s2-insert"
{
SELECT get_shard_id_for_distribution_column('to_split_table', 123456789);
INSERT INTO to_split_table VALUES (123456789, 1);
}
step "s2-update"
{
UPDATE to_split_table SET value = 111 WHERE id = 123456789;
}
step "s2-delete"
{
DELETE FROM to_split_table WHERE id = 123456789;
}
step "s2-select"
{
SELECT count(*) FROM to_split_table WHERE id = 123456789;
}
step "s2-end"
{
COMMIT;
}
step "s2-print-cluster"
{
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
}
session "s3"
// this advisory lock with (almost) random values are only used
// for testing purposes. For details, check Citus' logical replication
// source code
step "s3-acquire-advisory-lock"
{
SELECT pg_advisory_lock(44000, 55152);
}
step "s3-release-advisory-lock"
{
SELECT pg_advisory_unlock(44000, 55152);
}
##// nonblocking tests lie below ###
// Concurrent Insert:
// s3 holds advisory lock -> s1 starts non-blocking shard split and waits for advisory lock ->
// s2 inserts a row successfully demonstrating nonblocking split -> s3 releases the advisory lock
// -> s1 completes split -> result is reflected in new shards
permutation "s1-load-cache" "s2-print-cluster" "s3-acquire-advisory-lock" "s1-begin" "s2-begin" "s1-non-blocking-shard-split" "s2-insert" "s2-end" "s2-print-cluster" "s3-release-advisory-lock" "s1-end" "s2-print-cluster"
// Concurrent Update:
// s2 inserts a row to be updated later ->s3 holds advisory lock -> s1 starts non-blocking shard split and waits for advisory lock ->
// s2 udpates the row -> s3 releases the advisory lock
// -> s1 completes split -> result is reflected in new shards
permutation "s1-load-cache" "s2-insert" "s2-print-cluster" "s3-acquire-advisory-lock" "s1-begin" "s1-non-blocking-shard-split" "s2-update" "s3-release-advisory-lock" "s1-end" "s2-print-cluster"
// Concurrent Delete:
// s2 inserts a row to be deleted later ->s3 holds advisory lock -> s1 starts non-blocking shard split and waits for advisory lock ->
// s2 deletes the row -> s3 releases the advisory lock
// -> s1 completes split -> result is reflected in new shards
permutation "s1-load-cache" "s2-insert" "s2-print-cluster" "s3-acquire-advisory-lock" "s1-begin" "s1-non-blocking-shard-split" "s2-delete" "s3-release-advisory-lock" "s1-end" "s2-print-cluster"
// Same flow without loading cache
permutation "s2-print-cluster" "s3-acquire-advisory-lock" "s1-begin" "s2-begin" "s1-non-blocking-shard-split" "s2-insert" "s2-end" "s2-print-cluster" "s3-release-advisory-lock" "s1-end" "s2-print-cluster"
permutation "s2-insert" "s2-print-cluster" "s3-acquire-advisory-lock" "s1-begin" "s1-non-blocking-shard-split" "s2-update" "s3-release-advisory-lock" "s1-end" "s2-print-cluster"
permutation "s2-insert" "s2-print-cluster" "s3-acquire-advisory-lock" "s1-begin" "s1-non-blocking-shard-split" "s2-delete" "s3-release-advisory-lock" "s1-end" "s2-print-cluster"

View File

@ -7,6 +7,10 @@ test: tablespace
# Helpers for foreign key catalogs.
test: foreign_key_to_reference_table
# Split tests go here.
test: split_shard_replication_setup
test: split_shard_replication_setup_remote_local
test: split_shard_replication_setup_local
test: split_shard_replication_colocated_setup
test: worker_split_copy_test
test: worker_split_binary_copy_test
test: worker_split_text_copy_test
@ -16,3 +20,6 @@ test: citus_split_shard_by_split_points_failure
# Name citus_split_shard_by_split_points_columnar_partitioned was too long and being truncated.
# use citus_split_shard_columnar_partitioned instead.
test: citus_split_shard_columnar_partitioned
test: citus_non_blocking_split_shards
test: citus_non_blocking_split_shard_cleanup
test: citus_non_blocking_split_columnar

View File

@ -0,0 +1,294 @@
CREATE SCHEMA "citus_split_test_schema_columnar_partitioned";
SET search_path TO "citus_split_test_schema_columnar_partitioned";
SET citus.next_shard_id TO 8970000;
SET citus.next_placement_id TO 8770000;
SET citus.shard_count TO 1;
SET citus.shard_replication_factor TO 1;
-- BEGIN: Create table to split, along with other co-located tables. Add indexes, statistics etc.
CREATE TABLE sensors(
measureid integer,
eventdatetime date,
measure_data jsonb,
PRIMARY KEY (measureid, eventdatetime, measure_data))
PARTITION BY RANGE(eventdatetime);
-- Table access method is specified on child tables
CREATE TABLE sensorscolumnar(
measureid integer,
eventdatetime date,
measure_data jsonb,
PRIMARY KEY (measureid, eventdatetime, measure_data))
PARTITION BY RANGE(eventdatetime);
-- Create Partitions of table 'sensors'.
CREATE TABLE sensors_old PARTITION OF sensors FOR VALUES FROM ('2000-01-01') TO ('2020-01-01');
CREATE TABLE sensors_2020_01_01 PARTITION OF sensors FOR VALUES FROM ('2020-01-01') TO ('2020-02-01');
CREATE TABLE sensors_news PARTITION OF sensors FOR VALUES FROM ('2020-05-01') TO ('2025-01-01');
CREATE TABLE sensorscolumnar_old PARTITION OF sensorscolumnar FOR VALUES FROM ('2000-01-01') TO ('2020-01-01') USING COLUMNAR;
CREATE TABLE sensorscolumnar_2020_01_01 PARTITION OF sensorscolumnar FOR VALUES FROM ('2020-01-01') TO ('2020-02-01') USING COLUMNAR;
CREATE TABLE sensorscolumnar_news PARTITION OF sensorscolumnar FOR VALUES FROM ('2020-05-01') TO ('2025-01-01') USING COLUMNAR;
-- Create index on parent and child partitions.
CREATE INDEX index_on_parent ON sensors(lower(measureid::text));
CREATE INDEX index_on_child ON sensors_2020_01_01(lower(measure_data::text));
CREATE INDEX index_on_parent_columnar ON sensorscolumnar(lower(measureid::text));
CREATE INDEX index_on_child_columnar ON sensorscolumnar_2020_01_01(lower(measure_data::text));
ALTER INDEX index_on_parent ALTER COLUMN 1 SET STATISTICS 1000;
ALTER INDEX index_on_child ALTER COLUMN 1 SET STATISTICS 1000;
ALTER INDEX index_on_parent_columnar ALTER COLUMN 1 SET STATISTICS 1000;
ALTER INDEX index_on_child_columnar ALTER COLUMN 1 SET STATISTICS 1000;
-- Create statistics on parent and child partitions.
CREATE STATISTICS s1 (dependencies) ON measureid, eventdatetime FROM sensors;
CREATE STATISTICS s2 (dependencies) ON measureid, eventdatetime FROM sensors_2020_01_01;
CREATE STATISTICS s1_c (dependencies) ON measureid, eventdatetime FROM sensorscolumnar;
CREATE STATISTICS s2_c (dependencies) ON measureid, eventdatetime FROM sensorscolumnar_2020_01_01;
CLUSTER sensors_2020_01_01 USING index_on_child;
SELECT create_distributed_table('sensors', 'measureid');
SELECT create_distributed_table('sensorscolumnar', 'measureid');
-- create colocated distributed tables
CREATE TABLE colocated_dist_table (measureid integer PRIMARY KEY);
SELECT create_distributed_table('colocated_dist_table', 'measureid');
CLUSTER colocated_dist_table USING colocated_dist_table_pkey;
CREATE TABLE colocated_partitioned_table(
measureid integer,
eventdatetime date,
PRIMARY KEY (measureid, eventdatetime))
PARTITION BY RANGE(eventdatetime);
CREATE TABLE colocated_partitioned_table_2020_01_01 PARTITION OF colocated_partitioned_table FOR VALUES FROM ('2020-01-01') TO ('2020-02-01');
SELECT create_distributed_table('colocated_partitioned_table', 'measureid');
CLUSTER colocated_partitioned_table_2020_01_01 USING colocated_partitioned_table_2020_01_01_pkey;
-- create reference tables
CREATE TABLE reference_table (measureid integer PRIMARY KEY);
SELECT create_reference_table('reference_table');
SELECT shard.shardid, logicalrelid, shardminvalue, shardmaxvalue, nodename, nodeport
FROM pg_dist_shard AS shard
INNER JOIN pg_dist_placement placement ON shard.shardid = placement.shardid
INNER JOIN pg_dist_node node ON placement.groupid = node.groupid
INNER JOIN pg_catalog.pg_class cls ON shard.logicalrelid = cls.oid
INNER JOIN pg_catalog.pg_namespace ns ON cls.relnamespace = ns.oid
WHERE node.noderole = 'primary' AND ns.nspname = 'citus_split_test_schema_columnar_partitioned'
ORDER BY logicalrelid, shardminvalue::BIGINT, nodeport;
-- END: Create table to split, along with other co-located tables. Add indexes, statistics etc.
-- BEGIN: Create constraints for tables.
-- from parent to regular dist
ALTER TABLE sensors ADD CONSTRAINT fkey_from_parent_to_dist FOREIGN KEY (measureid) REFERENCES colocated_dist_table(measureid);
-- from parent to parent
ALTER TABLE sensors ADD CONSTRAINT fkey_from_parent_to_parent FOREIGN KEY (measureid, eventdatetime) REFERENCES colocated_partitioned_table(measureid, eventdatetime);
-- from parent to child
ALTER TABLE sensors ADD CONSTRAINT fkey_from_parent_to_child FOREIGN KEY (measureid, eventdatetime) REFERENCES colocated_partitioned_table_2020_01_01(measureid, eventdatetime);
ALTER TABLE sensors ADD CONSTRAINT fkey_from_parent_to_ref FOREIGN KEY (measureid) REFERENCES reference_table(measureid);
-- from child to regular dist
ALTER TABLE sensors_2020_01_01 ADD CONSTRAINT fkey_from_child_to_dist FOREIGN KEY (measureid) REFERENCES colocated_dist_table(measureid);
-- from child to parent
ALTER TABLE sensors_2020_01_01 ADD CONSTRAINT fkey_from_child_to_parent FOREIGN KEY (measureid,eventdatetime) REFERENCES colocated_partitioned_table(measureid,eventdatetime);
-- from child to child
ALTER TABLE sensors_2020_01_01 ADD CONSTRAINT fkey_from_child_to_child FOREIGN KEY (measureid,eventdatetime) REFERENCES colocated_partitioned_table_2020_01_01(measureid,eventdatetime);
ALTER TABLE sensors_2020_01_01 ADD CONSTRAINT fkey_from_child_to_ref FOREIGN KEY (measureid) REFERENCES reference_table(measureid);
-- No support for foreign keys, unique constraints, or exclusion constraints in columnar tables.
-- Please see: https://github.com/citusdata/citus/tree/main/src/backend/columnar/README.md
-- END: Create constraints for tables.
-- BEGIN: Load data into tables
INSERT INTO reference_table SELECT i FROM generate_series(0,1000)i;
INSERT INTO colocated_dist_table SELECT i FROM generate_series(0,1000)i;
INSERT INTO colocated_partitioned_table SELECT i, '2020-01-05' FROM generate_series(0,1000)i;
INSERT INTO sensors SELECT i, '2020-01-05', '{}' FROM generate_series(0,1000)i;
INSERT INTO sensorscolumnar SELECT i, '2020-01-05', '{}' FROM generate_series(0,1000)i;
-- END: Load data into tables
-- BEGIN: Show the current state on workers
\c - - - :worker_1_port
SET search_path TO "citus_split_test_schema_columnar_partitioned";
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like '%_89%'
ORDER BY 1, 2;
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like '%_89%' ORDER BY 1,2;
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema_columnar_partitioned')
)
ORDER BY stxname ASC;
\c - - - :worker_2_port
SET search_path TO "citus_split_test_schema_columnar_partitioned";
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like '%_89%'
ORDER BY 1, 2;
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like '%_89%' ORDER BY 1,2;
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema_columnar_partitioned')
)
ORDER BY stxname ASC;
-- END: Show the current state on workers
-- BEGIN: Split a shard along its co-located shards
\c - - - :master_port
SET search_path TO "citus_split_test_schema_columnar_partitioned";
SET citus.next_shard_id TO 8999000;
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
SELECT pg_catalog.citus_split_shard_by_split_points(
8970000,
ARRAY['-2120000000'],
ARRAY[:worker_1_node, :worker_2_node],
'force_logical');
-- END: Split a shard along its co-located shards
-- BEGIN: Validate Shard Info and Data
SELECT shard.shardid, logicalrelid, shardminvalue, shardmaxvalue, nodename, nodeport
FROM pg_dist_shard AS shard
INNER JOIN pg_dist_placement placement ON shard.shardid = placement.shardid
INNER JOIN pg_dist_node node ON placement.groupid = node.groupid
INNER JOIN pg_catalog.pg_class cls ON shard.logicalrelid = cls.oid
INNER JOIN pg_catalog.pg_namespace ns ON cls.relnamespace = ns.oid
WHERE node.noderole = 'primary' AND ns.nspname = 'citus_split_test_schema_columnar_partitioned'
ORDER BY logicalrelid, shardminvalue::BIGINT, nodeport;
SELECT count(*) FROM reference_table;
SELECT count(*) FROM colocated_partitioned_table;
SELECT count(*) FROM colocated_dist_table;
SELECT count(*) FROM sensors;
SELECT count(*) FROM sensorscolumnar;
-- END: Validate Shard Info and Data
-- BEGIN: Show the updated state on workers
\c - - - :worker_1_port
SET search_path TO "citus_split_test_schema_columnar_partitioned";
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like '%_89%'
ORDER BY 1, 2;
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like '%_89%' ORDER BY 1,2;
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema_columnar_partitioned')
)
ORDER BY stxname ASC;
\c - - - :worker_2_port
SET search_path TO "citus_split_test_schema_columnar_partitioned";
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like '%_89%'
ORDER BY 1, 2;
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like '%_89%' ORDER BY 1,2;
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema_columnar_partitioned')
)
ORDER BY stxname ASC;
-- END: Show the updated state on workers
-- BEGIN: Split a partition table directly
\c - - - :master_port
SET search_path TO "citus_split_test_schema_columnar_partitioned";
SET citus.next_shard_id TO 8999100;
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
SELECT pg_catalog.citus_split_shard_by_split_points(
8999002, -- sensors_old
ARRAY['-2127770000'],
ARRAY[:worker_1_node, :worker_2_node],
'auto');
-- END: Split a partition table directly
-- BEGIN: Validate Shard Info and Data
SELECT shard.shardid, logicalrelid, shardminvalue, shardmaxvalue, nodename, nodeport
FROM pg_dist_shard AS shard
INNER JOIN pg_dist_placement placement ON shard.shardid = placement.shardid
INNER JOIN pg_dist_node node ON placement.groupid = node.groupid
INNER JOIN pg_catalog.pg_class cls ON shard.logicalrelid = cls.oid
INNER JOIN pg_catalog.pg_namespace ns ON cls.relnamespace = ns.oid
WHERE node.noderole = 'primary' AND ns.nspname = 'citus_split_test_schema_columnar_partitioned'
ORDER BY logicalrelid, shardminvalue::BIGINT, nodeport;
SELECT count(*) FROM reference_table;
SELECT count(*) FROM colocated_partitioned_table;
SELECT count(*) FROM colocated_dist_table;
SELECT count(*) FROM sensors;
SELECT count(*) FROM sensorscolumnar;
-- END: Validate Shard Info and Data
-- BEGIN: Show the updated state on workers
\c - - - :worker_1_port
SET search_path TO "citus_split_test_schema_columnar_partitioned";
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like '%_89%'
ORDER BY 1, 2;
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like '%_89%' ORDER BY 1,2;
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema_columnar_partitioned')
)
ORDER BY stxname ASC;
\c - - - :worker_2_port
SET search_path TO "citus_split_test_schema_columnar_partitioned";
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like '%_89%'
ORDER BY 1, 2;
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like '%_89%' ORDER BY 1,2;
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema_columnar_partitioned')
)
ORDER BY stxname ASC;
-- END: Show the updated state on workers
--BEGIN : Cleanup
\c - postgres - :master_port
DROP SCHEMA "citus_split_test_schema_columnar_partitioned" CASCADE;
--END : Cleanup

View File

@ -0,0 +1,93 @@
/*
Citus non-blocking shard split test.
The test ensures after a error free non-blocking split operation, following objects
created for logical replication are cleaned up:
1) At source node: Publications, replication slots, dummy shards.
2) At destination node: Subscriptions, dummy shards.
*/
CREATE SCHEMA "citus_split_test_schema";
SET search_path TO "citus_split_test_schema";
SET citus.next_shard_id TO 8981000;
SET citus.next_placement_id TO 8610000;
SET citus.shard_count TO 2;
SET citus.shard_replication_factor TO 1;
-- BEGIN: Create table to split, along with other co-located tables. Add indexes, statistics etc.
CREATE TABLE sensors(
measureid integer,
eventdatetime date,
measure_data jsonb,
meaure_quantity decimal(15, 2),
measure_status char(1),
measure_comment varchar(44),
PRIMARY KEY (measureid, eventdatetime, measure_data));
SELECT create_distributed_table('sensors', 'measureid', colocate_with:='none');
INSERT INTO sensors SELECT i, '2020-01-05', '{}', 11011.10, 'A', 'I <3 Citus' FROM generate_series(0,1000)i;
-- BEGIN : Move one shard before we split it.
\c - postgres - :master_port
SET search_path TO "citus_split_test_schema";
SET citus.next_shard_id TO 8981007;
SET citus.defer_drop_after_shard_move TO OFF;
-- BEGIN : Set node id variables
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
-- END : Set node id variables
-- BEGIN : Split two shards
-- Perform 2 way split
SELECT pg_catalog.citus_split_shard_by_split_points(
8981000,
ARRAY['-1073741824'],
ARRAY[:worker_2_node, :worker_2_node],
'force_logical');
\c - - - :worker_1_port
SET search_path TO "citus_split_test_schema";
SET citus.show_shards_for_app_name_prefixes = '*';
-- Dummy shards should be cleaned up. 8981007, 8981008 are dummy shards
-- created at source.
SELECT count(*) FROM pg_class where relname like '%sensors_8981007%';
SELECT count(*) FROM pg_class where relname like '%sensors_8981008%';
-- Replication slots should be cleanedup at source
SELECT slot_name FROM pg_replication_slots;
-- Publications should be cleaned up on worker1
SELECT count(*) FROM pg_publication;
\c - - - :worker_2_port
SET search_path TO "citus_split_test_schema";
-- All subscriptions should be cleaned up.
SELECT count(*) FROM pg_subscription;
-- Trigger a 3-way local split.
\c - - - :master_port
SET search_path TO "citus_split_test_schema";
SELECT pg_catalog.citus_split_shard_by_split_points(
8981001,
ARRAY['536870911', '1610612735'],
ARRAY[:worker_2_node, :worker_2_node, :worker_2_node],
'force_logical');
\c - - - :worker_2_port
SET search_path TO "citus_split_test_schema";
-- Replication slots should be cleaned up
SELECT slot_name FROM pg_replication_slots;
-- Publications should be cleanedup
SELECT count(*) FROM pg_publication;
-- All subscriptions should be cleaned up.
SELECT count(*) FROM pg_subscription;
--BEGIN : Cleanup
\c - postgres - :master_port
DROP SCHEMA "citus_split_test_schema" CASCADE;
--END : Cleanup

View File

@ -0,0 +1,240 @@
/*
Citus Shard Split Test.The test is model similar to 'shard_move_constraints'.
Here is a high level overview of test plan:
1. Create a table 'sensors' (ShardCount = 2) to be split. Add indexes and statistics on this table.
2. Create two other tables: 'reference_table' and 'colocated_dist_table', co-located with sensors.
3. Create Foreign key constraints between the two co-located distributed tables.
4. Load data into the three tables.
5. Move one of the shards for 'sensors' to test ShardMove -> Split.
6. Trigger Split on both shards of 'sensors'. This will also split co-located tables.
7. Move one of the split shard to test Split -> ShardMove.
8. Split an already split shard second time on a different schema.
*/
CREATE SCHEMA "citus_split_test_schema";
CREATE ROLE test_shard_split_role WITH LOGIN;
GRANT USAGE, CREATE ON SCHEMA "citus_split_test_schema" TO test_shard_split_role;
SET ROLE test_shard_split_role;
SET search_path TO "citus_split_test_schema";
SET citus.next_shard_id TO 8981000;
SET citus.next_placement_id TO 8610000;
SET citus.shard_count TO 2;
SET citus.shard_replication_factor TO 1;
-- BEGIN: Create table to split, along with other co-located tables. Add indexes, statistics etc.
CREATE TABLE sensors(
measureid integer,
eventdatetime date,
measure_data jsonb,
meaure_quantity decimal(15, 2),
measure_status char(1),
measure_comment varchar(44),
PRIMARY KEY (measureid, eventdatetime, measure_data));
CREATE INDEX index_on_sensors ON sensors(lower(measureid::text));
ALTER INDEX index_on_sensors ALTER COLUMN 1 SET STATISTICS 1000;
CREATE INDEX hash_index_on_sensors ON sensors USING HASH((measure_data->'IsFailed'));
CREATE INDEX index_with_include_on_sensors ON sensors ((measure_data->'IsFailed')) INCLUDE (measure_data, eventdatetime, measure_status);
CREATE STATISTICS stats_on_sensors (dependencies) ON measureid, eventdatetime FROM sensors;
SELECT create_distributed_table('sensors', 'measureid', colocate_with:='none');
-- END: Create table to split, along with other co-located tables. Add indexes, statistics etc.
-- BEGIN: Create co-located distributed and reference tables.
CREATE TABLE reference_table (measureid integer PRIMARY KEY);
SELECT create_reference_table('reference_table');
CREATE TABLE colocated_dist_table (measureid integer PRIMARY KEY);
CLUSTER colocated_dist_table USING colocated_dist_table_pkey;
SELECT create_distributed_table('colocated_dist_table', 'measureid', colocate_with:='sensors');
CREATE TABLE table_with_index_rep_identity(key int NOT NULL);
CREATE UNIQUE INDEX uqx ON table_with_index_rep_identity(key);
ALTER TABLE table_with_index_rep_identity REPLICA IDENTITY USING INDEX uqx;
CLUSTER table_with_index_rep_identity USING uqx;
SELECT create_distributed_table('table_with_index_rep_identity', 'key', colocate_with:='sensors');
-- END: Create co-located distributed and reference tables.
-- BEGIN : Create Foreign key constraints.
ALTER TABLE sensors ADD CONSTRAINT fkey_table_to_dist FOREIGN KEY (measureid) REFERENCES colocated_dist_table(measureid);
-- END : Create Foreign key constraints.
-- BEGIN : Load data into tables.
INSERT INTO reference_table SELECT i FROM generate_series(0,1000)i;
INSERT INTO colocated_dist_table SELECT i FROM generate_series(0,1000)i;
INSERT INTO sensors SELECT i, '2020-01-05', '{}', 11011.10, 'A', 'I <3 Citus' FROM generate_series(0,1000)i;
SELECT COUNT(*) FROM sensors;
SELECT COUNT(*) FROM reference_table;
SELECT COUNT(*) FROM colocated_dist_table;
-- END: Load data into tables.
-- BEGIN : Display current state.
SELECT shard.shardid, logicalrelid, shardminvalue, shardmaxvalue, nodename, nodeport
FROM pg_dist_shard AS shard
INNER JOIN pg_dist_placement placement ON shard.shardid = placement.shardid
INNER JOIN pg_dist_node node ON placement.groupid = node.groupid
INNER JOIN pg_catalog.pg_class cls ON shard.logicalrelid = cls.oid
WHERE node.noderole = 'primary' AND (logicalrelid = 'sensors'::regclass OR logicalrelid = 'colocated_dist_table'::regclass OR logicalrelid = 'table_with_index_rep_identity'::regclass)
ORDER BY logicalrelid, shardminvalue::BIGINT;
\c - - - :worker_1_port
SET search_path TO "citus_split_test_schema", public, pg_catalog;
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like 'sensors_%'
ORDER BY 1, 2;
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'sensors_%' ORDER BY 1,2;
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'table_with_index_rep_identity_%' ORDER BY 1,2;
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema')
)
ORDER BY stxname ASC;
\c - - - :worker_2_port
SET search_path TO "citus_split_test_schema", public, pg_catalog;
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like 'sensors_%'
ORDER BY 1, 2;
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'sensors_%' ORDER BY 1,2;
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'table_with_index_rep_identity_%' ORDER BY 1,2;
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema')
)
ORDER BY stxname ASC;
-- END : Display current state
-- BEGIN : Move one shard before we split it.
\c - postgres - :master_port
SET ROLE test_shard_split_role;
SET search_path TO "citus_split_test_schema";
SET citus.next_shard_id TO 8981007;
SET citus.defer_drop_after_shard_move TO OFF;
SELECT citus_move_shard_placement(8981000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, shard_transfer_mode:='force_logical');
-- END : Move one shard before we split it.
-- BEGIN : Set node id variables
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
-- END : Set node id variables
-- BEGIN : Split two shards : One with move and One without move.
-- Perform 2 way split
SELECT pg_catalog.citus_split_shard_by_split_points(
8981000,
ARRAY['-1073741824'],
ARRAY[:worker_1_node, :worker_2_node],
'force_logical');
-- Perform 3 way split
SELECT pg_catalog.citus_split_shard_by_split_points(
8981001,
ARRAY['536870911', '1610612735'],
ARRAY[:worker_1_node, :worker_1_node, :worker_2_node],
'force_logical');
-- END : Split two shards : One with move and One without move.
-- BEGIN : Move a shard post split.
SELECT citus_move_shard_placement(8981007, 'localhost', :worker_1_port, 'localhost', :worker_2_port, shard_transfer_mode:='block_writes');
-- END : Move a shard post split.
-- BEGIN : Display current state.
SELECT shard.shardid, logicalrelid, shardminvalue, shardmaxvalue, nodename, nodeport
FROM pg_dist_shard AS shard
INNER JOIN pg_dist_placement placement ON shard.shardid = placement.shardid
INNER JOIN pg_dist_node node ON placement.groupid = node.groupid
INNER JOIN pg_catalog.pg_class cls ON shard.logicalrelid = cls.oid
WHERE node.noderole = 'primary' AND (logicalrelid = 'sensors'::regclass OR logicalrelid = 'colocated_dist_table'::regclass OR logicalrelid = 'table_with_index_rep_identity'::regclass)
ORDER BY logicalrelid, shardminvalue::BIGINT;
\c - - - :worker_1_port
SET search_path TO "citus_split_test_schema", public, pg_catalog;
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like 'sensors_%'
ORDER BY 1, 2;
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'sensors_%' ORDER BY 1,2;
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'table_with_index_rep_identity_%' ORDER BY 1,2;
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema')
)
ORDER BY stxname ASC;
\c - - - :worker_2_port
SET search_path TO "citus_split_test_schema", public, pg_catalog;
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like 'sensors_%'
ORDER BY 1, 2;
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'sensors_%' ORDER BY 1,2;
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'table_with_index_rep_identity_%' ORDER BY 1,2;
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema')
)
ORDER BY stxname ASC;
-- END : Display current state
-- BEGIN: Should be able to change/drop constraints
\c - postgres - :master_port
SET ROLE test_shard_split_role;
SET search_path TO "citus_split_test_schema";
ALTER INDEX index_on_sensors RENAME TO index_on_sensors_renamed;
ALTER INDEX index_on_sensors_renamed ALTER COLUMN 1 SET STATISTICS 200;
DROP STATISTICS stats_on_sensors;
DROP INDEX index_on_sensors_renamed;
ALTER TABLE sensors DROP CONSTRAINT fkey_table_to_dist;
-- END: Should be able to change/drop constraints
-- BEGIN: Split second time on another schema
SET search_path TO public;
SET citus.next_shard_id TO 8981031;
SELECT pg_catalog.citus_split_shard_by_split_points(
8981007,
ARRAY['-2100000000'],
ARRAY[:worker_1_node, :worker_2_node],
'force_logical');
SET search_path TO "citus_split_test_schema";
SELECT shard.shardid, logicalrelid, shardminvalue, shardmaxvalue, nodename, nodeport
FROM pg_dist_shard AS shard
INNER JOIN pg_dist_placement placement ON shard.shardid = placement.shardid
INNER JOIN pg_dist_node node ON placement.groupid = node.groupid
INNER JOIN pg_catalog.pg_class cls ON shard.logicalrelid = cls.oid
WHERE node.noderole = 'primary' AND (logicalrelid = 'sensors'::regclass OR logicalrelid = 'colocated_dist_table'::regclass OR logicalrelid = 'table_with_index_rep_identity'::regclass)
ORDER BY logicalrelid, shardminvalue::BIGINT;
-- END: Split second time on another schema
-- BEGIN: Validate Data Count
SELECT COUNT(*) FROM sensors;
SELECT COUNT(*) FROM reference_table;
SELECT COUNT(*) FROM colocated_dist_table;
-- END: Validate Data Count
--BEGIN : Cleanup
\c - postgres - :master_port
DROP SCHEMA "citus_split_test_schema" CASCADE;
--END : Cleanup

View File

@ -23,19 +23,7 @@ SELECT create_distributed_table('table_to_split','id');
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
-- UDF fails for any other shard_transfer_mode other than block_writes.
SELECT citus_split_shard_by_split_points(
49761302,
ARRAY['50'],
ARRAY[101, 201],
'auto');
SELECT citus_split_shard_by_split_points(
49761302,
ARRAY['50'],
ARRAY[101, 201],
'force_logical');
-- UDF fails for any other shard_transfer_mode other than block_writes/force_logical/auto.
SELECT citus_split_shard_by_split_points(
49761302,
ARRAY['50'],

View File

@ -0,0 +1,163 @@
\c - - - :master_port
CREATE USER myuser;
CREATE USER admin_user;
GRANT USAGE, CREATE ON SCHEMA split_shard_replication_setup_schema, public to myuser;
GRANT USAGE, CREATE ON SCHEMA split_shard_replication_setup_schema, public to admin_user;
SET search_path TO split_shard_replication_setup_schema;
SET citus.shard_replication_factor TO 1;
SET citus.shard_count TO 1;
SET citus.next_shard_id TO 4;
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
\c - myuser - -
SET search_path TO split_shard_replication_setup_schema;
SET citus.shard_replication_factor TO 1;
SET citus.shard_count TO 1;
SET citus.next_shard_id TO 4;
CREATE TABLE table_first (id bigserial PRIMARY KEY, value char);
SELECT create_distributed_table('table_first','id');
\c - admin_user - -
SET search_path TO split_shard_replication_setup_schema;
SET citus.next_shard_id TO 7;
SET citus.shard_count TO 1;
CREATE TABLE table_second (id bigserial PRIMARY KEY, value char);
SELECT create_distributed_table('table_second', 'id', colocate_with => 'table_first');
\c - myuser - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
CREATE TABLE table_first_5(id bigserial PRIMARY KEY, value char);
CREATE TABLE table_first_6(id bigserial PRIMARY KEY, value char);
\c - myuser - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
CREATE TABLE table_first_4(id bigserial PRIMARY KEY, value char);
CREATE TABLE table_first_5(id bigserial PRIMARY KEY, value char);
CREATE TABLE table_first_6(id bigserial PRIMARY KEY, value char);
\c - admin_user - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
CREATE TABLE table_second_8(id bigserial PRIMARY KEY, value char);
CREATE TABLE table_second_9(id bigserial PRIMARY KEY, value char);
\c - admin_user - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
CREATE TABLE table_second_7(id bigserial PRIMARY KEY, value char);
CREATE TABLE table_second_8(id bigserial PRIMARY KEY, value char);
CREATE TABLE table_second_9(id bigserial PRIMARY KEY, value char);
--- Test scenario one starts from here
--- 1. table_first and table_second are colocated tables.
--- 2. myuser is the owner table_first and admin_user is the owner of table_second.
--- 3. Shard table_first_4 and table_second_7 are colocated on worker1
--- 4. table_first_4 is split into table_first_5 and table_first_6 with target as worker2
--- 5. table_second_7 is split into table_second_8 and table_second_9 with target as worker2
--- 6. Create two publishers and two subscribers for respective table owners.
--- 7. Insert into table_first_4 and table_second_7 at source worker1
--- 8. Expect the results in child shards on worker2
-- Create publication at worker1
\c - postgres - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
CREATE PUBLICATION pub1 FOR TABLE table_first_4, table_first_5, table_first_6;
CREATE PUBLICATION pub2 FOR TABLE table_second_7, table_second_8, table_second_9;
SELECT count(*) FROM pg_catalog.worker_split_shard_replication_setup(ARRAY[
ROW(4, 'id', 5, '-2147483648', '-1', :worker_2_node)::pg_catalog.split_shard_info,
ROW(4, 'id', 6, '0', '2147483647', :worker_2_node)::pg_catalog.split_shard_info,
ROW(7, 'id', 8, '-2147483648', '-1', :worker_2_node)::pg_catalog.split_shard_info,
ROW(7, 'id', 9, '0', '2147483647', :worker_2_node)::pg_catalog.split_shard_info
]);
SELECT relowner AS table_owner_one FROM pg_class WHERE relname='table_first' \gset
SELECT relowner AS table_owner_two FROM pg_class WHERE relname='table_second' \gset
SELECT slot_name AS slot_for_first_owner FROM pg_create_logical_replication_slot(FORMAT('citus_shard_split_%s_%s', :worker_2_node, :table_owner_one), 'citus') \gset
SELECT slot_name AS slot_for_second_owner FROM pg_create_logical_replication_slot(FORMAT('citus_shard_split_%s_%s', :worker_2_node, :table_owner_two), 'citus') \gset
-- Create subscription at worker2 with copy_data to 'false'
\c - postgres - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
SET client_min_messages TO WARNING;
CREATE SUBSCRIPTION sub1
CONNECTION 'host=localhost port=57637 user=postgres dbname=regression'
PUBLICATION pub1
WITH (
create_slot=false,
enabled=true,
slot_name=:slot_for_first_owner,
copy_data=false);
\c - myuser - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
INSERT INTO table_first_4 VALUES(100, 'a');
INSERT INTO table_first_4 VALUES(400, 'a');
INSERT INTO table_first_4 VALUES(500, 'a');
SELECT wait_for_expected_rowcount_at_table('table_first_4', 3);
SELECT * FROM table_first_4;
\c - admin_user - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
INSERT INTO table_second_7 VALUES(100, 'a');
INSERT INTO table_second_7 VALUES(400, 'a');
SELECT wait_for_expected_rowcount_at_table('table_second_7', 2);
SELECT * FROM table_second_7;
-- expect data in table_first_5/6
\c - myuser - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
SELECT * FROM table_first_4;
SELECT wait_for_expected_rowcount_at_table('table_first_5', 1);
SELECT * FROM table_first_5;
SELECT wait_for_expected_rowcount_at_table('table_first_6', 2);
SELECT * FROM table_first_6;
-- should have zero rows in all the below tables as the subscription is not yet created for admin_user
\c - admin_user - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
SELECT * FROM table_second_7;
SELECT * FROM table_second_8;
SELECT * FROM table_second_9;
\c - postgres - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
SET client_min_messages TO WARNING;
CREATE SUBSCRIPTION sub2
CONNECTION 'host=localhost port=57637 user=postgres dbname=regression'
PUBLICATION pub2
WITH (
create_slot=false,
enabled=true,
slot_name=:slot_for_second_owner,
copy_data=false);
-- expect data
\c - admin_user - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
SELECT * FROM table_second_7;
SELECT wait_for_expected_rowcount_at_table('table_second_8', 1);
SELECT * FROM table_second_8;
SELECT wait_for_expected_rowcount_at_table('table_second_9', 1);
SELECT * FROM table_second_9;
\c - postgres - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
DROP PUBLICATION pub1;
DROP PUBLICATION pub2;
\c - postgres - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
SET client_min_messages TO ERROR;
DROP SUBSCRIPTION sub1;
DROP SUBSCRIPTION sub2;

View File

@ -0,0 +1,162 @@
CREATE SCHEMA split_shard_replication_setup_schema;
SET search_path TO split_shard_replication_setup_schema;
SET citus.shard_replication_factor TO 1;
SET citus.shard_count TO 1;
SET citus.next_shard_id TO 1;
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
CREATE OR REPLACE FUNCTION wait_for_expected_rowcount_at_table(tableName text, expectedCount integer) RETURNS void AS $$
DECLARE
actualCount integer;
BEGIN
EXECUTE FORMAT('SELECT COUNT(*) FROM %s', tableName) INTO actualCount;
WHILE expectedCount != actualCount LOOP
EXECUTE FORMAT('SELECT COUNT(*) FROM %s', tableName) INTO actualCount;
END LOOP;
END$$ LANGUAGE plpgsql;
CREATE OR REPLACE FUNCTION wait_for_updated_rowcount_at_table(tableName text, expectedCount integer) RETURNS void AS $$
DECLARE
actualCount integer;
BEGIN
EXECUTE FORMAT($query$SELECT COUNT(*) FROM %s WHERE value='b'$query$, tableName) INTO actualCount;
WHILE expectedCount != actualCount LOOP
EXECUTE FORMAT($query$SELECT COUNT(*) FROM %s WHERE value='b'$query$, tableName) INTO actualCount;
END LOOP;
END$$ LANGUAGE plpgsql;
-- Create distributed table (non co-located)
CREATE TABLE table_to_split (id bigserial PRIMARY KEY, value char);
SELECT create_distributed_table('table_to_split','id');
-- Test scenario one starts from here
-- 1. table_to_split is a citus distributed table
-- 2. Shard table_to_split_1 is located on worker1.
-- 3. table_to_split_1 is split into table_to_split_2 and table_to_split_3.
-- table_to_split_2/3 are located on worker2
-- 4. execute UDF split_shard_replication_setup on worker1 with below
-- params:
-- worker_split_shard_replication_setup
-- (
-- ARRAY[
-- ROW(1 /*source shardId */, 2 /* new shardId */,-2147483648 /* minHashValue */, -1 /* maxHasValue */ , 18 /* nodeId where new shard is placed */ ),
-- ROW(1, 3 , 0 , 2147483647, 18 )
-- ]
-- );
-- 5. Create Replication slot with 'citus'
-- 6. Setup Pub/Sub
-- 7. Insert into table_to_split_1 at source worker1
-- 8. Expect the results in either table_to_split_2 or table_to_split_3 at worker2
\c - - - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
CREATE TABLE table_to_split_1(id bigserial PRIMARY KEY, value char);
CREATE TABLE table_to_split_2(id bigserial PRIMARY KEY, value char);
CREATE TABLE table_to_split_3(id bigserial PRIMARY KEY, value char);
-- Create dummy shard tables(table_to_split_2/3b) at worker1
-- This is needed for Pub/Sub framework to work.
\c - - - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
CREATE TABLE table_to_split_2(id bigserial PRIMARY KEY, value char);
CREATE TABLE table_to_split_3(id bigserial PRIMARY KEY, value char);
-- Create publication at worker1
CREATE PUBLICATION pub1 FOR TABLE table_to_split_1, table_to_split_2, table_to_split_3;
SELECT count(*) FROM pg_catalog.worker_split_shard_replication_setup(ARRAY[
ROW(1, 'id', 2, '-2147483648', '-1', :worker_2_node)::pg_catalog.split_shard_info,
ROW(1, 'id', 3, '0', '2147483647', :worker_2_node)::pg_catalog.split_shard_info
]);
SELECT slot_name FROM pg_create_logical_replication_slot(FORMAT('citus_shard_split_%s_10', :worker_2_node), 'citus') \gset
-- Create subscription at worker2 with copy_data to 'false' and derived replication slot name
\c - - - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
CREATE SUBSCRIPTION sub1
CONNECTION 'host=localhost port=57637 user=postgres dbname=regression'
PUBLICATION pub1
WITH (
create_slot=false,
enabled=true,
slot_name=:slot_name,
copy_data=false);
-- No data is present at this moment in all the below tables at worker2
SELECT * FROM table_to_split_1;
SELECT * FROM table_to_split_2;
SELECT * FROM table_to_split_3;
-- Insert data in table_to_split_1 at worker1
\c - - - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
INSERT INTO table_to_split_1 values(100, 'a');
INSERT INTO table_to_split_1 values(400, 'a');
INSERT INTO table_to_split_1 values(500, 'a');
SELECT * FROM table_to_split_1;
SELECT * FROM table_to_split_2;
SELECT * FROM table_to_split_3;
-- Expect data to be present in shard 2 and shard 3 based on the hash value.
\c - - - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
SELECT * FROM table_to_split_1; -- should alwasy have zero rows
SELECT wait_for_expected_rowcount_at_table('table_to_split_2', 1);
SELECT * FROM table_to_split_2;
SELECT wait_for_expected_rowcount_at_table('table_to_split_3', 2);
SELECT * FROM table_to_split_3;
-- UPDATE data of table_to_split_1 from worker1
\c - - - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
UPDATE table_to_split_1 SET value='b' WHERE id = 100;
UPDATE table_to_split_1 SET value='b' WHERE id = 400;
UPDATE table_to_split_1 SET value='b' WHERE id = 500;
\c - - - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
SELECT * FROM table_to_split_1;
-- Value should be updated in table_to_split_2;
SELECT wait_for_updated_rowcount_at_table('table_to_split_2', 1);
SELECT * FROM table_to_split_2;
-- Value should be updated in table_to_split_3;
SELECT wait_for_updated_rowcount_at_table('table_to_split_3', 2);
SELECT * FROM table_to_split_3;
\c - - - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
DELETE FROM table_to_split_1;
-- Child shard rows should be deleted
\c - - - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
SELECT wait_for_expected_rowcount_at_table('table_to_split_1', 0);
SELECT * FROM table_to_split_1;
SELECT wait_for_expected_rowcount_at_table('table_to_split_2', 0);
SELECT * FROM table_to_split_2;
SELECT wait_for_expected_rowcount_at_table('table_to_split_3', 0);
SELECT * FROM table_to_split_3;
-- drop publication from worker1
\c - - - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
DROP PUBLICATION pub1;
\c - - - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
SET client_min_messages TO ERROR;
DROP SUBSCRIPTION sub1;

View File

@ -0,0 +1,61 @@
-- Test scenario (parent shard and child shards are located on same machine)
-- 1. table_to_split_1 is split into table_to_split_2 and table_to_split_3.
-- 2. table_to_split_1 is located on worker1.
-- 3. table_to_split_2 and table_to_split_3 are located on worker1
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
\c - - - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
SET client_min_messages TO ERROR;
-- Create publication at worker1
CREATE PUBLICATION pub1 for table table_to_split_1, table_to_split_2, table_to_split_3;
-- Worker1 is target for table_to_split_2 and table_to_split_3
SELECT count(*) FROM pg_catalog.worker_split_shard_replication_setup(ARRAY[
ROW(1, 'id', 2, '-2147483648', '-1', :worker_1_node)::pg_catalog.split_shard_info,
ROW(1, 'id', 3, '0', '2147483647', :worker_1_node)::pg_catalog.split_shard_info
]);
SELECT slot_name AS local_slot FROM pg_create_logical_replication_slot(FORMAT('citus_shard_split_%s_10', :worker_1_node), 'citus') \gset
-- Create subscription at worker1 with copy_data to 'false' a
BEGIN;
CREATE SUBSCRIPTION local_subscription
CONNECTION 'host=localhost port=57637 user=postgres dbname=regression'
PUBLICATION pub1
WITH (
create_slot=false,
enabled=true,
slot_name=:local_slot,
copy_data=false);
COMMIT;
INSERT INTO table_to_split_1 VALUES(100, 'a');
INSERT INTO table_to_split_1 VALUES(400, 'a');
INSERT INTO table_to_split_1 VALUES(500, 'a');
-- expect data to present in table_to_split_2/3 on worker1
SELECT * FROM table_to_split_1;
SELECT wait_for_expected_rowcount_at_table('table_to_split_2', 1);
SELECT * FROM table_to_split_2;
SELECT wait_for_expected_rowcount_at_table('table_to_split_3', 2);
SELECT * FROM table_to_split_3;
DELETE FROM table_to_split_1;
SELECT wait_for_expected_rowcount_at_table('table_to_split_1', 0);
SELECT * FROM table_to_split_1;
SELECT wait_for_expected_rowcount_at_table('table_to_split_2', 0);
SELECT * FROM table_to_split_2;
SELECT wait_for_expected_rowcount_at_table('table_to_split_3', 0);
SELECT * FROM table_to_split_3;
-- clean up
DROP SUBSCRIPTION local_subscription;
DROP PUBLICATION pub1;

View File

@ -0,0 +1,99 @@
-- Test scenario (Parent and one child on same node. Other child on different node)
-- 1. table_to_split_1 is split into table_to_split_2 and table_to_split_3.
-- 2. table_to_split_1 is located on worker1.
-- 3. table_to_split_2 is located on worker1 and table_to_split_3 is located on worker2
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
\c - - - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
-- Create publication at worker1
CREATE PUBLICATION pub1 FOR TABLE table_to_split_1, table_to_split_2, table_to_split_3;
SELECT count(*) FROM pg_catalog.worker_split_shard_replication_setup(ARRAY[
ROW(1, 'id', 2, '-2147483648', '-1', :worker_1_node)::pg_catalog.split_shard_info,
ROW(1, 'id', 3, '0', '2147483647', :worker_2_node)::pg_catalog.split_shard_info
]);
SELECT slot_name AS slot_for_worker1 FROM pg_create_logical_replication_slot(FORMAT('citus_shard_split_%s_10', :worker_1_node), 'citus') \gset
SELECT slot_name AS slot_for_worker2 FROM pg_create_logical_replication_slot(FORMAT('citus_shard_split_%s_10', :worker_2_node), 'citus') \gset
-- Create subscription at worker1 with copy_data to 'false' and 'slot_for_worker1'
CREATE SUBSCRIPTION sub_worker1
CONNECTION 'host=localhost port=57637 user=postgres dbname=regression'
PUBLICATION pub1
WITH (
create_slot=false,
enabled=true,
slot_name=:slot_for_worker1,
copy_data=false);
\c - - - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
-- Create subscription at worker2 with copy_data to 'false' and 'slot_for_worker2'
CREATE SUBSCRIPTION sub_worker2
CONNECTION 'host=localhost port=57637 user=postgres dbname=regression'
PUBLICATION pub1
WITH (
create_slot=false,
enabled=true,
slot_name=:slot_for_worker2,
copy_data=false);
-- No data is present at this moment in all the below tables at worker2
SELECT * FROM table_to_split_1;
SELECT * FROM table_to_split_2;
SELECT * FROM table_to_split_3;
-- Insert data in table_to_split_1 at worker1
\c - - - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
INSERT INTO table_to_split_1 VALUES(100, 'a');
INSERT INTO table_to_split_1 VALUES(400, 'a');
INSERT INTO table_to_split_1 VALUES(500, 'a');
UPDATE table_to_split_1 SET value='b' WHERE id = 400;
SELECT * FROM table_to_split_1;
-- expect data to present in table_to_split_2 on worker1 as its destination for value '400'
SELECT wait_for_expected_rowcount_at_table('table_to_split_2', 1);
SELECT * FROM table_to_split_2;
SELECT * FROM table_to_split_3;
-- Expect data to be present only in table_to_split3 on worker2
\c - - - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
SELECT * FROM table_to_split_1;
SELECT * FROM table_to_split_2;
SELECT wait_for_expected_rowcount_at_table('table_to_split_3', 2);
SELECT * FROM table_to_split_3;
-- delete all from table_to_split_1
\c - - - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
DELETE FROM table_to_split_1;
-- rows from table_to_split_2 should be deleted
SELECT wait_for_expected_rowcount_at_table('table_to_split_2', 0);
SELECT * FROM table_to_split_2;
-- rows from table_to_split_3 should be deleted
\c - - - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
SELECT wait_for_expected_rowcount_at_table('table_to_split_3', 0);
SELECT * FROM table_to_split_3;
\c - - - :worker_2_port
SET search_path TO split_shard_replication_setup_schema;
SET client_min_messages TO ERROR;
DROP SUBSCRIPTION sub_worker2;
-- drop publication from worker1
\c - - - :worker_1_port
SET search_path TO split_shard_replication_setup_schema;
SET client_min_messages TO ERROR;
DROP SUBSCRIPTION sub_worker1;
DROP PUBLICATION pub1;