diff --git a/src/backend/distributed/Makefile b/src/backend/distributed/Makefile index 69dbc0cfd..608f60c0b 100644 --- a/src/backend/distributed/Makefile +++ b/src/backend/distributed/Makefile @@ -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 diff --git a/src/backend/distributed/connection/connection_configuration.c b/src/backend/distributed/connection/connection_configuration.c index 3e3766a44..63dfebf8b 100644 --- a/src/backend/distributed/connection/connection_configuration.c +++ b/src/backend/distributed/connection/connection_configuration.c @@ -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 */ diff --git a/src/backend/distributed/connection/connection_management.c b/src/backend/distributed/connection/connection_management.c index bb42514ce..577378803 100644 --- a/src/backend/distributed/connection/connection_management.c +++ b/src/backend/distributed/connection/connection_management.c @@ -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); diff --git a/src/backend/distributed/metadata/metadata_cache.c b/src/backend/distributed/metadata/metadata_cache.c index 1520384f3..83fc33720 100644 --- a/src/backend/distributed/metadata/metadata_cache.c +++ b/src/backend/distributed/metadata/metadata_cache.c @@ -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. diff --git a/src/backend/distributed/metadata/metadata_utility.c b/src/backend/distributed/metadata/metadata_utility.c index 0b64f5726..ff85f6930 100644 --- a/src/backend/distributed/metadata/metadata_utility.c +++ b/src/backend/distributed/metadata/metadata_utility.c @@ -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 diff --git a/src/backend/distributed/operations/citus_split_shard_by_split_points.c b/src/backend/distributed/operations/citus_split_shard_by_split_points.c index 11e979900..9cfb80e18 100644 --- a/src/backend/distributed/operations/citus_split_shard_by_split_points.c +++ b/src/backend/distributed/operations/citus_split_shard_by_split_points.c @@ -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))); } diff --git a/src/backend/distributed/operations/shard_split.c b/src/backend/distributed/operations/shard_split.c index 76687434e..b69792514 100644 --- a/src/backend/distributed/operations/shard_split.c +++ b/src/backend/distributed/operations/shard_split.c @@ -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: + * . + */ + 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 : [] + * 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: + * + */ +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; +} diff --git a/src/backend/distributed/operations/worker_node_manager.c b/src/backend/distributed/operations/worker_node_manager.c index c516e27ef..9bab18c96 100644 --- a/src/backend/distributed/operations/worker_node_manager.c +++ b/src/backend/distributed/operations/worker_node_manager.c @@ -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. diff --git a/src/backend/distributed/operations/worker_split_shard_replication_setup_udf.c b/src/backend/distributed/operations/worker_split_shard_replication_setup_udf.c new file mode 100644 index 000000000..6b039283c --- /dev/null +++ b/src/backend/distributed/operations/worker_split_shard_replication_setup_udf.c @@ -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 + * 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); + } +} diff --git a/src/backend/distributed/replication/multi_logical_replication.c b/src/backend/distributed/replication/multi_logical_replication.c index 2919abd97..472ca6c73 100644 --- a/src/backend/distributed/replication/multi_logical_replication.c +++ b/src/backend/distributed/replication/multi_logical_replication.c @@ -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) + )); +} diff --git a/src/backend/distributed/shardsplit/shardsplit_decoder.c b/src/backend/distributed/shardsplit/shardsplit_decoder.c new file mode 100644 index 000000000..f6de28ca3 --- /dev/null +++ b/src/backend/distributed/shardsplit/shardsplit_decoder.c @@ -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); +} diff --git a/src/backend/distributed/shardsplit/shardsplit_logical_replication.c b/src/backend/distributed/shardsplit/shardsplit_logical_replication.c new file mode 100644 index 000000000..ef3034c03 --- /dev/null +++ b/src/backend/distributed/shardsplit/shardsplit_logical_replication.c @@ -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 + * ======= ------ ------ + * ------> |Shard2|-->|Shard1| + * ------ ------ + * + * ------ ------ + * ------> |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 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 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 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); + } +} diff --git a/src/backend/distributed/shardsplit/shardsplit_shared_memory.c b/src/backend/distributed/shardsplit/shardsplit_shared_memory.c new file mode 100644 index 000000000..c22a9a084 --- /dev/null +++ b/src/backend/distributed/shardsplit/shardsplit_shared_memory.c @@ -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; +} diff --git a/src/backend/distributed/shared_library_init.c b/src/backend/distributed/shared_library_init.c index a426639c4..a151dcd6b 100644 --- a/src/backend/distributed/shared_library_init.c +++ b/src/backend/distributed/shared_library_init.c @@ -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) { diff --git a/src/backend/distributed/sql/citus--11.0-3--11.1-1.sql b/src/backend/distributed/sql/citus--11.0-3--11.1-1.sql index e71f9362b..5d09a3aea 100644 --- a/src/backend/distributed/sql/citus--11.0-3--11.1-1.sql +++ b/src/backend/distributed/sql/citus--11.0-3--11.1-1.sql @@ -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" diff --git a/src/backend/distributed/sql/downgrades/citus--11.1-1--11.0-3.sql b/src/backend/distributed/sql/downgrades/citus--11.1-1--11.0-3.sql index 7261a31db..002ea471b 100644 --- a/src/backend/distributed/sql/downgrades/citus--11.1-1--11.0-3.sql +++ b/src/backend/distributed/sql/downgrades/citus--11.1-1--11.0-3.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); diff --git a/src/backend/distributed/sql/udfs/worker_split_shard_replication_setup/11.1-1.sql b/src/backend/distributed/sql/udfs/worker_split_shard_replication_setup/11.1-1.sql new file mode 100644 index 000000000..3ee131d45 --- /dev/null +++ b/src/backend/distributed/sql/udfs/worker_split_shard_replication_setup/11.1-1.sql @@ -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; diff --git a/src/backend/distributed/sql/udfs/worker_split_shard_replication_setup/latest.sql b/src/backend/distributed/sql/udfs/worker_split_shard_replication_setup/latest.sql new file mode 100644 index 000000000..3ee131d45 --- /dev/null +++ b/src/backend/distributed/sql/udfs/worker_split_shard_replication_setup/latest.sql @@ -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; diff --git a/src/backend/distributed/utils/citus_safe_lib.c b/src/backend/distributed/utils/citus_safe_lib.c index 466c598be..0b830fafc 100644 --- a/src/backend/distributed/utils/citus_safe_lib.c +++ b/src/backend/distributed/utils/citus_safe_lib.c @@ -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. diff --git a/src/include/distributed/citus_safe_lib.h b/src/include/distributed/citus_safe_lib.h index c934e931b..e039072a7 100644 --- a/src/include/distributed/citus_safe_lib.h +++ b/src/include/distributed/citus_safe_lib.h @@ -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 *)); diff --git a/src/include/distributed/connection_management.h b/src/include/distributed/connection_management.h index 5c2e07fe8..6c3b8ae8d 100644 --- a/src/include/distributed/connection_management.h +++ b/src/include/distributed/connection_management.h @@ -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 */ diff --git a/src/include/distributed/metadata_utility.h b/src/include/distributed/metadata_utility.h index bad361ae6..56e15d171 100644 --- a/src/include/distributed/metadata_utility.h +++ b/src/include/distributed/metadata_utility.h @@ -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); diff --git a/src/include/distributed/multi_logical_replication.h b/src/include/distributed/multi_logical_replication.h index 629179c54..783cd4daf 100644 --- a/src/include/distributed/multi_logical_replication.h +++ b/src/include/distributed/multi_logical_replication.h @@ -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_ */ diff --git a/src/include/distributed/shard_split.h b/src/include/distributed/shard_split.h index 790e3d612..11aeb6c00 100644 --- a/src/include/distributed/shard_split.h +++ b/src/include/distributed/shard_split.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. diff --git a/src/include/distributed/shardsplit_logical_replication.h b/src/include/distributed/shardsplit_logical_replication.h new file mode 100644 index 000000000..28138dd2a --- /dev/null +++ b/src/include/distributed/shardsplit_logical_replication.h @@ -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 + * 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 */ diff --git a/src/include/distributed/shardsplit_shared_memory.h b/src/include/distributed/shardsplit_shared_memory.h new file mode 100644 index 000000000..e4d6e910d --- /dev/null +++ b/src/include/distributed/shardsplit_shared_memory.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 */ diff --git a/src/include/distributed/worker_manager.h b/src/include/distributed/worker_manager.h index cf1458047..1c5b27f1f 100644 --- a/src/include/distributed/worker_manager.h +++ b/src/include/distributed/worker_manager.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); diff --git a/src/test/regress/enterprise_isolation_logicalrep_1_schedule b/src/test/regress/enterprise_isolation_logicalrep_1_schedule index 8a6f35af5..96cc9915e 100644 --- a/src/test/regress/enterprise_isolation_logicalrep_1_schedule +++ b/src/test/regress/enterprise_isolation_logicalrep_1_schedule @@ -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 diff --git a/src/test/regress/enterprise_split_schedule b/src/test/regress/enterprise_split_schedule new file mode 100644 index 000000000..6f216ea44 --- /dev/null +++ b/src/test/regress/enterprise_split_schedule @@ -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 diff --git a/src/test/regress/expected/citus_non_blocking_shard_split_cleanup.out b/src/test/regress/expected/citus_non_blocking_shard_split_cleanup.out new file mode 100644 index 000000000..2cfa1a716 --- /dev/null +++ b/src/test/regress/expected/citus_non_blocking_shard_split_cleanup.out @@ -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) + diff --git a/src/test/regress/expected/citus_non_blocking_split_columnar.out b/src/test/regress/expected/citus_non_blocking_split_columnar.out new file mode 100644 index 000000000..c54b3f827 --- /dev/null +++ b/src/test/regress/expected/citus_non_blocking_split_columnar.out @@ -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 diff --git a/src/test/regress/expected/citus_non_blocking_split_shard_cleanup.out b/src/test/regress/expected/citus_non_blocking_split_shard_cleanup.out new file mode 100644 index 000000000..afa3c89e4 --- /dev/null +++ b/src/test/regress/expected/citus_non_blocking_split_shard_cleanup.out @@ -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 diff --git a/src/test/regress/expected/citus_non_blocking_split_shards.out b/src/test/regress/expected/citus_non_blocking_split_shards.out new file mode 100644 index 000000000..7a3efbc7b --- /dev/null +++ b/src/test/regress/expected/citus_non_blocking_split_shards.out @@ -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 diff --git a/src/test/regress/expected/citus_split_shard_by_split_points_negative.out b/src/test/regress/expected/citus_split_shard_by_split_points_negative.out index 2445c1e18..1bb3fb12d 100644 --- a/src/test/regress/expected/citus_split_shard_by_split_points_negative.out +++ b/src/test/regress/expected/citus_split_shard_by_split_points_negative.out @@ -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'], diff --git a/src/test/regress/expected/isolation_non_blocking_shard_split.out b/src/test/regress/expected/isolation_non_blocking_shard_split.out new file mode 100644 index 000000000..24af03012 --- /dev/null +++ b/src/test/regress/expected/isolation_non_blocking_shard_split.out @@ -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'); + +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'); + +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'); + +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'); + +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); + +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'); + +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'); + +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'); + +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) + diff --git a/src/test/regress/expected/isolation_non_blocking_shard_split_fkey.out b/src/test/regress/expected/isolation_non_blocking_shard_split_fkey.out new file mode 100644 index 000000000..3821c9305 --- /dev/null +++ b/src/test/regress/expected/isolation_non_blocking_shard_split_fkey.out @@ -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'); + +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'); + +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'); + +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) + diff --git a/src/test/regress/expected/isolation_non_blocking_shard_split_with_index_as_replicaIdentity.out b/src/test/regress/expected/isolation_non_blocking_shard_split_with_index_as_replicaIdentity.out new file mode 100644 index 000000000..02b1fd9e4 --- /dev/null +++ b/src/test/regress/expected/isolation_non_blocking_shard_split_with_index_as_replicaIdentity.out @@ -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'); + +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'); + +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'); + +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'); + +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'); + +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'); + +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) + diff --git a/src/test/regress/expected/multi_extension.out b/src/test/regress/expected/multi_extension.out index 0067bdbad..92d54c064 100644 --- a/src/test/regress/expected/multi_extension.out +++ b/src/test/regress/expected/multi_extension.out @@ -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 diff --git a/src/test/regress/expected/split_shard_replication_colocated_setup.out b/src/test/regress/expected/split_shard_replication_colocated_setup.out new file mode 100644 index 000000000..29c535cd1 --- /dev/null +++ b/src/test/regress/expected/split_shard_replication_colocated_setup.out @@ -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; diff --git a/src/test/regress/expected/split_shard_replication_setup.out b/src/test/regress/expected/split_shard_replication_setup.out new file mode 100644 index 000000000..be0a46d4b --- /dev/null +++ b/src/test/regress/expected/split_shard_replication_setup.out @@ -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; diff --git a/src/test/regress/expected/split_shard_replication_setup_local.out b/src/test/regress/expected/split_shard_replication_setup_local.out new file mode 100644 index 000000000..77bdf8336 --- /dev/null +++ b/src/test/regress/expected/split_shard_replication_setup_local.out @@ -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; diff --git a/src/test/regress/expected/split_shard_replication_setup_remote_local.out b/src/test/regress/expected/split_shard_replication_setup_remote_local.out new file mode 100644 index 000000000..1f41519ca --- /dev/null +++ b/src/test/regress/expected/split_shard_replication_setup_remote_local.out @@ -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; diff --git a/src/test/regress/expected/split_shard_test_helpers.out b/src/test/regress/expected/split_shard_test_helpers.out new file mode 100644 index 000000000..ade92b566 --- /dev/null +++ b/src/test/regress/expected/split_shard_test_helpers.out @@ -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; diff --git a/src/test/regress/expected/upgrade_list_citus_objects.out b/src/test/regress/expected/upgrade_list_citus_objects.out index ecf0e0e4d..532b61517 100644 --- a/src/test/regress/expected/upgrade_list_citus_objects.out +++ b/src/test/regress/expected/upgrade_list_citus_objects.out @@ -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) diff --git a/src/test/regress/spec/isolation_non_blocking_shard_split.spec b/src/test/regress/spec/isolation_non_blocking_shard_split.spec new file mode 100644 index 000000000..deb1b61a8 --- /dev/null +++ b/src/test/regress/spec/isolation_non_blocking_shard_split.spec @@ -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" diff --git a/src/test/regress/spec/isolation_non_blocking_shard_split_fkey.spec b/src/test/regress/spec/isolation_non_blocking_shard_split_fkey.spec new file mode 100644 index 000000000..2aee89f53 --- /dev/null +++ b/src/test/regress/spec/isolation_non_blocking_shard_split_fkey.spec @@ -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" diff --git a/src/test/regress/spec/isolation_non_blocking_shard_split_with_index_as_replicaIdentity.spec b/src/test/regress/spec/isolation_non_blocking_shard_split_with_index_as_replicaIdentity.spec new file mode 100644 index 000000000..be4371e26 --- /dev/null +++ b/src/test/regress/spec/isolation_non_blocking_shard_split_with_index_as_replicaIdentity.spec @@ -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" diff --git a/src/test/regress/split_schedule b/src/test/regress/split_schedule index 37d9746f1..eaa8eb799 100644 --- a/src/test/regress/split_schedule +++ b/src/test/regress/split_schedule @@ -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 diff --git a/src/test/regress/sql/citus_non_blocking_split_columnar.sql b/src/test/regress/sql/citus_non_blocking_split_columnar.sql new file mode 100644 index 000000000..7d65997a3 --- /dev/null +++ b/src/test/regress/sql/citus_non_blocking_split_columnar.sql @@ -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 diff --git a/src/test/regress/sql/citus_non_blocking_split_shard_cleanup.sql b/src/test/regress/sql/citus_non_blocking_split_shard_cleanup.sql new file mode 100644 index 000000000..334c07b1a --- /dev/null +++ b/src/test/regress/sql/citus_non_blocking_split_shard_cleanup.sql @@ -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 diff --git a/src/test/regress/sql/citus_non_blocking_split_shards.sql b/src/test/regress/sql/citus_non_blocking_split_shards.sql new file mode 100644 index 000000000..d6569debf --- /dev/null +++ b/src/test/regress/sql/citus_non_blocking_split_shards.sql @@ -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 diff --git a/src/test/regress/sql/citus_split_shard_by_split_points_negative.sql b/src/test/regress/sql/citus_split_shard_by_split_points_negative.sql index ad76c5c7c..ebcd79c5e 100644 --- a/src/test/regress/sql/citus_split_shard_by_split_points_negative.sql +++ b/src/test/regress/sql/citus_split_shard_by_split_points_negative.sql @@ -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'], diff --git a/src/test/regress/sql/split_shard_replication_colocated_setup.sql b/src/test/regress/sql/split_shard_replication_colocated_setup.sql new file mode 100644 index 000000000..871bf49b5 --- /dev/null +++ b/src/test/regress/sql/split_shard_replication_colocated_setup.sql @@ -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; diff --git a/src/test/regress/sql/split_shard_replication_setup.sql b/src/test/regress/sql/split_shard_replication_setup.sql new file mode 100644 index 000000000..97e1d275f --- /dev/null +++ b/src/test/regress/sql/split_shard_replication_setup.sql @@ -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; + diff --git a/src/test/regress/sql/split_shard_replication_setup_local.sql b/src/test/regress/sql/split_shard_replication_setup_local.sql new file mode 100644 index 000000000..40cbd4063 --- /dev/null +++ b/src/test/regress/sql/split_shard_replication_setup_local.sql @@ -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; diff --git a/src/test/regress/sql/split_shard_replication_setup_remote_local.sql b/src/test/regress/sql/split_shard_replication_setup_remote_local.sql new file mode 100644 index 000000000..47e0c2aa3 --- /dev/null +++ b/src/test/regress/sql/split_shard_replication_setup_remote_local.sql @@ -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;