mirror of https://github.com/citusdata/citus.git
Introduce Non-Blocking Shard Split Workflow
parent
b67abdd28c
commit
e236711eea
|
@ -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
|
||||
|
||||
|
|
|
@ -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 */
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)));
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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)
|
||||
));
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -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)
|
||||
{
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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);
|
||||
|
|
25
src/backend/distributed/sql/udfs/worker_split_shard_replication_setup/11.1-1.sql
generated
Normal file
25
src/backend/distributed/sql/udfs/worker_split_shard_replication_setup/11.1-1.sql
generated
Normal 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;
|
|
@ -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;
|
|
@ -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.
|
||||
|
|
|
@ -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 *));
|
||||
|
|
|
@ -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 */
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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_ */
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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 */
|
|
@ -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 */
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
|
@ -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)
|
||||
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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'],
|
||||
|
|
|
@ -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)
|
||||
|
|
@ -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)
|
||||
|
|
@ -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)
|
||||
|
|
@ -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
|
||||
|
|
|
@ -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;
|
|
@ -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;
|
|
@ -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;
|
|
@ -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;
|
|
@ -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;
|
|
@ -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)
|
||||
|
||||
|
|
|
@ -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"
|
|
@ -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"
|
|
@ -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"
|
|
@ -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
|
||||
|
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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'],
|
||||
|
|
|
@ -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;
|
|
@ -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;
|
||||
|
|
@ -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;
|
|
@ -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;
|
Loading…
Reference in New Issue