Add non-blocking variant of create_distributed_table (#6087)

Added create_distributed_table_concurrently which is nonblocking variant of create_distributed_table.

It bases on the split API which takes advantage of logical replication to support nonblocking split operations.

Co-authored-by: Marco Slot <marco.slot@gmail.com>
Co-authored-by: aykutbozkurt <aykut.bozkurt1995@gmail.com>
pull/6273/head
Marco Slot 2022-08-30 14:35:40 +02:00 committed by GitHub
parent d68654680b
commit 6bb31c5d75
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
60 changed files with 4062 additions and 409 deletions

View File

@ -42,6 +42,7 @@
#include "distributed/colocation_utils.h" #include "distributed/colocation_utils.h"
#include "distributed/commands.h" #include "distributed/commands.h"
#include "distributed/deparser.h" #include "distributed/deparser.h"
#include "distributed/distributed_execution_locks.h"
#include "distributed/distribution_column.h" #include "distributed/distribution_column.h"
#include "distributed/listutils.h" #include "distributed/listutils.h"
#include "distributed/local_executor.h" #include "distributed/local_executor.h"
@ -59,12 +60,16 @@
#include "distributed/reference_table_utils.h" #include "distributed/reference_table_utils.h"
#include "distributed/relation_access_tracking.h" #include "distributed/relation_access_tracking.h"
#include "distributed/remote_commands.h" #include "distributed/remote_commands.h"
#include "distributed/repair_shards.h"
#include "distributed/resource_lock.h" #include "distributed/resource_lock.h"
#include "distributed/shard_rebalancer.h"
#include "distributed/shard_split.h"
#include "distributed/shared_library_init.h" #include "distributed/shared_library_init.h"
#include "distributed/shard_rebalancer.h" #include "distributed/shard_rebalancer.h"
#include "distributed/worker_protocol.h" #include "distributed/worker_protocol.h"
#include "distributed/worker_shard_visibility.h" #include "distributed/worker_shard_visibility.h"
#include "distributed/worker_transaction.h" #include "distributed/worker_transaction.h"
#include "distributed/utils/distribution_column_map.h"
#include "distributed/version_compat.h" #include "distributed/version_compat.h"
#include "executor/executor.h" #include "executor/executor.h"
#include "executor/spi.h" #include "executor/spi.h"
@ -76,6 +81,7 @@
#include "parser/parse_node.h" #include "parser/parse_node.h"
#include "parser/parse_relation.h" #include "parser/parse_relation.h"
#include "parser/parser.h" #include "parser/parser.h"
#include "postmaster/postmaster.h"
#include "storage/lmgr.h" #include "storage/lmgr.h"
#include "tcop/pquery.h" #include "tcop/pquery.h"
#include "tcop/tcopprot.h" #include "tcop/tcopprot.h"
@ -93,8 +99,18 @@
#define LOG_PER_TUPLE_AMOUNT 1000000 #define LOG_PER_TUPLE_AMOUNT 1000000
/* local function forward declarations */ /* local function forward declarations */
static void CreateDistributedTableConcurrently(Oid relationId,
char *distributionColumnName,
char distributionMethod,
char *colocateWithTableName,
int shardCount,
bool shardCountIsStrict);
static char DecideReplicationModel(char distributionMethod, char *colocateWithTableName, static char DecideReplicationModel(char distributionMethod, char *colocateWithTableName,
bool viaDeprecatedAPI); bool viaDeprecatedAPI);
static List * HashSplitPointsForShardList(List *shardList);
static List * HashSplitPointsForShardCount(int shardCount);
static List * WorkerNodesForShardList(List *shardList);
static List * RoundRobinWorkerNodeList(List *workerNodeList, int listLength);
static void CreateHashDistributedTableShards(Oid relationId, int shardCount, static void CreateHashDistributedTableShards(Oid relationId, int shardCount,
Oid colocatedTableId, bool localTableEmpty); Oid colocatedTableId, bool localTableEmpty);
static uint32 ColocationIdForNewTable(Oid relationId, Var *distributionColumn, static uint32 ColocationIdForNewTable(Oid relationId, Var *distributionColumn,
@ -105,9 +121,6 @@ static uint32 ColocationIdForNewTable(Oid relationId, Var *distributionColumn,
static void EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn, static void EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn,
char distributionMethod, uint32 colocationId, char distributionMethod, uint32 colocationId,
char replicationModel, bool viaDeprecatedAPI); char replicationModel, bool viaDeprecatedAPI);
static void EnsureTableCanBeColocatedWith(Oid relationId, char replicationModel,
Oid distributionColumnType,
Oid sourceRelationId);
static void EnsureLocalTableEmpty(Oid relationId); static void EnsureLocalTableEmpty(Oid relationId);
static void EnsureRelationHasNoTriggers(Oid relationId); static void EnsureRelationHasNoTriggers(Oid relationId);
static Oid SupportFunctionForColumn(Var *partitionColumn, Oid accessMethodId, static Oid SupportFunctionForColumn(Var *partitionColumn, Oid accessMethodId,
@ -117,6 +130,7 @@ static void EnsureLocalTableEmptyIfNecessary(Oid relationId, char distributionMe
static bool ShouldLocalTableBeEmpty(Oid relationId, char distributionMethod, bool static bool ShouldLocalTableBeEmpty(Oid relationId, char distributionMethod, bool
viaDeprecatedAPI); viaDeprecatedAPI);
static void EnsureCitusTableCanBeCreated(Oid relationOid); static void EnsureCitusTableCanBeCreated(Oid relationOid);
static void PropagatePrerequisiteObjectsForDistributedTable(Oid relationId);
static void EnsureDistributedSequencesHaveOneType(Oid relationId, static void EnsureDistributedSequencesHaveOneType(Oid relationId,
List *seqInfoList); List *seqInfoList);
static List * GetFKeyCreationCommandsRelationInvolvedWithTableType(Oid relationId, static List * GetFKeyCreationCommandsRelationInvolvedWithTableType(Oid relationId,
@ -134,9 +148,17 @@ static void DoCopyFromLocalTableIntoShards(Relation distributedRelation,
EState *estate); EState *estate);
static void ErrorIfTemporaryTable(Oid relationId); static void ErrorIfTemporaryTable(Oid relationId);
static void ErrorIfForeignTable(Oid relationOid); static void ErrorIfForeignTable(Oid relationOid);
static void SendAddLocalTableToMetadataCommandOutsideTransaction(Oid relationId);
static void EnsureDistributableTable(Oid relationId);
static void EnsureForeignKeysForDistributedTableConcurrently(Oid relationId);
static void EnsureColocateWithTableIsValid(Oid relationId, char distributionMethod,
char *distributionColumnName,
char *colocateWithTableName);
static void WarnIfTableHaveNoReplicaIdentity(Oid relationId);
/* exports for SQL callable functions */ /* exports for SQL callable functions */
PG_FUNCTION_INFO_V1(master_create_distributed_table); PG_FUNCTION_INFO_V1(master_create_distributed_table);
PG_FUNCTION_INFO_V1(create_distributed_table_concurrently);
PG_FUNCTION_INFO_V1(create_distributed_table); PG_FUNCTION_INFO_V1(create_distributed_table);
PG_FUNCTION_INFO_V1(create_reference_table); PG_FUNCTION_INFO_V1(create_reference_table);
@ -254,6 +276,589 @@ create_distributed_table(PG_FUNCTION_ARGS)
} }
/*
* create_distributed_concurrently gets a table name, distribution column,
* distribution method and colocate_with option, then it creates a
* distributed table.
*/
Datum
create_distributed_table_concurrently(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
if (PG_ARGISNULL(0) || PG_ARGISNULL(1) || PG_ARGISNULL(2) || PG_ARGISNULL(3))
{
PG_RETURN_VOID();
}
Oid relationId = PG_GETARG_OID(0);
text *distributionColumnText = PG_GETARG_TEXT_P(1);
char *distributionColumnName = text_to_cstring(distributionColumnText);
Oid distributionMethodOid = PG_GETARG_OID(2);
char distributionMethod = LookupDistributionMethod(distributionMethodOid);
text *colocateWithTableNameText = PG_GETARG_TEXT_P(3);
char *colocateWithTableName = text_to_cstring(colocateWithTableNameText);
bool shardCountIsStrict = false;
int shardCount = ShardCount;
if (!PG_ARGISNULL(4))
{
if (pg_strncasecmp(colocateWithTableName, "default", NAMEDATALEN) != 0 &&
pg_strncasecmp(colocateWithTableName, "none", NAMEDATALEN) != 0)
{
ereport(ERROR, (errmsg("Cannot use colocate_with with a table "
"and shard_count at the same time")));
}
shardCount = PG_GETARG_INT32(4);
/*
* if shard_count parameter is given than we have to
* make sure table has that many shards
*/
shardCountIsStrict = true;
}
CreateDistributedTableConcurrently(relationId, distributionColumnName,
distributionMethod,
colocateWithTableName,
shardCount,
shardCountIsStrict);
PG_RETURN_VOID();
}
/*
* CreateDistributedTableConcurrently distributes a table by first converting
* it to a Citus local table and then splitting the shard of the Citus local
* table.
*
* If anything goes wrong during the second phase, the table is left as a
* Citus local table.
*/
static void
CreateDistributedTableConcurrently(Oid relationId, char *distributionColumnName,
char distributionMethod,
char *colocateWithTableName,
int shardCount,
bool shardCountIsStrict)
{
/*
* We disallow create_distributed_table_concurrently in transaction blocks
* because we cannot handle preceding writes, and we block writes at the
* very end of the operation so the transaction should end immediately after.
*/
PreventInTransactionBlock(true, "create_distributed_table_concurrently");
/*
* do not allow multiple create_distributed_table_concurrently in the same
* transaction. We should do that check just here because concurrent local table
* conversion can cause issues.
*/
ErrorIfMultipleNonblockingMoveSplitInTheSameTransaction();
/* do not allow concurrent CreateDistributedTableConcurrently operations */
AcquireCreateDistributedTableConcurrentlyLock(relationId);
if (distributionMethod != DISTRIBUTE_BY_HASH)
{
ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("only hash-distributed tables can be distributed "
"without blocking writes")));
}
if (ShardReplicationFactor > 1)
{
ereport(ERROR, (errmsg("cannot distribute a table concurrently when "
"citus.shard_replication_factor > 1")));
}
EnsureCoordinatorIsInMetadata();
EnsureCitusTableCanBeCreated(relationId);
EnsureValidDistributionColumn(relationId, distributionColumnName);
/*
* Ensure table type is valid to be distributed. It should be either regular or citus local table.
*/
EnsureDistributableTable(relationId);
/*
* we rely on citus_add_local_table_to_metadata, so it can generate irrelevant messages.
* we want to error with a user friendly message if foreign keys are not supported.
* We can miss foreign key violations because we are not holding locks, so relation
* can be modified until we acquire the lock for the relation, but we do as much as we can
* to be user friendly on foreign key violation messages.
*/
EnsureForeignKeysForDistributedTableConcurrently(relationId);
bool viaDeprecatedAPI = false;
char replicationModel = DecideReplicationModel(distributionMethod,
colocateWithTableName,
viaDeprecatedAPI);
/*
* we fail transaction before local table conversion if the table could not be colocated with
* given table. We should make those checks after local table conversion by acquiring locks to
* the relation because the distribution column can be modified in that period.
*/
if (!IsColocateWithDefault(colocateWithTableName) && !IsColocateWithNone(
colocateWithTableName))
{
EnsureColocateWithTableIsValid(relationId, distributionMethod,
distributionColumnName,
colocateWithTableName);
}
/*
* Get name of the table before possibly replacing it in
* citus_add_local_table_to_metadata.
*/
char *tableName = get_rel_name(relationId);
Oid schemaId = get_rel_namespace(relationId);
char *schemaName = get_namespace_name(schemaId);
RangeVar *rangeVar = makeRangeVar(schemaName, tableName, -1);
/* If table is a regular table, then we need to add it into metadata. */
if (!IsCitusTable(relationId))
{
/*
* Before taking locks, convert the table into a Citus local table and commit
* to allow shard split to see the shard.
*/
SendAddLocalTableToMetadataCommandOutsideTransaction(relationId);
}
/*
* Lock target relation with a shard update exclusive lock to
* block DDL, but not writes.
*
* If there was a concurrent drop/rename, error out by setting missingOK = false.
*/
bool missingOK = false;
relationId = RangeVarGetRelid(rangeVar, ShareUpdateExclusiveLock, missingOK);
if (PartitionedTableNoLock(relationId))
{
/* also lock partitions */
LockPartitionRelations(relationId, ShareUpdateExclusiveLock);
}
WarnIfTableHaveNoReplicaIdentity(relationId);
List *shardList = LoadShardIntervalList(relationId);
/*
* It's technically possible for the table to have been concurrently
* distributed just after citus_add_local_table_to_metadata and just
* before acquiring the lock, so double check.
*/
if (list_length(shardList) != 1 ||
!IsCitusTableType(relationId, CITUS_LOCAL_TABLE))
{
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("table was concurrently modified")));
}
/*
* The table currently has one shard, we will split that shard to match the
* target distribution.
*/
ShardInterval *shardToSplit = (ShardInterval *) linitial(shardList);
PropagatePrerequisiteObjectsForDistributedTable(relationId);
/*
* we should re-evaluate distribution column values. It may have changed,
* because we did not lock the relation at the previous check before local
* table conversion.
*/
Var *distributionColumn = BuildDistributionKeyFromColumnName(relationId,
distributionColumnName,
NoLock);
Oid distributionColumnType = distributionColumn->vartype;
Oid distributionColumnCollation = distributionColumn->varcollid;
/* get an advisory lock to serialize concurrent default group creations */
if (IsColocateWithDefault(colocateWithTableName))
{
AcquireColocationDefaultLock();
}
/*
* At this stage, we only want to check for an existing co-location group.
* We cannot create a new co-location group until after replication slot
* creation in NonBlockingShardSplit.
*/
uint32 colocationId = FindColocateWithColocationId(relationId,
replicationModel,
distributionColumnType,
distributionColumnCollation,
shardCount,
shardCountIsStrict,
colocateWithTableName);
if (IsColocateWithDefault(colocateWithTableName) && (colocationId !=
INVALID_COLOCATION_ID))
{
/*
* we can release advisory lock if there is already a default entry for given params;
* else, we should keep it to prevent different default coloc entry creation by
* concurrent operations.
*/
ReleaseColocationDefaultLock();
}
EnsureRelationCanBeDistributed(relationId, distributionColumn, distributionMethod,
colocationId, replicationModel, viaDeprecatedAPI);
Oid colocatedTableId = InvalidOid;
if (colocationId != INVALID_COLOCATION_ID)
{
colocatedTableId = ColocatedTableId(colocationId);
}
List *workersForPlacementList;
List *shardSplitPointsList;
if (colocatedTableId != InvalidOid)
{
List *colocatedShardList = LoadShardIntervalList(colocatedTableId);
/*
* Match the shard ranges of an existing table.
*/
shardSplitPointsList = HashSplitPointsForShardList(colocatedShardList);
/*
* Find the node IDs of the shard placements.
*/
workersForPlacementList = WorkerNodesForShardList(colocatedShardList);
}
else
{
/*
* Generate a new set of #shardCount shards.
*/
shardSplitPointsList = HashSplitPointsForShardCount(shardCount);
/*
* Place shards in a round-robin fashion across all data nodes.
*/
List *workerNodeList = DistributedTablePlacementNodeList(NoLock);
workersForPlacementList = RoundRobinWorkerNodeList(workerNodeList, shardCount);
}
/*
* Make sure that existing reference tables have been replicated to all the nodes
* such that we can create foreign keys and joins work immediately after creation.
* We do this after applying all essential checks to error out early in case of
* user error.
*/
EnsureReferenceTablesExistOnAllNodes();
/*
* At this point, the table is a Citus local table, which means it does
* not have a partition column in the metadata. However, we cannot update
* the metadata here because that would prevent us from creating a replication
* slot to copy ongoing changes. Instead, we pass a hash that maps relation
* IDs to partition column vars.
*/
DistributionColumnMap *distributionColumnOverrides = CreateDistributionColumnMap();
AddDistributionColumnForRelation(distributionColumnOverrides, relationId,
distributionColumnName);
/*
* there is no colocation entries yet for local table, so we should
* check if table has any partition and add them to same colocation
* group
*/
List *sourceColocatedShardIntervalList = ListShardsUnderParentRelation(relationId);
SplitMode splitMode = NON_BLOCKING_SPLIT;
SplitOperation splitOperation = CREATE_DISTRIBUTED_TABLE;
SplitShard(
splitMode,
splitOperation,
shardToSplit->shardId,
shardSplitPointsList,
workersForPlacementList,
distributionColumnOverrides,
sourceColocatedShardIntervalList,
colocationId
);
}
/*
* EnsureForeignKeysForDistributedTableConcurrently ensures that referenced and referencing foreign
* keys for the given table are supported.
*
* We allow distributed -> reference
* distributed -> citus local
*
* We disallow reference -> distributed
* citus local -> distributed
* regular -> distributed
*
* Normally regular -> distributed is allowed but it is not allowed when we create the
* distributed table concurrently because we rely on conversion of regular table to citus local table,
* which errors with an unfriendly message.
*/
static void
EnsureForeignKeysForDistributedTableConcurrently(Oid relationId)
{
/*
* disallow citus local -> distributed fkeys.
* disallow reference -> distributed fkeys.
* disallow regular -> distributed fkeys.
*/
EnsureNoFKeyFromTableType(relationId, INCLUDE_CITUS_LOCAL_TABLES |
INCLUDE_REFERENCE_TABLES | INCLUDE_LOCAL_TABLES);
/*
* disallow distributed -> regular fkeys.
*/
EnsureNoFKeyToTableType(relationId, INCLUDE_LOCAL_TABLES);
}
/*
* EnsureColocateWithTableIsValid ensures given relation can be colocated with the table of given name.
*/
static void
EnsureColocateWithTableIsValid(Oid relationId, char distributionMethod,
char *distributionColumnName, char *colocateWithTableName)
{
bool viaDeprecatedAPI = false;
char replicationModel = DecideReplicationModel(distributionMethod,
colocateWithTableName,
viaDeprecatedAPI);
/*
* we fail transaction before local table conversion if the table could not be colocated with
* given table. We should make those checks after local table conversion by acquiring locks to
* the relation because the distribution column can be modified in that period.
*/
Oid distributionColumnType = ColumnTypeIdForRelationColumnName(relationId,
distributionColumnName);
text *colocateWithTableNameText = cstring_to_text(colocateWithTableName);
Oid colocateWithTableId = ResolveRelationId(colocateWithTableNameText, false);
EnsureTableCanBeColocatedWith(relationId, replicationModel,
distributionColumnType, colocateWithTableId);
}
/*
* AcquireCreateDistributedTableConcurrentlyLock does not allow concurrent create_distributed_table_concurrently
* operations.
*/
void
AcquireCreateDistributedTableConcurrentlyLock(Oid relationId)
{
LOCKTAG tag;
const bool sessionLock = false;
const bool dontWait = true;
SET_LOCKTAG_CITUS_OPERATION(tag, CITUS_CREATE_DISTRIBUTED_TABLE_CONCURRENTLY);
LockAcquireResult lockAcquired = LockAcquire(&tag, ExclusiveLock, sessionLock,
dontWait);
if (!lockAcquired)
{
ereport(ERROR, (errmsg("another create_distributed_table_concurrently "
"operation is in progress"),
errhint("Make sure that the concurrent operation has "
"finished and re-run the command")));
}
}
/*
* SendAddLocalTableToMetadataCommandOutsideTransaction executes metadata add local
* table command locally to avoid deadlock.
*/
static void
SendAddLocalTableToMetadataCommandOutsideTransaction(Oid relationId)
{
char *qualifiedRelationName = generate_qualified_relation_name(relationId);
/*
* we need to allow nested distributed execution, because we start a new distributed
* execution inside the pushed-down UDF citus_add_local_table_to_metadata. Normally
* citus does not allow that because it cannot guarantee correctness.
*/
StringInfo allowNestedDistributionCommand = makeStringInfo();
appendStringInfo(allowNestedDistributionCommand,
"SET LOCAL citus.allow_nested_distributed_execution to ON");
StringInfo addLocalTableToMetadataCommand = makeStringInfo();
appendStringInfo(addLocalTableToMetadataCommand,
"SELECT pg_catalog.citus_add_local_table_to_metadata(%s)",
quote_literal_cstr(qualifiedRelationName));
List *commands = list_make2(allowNestedDistributionCommand->data,
addLocalTableToMetadataCommand->data);
char *username = NULL;
SendCommandListToWorkerOutsideTransaction(LocalHostName, PostPortNumber, username,
commands);
}
/*
* WarnIfTableHaveNoReplicaIdentity notices user if the given table or its partitions (if any)
* do not have a replica identity which is required for logical replication to replicate
* UPDATE and DELETE commands during create_distributed_table_concurrently.
*/
void
WarnIfTableHaveNoReplicaIdentity(Oid relationId)
{
bool foundRelationWithNoReplicaIdentity = false;
/*
* Check for source relation's partitions if any. We do not need to check for the source relation
* because we can replicate partitioned table even if it does not have replica identity.
* Source table will have no data if it has partitions.
*/
if (PartitionedTable(relationId))
{
List *partitionList = PartitionList(relationId);
ListCell *partitionCell = NULL;
foreach(partitionCell, partitionList)
{
Oid partitionTableId = lfirst_oid(partitionCell);
if (!RelationCanPublishAllModifications(partitionTableId))
{
foundRelationWithNoReplicaIdentity = true;
break;
}
}
}
/* check for source relation if it is not partitioned */
else
{
if (!RelationCanPublishAllModifications(relationId))
{
foundRelationWithNoReplicaIdentity = true;
}
}
if (foundRelationWithNoReplicaIdentity)
{
char *relationName = get_rel_name(relationId);
ereport(NOTICE, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("relation %s does not have a REPLICA "
"IDENTITY or PRIMARY KEY", relationName),
errdetail("UPDATE and DELETE commands on the relation will "
"error out during create_distributed_table_concurrently unless "
"there is a REPLICA IDENTITY or PRIMARY KEY. "
"INSERT commands will still work.")));
}
}
/*
* HashSplitPointsForShardList returns a list of split points which match
* the shard ranges of the given list of shards;
*/
static List *
HashSplitPointsForShardList(List *shardList)
{
List *splitPointList = NIL;
ShardInterval *shardInterval = NULL;
foreach_ptr(shardInterval, shardList)
{
int32 shardMaxValue = DatumGetInt32(shardInterval->maxValue);
splitPointList = lappend_int(splitPointList, shardMaxValue);
}
/*
* Split point lists only include the upper boundaries.
*/
splitPointList = list_delete_last(splitPointList);
return splitPointList;
}
/*
* HashSplitPointsForShardCount returns a list of split points for a given
* shard count with roughly equal hash ranges.
*/
static List *
HashSplitPointsForShardCount(int shardCount)
{
List *splitPointList = NIL;
/* calculate the split of the hash space */
uint64 hashTokenIncrement = HASH_TOKEN_COUNT / shardCount;
/*
* Split points lists only include the upper boundaries, so we only
* go up to shardCount - 1 and do not have to apply the correction
* for the last shardmaxvalue.
*/
for (int64 shardIndex = 0; shardIndex < shardCount - 1; shardIndex++)
{
/* initialize the hash token space for this shard */
int32 shardMinValue = PG_INT32_MIN + (shardIndex * hashTokenIncrement);
int32 shardMaxValue = shardMinValue + (hashTokenIncrement - 1);
splitPointList = lappend_int(splitPointList, shardMaxValue);
}
return splitPointList;
}
/*
* WorkerNodesForShardList returns a list of node ids reflecting the locations of
* the given list of shards.
*/
static List *
WorkerNodesForShardList(List *shardList)
{
List *nodeIdList = NIL;
ShardInterval *shardInterval = NULL;
foreach_ptr(shardInterval, shardList)
{
WorkerNode *workerNode = ActiveShardPlacementWorkerNode(shardInterval->shardId);
nodeIdList = lappend_int(nodeIdList, workerNode->nodeId);
}
return nodeIdList;
}
/*
* RoundRobinWorkerNodeList round robins over the workers in the worker node list
* and adds node ids to a list of length listLength.
*/
static List *
RoundRobinWorkerNodeList(List *workerNodeList, int listLength)
{
List *nodeIdList = NIL;
for (int idx = 0; idx < listLength; idx++)
{
int nodeIdx = idx % list_length(workerNodeList);
WorkerNode *workerNode = (WorkerNode *) list_nth(workerNodeList, nodeIdx);
nodeIdList = lappend_int(nodeIdList, workerNode->nodeId);
}
return nodeIdList;
}
/* /*
* create_reference_table creates a distributed table with the given relationId. The * create_reference_table creates a distributed table with the given relationId. The
* created table has one shard and replication factor is set to the active worker * created table has one shard and replication factor is set to the active worker
@ -394,7 +999,6 @@ CreateDistributedTable(Oid relationId, char *distributionColumnName,
INCLUDE_ALL_TABLE_TYPES); INCLUDE_ALL_TABLE_TYPES);
relationId = DropFKeysAndUndistributeTable(relationId); relationId = DropFKeysAndUndistributeTable(relationId);
} }
/* /*
* To support foreign keys between reference tables and local tables, * To support foreign keys between reference tables and local tables,
* we drop & re-define foreign keys at the end of this function so * we drop & re-define foreign keys at the end of this function so
@ -431,21 +1035,9 @@ CreateDistributedTable(Oid relationId, char *distributionColumnName,
LockRelationOid(relationId, ExclusiveLock); LockRelationOid(relationId, ExclusiveLock);
/* EnsureTableNotDistributed(relationId);
* Ensure that the sequences used in column defaults of the table
* have proper types
*/
EnsureRelationHasCompatibleSequenceTypes(relationId);
/* PropagatePrerequisiteObjectsForDistributedTable(relationId);
* distributed tables might have dependencies on different objects, since we create
* shards for a distributed table via multiple sessions these objects will be created
* via their own connection and committed immediately so they become visible to all
* sessions creating shards.
*/
ObjectAddress *tableAddress = palloc0(sizeof(ObjectAddress));
ObjectAddressSet(*tableAddress, RelationRelationId, relationId);
EnsureAllObjectDependenciesExistOnAllNodes(list_make1(tableAddress));
char replicationModel = DecideReplicationModel(distributionMethod, char replicationModel = DecideReplicationModel(distributionMethod,
colocateWithTableName, colocateWithTableName,
@ -453,7 +1045,7 @@ CreateDistributedTable(Oid relationId, char *distributionColumnName,
Var *distributionColumn = BuildDistributionKeyFromColumnName(relationId, Var *distributionColumn = BuildDistributionKeyFromColumnName(relationId,
distributionColumnName, distributionColumnName,
ExclusiveLock); NoLock);
/* /*
* ColocationIdForNewTable assumes caller acquires lock on relationId. In our case, * ColocationIdForNewTable assumes caller acquires lock on relationId. In our case,
@ -582,6 +1174,31 @@ CreateDistributedTable(Oid relationId, char *distributionColumnName,
} }
/*
* PropagatePrerequisiteObjectsForDistributedTable ensures we can create shards
* on all nodes by ensuring all dependent objects exist on all node.
*/
static void
PropagatePrerequisiteObjectsForDistributedTable(Oid relationId)
{
/*
* Ensure that the sequences used in column defaults of the table
* have proper types
*/
EnsureRelationHasCompatibleSequenceTypes(relationId);
/*
* distributed tables might have dependencies on different objects, since we create
* shards for a distributed table via multiple sessions these objects will be created
* via their own connection and committed immediately so they become visible to all
* sessions creating shards.
*/
ObjectAddress *tableAddress = palloc0(sizeof(ObjectAddress));
ObjectAddressSet(*tableAddress, RelationRelationId, relationId);
EnsureAllObjectDependenciesExistOnAllNodes(list_make1(tableAddress));
}
/* /*
* EnsureSequenceTypeSupported ensures that the type of the column that uses * EnsureSequenceTypeSupported ensures that the type of the column that uses
* a sequence on its DEFAULT is consistent with previous uses (if any) of the * a sequence on its DEFAULT is consistent with previous uses (if any) of the
@ -956,82 +1573,56 @@ ColocationIdForNewTable(Oid relationId, Var *distributionColumn,
*/ */
Assert(distributionMethod == DISTRIBUTE_BY_HASH); Assert(distributionMethod == DISTRIBUTE_BY_HASH);
Relation pgDistColocation = table_open(DistColocationRelationId(), ExclusiveLock);
Oid distributionColumnType = distributionColumn->vartype; Oid distributionColumnType = distributionColumn->vartype;
Oid distributionColumnCollation = get_typcollation(distributionColumnType); Oid distributionColumnCollation = get_typcollation(distributionColumnType);
bool createdColocationGroup = false;
if (pg_strncasecmp(colocateWithTableName, "default", NAMEDATALEN) == 0) /* get an advisory lock to serialize concurrent default group creations */
if (IsColocateWithDefault(colocateWithTableName))
{ {
/* check for default colocation group */ AcquireColocationDefaultLock();
colocationId = ColocationId(shardCount, ShardReplicationFactor, }
distributionColumnType,
distributionColumnCollation);
colocationId = FindColocateWithColocationId(relationId,
replicationModel,
distributionColumnType,
distributionColumnCollation,
shardCount,
shardCountIsStrict,
colocateWithTableName);
if (IsColocateWithDefault(colocateWithTableName) && (colocationId !=
INVALID_COLOCATION_ID))
{
/* /*
* if the shardCount is strict then we check if the shard count * we can release advisory lock if there is already a default entry for given params;
* of the colocated table is actually shardCount * else, we should keep it to prevent different default coloc entry creation by
* concurrent operations.
*/ */
if (shardCountIsStrict && colocationId != INVALID_COLOCATION_ID) ReleaseColocationDefaultLock();
{ }
Oid colocatedTableId = ColocatedTableId(colocationId);
if (colocationId == INVALID_COLOCATION_ID)
if (colocatedTableId != InvalidOid) {
{ if (IsColocateWithDefault(colocateWithTableName))
CitusTableCacheEntry *cacheEntry =
GetCitusTableCacheEntry(colocatedTableId);
int colocatedTableShardCount = cacheEntry->shardIntervalArrayLength;
if (colocatedTableShardCount != shardCount)
{
colocationId = INVALID_COLOCATION_ID;
}
}
}
if (colocationId == INVALID_COLOCATION_ID)
{ {
/*
* Generate a new colocation ID and insert a pg_dist_colocation
* record.
*/
colocationId = CreateColocationGroup(shardCount, ShardReplicationFactor,
distributionColumnType,
distributionColumnCollation);
}
else if (IsColocateWithNone(colocateWithTableName))
{
/*
* Generate a new colocation ID and insert a pg_dist_colocation
* record.
*/
colocationId = CreateColocationGroup(shardCount, ShardReplicationFactor, colocationId = CreateColocationGroup(shardCount, ShardReplicationFactor,
distributionColumnType, distributionColumnType,
distributionColumnCollation); distributionColumnCollation);
createdColocationGroup = true;
} }
}
else if (IsColocateWithNone(colocateWithTableName))
{
colocationId = CreateColocationGroup(shardCount, ShardReplicationFactor,
distributionColumnType,
distributionColumnCollation);
createdColocationGroup = true;
}
else
{
text *colocateWithTableNameText = cstring_to_text(colocateWithTableName);
Oid sourceRelationId = ResolveRelationId(colocateWithTableNameText, false);
EnsureTableCanBeColocatedWith(relationId, replicationModel,
distributionColumnType, sourceRelationId);
colocationId = TableColocationId(sourceRelationId);
}
/*
* If we created a new colocation group then we need to keep the lock to
* prevent a concurrent create_distributed_table call from creating another
* colocation group with the same parameters. If we're using an existing
* colocation group then other transactions will use the same one.
*/
if (createdColocationGroup)
{
/* keep the exclusive lock */
table_close(pgDistColocation, NoLock);
}
else
{
/* release the exclusive lock */
table_close(pgDistColocation, ExclusiveLock);
} }
} }
@ -1055,7 +1646,6 @@ EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn,
{ {
Oid parentRelationId = InvalidOid; Oid parentRelationId = InvalidOid;
EnsureTableNotDistributed(relationId);
EnsureLocalTableEmptyIfNecessary(relationId, distributionMethod, viaDeprecatedAPI); EnsureLocalTableEmptyIfNecessary(relationId, distributionMethod, viaDeprecatedAPI);
/* user really wants triggers? */ /* user really wants triggers? */
@ -1131,13 +1721,13 @@ EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn,
} }
} }
if (PartitionTable(relationId)) if (PartitionTableNoLock(relationId))
{ {
parentRelationId = PartitionParentOid(relationId); parentRelationId = PartitionParentOid(relationId);
} }
/* partitions cannot be distributed if their parent is not distributed */ /* partitions cannot be distributed if their parent is not distributed */
if (PartitionTable(relationId) && !IsCitusTable(parentRelationId)) if (PartitionTableNoLock(relationId) && !IsCitusTable(parentRelationId))
{ {
char *parentRelationName = get_rel_name(parentRelationId); char *parentRelationName = get_rel_name(parentRelationId);
@ -1155,7 +1745,7 @@ EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn,
* reach this point because, we call CreateDistributedTable for partitions if their * reach this point because, we call CreateDistributedTable for partitions if their
* parent table is distributed. * parent table is distributed.
*/ */
if (PartitionedTable(relationId)) if (PartitionedTableNoLock(relationId))
{ {
/* we cannot distribute partitioned tables with master_create_distributed_table */ /* we cannot distribute partitioned tables with master_create_distributed_table */
if (viaDeprecatedAPI) if (viaDeprecatedAPI)
@ -1174,7 +1764,7 @@ EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn,
} }
/* we don't support distributing tables with multi-level partitioning */ /* we don't support distributing tables with multi-level partitioning */
if (PartitionTable(relationId)) if (PartitionTableNoLock(relationId))
{ {
char *parentRelationName = get_rel_name(parentRelationId); char *parentRelationName = get_rel_name(parentRelationId);
@ -1227,55 +1817,6 @@ ErrorIfTableIsACatalogTable(Relation relation)
} }
/*
* EnsureTableCanBeColocatedWith checks whether a given replication model and
* distribution column type is suitable to distribute a table to be colocated
* with given source table.
*
* We only pass relationId to provide meaningful error messages.
*/
static void
EnsureTableCanBeColocatedWith(Oid relationId, char replicationModel,
Oid distributionColumnType, Oid sourceRelationId)
{
CitusTableCacheEntry *sourceTableEntry = GetCitusTableCacheEntry(sourceRelationId);
char sourceReplicationModel = sourceTableEntry->replicationModel;
Var *sourceDistributionColumn = DistPartitionKeyOrError(sourceRelationId);
if (!IsCitusTableTypeCacheEntry(sourceTableEntry, HASH_DISTRIBUTED))
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot distribute relation"),
errdetail("Currently, colocate_with option is only supported "
"for hash distributed tables.")));
}
if (sourceReplicationModel != replicationModel)
{
char *relationName = get_rel_name(relationId);
char *sourceRelationName = get_rel_name(sourceRelationId);
ereport(ERROR, (errmsg("cannot colocate tables %s and %s",
sourceRelationName, relationName),
errdetail("Replication models don't match for %s and %s.",
sourceRelationName, relationName)));
}
Oid sourceDistributionColumnType = sourceDistributionColumn->vartype;
if (sourceDistributionColumnType != distributionColumnType)
{
char *relationName = get_rel_name(relationId);
char *sourceRelationName = get_rel_name(sourceRelationId);
ereport(ERROR, (errmsg("cannot colocate tables %s and %s",
sourceRelationName, relationName),
errdetail("Distribution column types don't match for "
"%s and %s.", sourceRelationName,
relationName)));
}
}
/* /*
* EnsureLocalTableEmptyIfNecessary errors out if the function should be empty * EnsureLocalTableEmptyIfNecessary errors out if the function should be empty
* according to ShouldLocalTableBeEmpty but it is not. * according to ShouldLocalTableBeEmpty but it is not.
@ -1350,6 +1891,27 @@ EnsureLocalTableEmpty(Oid relationId)
} }
/*
* EnsureDistributableTable ensures the given table type is appropriate to
* be distributed. Table type should be regular or citus local table.
*/
static void
EnsureDistributableTable(Oid relationId)
{
bool isLocalTable = IsCitusTableType(relationId, CITUS_LOCAL_TABLE);
bool isRegularTable = !IsCitusTableType(relationId, ANY_CITUS_TABLE_TYPE);
if (!isLocalTable && !isRegularTable)
{
char *relationName = get_rel_name(relationId);
ereport(ERROR, (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
errmsg("table \"%s\" is already distributed",
relationName)));
}
}
/* /*
* EnsureTableNotDistributed errors out if the table is distributed. * EnsureTableNotDistributed errors out if the table is distributed.
*/ */

View File

@ -97,6 +97,66 @@ ConstraintIsAForeignKeyToReferenceTable(char *inputConstaintName, Oid relationId
} }
/*
* EnsureNoFKeyFromTableType ensures that given relation is not referenced by any table specified
* by table type flag.
*/
void
EnsureNoFKeyFromTableType(Oid relationId, int tableTypeFlag)
{
int flags = INCLUDE_REFERENCED_CONSTRAINTS | EXCLUDE_SELF_REFERENCES |
tableTypeFlag;
List *referencedFKeyOids = GetForeignKeyOids(relationId, flags);
if (list_length(referencedFKeyOids) > 0)
{
Oid referencingFKeyOid = linitial_oid(referencedFKeyOids);
Oid referencingTableId = GetReferencingTableId(referencingFKeyOid);
char *referencingRelName = get_rel_name(referencingTableId);
char *referencedRelName = get_rel_name(relationId);
char *referencingTableTypeName = GetTableTypeName(referencingTableId);
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("relation %s is referenced by a foreign key from %s",
referencedRelName, referencingRelName),
errdetail(
"foreign keys from a %s to a distributed table are not supported.",
referencingTableTypeName)));
}
}
/*
* EnsureNoFKeyToTableType ensures that given relation is not referencing by any table specified
* by table type flag.
*/
void
EnsureNoFKeyToTableType(Oid relationId, int tableTypeFlag)
{
int flags = INCLUDE_REFERENCING_CONSTRAINTS | EXCLUDE_SELF_REFERENCES |
tableTypeFlag;
List *referencingFKeyOids = GetForeignKeyOids(relationId, flags);
if (list_length(referencingFKeyOids) > 0)
{
Oid referencedFKeyOid = linitial_oid(referencingFKeyOids);
Oid referencedTableId = GetReferencedTableId(referencedFKeyOid);
char *referencedRelName = get_rel_name(referencedTableId);
char *referencingRelName = get_rel_name(relationId);
char *referencedTableTypeName = GetTableTypeName(referencedTableId);
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("relation %s is referenced by a foreign key from %s",
referencedRelName, referencingRelName),
errdetail(
"foreign keys from a distributed table to a %s are not supported.",
referencedTableTypeName)));
}
}
/* /*
* ErrorIfUnsupportedForeignConstraintExists runs checks related to foreign * ErrorIfUnsupportedForeignConstraintExists runs checks related to foreign
* constraints and errors out if it is not possible to create one of the * constraints and errors out if it is not possible to create one of the

View File

@ -493,6 +493,49 @@ IsCitusTableTypeInternal(char partitionMethod, char replicationModel,
} }
/*
* GetTableTypeName returns string representation of the table type.
*/
char *
GetTableTypeName(Oid tableId)
{
bool regularTable = false;
char partitionMethod = ' ';
char replicationModel = ' ';
if (IsCitusTable(tableId))
{
CitusTableCacheEntry *referencingCacheEntry = GetCitusTableCacheEntry(tableId);
partitionMethod = referencingCacheEntry->partitionMethod;
replicationModel = referencingCacheEntry->replicationModel;
}
else
{
regularTable = true;
}
if (regularTable)
{
return "regular table";
}
else if (partitionMethod == 'h')
{
return "distributed table";
}
else if (partitionMethod == 'n' && replicationModel == 't')
{
return "reference table";
}
else if (partitionMethod == 'n' && replicationModel != 't')
{
return "citus local table";
}
else
{
return "unknown table";
}
}
/* /*
* IsCitusTable returns whether relationId is a distributed relation or * IsCitusTable returns whether relationId is a distributed relation or
* not. * not.

View File

@ -159,6 +159,7 @@ PG_FUNCTION_INFO_V1(worker_record_sequence_dependency);
* or regular users as long as the regular user owns the input object. * or regular users as long as the regular user owns the input object.
*/ */
PG_FUNCTION_INFO_V1(citus_internal_add_partition_metadata); PG_FUNCTION_INFO_V1(citus_internal_add_partition_metadata);
PG_FUNCTION_INFO_V1(citus_internal_delete_partition_metadata);
PG_FUNCTION_INFO_V1(citus_internal_add_shard_metadata); PG_FUNCTION_INFO_V1(citus_internal_add_shard_metadata);
PG_FUNCTION_INFO_V1(citus_internal_add_placement_metadata); PG_FUNCTION_INFO_V1(citus_internal_add_placement_metadata);
PG_FUNCTION_INFO_V1(citus_internal_update_placement_metadata); PG_FUNCTION_INFO_V1(citus_internal_update_placement_metadata);
@ -1211,6 +1212,24 @@ DistributionDeleteCommand(const char *schemaName, const char *tableName)
} }
/*
* DistributionDeleteMetadataCommand returns a query to delete pg_dist_partition
* metadata from a worker node for a given table.
*/
char *
DistributionDeleteMetadataCommand(Oid relationId)
{
StringInfo deleteCommand = makeStringInfo();
char *qualifiedRelationName = generate_qualified_relation_name(relationId);
appendStringInfo(deleteCommand,
"SELECT pg_catalog.citus_internal_delete_partition_metadata(%s)",
quote_literal_cstr(qualifiedRelationName));
return deleteCommand->data;
}
/* /*
* TableOwnerResetCommand generates a commands that can be executed * TableOwnerResetCommand generates a commands that can be executed
* to reset the table owner. * to reset the table owner.
@ -3199,6 +3218,35 @@ EnsurePartitionMetadataIsSane(Oid relationId, char distributionMethod, int coloc
} }
/*
* citus_internal_delete_partition_metadata is an internal UDF to
* delete a row in pg_dist_partition.
*/
Datum
citus_internal_delete_partition_metadata(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
PG_ENSURE_ARGNOTNULL(0, "relation");
Oid relationId = PG_GETARG_OID(0);
/* only owner of the table (or superuser) is allowed to add the Citus metadata */
EnsureTableOwner(relationId);
/* we want to serialize all the metadata changes to this table */
LockRelationOid(relationId, ShareUpdateExclusiveLock);
if (!ShouldSkipMetadataChecks())
{
EnsureCoordinatorInitiatedOperation();
}
DeletePartitionRow(relationId);
PG_RETURN_VOID();
}
/* /*
* citus_internal_add_shard_metadata is an internal UDF to * citus_internal_add_shard_metadata is an internal UDF to
* add a row to pg_dist_shard. * add a row to pg_dist_shard.

View File

@ -39,6 +39,7 @@
#include "distributed/metadata_utility.h" #include "distributed/metadata_utility.h"
#include "distributed/coordinator_protocol.h" #include "distributed/coordinator_protocol.h"
#include "distributed/metadata_cache.h" #include "distributed/metadata_cache.h"
#include "distributed/metadata_sync.h"
#include "distributed/multi_join_order.h" #include "distributed/multi_join_order.h"
#include "distributed/multi_logical_optimizer.h" #include "distributed/multi_logical_optimizer.h"
#include "distributed/multi_partitioning_utils.h" #include "distributed/multi_partitioning_utils.h"
@ -2153,6 +2154,96 @@ UpdatePgDistPartitionAutoConverted(Oid citusTableId, bool autoConverted)
} }
/*
* UpdateDistributionColumnGlobally sets the distribution column and colocation ID
* for a table in pg_dist_partition on all nodes
*/
void
UpdateDistributionColumnGlobally(Oid relationId, char distributionMethod,
Var *distributionColumn, int colocationId)
{
UpdateDistributionColumn(relationId, distributionMethod, distributionColumn,
colocationId);
if (ShouldSyncTableMetadata(relationId))
{
/* we use delete+insert because syncing uses specialized RPCs */
char *deleteMetadataCommand = DistributionDeleteMetadataCommand(relationId);
SendCommandToWorkersWithMetadata(deleteMetadataCommand);
/* pick up the new metadata (updated above) */
CitusTableCacheEntry *cacheEntry = GetCitusTableCacheEntry(relationId);
char *insertMetadataCommand = DistributionCreateCommand(cacheEntry);
SendCommandToWorkersWithMetadata(insertMetadataCommand);
}
}
/*
* UpdateDistributionColumn sets the distribution column and colocation ID for a table
* in pg_dist_partition.
*/
void
UpdateDistributionColumn(Oid relationId, char distributionMethod, Var *distributionColumn,
int colocationId)
{
ScanKeyData scanKey[1];
int scanKeyCount = 1;
bool indexOK = true;
Datum values[Natts_pg_dist_partition];
bool isnull[Natts_pg_dist_partition];
bool replace[Natts_pg_dist_partition];
Relation pgDistPartition = table_open(DistPartitionRelationId(), RowExclusiveLock);
TupleDesc tupleDescriptor = RelationGetDescr(pgDistPartition);
ScanKeyInit(&scanKey[0], Anum_pg_dist_partition_logicalrelid,
BTEqualStrategyNumber, F_OIDEQ, ObjectIdGetDatum(relationId));
SysScanDesc scanDescriptor = systable_beginscan(pgDistPartition,
DistPartitionLogicalRelidIndexId(),
indexOK,
NULL, scanKeyCount, scanKey);
HeapTuple heapTuple = systable_getnext(scanDescriptor);
if (!HeapTupleIsValid(heapTuple))
{
ereport(ERROR, (errmsg("could not find valid entry for citus table with oid: %u",
relationId)));
}
memset(replace, 0, sizeof(replace));
replace[Anum_pg_dist_partition_partmethod - 1] = true;
values[Anum_pg_dist_partition_partmethod - 1] = CharGetDatum(distributionMethod);
isnull[Anum_pg_dist_partition_partmethod - 1] = false;
replace[Anum_pg_dist_partition_colocationid - 1] = true;
values[Anum_pg_dist_partition_colocationid - 1] = UInt32GetDatum(colocationId);
isnull[Anum_pg_dist_partition_colocationid - 1] = false;
replace[Anum_pg_dist_partition_autoconverted - 1] = true;
values[Anum_pg_dist_partition_autoconverted - 1] = BoolGetDatum(false);
isnull[Anum_pg_dist_partition_autoconverted - 1] = false;
char *distributionColumnString = nodeToString((Node *) distributionColumn);
replace[Anum_pg_dist_partition_partkey - 1] = true;
values[Anum_pg_dist_partition_partkey - 1] =
CStringGetTextDatum(distributionColumnString);
isnull[Anum_pg_dist_partition_partkey - 1] = false;
heapTuple = heap_modify_tuple(heapTuple, tupleDescriptor, values, isnull, replace);
CatalogTupleUpdate(pgDistPartition, &heapTuple->t_self, heapTuple);
CitusInvalidateRelcacheByRelid(relationId);
CommandCounterIncrement();
systable_endscan(scanDescriptor);
table_close(pgDistPartition, NoLock);
}
/* /*
* Check that the current user has `mode` permissions on relationId, error out * Check that the current user has `mode` permissions on relationId, error out
* if not. Superusers always have such permissions. * if not. Superusers always have such permissions.

View File

@ -2546,6 +2546,24 @@ EnsureCoordinator(void)
} }
/*
* EnsureCoordinatorIsInMetadata checks whether the coordinator is added to the
* metadata, which is required for many operations.
*/
void
EnsureCoordinatorIsInMetadata(void)
{
bool isCoordinatorInMetadata = false;
PrimaryNodeForGroup(COORDINATOR_GROUP_ID, &isCoordinatorInMetadata);
if (!isCoordinatorInMetadata)
{
ereport(ERROR, (errmsg("coordinator is not added to the metadata"),
errhint("Use SELECT citus_set_coordinator_host('<hostname>') "
"to configure the coordinator hostname")));
}
}
/* /*
* InsertCoordinatorIfClusterEmpty can be used to ensure Citus tables can be * InsertCoordinatorIfClusterEmpty can be used to ensure Citus tables can be
* created even on a node that has just performed CREATE EXTENSION citus; * created even on a node that has just performed CREATE EXTENSION citus;

View File

@ -23,6 +23,7 @@
#include "distributed/connection_management.h" #include "distributed/connection_management.h"
#include "distributed/remote_commands.h" #include "distributed/remote_commands.h"
#include "distributed/shard_split.h" #include "distributed/shard_split.h"
#include "distributed/utils/distribution_column_map.h"
/* declarations for dynamic loading */ /* declarations for dynamic loading */
PG_FUNCTION_INFO_V1(citus_split_shard_by_split_points); PG_FUNCTION_INFO_V1(citus_split_shard_by_split_points);
@ -52,12 +53,17 @@ citus_split_shard_by_split_points(PG_FUNCTION_ARGS)
Oid shardTransferModeOid = PG_GETARG_OID(3); Oid shardTransferModeOid = PG_GETARG_OID(3);
SplitMode shardSplitMode = LookupSplitMode(shardTransferModeOid); SplitMode shardSplitMode = LookupSplitMode(shardTransferModeOid);
DistributionColumnMap *distributionColumnOverrides = NULL;
List *sourceColocatedShardIntervalList = NIL;
SplitShard( SplitShard(
shardSplitMode, shardSplitMode,
SHARD_SPLIT_API, SHARD_SPLIT_API,
shardIdToSplit, shardIdToSplit,
shardSplitPointsList, shardSplitPointsList,
nodeIdsForPlacementList); nodeIdsForPlacementList,
distributionColumnOverrides,
sourceColocatedShardIntervalList,
INVALID_COLOCATION_ID);
PG_RETURN_VOID(); PG_RETURN_VOID();
} }

View File

@ -33,6 +33,7 @@
#include "distributed/worker_transaction.h" #include "distributed/worker_transaction.h"
#include "distributed/version_compat.h" #include "distributed/version_compat.h"
#include "distributed/shard_split.h" #include "distributed/shard_split.h"
#include "distributed/utils/distribution_column_map.h"
#include "nodes/pg_list.h" #include "nodes/pg_list.h"
#include "storage/lock.h" #include "storage/lock.h"
#include "utils/builtins.h" #include "utils/builtins.h"
@ -163,12 +164,17 @@ isolate_tenant_to_new_shard(PG_FUNCTION_ARGS)
nodeIdsForPlacementList = lappend_int(nodeIdsForPlacementList, sourceNodeId); nodeIdsForPlacementList = lappend_int(nodeIdsForPlacementList, sourceNodeId);
} }
DistributionColumnMap *distributionColumnOverrides = NULL;
List *sourceColocatedShardIntervalList = NIL;
SplitMode splitMode = LookupSplitMode(shardTransferModeOid); SplitMode splitMode = LookupSplitMode(shardTransferModeOid);
SplitShard(splitMode, SplitShard(splitMode,
ISOLATE_TENANT_TO_NEW_SHARD, ISOLATE_TENANT_TO_NEW_SHARD,
sourceShard->shardId, sourceShard->shardId,
shardSplitPointsList, shardSplitPointsList,
nodeIdsForPlacementList); nodeIdsForPlacementList,
distributionColumnOverrides,
sourceColocatedShardIntervalList,
INVALID_COLOCATION_ID);
cacheEntry = GetCitusTableCacheEntry(relationId); cacheEntry = GetCitusTableCacheEntry(relationId);
ShardInterval *newShard = FindShardInterval(tenantIdDatum, cacheEntry); ShardInterval *newShard = FindShardInterval(tenantIdDatum, cacheEntry);

View File

@ -71,7 +71,6 @@ typedef struct ShardCommandList
} ShardCommandList; } ShardCommandList;
/* local function forward declarations */ /* local function forward declarations */
static bool RelationCanPublishAllModifications(Oid relationId);
static bool CanUseLogicalReplication(Oid relationId, char shardReplicationMode); static bool CanUseLogicalReplication(Oid relationId, char shardReplicationMode);
static void ErrorIfTableCannotBeReplicated(Oid relationId); static void ErrorIfTableCannotBeReplicated(Oid relationId);
static void ErrorIfTargetNodeIsNotSafeToCopyTo(const char *targetNodeName, static void ErrorIfTargetNodeIsNotSafeToCopyTo(const char *targetNodeName,
@ -635,7 +634,7 @@ VerifyTablesHaveReplicaIdentity(List *colocatedTableList)
* RelationCanPublishAllModifications returns true if the relation is safe to publish * RelationCanPublishAllModifications returns true if the relation is safe to publish
* all modification while being replicated via logical replication. * all modification while being replicated via logical replication.
*/ */
static bool bool
RelationCanPublishAllModifications(Oid relationId) RelationCanPublishAllModifications(Oid relationId)
{ {
Relation relation = RelationIdGetRelation(relationId); Relation relation = RelationIdGetRelation(relationId);

View File

@ -30,6 +30,7 @@
#include "distributed/shard_split.h" #include "distributed/shard_split.h"
#include "distributed/reference_table_utils.h" #include "distributed/reference_table_utils.h"
#include "distributed/repair_shards.h" #include "distributed/repair_shards.h"
#include "distributed/resource_lock.h"
#include "distributed/multi_partitioning_utils.h" #include "distributed/multi_partitioning_utils.h"
#include "distributed/worker_manager.h" #include "distributed/worker_manager.h"
#include "distributed/worker_transaction.h" #include "distributed/worker_transaction.h"
@ -37,6 +38,7 @@
#include "distributed/pg_dist_shard.h" #include "distributed/pg_dist_shard.h"
#include "distributed/metadata_sync.h" #include "distributed/metadata_sync.h"
#include "distributed/multi_physical_planner.h" #include "distributed/multi_physical_planner.h"
#include "distributed/utils/distribution_column_map.h"
#include "commands/dbcommands.h" #include "commands/dbcommands.h"
#include "distributed/shardsplit_logical_replication.h" #include "distributed/shardsplit_logical_replication.h"
#include "distributed/deparse_shard_query.h" #include "distributed/deparse_shard_query.h"
@ -71,12 +73,8 @@ static void ErrorIfCannotSplitShardExtended(SplitOperation splitOperation,
ShardInterval *shardIntervalToSplit, ShardInterval *shardIntervalToSplit,
List *shardSplitPointsList, List *shardSplitPointsList,
List *nodeIdsForPlacementList); List *nodeIdsForPlacementList);
static void CreateAndCopySplitShardsForShardGroup( static void ErrorIfModificationAndSplitInTheSameTransaction(SplitOperation
HTAB *mapOfShardToPlacementCreatedByWorkflow, splitOperation);
WorkerNode *sourceShardNode,
List *sourceColocatedShardIntervalList,
List *shardGroupSplitIntervalListList,
List *workersForPlacementList);
static void CreateSplitShardsForShardGroup(HTAB *mapOfShardToPlacementCreatedByWorkflow, static void CreateSplitShardsForShardGroup(HTAB *mapOfShardToPlacementCreatedByWorkflow,
List *shardGroupSplitIntervalListList, List *shardGroupSplitIntervalListList,
List *workersForPlacementList); List *workersForPlacementList);
@ -98,23 +96,35 @@ static void CreateSplitIntervalsForShard(ShardInterval *sourceShard,
List *splitPointsForShard, List *splitPointsForShard,
List **shardSplitChildrenIntervalList); List **shardSplitChildrenIntervalList);
static void BlockingShardSplit(SplitOperation splitOperation, static void BlockingShardSplit(SplitOperation splitOperation,
ShardInterval *shardIntervalToSplit, uint64 splitWorkflowId,
List *sourceColocatedShardIntervalList,
List *shardSplitPointsList, List *shardSplitPointsList,
List *workersForPlacementList); List *workersForPlacementList,
DistributionColumnMap *distributionColumnOverrides);
static void NonBlockingShardSplit(SplitOperation splitOperation, static void NonBlockingShardSplit(SplitOperation splitOperation,
ShardInterval *shardIntervalToSplit, uint64 splitWorkflowId,
List *sourceColocatedShardIntervalList,
List *shardSplitPointsList, List *shardSplitPointsList,
List *workersForPlacementList); List *workersForPlacementList,
DistributionColumnMap *distributionColumnOverrides,
uint32 targetColocationId);
static void DoSplitCopy(WorkerNode *sourceShardNode, static void DoSplitCopy(WorkerNode *sourceShardNode,
List *sourceColocatedShardIntervalList, List *sourceColocatedShardIntervalList,
List *shardGroupSplitIntervalListList, List *shardGroupSplitIntervalListList,
List *workersForPlacementList, List *workersForPlacementList,
char *snapShotName); char *snapShotName,
DistributionColumnMap *distributionColumnOverrides);
static StringInfo CreateSplitCopyCommand(ShardInterval *sourceShardSplitInterval, static StringInfo CreateSplitCopyCommand(ShardInterval *sourceShardSplitInterval,
char *distributionColumnName,
List *splitChildrenShardIntervalList, List *splitChildrenShardIntervalList,
List *workersForPlacementList); List *workersForPlacementList);
static Task * CreateSplitCopyTask(StringInfo splitCopyUdfCommand, char *snapshotName, int static Task * CreateSplitCopyTask(StringInfo splitCopyUdfCommand, char *snapshotName, int
taskId, uint64 jobId); taskId, uint64 jobId);
static void UpdateDistributionColumnsForShardGroup(List *colocatedShardList,
DistributionColumnMap *distCols,
char distributionMethod,
int shardCount,
uint32 colocationId);
static void InsertSplitChildrenShardMetadata(List *shardGroupSplitIntervalListList, static void InsertSplitChildrenShardMetadata(List *shardGroupSplitIntervalListList,
List *workersForPlacementList); List *workersForPlacementList);
static void CreatePartitioningHierarchy(List *shardGroupSplitIntervalListList, static void CreatePartitioningHierarchy(List *shardGroupSplitIntervalListList,
@ -126,30 +136,38 @@ static HTAB * CreateEmptyMapForShardsCreatedByWorkflow();
static Task * CreateTaskForDDLCommandList(List *ddlCommandList, WorkerNode *workerNode); static Task * CreateTaskForDDLCommandList(List *ddlCommandList, WorkerNode *workerNode);
static StringInfo CreateSplitShardReplicationSetupUDF( static StringInfo CreateSplitShardReplicationSetupUDF(
List *sourceColocatedShardIntervalList, List *shardGroupSplitIntervalListList, List *sourceColocatedShardIntervalList, List *shardGroupSplitIntervalListList,
List *destinationWorkerNodesList); List *destinationWorkerNodesList,
DistributionColumnMap *
distributionColumnOverrides);
static List * ParseReplicationSlotInfoFromResult(PGresult *result); static List * ParseReplicationSlotInfoFromResult(PGresult *result);
static List * ExecuteSplitShardReplicationSetupUDF(WorkerNode *sourceWorkerNode, static List * ExecuteSplitShardReplicationSetupUDF(WorkerNode *sourceWorkerNode,
List *sourceColocatedShardIntervalList, List *sourceColocatedShardIntervalList,
List *shardGroupSplitIntervalListList, List *shardGroupSplitIntervalListList,
List *destinationWorkerNodesList); List *destinationWorkerNodesList,
DistributionColumnMap *
distributionColumnOverrides);
static void ExecuteSplitShardReleaseSharedMemory(WorkerNode *sourceWorkerNode); static void ExecuteSplitShardReleaseSharedMemory(WorkerNode *sourceWorkerNode);
static void AddDummyShardEntryInMap(HTAB *mapOfDummyShards, uint32 targetNodeId, static void AddDummyShardEntryInMap(HTAB *mapOfDummyShards, uint32 targetNodeId,
ShardInterval *shardInterval); ShardInterval *shardInterval);
static void DropDummyShards(HTAB *mapOfDummyShardToPlacement); static void DropDummyShards(HTAB *mapOfDummyShardToPlacement);
static void DropDummyShard(MultiConnection *connection, ShardInterval *shardInterval); static void DropDummyShard(MultiConnection *connection, ShardInterval *shardInterval);
static uint64 GetNextShardIdForSplitChild(void); static uint64 GetNextShardIdForSplitChild(void);
static void AcquireNonblockingSplitLock(Oid relationId);
static List * GetWorkerNodesFromWorkerIds(List *nodeIdsForPlacementList);
/* Customize error message strings based on operation type */ /* Customize error message strings based on operation type */
static const char *const SplitOperationName[] = static const char *const SplitOperationName[] =
{ {
[SHARD_SPLIT_API] = "split", [SHARD_SPLIT_API] = "split",
[ISOLATE_TENANT_TO_NEW_SHARD] = "isolate", [ISOLATE_TENANT_TO_NEW_SHARD] = "isolate",
[CREATE_DISTRIBUTED_TABLE] = "create"
}; };
static const char *const SplitTargetName[] = static const char *const SplitTargetName[] =
{ {
[SHARD_SPLIT_API] = "shard", [SHARD_SPLIT_API] = "shard",
[ISOLATE_TENANT_TO_NEW_SHARD] = "tenant", [ISOLATE_TENANT_TO_NEW_SHARD] = "tenant",
[CREATE_DISTRIBUTED_TABLE] = "distributed table"
}; };
/* Function definitions */ /* Function definitions */
@ -230,6 +248,12 @@ ErrorIfCannotSplitShardExtended(SplitOperation splitOperation,
List *shardSplitPointsList, List *shardSplitPointsList,
List *nodeIdsForPlacementList) List *nodeIdsForPlacementList)
{ {
/* we should not perform checks for create distributed table operation */
if (splitOperation == CREATE_DISTRIBUTED_TABLE)
{
return;
}
CitusTableCacheEntry *cachedTableEntry = GetCitusTableCacheEntry( CitusTableCacheEntry *cachedTableEntry = GetCitusTableCacheEntry(
shardIntervalToSplit->relationId); shardIntervalToSplit->relationId);
@ -352,20 +376,11 @@ ErrorIfCannotSplitShardExtended(SplitOperation splitOperation,
/* /*
* SplitShard API to split a given shard (or shard group) based on specified split points * ErrorIfModificationAndSplitInTheSameTransaction will error if we detect split operation
* to a set of destination nodes. * in the same transaction which has modification before.
* 'splitMode' : Mode of split operation.
* 'splitOperation' : Customer operation that triggered split.
* 'shardInterval' : Source shard interval to be split.
* 'shardSplitPointsList' : Split Points list for the source 'shardInterval'.
* 'nodeIdsForPlacementList' : Placement list corresponding to split children.
*/ */
void static void
SplitShard(SplitMode splitMode, ErrorIfModificationAndSplitInTheSameTransaction(SplitOperation splitOperation)
SplitOperation splitOperation,
uint64 shardIdToSplit,
List *shardSplitPointsList,
List *nodeIdsForPlacementList)
{ {
if (XactModificationLevel > XACT_MODIFICATION_NONE) if (XactModificationLevel > XACT_MODIFICATION_NONE)
{ {
@ -375,13 +390,82 @@ SplitShard(SplitMode splitMode,
SplitOperationName[splitOperation], SplitOperationName[splitOperation],
SplitTargetName[splitOperation]))); SplitTargetName[splitOperation])));
} }
else if (PlacementMovedUsingLogicalReplicationInTX) }
/*
* ErrorIfMultipleNonblockingMoveSplitInTheSameTransaction will error if we detect multiple
* nonblocking shard movements/splits in the same transaction.
*/
void
ErrorIfMultipleNonblockingMoveSplitInTheSameTransaction(void)
{
if (PlacementMovedUsingLogicalReplicationInTX)
{ {
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("multiple shard movements/splits via logical " errmsg("multiple shard movements/splits via logical "
"replication in the same transaction is currently " "replication in the same transaction is currently "
"not supported"))); "not supported")));
} }
}
/*
* GetWorkerNodesFromWorkerIds returns list of worker nodes given a list
* of worker ids. It will error if any node id is invalid.
*/
static List *
GetWorkerNodesFromWorkerIds(List *nodeIdsForPlacementList)
{
List *workersForPlacementList = NIL;
int32 nodeId;
foreach_int(nodeId, nodeIdsForPlacementList)
{
uint32 nodeIdValue = (uint32) nodeId;
WorkerNode *workerNode = LookupNodeByNodeId(nodeIdValue);
/* NodeId in Citus are unsigned and range from [1, 4294967296]. */
if (nodeIdValue < 1 || workerNode == NULL)
{
ereport(ERROR, (errcode(ERRCODE_ACTIVE_SQL_TRANSACTION),
errmsg("Invalid Node Id '%u'.", nodeIdValue)));
}
workersForPlacementList =
lappend(workersForPlacementList, (void *) workerNode);
}
return workersForPlacementList;
}
/*
* SplitShard API to split a given shard (or shard group) based on specified split points
* to a set of destination nodes.
* 'splitMode' : Mode of split operation.
* 'splitOperation' : Customer operation that triggered split.
* 'shardInterval' : Source shard interval to be split.
* 'shardSplitPointsList' : Split Points list for the source 'shardInterval'.
* 'nodeIdsForPlacementList' : Placement list corresponding to split children.
* 'distributionColumnList' : Maps relation IDs to distribution columns.
* If not specified, the distribution column is read
* from the metadata.
* 'colocatedShardIntervalList' : Shard interval list for colocation group. (only used for
* create_distributed_table_concurrently).
* 'targetColocationId' : Specifies the colocation ID (only used for
* create_distributed_table_concurrently).
*/
void
SplitShard(SplitMode splitMode,
SplitOperation splitOperation,
uint64 shardIdToSplit,
List *shardSplitPointsList,
List *nodeIdsForPlacementList,
DistributionColumnMap *distributionColumnOverrides,
List *colocatedShardIntervalList,
uint32 targetColocationId)
{
ErrorIfModificationAndSplitInTheSameTransaction(splitOperation);
ShardInterval *shardIntervalToSplit = LoadShardInterval(shardIdToSplit); ShardInterval *shardIntervalToSplit = LoadShardInterval(shardIdToSplit);
List *colocatedTableList = ColocatedTableList(shardIntervalToSplit->relationId); List *colocatedTableList = ColocatedTableList(shardIntervalToSplit->relationId);
@ -391,6 +475,7 @@ SplitShard(SplitMode splitMode,
VerifyTablesHaveReplicaIdentity(colocatedTableList); VerifyTablesHaveReplicaIdentity(colocatedTableList);
} }
/* Acquire global lock to prevent concurrent split on the same colocation group or relation */
Oid relationId = RelationIdForShard(shardIdToSplit); Oid relationId = RelationIdForShard(shardIdToSplit);
AcquirePlacementColocationLock(relationId, ExclusiveLock, "split"); AcquirePlacementColocationLock(relationId, ExclusiveLock, "split");
@ -407,47 +492,50 @@ SplitShard(SplitMode splitMode,
LockRelationOid(colocatedTableId, ShareUpdateExclusiveLock); LockRelationOid(colocatedTableId, ShareUpdateExclusiveLock);
} }
ErrorIfCannotSplitShard(SHARD_SPLIT_API, shardIntervalToSplit); ErrorIfCannotSplitShard(splitOperation, shardIntervalToSplit);
ErrorIfCannotSplitShardExtended( ErrorIfCannotSplitShardExtended(
SHARD_SPLIT_API, splitOperation,
shardIntervalToSplit, shardIntervalToSplit,
shardSplitPointsList, shardSplitPointsList,
nodeIdsForPlacementList); nodeIdsForPlacementList);
List *workersForPlacementList = NIL; List *workersForPlacementList = GetWorkerNodesFromWorkerIds(nodeIdsForPlacementList);
Datum nodeId;
foreach_int(nodeId, nodeIdsForPlacementList) List *sourceColocatedShardIntervalList = NIL;
if (colocatedShardIntervalList == NIL)
{ {
uint32 nodeIdValue = DatumGetUInt32(nodeId); sourceColocatedShardIntervalList = ColocatedShardIntervalList(
WorkerNode *workerNode = LookupNodeByNodeId(nodeIdValue); shardIntervalToSplit);
/* NodeId in Citus are unsigned and range from [1, 4294967296]. */
if (nodeIdValue < 1 || workerNode == NULL)
{
ereport(ERROR, (errcode(ERRCODE_ACTIVE_SQL_TRANSACTION),
errmsg("Invalid Node Id '%u'.", nodeIdValue)));
}
workersForPlacementList =
lappend(workersForPlacementList, (void *) workerNode);
} }
else
{
sourceColocatedShardIntervalList = colocatedShardIntervalList;
}
/* use the user-specified shard ID as the split workflow ID */
uint64 splitWorkflowId = shardIntervalToSplit->shardId;
if (splitMode == BLOCKING_SPLIT) if (splitMode == BLOCKING_SPLIT)
{ {
EnsureReferenceTablesExistOnAllNodesExtended(TRANSFER_MODE_BLOCK_WRITES); EnsureReferenceTablesExistOnAllNodesExtended(TRANSFER_MODE_BLOCK_WRITES);
BlockingShardSplit( BlockingShardSplit(
splitOperation, splitOperation,
shardIntervalToSplit, splitWorkflowId,
sourceColocatedShardIntervalList,
shardSplitPointsList, shardSplitPointsList,
workersForPlacementList); workersForPlacementList,
distributionColumnOverrides);
} }
else else
{ {
NonBlockingShardSplit( NonBlockingShardSplit(
splitOperation, splitOperation,
shardIntervalToSplit, splitWorkflowId,
sourceColocatedShardIntervalList,
shardSplitPointsList, shardSplitPointsList,
workersForPlacementList); workersForPlacementList,
distributionColumnOverrides,
targetColocationId);
PlacementMovedUsingLogicalReplicationInTX = true; PlacementMovedUsingLogicalReplicationInTX = true;
} }
@ -521,20 +609,20 @@ CreateEmptyMapForShardsCreatedByWorkflow()
/* /*
* SplitShard API to split a given shard (or shard group) in blocking fashion * SplitShard API to split a given shard (or shard group) in blocking fashion
* based on specified split points to a set of destination nodes. * based on specified split points to a set of destination nodes.
* 'splitOperation' : Customer operation that triggered split. * splitOperation : Customer operation that triggered split.
* 'shardIntervalToSplit' : Source shard interval to be split. * splitWorkflowId : Number used to identify split workflow in names.
* 'shardSplitPointsList' : Split Points list for the source 'shardInterval'. * sourceColocatedShardIntervalList : Source shard group to be split.
* 'workersForPlacementList' : Placement list corresponding to split children. * shardSplitPointsList : Split Points list for the source 'shardInterval'.
* workersForPlacementList : Placement list corresponding to split children.
*/ */
static void static void
BlockingShardSplit(SplitOperation splitOperation, BlockingShardSplit(SplitOperation splitOperation,
ShardInterval *shardIntervalToSplit, uint64 splitWorkflowId,
List *sourceColocatedShardIntervalList,
List *shardSplitPointsList, List *shardSplitPointsList,
List *workersForPlacementList) List *workersForPlacementList,
DistributionColumnMap *distributionColumnOverrides)
{ {
List *sourceColocatedShardIntervalList = ColocatedShardIntervalList(
shardIntervalToSplit);
BlockWritesToShardList(sourceColocatedShardIntervalList); BlockWritesToShardList(sourceColocatedShardIntervalList);
/* First create shard interval metadata for split children */ /* First create shard interval metadata for split children */
@ -543,29 +631,29 @@ BlockingShardSplit(SplitOperation splitOperation,
shardSplitPointsList); shardSplitPointsList);
/* Only single placement allowed (already validated RelationReplicationFactor = 1) */ /* Only single placement allowed (already validated RelationReplicationFactor = 1) */
List *sourcePlacementList = ActiveShardPlacementList(shardIntervalToSplit->shardId); ShardInterval *firstShard = linitial(sourceColocatedShardIntervalList);
Assert(sourcePlacementList->length == 1); WorkerNode *sourceShardNode =
ShardPlacement *sourceShardPlacement = (ShardPlacement *) linitial( ActiveShardPlacementWorkerNode(firstShard->shardId);
sourcePlacementList);
WorkerNode *sourceShardToCopyNode = FindNodeWithNodeId(sourceShardPlacement->nodeId,
false /* missingOk */);
HTAB *mapOfShardToPlacementCreatedByWorkflow = HTAB *mapOfShardToPlacementCreatedByWorkflow =
CreateEmptyMapForShardsCreatedByWorkflow(); CreateEmptyMapForShardsCreatedByWorkflow();
PG_TRY(); PG_TRY();
{ {
/* /* Physically create split children. */
* Physically create split children, perform split copy and create auxiliary structures. CreateSplitShardsForShardGroup(mapOfShardToPlacementCreatedByWorkflow,
* This includes: indexes, replicaIdentity. triggers and statistics. shardGroupSplitIntervalListList,
* Foreign key constraints are created after Metadata changes (see CreateForeignKeyConstraints). workersForPlacementList);
*/
CreateAndCopySplitShardsForShardGroup( /* For Blocking split, copy isn't snapshotted */
mapOfShardToPlacementCreatedByWorkflow, char *snapshotName = NULL;
sourceShardToCopyNode, DoSplitCopy(sourceShardNode, sourceColocatedShardIntervalList,
sourceColocatedShardIntervalList, shardGroupSplitIntervalListList, workersForPlacementList,
shardGroupSplitIntervalListList, snapshotName, distributionColumnOverrides);
workersForPlacementList);
/* Create auxiliary structures (indexes, stats, replicaindentities, triggers) */
CreateAuxiliaryStructuresForShardGroup(shardGroupSplitIntervalListList,
workersForPlacementList,
true /* includeReplicaIdentity*/);
/* /*
* Up to this point, we performed various subtransactions that may * Up to this point, we performed various subtransactions that may
@ -608,6 +696,7 @@ BlockingShardSplit(SplitOperation splitOperation,
} }
PG_END_TRY(); PG_END_TRY();
CitusInvalidateRelcacheByRelid(DistShardRelationId()); CitusInvalidateRelcacheByRelid(DistShardRelationId());
} }
@ -731,33 +820,6 @@ CreateAuxiliaryStructuresForShardGroup(List *shardGroupSplitIntervalListList,
} }
/*
* Create ShardGroup split children, perform copy and create auxiliary structures
* on a list of corresponding workers.
*/
static void
CreateAndCopySplitShardsForShardGroup(HTAB *mapOfShardToPlacementCreatedByWorkflow,
WorkerNode *sourceShardNode,
List *sourceColocatedShardIntervalList,
List *shardGroupSplitIntervalListList,
List *workersForPlacementList)
{
CreateSplitShardsForShardGroup(mapOfShardToPlacementCreatedByWorkflow,
shardGroupSplitIntervalListList,
workersForPlacementList);
/* For Blocking split, copy isn't snapshotted */
char *snapshotName = NULL;
DoSplitCopy(sourceShardNode, sourceColocatedShardIntervalList,
shardGroupSplitIntervalListList, workersForPlacementList, snapshotName);
/* Create auxiliary structures (indexes, stats, replicaindentities, triggers) */
CreateAuxiliaryStructuresForShardGroup(shardGroupSplitIntervalListList,
workersForPlacementList,
true /* includeReplicaIdentity*/);
}
/* /*
* Perform Split Copy from source shard(s) to split children. * Perform Split Copy from source shard(s) to split children.
* 'sourceShardNode' : Source shard worker node. * 'sourceShardNode' : Source shard worker node.
@ -768,7 +830,7 @@ CreateAndCopySplitShardsForShardGroup(HTAB *mapOfShardToPlacementCreatedByWorkfl
static void static void
DoSplitCopy(WorkerNode *sourceShardNode, List *sourceColocatedShardIntervalList, DoSplitCopy(WorkerNode *sourceShardNode, List *sourceColocatedShardIntervalList,
List *shardGroupSplitIntervalListList, List *destinationWorkerNodesList, List *shardGroupSplitIntervalListList, List *destinationWorkerNodesList,
char *snapShotName) char *snapShotName, DistributionColumnMap *distributionColumnOverrides)
{ {
ShardInterval *sourceShardIntervalToCopy = NULL; ShardInterval *sourceShardIntervalToCopy = NULL;
List *splitShardIntervalList = NIL; List *splitShardIntervalList = NIL;
@ -783,25 +845,40 @@ DoSplitCopy(WorkerNode *sourceShardNode, List *sourceColocatedShardIntervalList,
* data themselves. Their partitions do contain data, but those are * data themselves. Their partitions do contain data, but those are
* different colocated shards that will be copied seperately. * different colocated shards that will be copied seperately.
*/ */
if (!PartitionedTable(sourceShardIntervalToCopy->relationId)) if (PartitionedTable(sourceShardIntervalToCopy->relationId))
{ {
StringInfo splitCopyUdfCommand = CreateSplitCopyCommand( continue;
sourceShardIntervalToCopy,
splitShardIntervalList,
destinationWorkerNodesList);
/* 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);
taskId++;
} }
Oid relationId = sourceShardIntervalToCopy->relationId;
Var *distributionColumn =
GetDistributionColumnWithOverrides(relationId,
distributionColumnOverrides);
Assert(distributionColumn != NULL);
bool missingOK = false;
char *distributionColumnName = get_attname(relationId,
distributionColumn->varattno,
missingOK);
StringInfo splitCopyUdfCommand = CreateSplitCopyCommand(
sourceShardIntervalToCopy,
distributionColumnName,
splitShardIntervalList,
destinationWorkerNodesList);
/* 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);
taskId++;
} }
ExecuteTaskListOutsideTransaction(ROW_MODIFY_NONE, splitCopyTaskList, ExecuteTaskListOutsideTransaction(ROW_MODIFY_NONE, splitCopyTaskList,
@ -834,6 +911,7 @@ DoSplitCopy(WorkerNode *sourceShardNode, List *sourceColocatedShardIntervalList,
*/ */
static StringInfo static StringInfo
CreateSplitCopyCommand(ShardInterval *sourceShardSplitInterval, CreateSplitCopyCommand(ShardInterval *sourceShardSplitInterval,
char *distributionColumnName,
List *splitChildrenShardIntervalList, List *splitChildrenShardIntervalList,
List *destinationWorkerNodesList) List *destinationWorkerNodesList)
{ {
@ -865,8 +943,9 @@ CreateSplitCopyCommand(ShardInterval *sourceShardSplitInterval,
appendStringInfo(splitCopyInfoArray, "]"); appendStringInfo(splitCopyInfoArray, "]");
StringInfo splitCopyUdf = makeStringInfo(); StringInfo splitCopyUdf = makeStringInfo();
appendStringInfo(splitCopyUdf, "SELECT pg_catalog.worker_split_copy(%lu, %s);", appendStringInfo(splitCopyUdf, "SELECT pg_catalog.worker_split_copy(%lu, %s, %s);",
sourceShardSplitInterval->shardId, sourceShardSplitInterval->shardId,
quote_literal_cstr(distributionColumnName),
splitCopyInfoArray->data); splitCopyInfoArray->data);
return splitCopyUdf; return splitCopyUdf;
@ -976,8 +1055,19 @@ CreateSplitIntervalsForShard(ShardInterval *sourceShard,
int shardIntervalCount = list_length(splitPointsForShard) + 1; int shardIntervalCount = list_length(splitPointsForShard) + 1;
ListCell *splitPointCell = list_head(splitPointsForShard); ListCell *splitPointCell = list_head(splitPointsForShard);
int32 splitParentMaxValue = DatumGetInt32(sourceShard->maxValue); int32 splitParentMaxValue = DatumGetInt32(sourceShard->maxValue);
int32 currentSplitChildMinValue = DatumGetInt32(sourceShard->minValue); int32 currentSplitChildMinValue = DatumGetInt32(sourceShard->minValue);
/* if we are splitting a Citus local table, assume whole shard range */
if (!sourceShard->maxValueExists)
{
splitParentMaxValue = PG_INT32_MAX;
}
if (!sourceShard->minValueExists)
{
currentSplitChildMinValue = PG_INT32_MIN;
}
for (int index = 0; index < shardIntervalCount; index++) for (int index = 0; index < shardIntervalCount; index++)
{ {
ShardInterval *splitChildShardInterval = CopyShardInterval(sourceShard); ShardInterval *splitChildShardInterval = CopyShardInterval(sourceShard);
@ -1008,6 +1098,54 @@ CreateSplitIntervalsForShard(ShardInterval *sourceShard,
} }
/*
* UpdateDistributionColumnsForShardGroup globally updates the pg_dist_partition metadata
* for each relation that has a shard in colocatedShardList.
*
* This is used primarily for Citus local -> distributed table conversion
* in create_distributed_table_concurrently.
*
* It would be nicer to keep this separate from shard split, but we need to do the
* update at exactly the right point in the shard split process, namely after
* replication slot creation and before inserting shard metadata, which itself
* needs to happen before foreign key creation (mainly because the foreign key
* functions depend on metadata).
*/
static void
UpdateDistributionColumnsForShardGroup(List *colocatedShardList,
DistributionColumnMap *distributionColumnMap,
char distributionMethod,
int shardCount,
uint32 colocationId)
{
ShardInterval *shardInterval = NULL;
foreach_ptr(shardInterval, colocatedShardList)
{
Oid relationId = shardInterval->relationId;
Var *distributionColumn = GetDistributionColumnFromMap(distributionColumnMap,
relationId);
/* we should have an entry for every relation ID in the colocation group */
Assert(distributionColumn != NULL);
if (colocationId == INVALID_COLOCATION_ID)
{
/*
* Getting here with an invalid co-location ID means that no
* appropriate co-location group exists yet.
*/
colocationId = CreateColocationGroup(shardCount,
ShardReplicationFactor,
distributionColumn->vartype,
distributionColumn->varcollid);
}
UpdateDistributionColumnGlobally(relationId, distributionMethod,
distributionColumn, colocationId);
}
}
/* /*
* Insert new shard and placement metadata. * Insert new shard and placement metadata.
* Sync the Metadata with all nodes if enabled. * Sync the Metadata with all nodes if enabled.
@ -1265,32 +1403,73 @@ TryDropSplitShardsOnFailure(HTAB *mapOfShardToPlacementCreatedByWorkflow)
/* /*
* SplitShard API to split a given shard (or shard group) in non-blocking fashion * AcquireNonblockingSplitLock does not allow concurrent nonblocking splits, because we share memory and
* based on specified split points to a set of destination nodes. * replication slots.
* 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 static void
NonBlockingShardSplit(SplitOperation splitOperation, AcquireNonblockingSplitLock(Oid relationId)
ShardInterval *shardIntervalToSplit,
List *shardSplitPointsList,
List *workersForPlacementList)
{ {
LOCKTAG tag;
const bool sessionLock = false;
const bool dontWait = true;
SET_LOCKTAG_CITUS_OPERATION(tag, CITUS_NONBLOCKING_SPLIT);
LockAcquireResult lockAcquired = LockAcquire(&tag, ExclusiveLock, sessionLock,
dontWait);
if (!lockAcquired)
{
ereport(ERROR, (errmsg("could not acquire the lock required to split "
"concurrently %s.", generate_qualified_relation_name(
relationId)),
errdetail("It means that either a concurrent shard move "
"or distributed table creation is happening."),
errhint("Make sure that the concurrent operation has "
"finished and re-run the command")));
}
}
/*
* SplitShard API to split a given shard (or shard group) in non-blocking fashion
* based on specified split points to a set of destination nodes.
* splitOperation : Customer operation that triggered split.
* splitWorkflowId : Number used to identify split workflow in names.
* sourceColocatedShardIntervalList : Source shard group to be split.
* shardSplitPointsList : Split Points list for the source 'shardInterval'.
* workersForPlacementList : Placement list corresponding to split children.
* distributionColumnList : Maps relation IDs to distribution columns.
* If not specified, the distribution column is read
* from the metadata.
* targetColocationId : Specifies the colocation ID (only used for
* create_distributed_table_concurrently).
*/
void
NonBlockingShardSplit(SplitOperation splitOperation,
uint64 splitWorkflowId,
List *sourceColocatedShardIntervalList,
List *shardSplitPointsList,
List *workersForPlacementList,
DistributionColumnMap *distributionColumnOverrides,
uint32 targetColocationId)
{
ErrorIfMultipleNonblockingMoveSplitInTheSameTransaction();
char *superUser = CitusExtensionOwnerName(); char *superUser = CitusExtensionOwnerName();
char *databaseName = get_database_name(MyDatabaseId); char *databaseName = get_database_name(MyDatabaseId);
List *sourceColocatedShardIntervalList = ColocatedShardIntervalList(
shardIntervalToSplit);
/* First create shard interval metadata for split children */ /* First create shard interval metadata for split children */
List *shardGroupSplitIntervalListList = CreateSplitIntervalsForShardGroup( List *shardGroupSplitIntervalListList = CreateSplitIntervalsForShardGroup(
sourceColocatedShardIntervalList, sourceColocatedShardIntervalList,
shardSplitPointsList); shardSplitPointsList);
WorkerNode *sourceShardToCopyNode = ActiveShardPlacementWorkerNode( ShardInterval *firstShard = linitial(sourceColocatedShardIntervalList);
shardIntervalToSplit->shardId);
/* Acquire global lock to prevent concurrent nonblocking splits */
AcquireNonblockingSplitLock(firstShard->relationId);
WorkerNode *sourceShardToCopyNode =
ActiveShardPlacementWorkerNode(firstShard->shardId);
/* Create hashmap to group shards for publication-subscription management */ /* Create hashmap to group shards for publication-subscription management */
HTAB *publicationInfoHash = CreateShardSplitInfoMapForPublication( HTAB *publicationInfoHash = CreateShardSplitInfoMapForPublication(
@ -1352,13 +1531,13 @@ NonBlockingShardSplit(SplitOperation splitOperation,
/* 4) Create Publications. */ /* 4) Create Publications. */
CreatePublications(sourceConnection, publicationInfoHash); CreatePublications(sourceConnection, publicationInfoHash);
/* 5) Execute 'worker_split_shard_replication_setup UDF */ /* 5) Execute 'worker_split_shard_replication_setup UDF */
List *replicationSlotInfoList = ExecuteSplitShardReplicationSetupUDF( List *replicationSlotInfoList = ExecuteSplitShardReplicationSetupUDF(
sourceShardToCopyNode, sourceShardToCopyNode,
sourceColocatedShardIntervalList, sourceColocatedShardIntervalList,
shardGroupSplitIntervalListList, shardGroupSplitIntervalListList,
workersForPlacementList); workersForPlacementList,
distributionColumnOverrides);
/* /*
* Subscriber flow starts from here. * Subscriber flow starts from here.
@ -1400,7 +1579,7 @@ NonBlockingShardSplit(SplitOperation splitOperation,
/* 8) Do snapshotted Copy */ /* 8) Do snapshotted Copy */
DoSplitCopy(sourceShardToCopyNode, sourceColocatedShardIntervalList, DoSplitCopy(sourceShardToCopyNode, sourceColocatedShardIntervalList,
shardGroupSplitIntervalListList, workersForPlacementList, shardGroupSplitIntervalListList, workersForPlacementList,
snapshot); snapshot, distributionColumnOverrides);
/* /*
* 9) Create replica identities, this needs to be done before enabling * 9) Create replica identities, this needs to be done before enabling
@ -1446,7 +1625,6 @@ NonBlockingShardSplit(SplitOperation splitOperation,
/* 19) Drop Publications */ /* 19) Drop Publications */
DropPublications(sourceConnection, publicationInfoHash); DropPublications(sourceConnection, publicationInfoHash);
/* /*
* 20) Drop old shards and delete related metadata. Have to do that before * 20) Drop old shards and delete related metadata. Have to do that before
* creating the new shard metadata, because there's cross-checks * creating the new shard metadata, because there's cross-checks
@ -1454,7 +1632,40 @@ NonBlockingShardSplit(SplitOperation splitOperation,
*/ */
DropShardList(sourceColocatedShardIntervalList); DropShardList(sourceColocatedShardIntervalList);
/* 21) Insert new shard and placement metdata */ /*
* 21) In case of create_distributed_table_concurrently, which converts
* a Citus local table to a distributed table, update the distributed
* table metadata now.
*
* We would rather have this be outside of the scope of NonBlockingShardSplit,
* but we cannot make metadata changes before replication slot creation, and
* we cannot create the replication slot before creating new shards and
* corresponding publications, because the decoder uses a catalog snapshot
* from the time of the slot creation, which means it would not be able to see
* the shards or publications when replication starts if it was created before.
*
* We also cannot easily move metadata changes to be after this function,
* because CreateForeignKeyConstraints relies on accurate metadata and
* we also want to perform the clean-up logic in PG_CATCH in case of
* failure.
*
* Hence, this appears to be the only suitable spot for updating
* pg_dist_partition and pg_dist_colocation.
*/
if (splitOperation == CREATE_DISTRIBUTED_TABLE)
{
/* we currently only use split for hash-distributed tables */
char distributionMethod = DISTRIBUTE_BY_HASH;
int shardCount = list_length(shardSplitPointsList) + 1;
UpdateDistributionColumnsForShardGroup(sourceColocatedShardIntervalList,
distributionColumnOverrides,
distributionMethod,
shardCount,
targetColocationId);
}
/* 22) Insert new shard and placement metdata */
InsertSplitChildrenShardMetadata(shardGroupSplitIntervalListList, InsertSplitChildrenShardMetadata(shardGroupSplitIntervalListList,
workersForPlacementList); workersForPlacementList);
@ -1462,7 +1673,7 @@ NonBlockingShardSplit(SplitOperation splitOperation,
workersForPlacementList); workersForPlacementList);
/* /*
* 22) Create foreign keys if exists after the metadata changes happening in * 23) Create foreign keys if exists after the metadata changes happening in
* DropShardList() and InsertSplitChildrenShardMetadata() because the foreign * DropShardList() and InsertSplitChildrenShardMetadata() because the foreign
* key creation depends on the new metadata. * key creation depends on the new metadata.
*/ */
@ -1470,7 +1681,7 @@ NonBlockingShardSplit(SplitOperation splitOperation,
workersForPlacementList); workersForPlacementList);
/* /*
* 23) Drop dummy shards. * 24) Drop dummy shards.
*/ */
DropDummyShards(mapOfDummyShardToPlacement); DropDummyShards(mapOfDummyShardToPlacement);
@ -1654,12 +1865,14 @@ static List *
ExecuteSplitShardReplicationSetupUDF(WorkerNode *sourceWorkerNode, ExecuteSplitShardReplicationSetupUDF(WorkerNode *sourceWorkerNode,
List *sourceColocatedShardIntervalList, List *sourceColocatedShardIntervalList,
List *shardGroupSplitIntervalListList, List *shardGroupSplitIntervalListList,
List *destinationWorkerNodesList) List *destinationWorkerNodesList,
DistributionColumnMap *distributionColumnOverrides)
{ {
StringInfo splitShardReplicationUDF = CreateSplitShardReplicationSetupUDF( StringInfo splitShardReplicationUDF = CreateSplitShardReplicationSetupUDF(
sourceColocatedShardIntervalList, sourceColocatedShardIntervalList,
shardGroupSplitIntervalListList, shardGroupSplitIntervalListList,
destinationWorkerNodesList); destinationWorkerNodesList,
distributionColumnOverrides);
/* Force a new connection to execute the UDF */ /* Force a new connection to execute the UDF */
int connectionFlags = 0; int connectionFlags = 0;
@ -1756,7 +1969,8 @@ ExecuteSplitShardReleaseSharedMemory(WorkerNode *sourceWorkerNode)
StringInfo StringInfo
CreateSplitShardReplicationSetupUDF(List *sourceColocatedShardIntervalList, CreateSplitShardReplicationSetupUDF(List *sourceColocatedShardIntervalList,
List *shardGroupSplitIntervalListList, List *shardGroupSplitIntervalListList,
List *destinationWorkerNodesList) List *destinationWorkerNodesList,
DistributionColumnMap *distributionColumnOverrides)
{ {
StringInfo splitChildrenRows = makeStringInfo(); StringInfo splitChildrenRows = makeStringInfo();
@ -1768,11 +1982,15 @@ CreateSplitShardReplicationSetupUDF(List *sourceColocatedShardIntervalList,
{ {
int64 sourceShardId = sourceShardIntervalToCopy->shardId; int64 sourceShardId = sourceShardIntervalToCopy->shardId;
Oid relationId = sourceShardIntervalToCopy->relationId; Oid relationId = sourceShardIntervalToCopy->relationId;
Var *partitionColumn = DistPartitionKey(relationId);
Var *distributionColumn =
GetDistributionColumnWithOverrides(relationId,
distributionColumnOverrides);
bool missingOK = false; bool missingOK = false;
char *partitionColumnName = char *distributionColumnName =
get_attname(relationId, partitionColumn->varattno, missingOK); get_attname(relationId, distributionColumn->varattno,
missingOK);
ShardInterval *splitChildShardInterval = NULL; ShardInterval *splitChildShardInterval = NULL;
WorkerNode *destinationWorkerNode = NULL; WorkerNode *destinationWorkerNode = NULL;
@ -1795,7 +2013,7 @@ CreateSplitShardReplicationSetupUDF(List *sourceColocatedShardIntervalList,
appendStringInfo(splitChildrenRows, appendStringInfo(splitChildrenRows,
"ROW(%lu, %s, %lu, %s, %s, %u)::pg_catalog.split_shard_info", "ROW(%lu, %s, %lu, %s, %s, %u)::pg_catalog.split_shard_info",
sourceShardId, sourceShardId,
quote_literal_cstr(partitionColumnName), quote_literal_cstr(distributionColumnName),
splitChildShardInterval->shardId, splitChildShardInterval->shardId,
quote_literal_cstr(minValueString->data), quote_literal_cstr(minValueString->data),
quote_literal_cstr(maxValueString->data), quote_literal_cstr(maxValueString->data),

View File

@ -9,15 +9,16 @@
#include "postgres.h" #include "postgres.h"
#include "pg_version_compat.h" #include "pg_version_compat.h"
#include "distributed/citus_ruleutils.h"
#include "distributed/distribution_column.h"
#include "distributed/intermediate_results.h"
#include "distributed/listutils.h"
#include "distributed/multi_executor.h"
#include "distributed/utils/array_type.h"
#include "distributed/worker_shard_copy.h"
#include "utils/lsyscache.h" #include "utils/lsyscache.h"
#include "utils/array.h" #include "utils/array.h"
#include "utils/builtins.h" #include "utils/builtins.h"
#include "distributed/utils/array_type.h"
#include "distributed/listutils.h"
#include "distributed/multi_executor.h"
#include "distributed/worker_shard_copy.h"
#include "distributed/intermediate_results.h"
#include "distributed/citus_ruleutils.h"
PG_FUNCTION_INFO_V1(worker_split_copy); PG_FUNCTION_INFO_V1(worker_split_copy);
@ -38,6 +39,7 @@ static DestReceiver ** CreateShardCopyDestReceivers(EState *estate,
static DestReceiver * CreatePartitionedSplitCopyDestReceiver(EState *executor, static DestReceiver * CreatePartitionedSplitCopyDestReceiver(EState *executor,
ShardInterval * ShardInterval *
shardIntervalToSplitCopy, shardIntervalToSplitCopy,
char *partitionColumnName,
List *splitCopyInfoList); List *splitCopyInfoList);
static void BuildMinMaxRangeArrays(List *splitCopyInfoList, ArrayType **minValueArray, static void BuildMinMaxRangeArrays(List *splitCopyInfoList, ArrayType **minValueArray,
ArrayType **maxValueArray); ArrayType **maxValueArray);
@ -54,7 +56,10 @@ worker_split_copy(PG_FUNCTION_ARGS)
uint64 shardIdToSplitCopy = DatumGetUInt64(PG_GETARG_DATUM(0)); uint64 shardIdToSplitCopy = DatumGetUInt64(PG_GETARG_DATUM(0));
ShardInterval *shardIntervalToSplitCopy = LoadShardInterval(shardIdToSplitCopy); ShardInterval *shardIntervalToSplitCopy = LoadShardInterval(shardIdToSplitCopy);
ArrayType *splitCopyInfoArrayObject = PG_GETARG_ARRAYTYPE_P(1); text *partitionColumnText = PG_GETARG_TEXT_P(1);
char *partitionColumnName = text_to_cstring(partitionColumnText);
ArrayType *splitCopyInfoArrayObject = PG_GETARG_ARRAYTYPE_P(2);
bool arrayHasNull = ARR_HASNULL(splitCopyInfoArrayObject); bool arrayHasNull = ARR_HASNULL(splitCopyInfoArrayObject);
if (arrayHasNull) if (arrayHasNull)
{ {
@ -82,6 +87,7 @@ worker_split_copy(PG_FUNCTION_ARGS)
EState *executor = CreateExecutorState(); EState *executor = CreateExecutorState();
DestReceiver *splitCopyDestReceiver = CreatePartitionedSplitCopyDestReceiver(executor, DestReceiver *splitCopyDestReceiver = CreatePartitionedSplitCopyDestReceiver(executor,
shardIntervalToSplitCopy, shardIntervalToSplitCopy,
partitionColumnName,
splitCopyInfoList); splitCopyInfoList);
Oid sourceShardToCopySchemaOId = get_rel_namespace( Oid sourceShardToCopySchemaOId = get_rel_namespace(
@ -228,6 +234,7 @@ CreateShardCopyDestReceivers(EState *estate, ShardInterval *shardIntervalToSplit
static DestReceiver * static DestReceiver *
CreatePartitionedSplitCopyDestReceiver(EState *estate, CreatePartitionedSplitCopyDestReceiver(EState *estate,
ShardInterval *shardIntervalToSplitCopy, ShardInterval *shardIntervalToSplitCopy,
char *partitionColumnName,
List *splitCopyInfoList) List *splitCopyInfoList)
{ {
/* Create underlying ShardCopyDestReceivers */ /* Create underlying ShardCopyDestReceivers */
@ -240,10 +247,17 @@ CreatePartitionedSplitCopyDestReceiver(EState *estate,
ArrayType *minValuesArray = NULL; ArrayType *minValuesArray = NULL;
ArrayType *maxValuesArray = NULL; ArrayType *maxValuesArray = NULL;
BuildMinMaxRangeArrays(splitCopyInfoList, &minValuesArray, &maxValuesArray); BuildMinMaxRangeArrays(splitCopyInfoList, &minValuesArray, &maxValuesArray);
CitusTableCacheEntry *cacheEntry = GetCitusTableCacheEntry(
shardIntervalToSplitCopy->relationId); /* we currently only support hash-distribution */
char partitionMethod = cacheEntry->partitionMethod; char partitionMethod = DISTRIBUTE_BY_HASH;
Var *partitionColumn = cacheEntry->partitionColumn;
/* synthetically build the partition column by looking at shard columns */
uint64 shardId = shardIntervalToSplitCopy->shardId;
bool missingOK = false;
Oid shardRelationId = LookupShardRelationFromCatalog(shardId, missingOK);
Var *partitionColumn = BuildDistributionKeyFromColumnName(shardRelationId,
partitionColumnName,
AccessShareLock);
CitusTableCacheEntry *shardSearchInfo = CitusTableCacheEntry *shardSearchInfo =
QueryTupleShardSearchInfo(minValuesArray, maxValuesArray, QueryTupleShardSearchInfo(minValuesArray, maxValuesArray,

View File

@ -14,6 +14,7 @@
#include "replication/logical.h" #include "replication/logical.h"
#include "utils/typcache.h" #include "utils/typcache.h"
extern void _PG_output_plugin_init(OutputPluginCallbacks *cb); extern void _PG_output_plugin_init(OutputPluginCallbacks *cb);
static LogicalDecodeChangeCB pgoutputChangeCB; static LogicalDecodeChangeCB pgoutputChangeCB;
@ -216,8 +217,9 @@ GetHashValueForIncomingTuple(Relation sourceShardRelation,
TYPECACHE_HASH_PROC_FINFO); TYPECACHE_HASH_PROC_FINFO);
/* get hashed value of the distribution value */ /* get hashed value of the distribution value */
Datum hashedValueDatum = FunctionCall1(&(typeEntry->hash_proc_finfo), Datum hashedValueDatum = FunctionCall1Coll(&(typeEntry->hash_proc_finfo),
partitionColumnValue); typeEntry->typcollation,
partitionColumnValue);
return DatumGetInt32(hashedValueDatum); return DatumGetInt32(hashedValueDatum);
} }

View File

@ -1,4 +1,6 @@
#include "udfs/citus_locks/11.1-1.sql" #include "udfs/citus_locks/11.1-1.sql"
#include "udfs/create_distributed_table_concurrently/11.1-1.sql"
#include "udfs/citus_internal_delete_partition_metadata/11.1-1.sql"
DROP FUNCTION pg_catalog.worker_create_schema(bigint,text); DROP FUNCTION pg_catalog.worker_create_schema(bigint,text);
DROP FUNCTION pg_catalog.worker_cleanup_job_schema_cache(); DROP FUNCTION pg_catalog.worker_cleanup_job_schema_cache();

View File

@ -70,6 +70,7 @@ DROP FUNCTION pg_catalog.citus_split_shard_by_split_points(
shard_transfer_mode citus.shard_transfer_mode); shard_transfer_mode citus.shard_transfer_mode);
DROP FUNCTION pg_catalog.worker_split_copy( DROP FUNCTION pg_catalog.worker_split_copy(
source_shard_id bigint, source_shard_id bigint,
distribution_column text,
splitCopyInfos pg_catalog.split_copy_info[]); splitCopyInfos pg_catalog.split_copy_info[]);
DROP TYPE pg_catalog.split_copy_info; DROP TYPE pg_catalog.split_copy_info;
@ -97,3 +98,5 @@ DROP FUNCTION pg_catalog.replicate_reference_tables(citus.shard_transfer_mode);
DROP FUNCTION pg_catalog.isolate_tenant_to_new_shard(table_name regclass, tenant_id "any", cascade_option text, shard_transfer_mode citus.shard_transfer_mode); DROP FUNCTION pg_catalog.isolate_tenant_to_new_shard(table_name regclass, tenant_id "any", cascade_option text, shard_transfer_mode citus.shard_transfer_mode);
#include "../udfs/isolate_tenant_to_new_shard/8.0-1.sql" #include "../udfs/isolate_tenant_to_new_shard/8.0-1.sql"
DROP FUNCTION pg_catalog.create_distributed_table_concurrently;
DROP FUNCTION pg_catalog.citus_internal_delete_partition_metadata(regclass);

View File

@ -0,0 +1,7 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_delete_partition_metadata(table_name regclass)
RETURNS void
LANGUAGE C STRICT
AS 'MODULE_PATHNAME';
COMMENT ON FUNCTION pg_catalog.citus_internal_delete_partition_metadata(regclass) IS
'Deletes a row from pg_dist_partition with table ownership checks';

View File

@ -0,0 +1,7 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_delete_partition_metadata(table_name regclass)
RETURNS void
LANGUAGE C STRICT
AS 'MODULE_PATHNAME';
COMMENT ON FUNCTION pg_catalog.citus_internal_delete_partition_metadata(regclass) IS
'Deletes a row from pg_dist_partition with table ownership checks';

View File

@ -0,0 +1,14 @@
CREATE FUNCTION pg_catalog.create_distributed_table_concurrently(table_name regclass,
distribution_column text,
distribution_type citus.distribution_type DEFAULT 'hash',
colocate_with text DEFAULT 'default',
shard_count int DEFAULT NULL)
RETURNS void
LANGUAGE C
AS 'MODULE_PATHNAME', $$create_distributed_table_concurrently$$;
COMMENT ON FUNCTION pg_catalog.create_distributed_table_concurrently(table_name regclass,
distribution_column text,
distribution_type citus.distribution_type,
colocate_with text,
shard_count int)
IS 'creates a distributed table and avoids blocking writes';

View File

@ -0,0 +1,14 @@
CREATE FUNCTION pg_catalog.create_distributed_table_concurrently(table_name regclass,
distribution_column text,
distribution_type citus.distribution_type DEFAULT 'hash',
colocate_with text DEFAULT 'default',
shard_count int DEFAULT NULL)
RETURNS void
LANGUAGE C
AS 'MODULE_PATHNAME', $$create_distributed_table_concurrently$$;
COMMENT ON FUNCTION pg_catalog.create_distributed_table_concurrently(table_name regclass,
distribution_column text,
distribution_type citus.distribution_type,
colocate_with text,
shard_count int)
IS 'creates a distributed table and avoids blocking writes';

View File

@ -14,9 +14,10 @@ ALTER TYPE citus.split_copy_info SET SCHEMA pg_catalog;
CREATE OR REPLACE FUNCTION pg_catalog.worker_split_copy( CREATE OR REPLACE FUNCTION pg_catalog.worker_split_copy(
source_shard_id bigint, source_shard_id bigint,
distribution_column text,
splitCopyInfos pg_catalog.split_copy_info[]) splitCopyInfos pg_catalog.split_copy_info[])
RETURNS void RETURNS void
LANGUAGE C STRICT LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$worker_split_copy$$; AS 'MODULE_PATHNAME', $$worker_split_copy$$;
COMMENT ON FUNCTION pg_catalog.worker_split_copy(source_shard_id bigint, splitCopyInfos pg_catalog.split_copy_info[]) COMMENT ON FUNCTION pg_catalog.worker_split_copy(source_shard_id bigint, distribution_column text, splitCopyInfos pg_catalog.split_copy_info[])
IS 'Perform split copy for shard'; IS 'Perform split copy for shard';

View File

@ -14,9 +14,10 @@ ALTER TYPE citus.split_copy_info SET SCHEMA pg_catalog;
CREATE OR REPLACE FUNCTION pg_catalog.worker_split_copy( CREATE OR REPLACE FUNCTION pg_catalog.worker_split_copy(
source_shard_id bigint, source_shard_id bigint,
distribution_column text,
splitCopyInfos pg_catalog.split_copy_info[]) splitCopyInfos pg_catalog.split_copy_info[])
RETURNS void RETURNS void
LANGUAGE C STRICT LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$worker_split_copy$$; AS 'MODULE_PATHNAME', $$worker_split_copy$$;
COMMENT ON FUNCTION pg_catalog.worker_split_copy(source_shard_id bigint, splitCopyInfos pg_catalog.split_copy_info[]) COMMENT ON FUNCTION pg_catalog.worker_split_copy(source_shard_id bigint, distribution_column text, splitCopyInfos pg_catalog.split_copy_info[])
IS 'Perform split copy for shard'; IS 'Perform split copy for shard';

View File

@ -114,7 +114,8 @@ IntegerArrayTypeToList(ArrayType *arrayObject)
for (int index = 0; index < arrayObjectCount; index++) for (int index = 0; index < arrayObjectCount; index++)
{ {
list = lappend_int(list, datumObjectArray[index]); int32 intObject = DatumGetInt32(datumObjectArray[index]);
list = lappend_int(list, intObject);
} }
return list; return list;

View File

@ -53,6 +53,7 @@ static void DeleteColocationGroup(uint32 colocationId);
static uint32 CreateColocationGroupForRelation(Oid sourceRelationId); static uint32 CreateColocationGroupForRelation(Oid sourceRelationId);
static void BreakColocation(Oid sourceRelationId); static void BreakColocation(Oid sourceRelationId);
/* exports for SQL callable functions */ /* exports for SQL callable functions */
PG_FUNCTION_INFO_V1(mark_tables_colocated); PG_FUNCTION_INFO_V1(mark_tables_colocated);
PG_FUNCTION_INFO_V1(get_colocated_shard_array); PG_FUNCTION_INFO_V1(get_colocated_shard_array);
@ -142,6 +143,17 @@ IsColocateWithNone(char *colocateWithTableName)
} }
/*
* IsColocateWithDefault returns true if the given table is
* the special keyword "default".
*/
bool
IsColocateWithDefault(char *colocateWithTableName)
{
return pg_strncasecmp(colocateWithTableName, "default", NAMEDATALEN) == 0;
}
/* /*
* BreakColocation breaks the colocations of the given relation id. * BreakColocation breaks the colocations of the given relation id.
* If t1, t2 and t3 are colocated and we call this function with t2, * If t1, t2 and t3 are colocated and we call this function with t2,
@ -564,6 +576,39 @@ ColocationId(int shardCount, int replicationFactor, Oid distributionColumnType,
} }
/*
* AcquireColocationDefaultLock serializes concurrent creation of a colocation entry
* for default group.
*/
void
AcquireColocationDefaultLock(void)
{
LOCKTAG tag;
const bool sessionLock = false;
const bool dontWait = false;
SET_LOCKTAG_CITUS_OPERATION(tag, CITUS_CREATE_COLOCATION_DEFAULT);
(void) LockAcquire(&tag, ExclusiveLock, sessionLock, dontWait);
}
/*
* ReleaseColocationDefaultLock releases the lock for concurrent creation of a colocation entry
* for default group.
*/
void
ReleaseColocationDefaultLock(void)
{
LOCKTAG tag;
const bool sessionLock = false;
SET_LOCKTAG_CITUS_OPERATION(tag, CITUS_CREATE_COLOCATION_DEFAULT);
LockRelease(&tag, ExclusiveLock, sessionLock);
}
/* /*
* CreateColocationGroup creates a new colocation id and writes it into * CreateColocationGroup creates a new colocation id and writes it into
* pg_dist_colocation with the given configuration. It also returns the created * pg_dist_colocation with the given configuration. It also returns the created
@ -619,7 +664,7 @@ InsertColocationGroupLocally(uint32 colocationId, int shardCount, int replicatio
/* increment the counter so that next command can see the row */ /* increment the counter so that next command can see the row */
CommandCounterIncrement(); CommandCounterIncrement();
table_close(pgDistColocation, RowExclusiveLock); table_close(pgDistColocation, NoLock);
} }
@ -1271,5 +1316,110 @@ DeleteColocationGroupLocally(uint32 colocationId)
} }
systable_endscan(scanDescriptor); systable_endscan(scanDescriptor);
table_close(pgDistColocation, RowExclusiveLock); table_close(pgDistColocation, NoLock);
}
/*
* FindColocateWithColocationId tries to find a colocation ID for a given
* colocate_with clause passed to create_distributed_table.
*/
uint32
FindColocateWithColocationId(Oid relationId, char replicationModel,
Oid distributionColumnType,
Oid distributionColumnCollation,
int shardCount, bool shardCountIsStrict,
char *colocateWithTableName)
{
uint32 colocationId = INVALID_COLOCATION_ID;
if (IsColocateWithDefault(colocateWithTableName))
{
/* check for default colocation group */
colocationId = ColocationId(shardCount, ShardReplicationFactor,
distributionColumnType,
distributionColumnCollation);
/*
* if the shardCount is strict then we check if the shard count
* of the colocated table is actually shardCount
*/
if (shardCountIsStrict && colocationId != INVALID_COLOCATION_ID)
{
Oid colocatedTableId = ColocatedTableId(colocationId);
if (colocatedTableId != InvalidOid)
{
CitusTableCacheEntry *cacheEntry =
GetCitusTableCacheEntry(colocatedTableId);
int colocatedTableShardCount = cacheEntry->shardIntervalArrayLength;
if (colocatedTableShardCount != shardCount)
{
colocationId = INVALID_COLOCATION_ID;
}
}
}
}
else if (!IsColocateWithNone(colocateWithTableName))
{
text *colocateWithTableNameText = cstring_to_text(colocateWithTableName);
Oid sourceRelationId = ResolveRelationId(colocateWithTableNameText, false);
EnsureTableCanBeColocatedWith(relationId, replicationModel,
distributionColumnType, sourceRelationId);
colocationId = TableColocationId(sourceRelationId);
}
return colocationId;
}
/*
* EnsureTableCanBeColocatedWith checks whether a given replication model and
* distribution column type is suitable to distribute a table to be colocated
* with given source table.
*
* We only pass relationId to provide meaningful error messages.
*/
void
EnsureTableCanBeColocatedWith(Oid relationId, char replicationModel,
Oid distributionColumnType, Oid sourceRelationId)
{
CitusTableCacheEntry *sourceTableEntry = GetCitusTableCacheEntry(sourceRelationId);
char sourceReplicationModel = sourceTableEntry->replicationModel;
Var *sourceDistributionColumn = DistPartitionKeyOrError(sourceRelationId);
if (!IsCitusTableTypeCacheEntry(sourceTableEntry, HASH_DISTRIBUTED))
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot distribute relation"),
errdetail("Currently, colocate_with option is only supported "
"for hash distributed tables.")));
}
if (sourceReplicationModel != replicationModel)
{
char *relationName = get_rel_name(relationId);
char *sourceRelationName = get_rel_name(sourceRelationId);
ereport(ERROR, (errmsg("cannot colocate tables %s and %s",
sourceRelationName, relationName),
errdetail("Replication models don't match for %s and %s.",
sourceRelationName, relationName)));
}
Oid sourceDistributionColumnType = sourceDistributionColumn->vartype;
if (sourceDistributionColumnType != distributionColumnType)
{
char *relationName = get_rel_name(relationId);
char *sourceRelationName = get_rel_name(sourceRelationId);
ereport(ERROR, (errmsg("cannot colocate tables %s and %s",
sourceRelationName, relationName),
errdetail("Distribution column types don't match for "
"%s and %s.", sourceRelationName,
relationName)));
}
} }

View File

@ -24,6 +24,7 @@
#include "nodes/nodes.h" #include "nodes/nodes.h"
#include "nodes/primnodes.h" #include "nodes/primnodes.h"
#include "parser/scansup.h" #include "parser/scansup.h"
#include "parser/parse_relation.h"
#include "utils/builtins.h" #include "utils/builtins.h"
#include "utils/elog.h" #include "utils/elog.h"
#include "utils/errcodes.h" #include "utils/errcodes.h"
@ -123,7 +124,7 @@ column_to_column_name(PG_FUNCTION_ARGS)
Var * Var *
BuildDistributionKeyFromColumnName(Oid relationId, char *columnName, LOCKMODE lockMode) BuildDistributionKeyFromColumnName(Oid relationId, char *columnName, LOCKMODE lockMode)
{ {
Relation relation = try_relation_open(relationId, ExclusiveLock); Relation relation = try_relation_open(relationId, lockMode);
if (relation == NULL) if (relation == NULL)
{ {
@ -172,6 +173,76 @@ BuildDistributionKeyFromColumnName(Oid relationId, char *columnName, LOCKMODE lo
} }
/*
* EnsureValidDistributionColumn Errors out if the
* specified column does not exist or is not suitable to be used as a
* distribution column. It does not hold locks.
*/
void
EnsureValidDistributionColumn(Oid relationId, char *columnName)
{
Relation relation = try_relation_open(relationId, AccessShareLock);
if (relation == NULL)
{
ereport(ERROR, (errmsg("relation does not exist")));
}
char *tableName = get_rel_name(relationId);
/* it'd probably better to downcase identifiers consistent with SQL case folding */
truncate_identifier(columnName, strlen(columnName), true);
/* lookup column definition */
HeapTuple columnTuple = SearchSysCacheAttName(relationId, columnName);
if (!HeapTupleIsValid(columnTuple))
{
ereport(ERROR, (errcode(ERRCODE_UNDEFINED_COLUMN),
errmsg("column \"%s\" of relation \"%s\" does not exist",
columnName, tableName)));
}
Form_pg_attribute columnForm = (Form_pg_attribute) GETSTRUCT(columnTuple);
/* check if the column may be referenced in the distribution key */
if (columnForm->attnum <= 0)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot reference system column \"%s\" in relation \"%s\"",
columnName, tableName)));
}
ReleaseSysCache(columnTuple);
relation_close(relation, AccessShareLock);
}
/*
* ColumnTypeIdForRelationColumnName returns type id for the given relation's column name.
*/
Oid
ColumnTypeIdForRelationColumnName(Oid relationId, char *columnName)
{
Assert(columnName != NULL);
AttrNumber attNum = get_attnum(relationId, columnName);
if (attNum == InvalidAttrNumber)
{
ereport(ERROR, (errmsg("invalid attr %s", columnName)));
}
Relation relation = relation_open(relationId, AccessShareLock);
Oid typeId = attnumTypeId(relation, attNum);
relation_close(relation, AccessShareLock);
return typeId;
}
/* /*
* ColumnToColumnName returns the human-readable name of a column given a * ColumnToColumnName returns the human-readable name of a column given a
* relation identifier and the column's internal (Var) representation. * relation identifier and the column's internal (Var) representation.

View File

@ -0,0 +1,139 @@
/*-------------------------------------------------------------------------
*
* distribution_column_map.c
* Implementation of a relation OID to distribution column map.
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "common/hashfn.h"
#include "distributed/distribution_column.h"
#include "distributed/listutils.h"
#include "distributed/multi_join_order.h"
#include "distributed/multi_partitioning_utils.h"
#include "distributed/utils/distribution_column_map.h"
#include "nodes/primnodes.h"
/*
* RelationIdDistributionColumnMapEntry is used to map relation IDs to
* distribution column Vars.
*/
typedef struct RelationIdDistributionColumnMapEntry
{
/* OID of the relation */
Oid relationId;
/* a Var describing the distribution column */
Var *distributionColumn;
} RelationIdDistributionColumnMapEntry;
/*
* CreateDistributionColumnMap creates an empty (OID -> distribution column Var) map.
*/
DistributionColumnMap *
CreateDistributionColumnMap(void)
{
HASHCTL info = { 0 };
info.keysize = sizeof(Oid);
info.entrysize = sizeof(RelationIdDistributionColumnMapEntry);
info.hash = oid_hash;
info.hcxt = CurrentMemoryContext;
uint32 hashFlags = (HASH_ELEM | HASH_FUNCTION | HASH_CONTEXT);
HTAB *distributionColumnMap = hash_create("Distribution Column Map", 32,
&info, hashFlags);
return distributionColumnMap;
}
/*
* AddDistributionColumnForRelation adds the given OID and its distribution column
* to the hash, as well as any child partitions.
*/
void
AddDistributionColumnForRelation(DistributionColumnMap *distributionColumnMap,
Oid relationId,
char *distributionColumnName)
{
bool entryFound = false;
RelationIdDistributionColumnMapEntry *entry =
hash_search(distributionColumnMap, &relationId, HASH_ENTER, &entryFound);
Assert(!entryFound);
entry->distributionColumn =
BuildDistributionKeyFromColumnName(relationId, distributionColumnName, NoLock);
if (PartitionedTable(relationId))
{
/*
* Recursively add partitions as well.
*/
List *partitionList = PartitionList(relationId);
Oid partitionRelationId = InvalidOid;
foreach_oid(partitionRelationId, partitionList)
{
AddDistributionColumnForRelation(distributionColumnMap, partitionRelationId,
distributionColumnName);
}
}
}
/*
* GetDistributionColumnFromMap returns the distribution column for a given
* relation ID from the distribution column map.
*/
Var *
GetDistributionColumnFromMap(DistributionColumnMap *distributionColumnMap,
Oid relationId)
{
bool entryFound = false;
RelationIdDistributionColumnMapEntry *entry =
hash_search(distributionColumnMap, &relationId, HASH_FIND, &entryFound);
if (entryFound)
{
return entry->distributionColumn;
}
else
{
return NULL;
}
}
/*
* GetDistributionColumnWithOverrides returns the distribution column for a given
* relation from the distribution column overrides map, or the metadata if no
* override is specified.
*/
Var *
GetDistributionColumnWithOverrides(Oid relationId,
DistributionColumnMap *distributionColumnOverrides)
{
Var *distributionColumn = NULL;
if (distributionColumnOverrides != NULL)
{
distributionColumn = GetDistributionColumnFromMap(distributionColumnOverrides,
relationId);
if (distributionColumn != NULL)
{
return distributionColumn;
}
}
/* no override defined, use distribution column from metadata */
return DistPartitionKey(relationId);
}

View File

@ -996,11 +996,19 @@ IsParentTable(Oid relationId)
systable_endscan(scan); systable_endscan(scan);
table_close(pgInherits, AccessShareLock); table_close(pgInherits, AccessShareLock);
if (tableInherited && PartitionedTable(relationId)) Relation relation = try_relation_open(relationId, AccessShareLock);
if (relation == NULL)
{
ereport(ERROR, (errmsg("relation with OID %u does not exist", relationId)));
}
if (tableInherited && PartitionedTableNoLock(relationId))
{ {
tableInherited = false; tableInherited = false;
} }
relation_close(relation, AccessShareLock);
return tableInherited; return tableInherited;
} }
@ -1291,3 +1299,29 @@ PartitionBound(Oid partitionId)
return partitionBoundString; return partitionBoundString;
} }
/*
* ListShardsUnderParentRelation returns a list of ShardInterval for every
* shard under a given relation, meaning it includes the shards of child
* tables in a partitioning hierarchy.
*/
List *
ListShardsUnderParentRelation(Oid relationId)
{
List *shardList = LoadShardIntervalList(relationId);
if (PartitionedTable(relationId))
{
List *partitionList = PartitionList(relationId);
Oid partitionRelationId = InvalidOid;
foreach_oid(partitionRelationId, partitionList)
{
List *childShardList = ListShardsUnderParentRelation(partitionRelationId);
shardList = list_concat(shardList, childShardList);
}
}
return shardList;
}

View File

@ -36,6 +36,7 @@ extern void InsertColocationGroupLocally(uint32 colocationId, int shardCount,
Oid distributionColumnType, Oid distributionColumnType,
Oid distributionColumnCollation); Oid distributionColumnCollation);
extern bool IsColocateWithNone(char *colocateWithTableName); extern bool IsColocateWithNone(char *colocateWithTableName);
extern bool IsColocateWithDefault(char *colocateWithTableName);
extern uint32 GetNextColocationId(void); extern uint32 GetNextColocationId(void);
extern void ErrorIfShardPlacementsNotColocated(Oid leftRelationId, Oid rightRelationId); extern void ErrorIfShardPlacementsNotColocated(Oid leftRelationId, Oid rightRelationId);
extern void CheckReplicationModel(Oid sourceRelationId, Oid targetRelationId); extern void CheckReplicationModel(Oid sourceRelationId, Oid targetRelationId);
@ -48,5 +49,15 @@ extern void UpdateRelationColocationGroup(Oid distributedRelationId, uint32 colo
extern void DeleteColocationGroupIfNoTablesBelong(uint32 colocationId); extern void DeleteColocationGroupIfNoTablesBelong(uint32 colocationId);
extern List * ColocationGroupTableList(uint32 colocationId, uint32 count); extern List * ColocationGroupTableList(uint32 colocationId, uint32 count);
extern void DeleteColocationGroupLocally(uint32 colocationId); extern void DeleteColocationGroupLocally(uint32 colocationId);
extern uint32 FindColocateWithColocationId(Oid relationId, char replicationModel,
Oid distributionColumnType,
Oid distributionColumnCollation,
int shardCount, bool shardCountIsStrict,
char *colocateWithTableName);
extern void EnsureTableCanBeColocatedWith(Oid relationId, char replicationModel,
Oid distributionColumnType,
Oid sourceRelationId);
extern void AcquireColocationDefaultLock(void);
extern void ReleaseColocationDefaultLock(void);
#endif /* COLOCATION_UTILS_H_ */ #endif /* COLOCATION_UTILS_H_ */

View File

@ -258,6 +258,8 @@ extern void ErrorIfUnsupportedForeignConstraintExists(Relation relation,
char distributionMethod, char distributionMethod,
Var *distributionColumn, Var *distributionColumn,
uint32 colocationId); uint32 colocationId);
extern void EnsureNoFKeyFromTableType(Oid relationId, int tableTypeFlag);
extern void EnsureNoFKeyToTableType(Oid relationId, int tableTypeFlag);
extern void ErrorOutForFKeyBetweenPostgresAndCitusLocalTable(Oid localTableId); extern void ErrorOutForFKeyBetweenPostgresAndCitusLocalTable(Oid localTableId);
extern bool ColumnReferencedByAnyForeignKey(char *columnName, Oid relationId); extern bool ColumnReferencedByAnyForeignKey(char *columnName, Oid relationId);
extern bool ColumnAppearsInForeignKey(char *columnName, Oid relationId); extern bool ColumnAppearsInForeignKey(char *columnName, Oid relationId);

View File

@ -23,5 +23,7 @@ extern Var * BuildDistributionKeyFromColumnName(Oid relationId,
char *columnName, char *columnName,
LOCKMODE lockMode); LOCKMODE lockMode);
extern char * ColumnToColumnName(Oid relationId, Node *columnNode); extern char * ColumnToColumnName(Oid relationId, Node *columnNode);
extern Oid ColumnTypeIdForRelationColumnName(Oid relationId, char *columnName);
extern void EnsureValidDistributionColumn(Oid relationId, char *columnName);
#endif /* DISTRIBUTION_COLUMN_H */ #endif /* DISTRIBUTION_COLUMN_H */

View File

@ -143,6 +143,7 @@ extern List * AllCitusTableIds(void);
extern bool IsCitusTableType(Oid relationId, CitusTableType tableType); extern bool IsCitusTableType(Oid relationId, CitusTableType tableType);
extern bool IsCitusTableTypeCacheEntry(CitusTableCacheEntry *tableEtnry, extern bool IsCitusTableTypeCacheEntry(CitusTableCacheEntry *tableEtnry,
CitusTableType tableType); CitusTableType tableType);
extern char * GetTableTypeName(Oid tableId);
extern void SetCreateCitusTransactionLevel(int val); extern void SetCreateCitusTransactionLevel(int val);
extern int GetCitusCreationLevel(void); extern int GetCitusCreationLevel(void);

View File

@ -69,6 +69,7 @@ extern char * MarkObjectsDistributedCreateCommand(List *addresses,
extern char * DistributionCreateCommand(CitusTableCacheEntry *cacheEntry); extern char * DistributionCreateCommand(CitusTableCacheEntry *cacheEntry);
extern char * DistributionDeleteCommand(const char *schemaName, extern char * DistributionDeleteCommand(const char *schemaName,
const char *tableName); const char *tableName);
extern char * DistributionDeleteMetadataCommand(Oid relationId);
extern char * TableOwnerResetCommand(Oid distributedRelationId); extern char * TableOwnerResetCommand(Oid distributedRelationId);
extern char * NodeListInsertCommand(List *workerNodeList); extern char * NodeListInsertCommand(List *workerNodeList);
extern List * ShardListInsertCommand(List *shardIntervalList); extern List * ShardListInsertCommand(List *shardIntervalList);

View File

@ -248,6 +248,10 @@ extern void InsertIntoPgDistPartition(Oid relationId, char distributionMethod,
Var *distributionColumn, uint32 colocationId, Var *distributionColumn, uint32 colocationId,
char replicationModel, bool autoConverted); char replicationModel, bool autoConverted);
extern void UpdatePgDistPartitionAutoConverted(Oid citusTableId, bool autoConverted); extern void UpdatePgDistPartitionAutoConverted(Oid citusTableId, bool autoConverted);
extern void UpdateDistributionColumnGlobally(Oid relationId, char distributionMethod,
Var *distributionColumn, int colocationId);
extern void UpdateDistributionColumn(Oid relationId, char distributionMethod,
Var *distributionColumn, int colocationId);
extern void DeletePartitionRow(Oid distributedRelationId); extern void DeletePartitionRow(Oid distributedRelationId);
extern void DeleteShardRow(uint64 shardId); extern void DeleteShardRow(uint64 shardId);
extern void UpdateShardPlacementState(uint64 placementId, char shardState); extern void UpdateShardPlacementState(uint64 placementId, char shardState);

View File

@ -30,5 +30,6 @@ extern char * GeneratePartitioningInformation(Oid tableId);
extern void FixPartitionConstraintsOnWorkers(Oid relationId); extern void FixPartitionConstraintsOnWorkers(Oid relationId);
extern void FixLocalPartitionConstraints(Oid relationId, int64 shardId); extern void FixLocalPartitionConstraints(Oid relationId, int64 shardId);
extern void FixPartitionShardIndexNames(Oid relationId, Oid parentIndexOid); extern void FixPartitionShardIndexNames(Oid relationId, Oid parentIndexOid);
extern List * ListShardsUnderParentRelation(Oid relationId);
#endif /* MULTI_PARTITIONING_UTILS_H_ */ #endif /* MULTI_PARTITIONING_UTILS_H_ */

View File

@ -17,3 +17,4 @@ extern void ErrorIfMoveUnsupportedTableType(Oid relationId);
extern void CopyShardsToNode(WorkerNode *sourceNode, WorkerNode *targetNode, extern void CopyShardsToNode(WorkerNode *sourceNode, WorkerNode *targetNode,
List *shardIntervalList, char *snapshotName); List *shardIntervalList, char *snapshotName);
extern void VerifyTablesHaveReplicaIdentity(List *colocatedTableList); extern void VerifyTablesHaveReplicaIdentity(List *colocatedTableList);
extern bool RelationCanPublishAllModifications(Oid relationId);

View File

@ -48,7 +48,10 @@ typedef enum AdvisoryLocktagClass
/* CitusOperations has constants for citus operations */ /* CitusOperations has constants for citus operations */
typedef enum CitusOperations typedef enum CitusOperations
{ {
CITUS_TRANSACTION_RECOVERY = 0 CITUS_TRANSACTION_RECOVERY = 0,
CITUS_NONBLOCKING_SPLIT = 1,
CITUS_CREATE_DISTRIBUTED_TABLE_CONCURRENTLY = 2,
CITUS_CREATE_COLOCATION_DEFAULT = 3
} CitusOperations; } CitusOperations;
/* reuse advisory lock, but with different, unused field 4 (4)*/ /* reuse advisory lock, but with different, unused field 4 (4)*/
@ -177,6 +180,8 @@ extern void SerializeNonCommutativeWrites(List *shardIntervalList, LOCKMODE lock
extern void LockRelationShardResources(List *relationShardList, LOCKMODE lockMode); extern void LockRelationShardResources(List *relationShardList, LOCKMODE lockMode);
extern List * GetSortedReferenceShardIntervals(List *relationList); extern List * GetSortedReferenceShardIntervals(List *relationList);
void AcquireCreateDistributedTableConcurrentlyLock(Oid relationId);
/* Lock parent table's colocated shard resource */ /* Lock parent table's colocated shard resource */
extern void LockParentShardResourceIfPartition(List *shardIntervalList, extern void LockParentShardResourceIfPartition(List *shardIntervalList,
LOCKMODE lockMode); LOCKMODE lockMode);

View File

@ -12,6 +12,8 @@
#ifndef SHARDSPLIT_H_ #ifndef SHARDSPLIT_H_
#define SHARDSPLIT_H_ #define SHARDSPLIT_H_
#include "distributed/utils/distribution_column_map.h"
/* Split Modes supported by Shard Split API */ /* Split Modes supported by Shard Split API */
typedef enum SplitMode typedef enum SplitMode
{ {
@ -28,10 +30,10 @@ typedef enum SplitMode
typedef enum SplitOperation typedef enum SplitOperation
{ {
SHARD_SPLIT_API = 0, SHARD_SPLIT_API = 0,
ISOLATE_TENANT_TO_NEW_SHARD ISOLATE_TENANT_TO_NEW_SHARD,
CREATE_DISTRIBUTED_TABLE
} SplitOperation; } SplitOperation;
/* /*
* SplitShard API to split a given shard (or shard group) using split mode and * SplitShard API to split a given shard (or shard group) using split mode and
* specified split points to a set of destination nodes. * specified split points to a set of destination nodes.
@ -40,10 +42,15 @@ extern void SplitShard(SplitMode splitMode,
SplitOperation splitOperation, SplitOperation splitOperation,
uint64 shardIdToSplit, uint64 shardIdToSplit,
List *shardSplitPointsList, List *shardSplitPointsList,
List *nodeIdsForPlacementList); List *nodeIdsForPlacementList,
DistributionColumnMap *distributionColumnOverrides,
List *colocatedShardIntervalList,
uint32 targetColocationId);
extern void DropShardList(List *shardIntervalList); extern void DropShardList(List *shardIntervalList);
extern SplitMode LookupSplitMode(Oid shardTransferModeOid); extern SplitMode LookupSplitMode(Oid shardTransferModeOid);
extern void ErrorIfMultipleNonblockingMoveSplitInTheSameTransaction(void);
#endif /* SHARDSPLIT_H_ */ #endif /* SHARDSPLIT_H_ */

View File

@ -47,4 +47,5 @@ extern void DropShardSplitPublications(MultiConnection *sourceConnection,
extern void DropShardSplitSubsriptions(List *shardSplitSubscribersMetadataList); extern void DropShardSplitSubsriptions(List *shardSplitSubscribersMetadataList);
extern void DropShardSplitReplicationSlots(MultiConnection *sourceConnection, extern void DropShardSplitReplicationSlots(MultiConnection *sourceConnection,
List *replicationSlotInfoList); List *replicationSlotInfoList);
#endif /* SHARDSPLIT_LOGICAL_REPLICATION_H */ #endif /* SHARDSPLIT_LOGICAL_REPLICATION_H */

View File

@ -0,0 +1,32 @@
/*-------------------------------------------------------------------------
*
* distribution_column_map.h
* Declarations for a relation OID to distribution column hash.
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#ifndef DISTRIBUTION_COLUMN_HASH_H
#define DISTRIBUTION_COLUMN_HASH_H
#include "postgres.h"
#include "nodes/primnodes.h"
#include "utils/hsearch.h"
typedef HTAB DistributionColumnMap;
extern DistributionColumnMap * CreateDistributionColumnMap(void);
extern void AddDistributionColumnForRelation(DistributionColumnMap *distributionColumns,
Oid relationId,
char *distributionColumnName);
extern Var * GetDistributionColumnFromMap(DistributionColumnMap *distributionColumnMap,
Oid relationId);
extern Var * GetDistributionColumnWithOverrides(Oid relationId,
DistributionColumnMap *overrides);
#endif /* DISTRIBUTION_COLUMN_HASH_H */

View File

@ -90,6 +90,7 @@ extern WorkerNode * FindWorkerNodeAnyCluster(const char *nodeName, int32 nodePor
extern WorkerNode * FindNodeWithNodeId(int nodeId, bool missingOk); extern WorkerNode * FindNodeWithNodeId(int nodeId, bool missingOk);
extern List * ReadDistNode(bool includeNodesFromOtherClusters); extern List * ReadDistNode(bool includeNodesFromOtherClusters);
extern void EnsureCoordinator(void); extern void EnsureCoordinator(void);
extern void EnsureCoordinatorIsInMetadata(void);
extern void InsertCoordinatorIfClusterEmpty(void); extern void InsertCoordinatorIfClusterEmpty(void);
extern uint32 GroupForNode(char *nodeName, int32 nodePort); extern uint32 GroupForNode(char *nodeName, int32 nodePort);
extern WorkerNode * PrimaryNodeForGroup(int32 groupId, bool *groupContainsNodes); extern WorkerNode * PrimaryNodeForGroup(int32 groupId, bool *groupContainsNodes);

View File

@ -0,0 +1,288 @@
create schema create_distributed_table_concurrently;
set search_path to create_distributed_table_concurrently;
set citus.shard_replication_factor to 1;
-- make sure we have the coordinator in the metadata
SELECT 1 FROM citus_set_coordinator_host('localhost', :master_port);
?column?
---------------------------------------------------------------------
1
(1 row)
create table ref (id int primary key);
select create_reference_table('ref');
create_reference_table
---------------------------------------------------------------------
(1 row)
insert into ref select s from generate_series(0,9) s;
create table test (key text, id int references ref (id) on delete cascade, t timestamptz default now()) partition by range (t);
create table test_1 partition of test for values from ('2022-01-01') to ('2022-12-31');
create table test_2 partition of test for values from ('2023-01-01') to ('2023-12-31');
insert into test (key,id,t) select s,s%10, '2022-01-01'::date + interval '1 year' * (s%2) from generate_series(1,100) s;
create table nocolo (x int, y int);
-- test error conditions
select create_distributed_table_concurrently('test','key', 'append');
ERROR: only hash-distributed tables can be distributed without blocking writes
select create_distributed_table_concurrently('test','key', 'range');
ERROR: only hash-distributed tables can be distributed without blocking writes
select create_distributed_table_concurrently('test','noexists', 'hash');
ERROR: column "noexists" of relation "test" does not exist
select create_distributed_table_concurrently(0,'key');
ERROR: relation with OID XXXX does not exist
select create_distributed_table_concurrently('ref','id');
ERROR: table "ref" is already distributed
set citus.shard_replication_factor to 2;
select create_distributed_table_concurrently('test','key', 'hash');
ERROR: cannot distribute a table concurrently when citus.shard_replication_factor > 1
set citus.shard_replication_factor to 1;
begin;
select create_distributed_table_concurrently('test','key');
ERROR: create_distributed_table_concurrently cannot run inside a transaction block
rollback;
select create_distributed_table_concurrently('test','key'), create_distributed_table_concurrently('test','key');
NOTICE: relation test does not have a REPLICA IDENTITY or PRIMARY KEY
DETAIL: UPDATE and DELETE commands on the relation will error out during create_distributed_table_concurrently unless there is a REPLICA IDENTITY or PRIMARY KEY. INSERT commands will still work.
ERROR: multiple shard movements/splits via logical replication in the same transaction is currently not supported
select create_distributed_table_concurrently('nocolo','x');
NOTICE: relation nocolo does not have a REPLICA IDENTITY or PRIMARY KEY
DETAIL: UPDATE and DELETE commands on the relation will error out during create_distributed_table_concurrently unless there is a REPLICA IDENTITY or PRIMARY KEY. INSERT commands will still work.
create_distributed_table_concurrently
---------------------------------------------------------------------
(1 row)
select create_distributed_table_concurrently('test','key', colocate_with := 'nocolo');
ERROR: cannot colocate tables nocolo and test
DETAIL: Distribution column types don't match for nocolo and test.
select create_distributed_table_concurrently('test','key', colocate_with := 'noexists');
ERROR: relation "noexists" does not exist
-- use colocate_with "default"
select create_distributed_table_concurrently('test','key', shard_count := 11);
NOTICE: relation test does not have a REPLICA IDENTITY or PRIMARY KEY
DETAIL: UPDATE and DELETE commands on the relation will error out during create_distributed_table_concurrently unless there is a REPLICA IDENTITY or PRIMARY KEY. INSERT commands will still work.
create_distributed_table_concurrently
---------------------------------------------------------------------
(1 row)
select shardcount from pg_dist_partition p join pg_dist_colocation c using (colocationid) where logicalrelid = 'test'::regclass;
shardcount
---------------------------------------------------------------------
11
(1 row)
select count(*) from pg_dist_shard where logicalrelid = 'test'::regclass;
count
---------------------------------------------------------------------
11
(1 row)
-- verify queries still work
select count(*) from test;
count
---------------------------------------------------------------------
100
(1 row)
select key, id from test where key = '1';
key | id
---------------------------------------------------------------------
1 | 1
(1 row)
select count(*) from test_1;
count
---------------------------------------------------------------------
50
(1 row)
-- verify that the foreign key to reference table was created
begin;
delete from ref;
select count(*) from test;
count
---------------------------------------------------------------------
0
(1 row)
rollback;
-- verify that we can undistribute the table
begin;
select undistribute_table('test', cascade_via_foreign_keys := true);
NOTICE: converting the partitions of create_distributed_table_concurrently.test
NOTICE: creating a new table for create_distributed_table_concurrently.test
NOTICE: dropping the old create_distributed_table_concurrently.test
NOTICE: renaming the new table to create_distributed_table_concurrently.test
NOTICE: creating a new table for create_distributed_table_concurrently.ref
NOTICE: moving the data of create_distributed_table_concurrently.ref
NOTICE: dropping the old create_distributed_table_concurrently.ref
NOTICE: renaming the new table to create_distributed_table_concurrently.ref
NOTICE: creating a new table for create_distributed_table_concurrently.test_1
NOTICE: moving the data of create_distributed_table_concurrently.test_1
NOTICE: dropping the old create_distributed_table_concurrently.test_1
NOTICE: renaming the new table to create_distributed_table_concurrently.test_1
NOTICE: creating a new table for create_distributed_table_concurrently.test_2
NOTICE: moving the data of create_distributed_table_concurrently.test_2
NOTICE: dropping the old create_distributed_table_concurrently.test_2
NOTICE: renaming the new table to create_distributed_table_concurrently.test_2
undistribute_table
---------------------------------------------------------------------
(1 row)
rollback;
-- verify that we can co-locate with create_distributed_table_concurrently
create table test2 (x text primary key, y text);
insert into test2 (x,y) select s,s from generate_series(1,100) s;
select create_distributed_table_concurrently('test2','x', colocate_with := 'test');
create_distributed_table_concurrently
---------------------------------------------------------------------
(1 row)
-- verify co-located joins work
select count(*) from test join test2 on (key = x);
count
---------------------------------------------------------------------
100
(1 row)
select id, y from test join test2 on (key = x) where key = '1';
id | y
---------------------------------------------------------------------
1 | 1
(1 row)
-- verify co-locaed foreign keys work
alter table test add constraint fk foreign key (key) references test2 (x);
-------foreign key tests among different table types--------
-- verify we do not allow foreign keys from reference table to distributed table concurrently
create table ref_table1(id int);
create table dist_table1(id int primary key);
select create_reference_table('ref_table1');
create_reference_table
---------------------------------------------------------------------
(1 row)
alter table ref_table1 add constraint fkey foreign key (id) references dist_table1(id);
select create_distributed_table_concurrently('dist_table1', 'id');
ERROR: relation dist_table1 is referenced by a foreign key from ref_table1
DETAIL: foreign keys from a reference table to a distributed table are not supported.
-- verify we do not allow foreign keys from citus local table to distributed table concurrently
create table citus_local_table1(id int);
select citus_add_local_table_to_metadata('citus_local_table1');
citus_add_local_table_to_metadata
---------------------------------------------------------------------
(1 row)
create table dist_table2(id int primary key);
alter table citus_local_table1 add constraint fkey foreign key (id) references dist_table2(id);
select create_distributed_table_concurrently('dist_table2', 'id');
ERROR: relation dist_table2 is referenced by a foreign key from citus_local_table1
DETAIL: foreign keys from a citus local table to a distributed table are not supported.
-- verify we do not allow foreign keys from regular table to distributed table concurrently
create table local_table1(id int);
create table dist_table3(id int primary key);
alter table local_table1 add constraint fkey foreign key (id) references dist_table3(id);
select create_distributed_table_concurrently('dist_table3', 'id');
ERROR: relation dist_table3 is referenced by a foreign key from local_table1
DETAIL: foreign keys from a regular table to a distributed table are not supported.
-- verify we allow foreign keys from distributed table to reference table concurrently
create table ref_table2(id int primary key);
select create_reference_table('ref_table2');
create_reference_table
---------------------------------------------------------------------
(1 row)
create table dist_table4(id int references ref_table2(id));
select create_distributed_table_concurrently('dist_table4', 'id');
NOTICE: relation dist_table4 does not have a REPLICA IDENTITY or PRIMARY KEY
DETAIL: UPDATE and DELETE commands on the relation will error out during create_distributed_table_concurrently unless there is a REPLICA IDENTITY or PRIMARY KEY. INSERT commands will still work.
create_distributed_table_concurrently
---------------------------------------------------------------------
(1 row)
insert into ref_table2 select s from generate_series(1,100) s;
insert into dist_table4 select s from generate_series(1,100) s;
select count(*) as total from dist_table4;
total
---------------------------------------------------------------------
100
(1 row)
-- verify we do not allow foreign keys from distributed table to citus local table concurrently
create table citus_local_table2(id int primary key);
select citus_add_local_table_to_metadata('citus_local_table2');
citus_add_local_table_to_metadata
---------------------------------------------------------------------
(1 row)
create table dist_table5(id int references citus_local_table2(id));
select create_distributed_table_concurrently('dist_table5', 'id');
NOTICE: relation dist_table5 does not have a REPLICA IDENTITY or PRIMARY KEY
DETAIL: UPDATE and DELETE commands on the relation will error out during create_distributed_table_concurrently unless there is a REPLICA IDENTITY or PRIMARY KEY. INSERT commands will still work.
ERROR: cannot create foreign key constraint since relations are not colocated or not referencing a reference table
DETAIL: A distributed table can only have foreign keys if it is referencing another colocated hash distributed table or a reference table
-- verify we do not allow foreign keys from distributed table to regular table concurrently
create table local_table2(id int primary key);
create table dist_table6(id int references local_table2(id));
select create_distributed_table_concurrently('dist_table6', 'id');
ERROR: relation local_table2 is referenced by a foreign key from dist_table6
DETAIL: foreign keys from a distributed table to a regular table are not supported.
-------foreign key tests among different table types--------
-- columnar tests --
-- create table with partitions
create table test_columnar (id int) partition by range (id);
create table test_columnar_1 partition of test_columnar for values from (1) to (51);
create table test_columnar_2 partition of test_columnar for values from (51) to (101) using columnar;
-- load some data
insert into test_columnar (id) select s from generate_series(1,100) s;
-- distribute table
select create_distributed_table_concurrently('test_columnar','id');
NOTICE: relation test_columnar does not have a REPLICA IDENTITY or PRIMARY KEY
DETAIL: UPDATE and DELETE commands on the relation will error out during create_distributed_table_concurrently unless there is a REPLICA IDENTITY or PRIMARY KEY. INSERT commands will still work.
create_distributed_table_concurrently
---------------------------------------------------------------------
(1 row)
-- verify queries still work
select count(*) from test_columnar;
count
---------------------------------------------------------------------
100
(1 row)
select id from test_columnar where id = 1;
id
---------------------------------------------------------------------
1
(1 row)
select id from test_columnar where id = 51;
id
---------------------------------------------------------------------
51
(1 row)
select count(*) from test_columnar_1;
count
---------------------------------------------------------------------
50
(1 row)
select count(*) from test_columnar_2;
count
---------------------------------------------------------------------
50
(1 row)
-- columnar tests --
set client_min_messages to warning;
drop schema create_distributed_table_concurrently cascade;

View File

@ -0,0 +1,203 @@
--
-- failure_create_distributed_table_concurrently adds failure tests for creating distributed table concurrently without data.
--
-- due to different libpq versions
-- some warning messages differ
-- between local and CI
SET client_min_messages TO ERROR;
-- setup db
CREATE SCHEMA IF NOT EXISTS create_dist_tbl_con;
SET SEARCH_PATH = create_dist_tbl_con;
SET citus.shard_count TO 2;
SET citus.shard_replication_factor TO 1;
SET citus.max_adaptive_executor_pool_size TO 1;
SELECT pg_backend_pid() as pid \gset
-- make sure coordinator is in the metadata
SELECT citus_set_coordinator_host('localhost', 57636);
citus_set_coordinator_host
---------------------------------------------------------------------
(1 row)
-- create table that will be distributed concurrently
CREATE TABLE table_1 (id int PRIMARY KEY);
-- START OF TESTS
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
---------------------------------------------------------------------
(1 row)
-- failure on shard table creation
SELECT citus.mitmproxy('conn.onQuery(query="CREATE TABLE create_dist_tbl_con.table_1").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table_concurrently('table_1', 'id');
ERROR: connection not open
CONTEXT: while executing command on localhost:xxxxx
-- cancellation on shard table creation
SELECT citus.mitmproxy('conn.onQuery(query="CREATE TABLE create_dist_tbl_con.table_1").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table_concurrently('table_1', 'id');
ERROR: canceling statement due to user request
-- failure on table constraints on replica identity creation
SELECT citus.mitmproxy('conn.onQuery(query="ALTER TABLE create_dist_tbl_con.table_1 ADD CONSTRAINT").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table_concurrently('table_1', 'id');
ERROR: connection not open
CONTEXT: while executing command on localhost:xxxxx
-- cancellation on table constraints on replica identity creation
SELECT citus.mitmproxy('conn.onQuery(query="ALTER TABLE create_dist_tbl_con.table_1 ADD CONSTRAINT").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table_concurrently('table_1', 'id');
ERROR: canceling statement due to user request
-- failure on subscription creation
SELECT citus.mitmproxy('conn.onQuery(query="CREATE SUBSCRIPTION").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table_concurrently('table_1', 'id');
ERROR: connection not open
CONTEXT: while executing command on localhost:xxxxx
-- cancellation on subscription creation
SELECT citus.mitmproxy('conn.onQuery(query="CREATE SUBSCRIPTION").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table_concurrently('table_1', 'id');
ERROR: canceling statement due to user request
-- failure on catching up LSN
SELECT citus.mitmproxy('conn.onQuery(query="SELECT min\(latest_end_lsn\) FROM pg_stat_subscription").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table_concurrently('table_1', 'id');
ERROR: connection not open
CONTEXT: while executing command on localhost:xxxxx
-- cancellation on catching up LSN
SELECT citus.mitmproxy('conn.onQuery(query="SELECT min\(latest_end_lsn\) FROM pg_stat_subscription").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table_concurrently('table_1', 'id');
ERROR: canceling statement due to user request
-- Comment out below flaky tests. It is caused by shard split cleanup which does not work properly yet.
-- -- failure on dropping subscription
-- SELECT citus.mitmproxy('conn.onQuery(query="DROP SUBSCRIPTION").kill()');
-- SELECT create_distributed_table_concurrently('table_1', 'id');
-- -- cancellation on dropping subscription
-- SELECT citus.mitmproxy('conn.onQuery(query="DROP SUBSCRIPTION").cancel(' || :pid || ')');
-- SELECT create_distributed_table_concurrently('table_1', 'id');
-- -- failure on dropping old shard
-- SELECT citus.mitmproxy('conn.onQuery(query="DROP TABLE IF EXISTS create_dist_tbl_con.table_1").kill()');
-- SELECT create_distributed_table_concurrently('table_1', 'id');
-- -- cancellation on dropping old shard
-- SELECT citus.mitmproxy('conn.onQuery(query="DROP TABLE IF EXISTS create_dist_tbl_con.table_1").cancel(' || :pid || ')');
-- SELECT create_distributed_table_concurrently('table_1', 'id');
-- failure on transaction begin
SELECT citus.mitmproxy('conn.onQuery(query="BEGIN").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table_concurrently('table_1', 'id');
ERROR: failure on connection marked as essential: localhost:xxxxx
-- failure on transaction begin
SELECT citus.mitmproxy('conn.onQuery(query="BEGIN").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table_concurrently('table_1', 'id');
ERROR: canceling statement due to user request
-- failure on transaction commit
SELECT citus.mitmproxy('conn.onQuery(query="COMMIT").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table_concurrently('table_1', 'id');
ERROR: failure on connection marked as essential: localhost:xxxxx
-- failure on transaction commit
SELECT citus.mitmproxy('conn.onQuery(query="COMMIT").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table_concurrently('table_1', 'id');
ERROR: canceling statement due to user request
-- failure on prepare transaction
SELECT citus.mitmproxy('conn.onQuery(query="PREPARE TRANSACTION").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table_concurrently('table_1', 'id');
ERROR: connection not open
CONTEXT: while executing command on localhost:xxxxx
-- failure on prepare transaction
SELECT citus.mitmproxy('conn.onQuery(query="PREPARE TRANSACTION").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table_concurrently('table_1', 'id');
ERROR: canceling statement due to user request
-- END OF TESTS
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
---------------------------------------------------------------------
(1 row)
-- Verify that the table can be distributed concurrently after unsuccessful attempts
SELECT create_distributed_table_concurrently('table_1', 'id');
create_distributed_table_concurrently
---------------------------------------------------------------------
(1 row)
SELECT * FROM pg_dist_shard WHERE logicalrelid = 'table_1'::regclass;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue
---------------------------------------------------------------------
table_1 | 1880096 | t | -2147483648 | -1
table_1 | 1880097 | t | 0 | 2147483647
(2 rows)
DROP SCHEMA create_dist_tbl_con CASCADE;
SET search_path TO default;
SELECT citus_remove_node('localhost', 57636);
citus_remove_node
---------------------------------------------------------------------
(1 row)

View File

@ -0,0 +1,763 @@
unused step name: s1-create-concurrently-table_2
Parsed test spec with 4 sessions
starting permutation: s1-truncate s3-acquire-split-advisory-lock s1-settings s2-settings s1-create-concurrently-table_1 s2-begin s2-insert s2-commit s3-release-split-advisory-lock s2-print-status
step s1-truncate:
TRUNCATE table_1;
step s3-acquire-split-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-settings:
-- session needs to have replication factor set to 1, can't do in setup
SET citus.shard_count TO 4;
SET citus.shard_replication_factor TO 1;
step s2-settings:
-- session needs to have replication factor set to 1, can't do in setup
SET citus.shard_count TO 4;
SET citus.shard_replication_factor TO 1;
step s1-create-concurrently-table_1:
SELECT create_distributed_table_concurrently('table_1', 'id');
<waiting ...>
step s2-begin:
BEGIN;
step s2-insert:
INSERT INTO table_1 SELECT s FROM generate_series(1,20) s;
step s2-commit:
COMMIT;
step s3-release-split-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s1-create-concurrently-table_1: <... completed>
create_distributed_table_concurrently
---------------------------------------------------------------------
(1 row)
step s2-print-status:
-- sanity check on partitions
SELECT * FROM pg_dist_shard
WHERE logicalrelid = 'table_1'::regclass OR logicalrelid = 'table_2'::regclass
ORDER BY shardminvalue::BIGINT, logicalrelid;
-- sanity check on total elements in the table
SELECT COUNT(*) FROM table_1;
logicalrelid|shardid|shardstorage|shardminvalue|shardmaxvalue
---------------------------------------------------------------------
table_1 |1400294|t | -2147483648| -1073741825
table_1 |1400295|t | -1073741824| -1
table_1 |1400296|t | 0| 1073741823
table_1 |1400297|t | 1073741824| 2147483647
(4 rows)
count
---------------------------------------------------------------------
20
(1 row)
citus_remove_node
---------------------------------------------------------------------
(1 row)
starting permutation: s1-truncate s3-acquire-split-advisory-lock s1-create-concurrently-table_1 s2-begin s2-insert s2-update s2-commit s3-release-split-advisory-lock s2-print-status
step s1-truncate:
TRUNCATE table_1;
step s3-acquire-split-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-create-concurrently-table_1:
SELECT create_distributed_table_concurrently('table_1', 'id');
<waiting ...>
step s2-begin:
BEGIN;
step s2-insert:
INSERT INTO table_1 SELECT s FROM generate_series(1,20) s;
step s2-update:
UPDATE table_1 SET id = 21 WHERE id = 20;
step s2-commit:
COMMIT;
step s3-release-split-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s1-create-concurrently-table_1: <... completed>
create_distributed_table_concurrently
---------------------------------------------------------------------
(1 row)
step s2-print-status:
-- sanity check on partitions
SELECT * FROM pg_dist_shard
WHERE logicalrelid = 'table_1'::regclass OR logicalrelid = 'table_2'::regclass
ORDER BY shardminvalue::BIGINT, logicalrelid;
-- sanity check on total elements in the table
SELECT COUNT(*) FROM table_1;
logicalrelid|shardid|shardstorage|shardminvalue|shardmaxvalue
---------------------------------------------------------------------
table_1 |1400299|t | -2147483648| -1073741825
table_1 |1400300|t | -1073741824| -1
table_1 |1400301|t | 0| 1073741823
table_1 |1400302|t | 1073741824| 2147483647
(4 rows)
count
---------------------------------------------------------------------
20
(1 row)
citus_remove_node
---------------------------------------------------------------------
(1 row)
starting permutation: s1-truncate s3-acquire-split-advisory-lock s1-create-concurrently-table_1 s2-begin s2-insert s2-delete s2-commit s3-release-split-advisory-lock s2-print-status
step s1-truncate:
TRUNCATE table_1;
step s3-acquire-split-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-create-concurrently-table_1:
SELECT create_distributed_table_concurrently('table_1', 'id');
<waiting ...>
step s2-begin:
BEGIN;
step s2-insert:
INSERT INTO table_1 SELECT s FROM generate_series(1,20) s;
step s2-delete:
DELETE FROM table_1 WHERE id = 11;
step s2-commit:
COMMIT;
step s3-release-split-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s1-create-concurrently-table_1: <... completed>
create_distributed_table_concurrently
---------------------------------------------------------------------
(1 row)
step s2-print-status:
-- sanity check on partitions
SELECT * FROM pg_dist_shard
WHERE logicalrelid = 'table_1'::regclass OR logicalrelid = 'table_2'::regclass
ORDER BY shardminvalue::BIGINT, logicalrelid;
-- sanity check on total elements in the table
SELECT COUNT(*) FROM table_1;
logicalrelid|shardid|shardstorage|shardminvalue|shardmaxvalue
---------------------------------------------------------------------
table_1 |1400304|t | -2147483648| -1073741825
table_1 |1400305|t | -1073741824| -1
table_1 |1400306|t | 0| 1073741823
table_1 |1400307|t | 1073741824| 2147483647
(4 rows)
count
---------------------------------------------------------------------
19
(1 row)
citus_remove_node
---------------------------------------------------------------------
(1 row)
starting permutation: s1-truncate s3-acquire-split-advisory-lock s1-create-concurrently-table_1 s2-begin s2-insert s2-copy s2-commit s3-release-split-advisory-lock s2-print-status
step s1-truncate:
TRUNCATE table_1;
step s3-acquire-split-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-create-concurrently-table_1:
SELECT create_distributed_table_concurrently('table_1', 'id');
<waiting ...>
step s2-begin:
BEGIN;
step s2-insert:
INSERT INTO table_1 SELECT s FROM generate_series(1,20) s;
step s2-copy:
COPY table_1 FROM PROGRAM 'echo 30 && echo 31 && echo 32 && echo 33 && echo 34 && echo 35 && echo 36 && echo 37 && echo 38';
step s2-commit:
COMMIT;
step s3-release-split-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s1-create-concurrently-table_1: <... completed>
create_distributed_table_concurrently
---------------------------------------------------------------------
(1 row)
step s2-print-status:
-- sanity check on partitions
SELECT * FROM pg_dist_shard
WHERE logicalrelid = 'table_1'::regclass OR logicalrelid = 'table_2'::regclass
ORDER BY shardminvalue::BIGINT, logicalrelid;
-- sanity check on total elements in the table
SELECT COUNT(*) FROM table_1;
logicalrelid|shardid|shardstorage|shardminvalue|shardmaxvalue
---------------------------------------------------------------------
table_1 |1400309|t | -2147483648| -1073741825
table_1 |1400310|t | -1073741824| -1
table_1 |1400311|t | 0| 1073741823
table_1 |1400312|t | 1073741824| 2147483647
(4 rows)
count
---------------------------------------------------------------------
29
(1 row)
citus_remove_node
---------------------------------------------------------------------
(1 row)
starting permutation: s3-acquire-split-advisory-lock s1-create-concurrently-table_1 s2-insert s2-reindex-concurrently s4-print-waiting-locks s3-release-split-advisory-lock
step s3-acquire-split-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-create-concurrently-table_1:
SELECT create_distributed_table_concurrently('table_1', 'id');
<waiting ...>
step s2-insert:
INSERT INTO table_1 SELECT s FROM generate_series(1,20) s;
step s2-reindex-concurrently:
REINDEX TABLE CONCURRENTLY table_1;
<waiting ...>
step s4-print-waiting-locks:
SELECT mode, relation::regclass, granted FROM pg_locks
WHERE relation = 'table_1'::regclass OR relation = 'table_2'::regclass
ORDER BY mode, relation, granted;
mode |relation|granted
---------------------------------------------------------------------
AccessShareLock |table_1 |t
ShareUpdateExclusiveLock|table_1 |f
ShareUpdateExclusiveLock|table_1 |t
(3 rows)
step s3-release-split-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s1-create-concurrently-table_1: <... completed>
create_distributed_table_concurrently
---------------------------------------------------------------------
(1 row)
step s2-reindex-concurrently: <... completed>
citus_remove_node
---------------------------------------------------------------------
(1 row)
starting permutation: s3-acquire-split-advisory-lock s1-create-concurrently-table_1 s2-insert s2-reindex s4-print-waiting-locks s3-release-split-advisory-lock
step s3-acquire-split-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-create-concurrently-table_1:
SELECT create_distributed_table_concurrently('table_1', 'id');
<waiting ...>
step s2-insert:
INSERT INTO table_1 SELECT s FROM generate_series(1,20) s;
step s2-reindex:
REINDEX TABLE table_1;
<waiting ...>
step s4-print-waiting-locks:
SELECT mode, relation::regclass, granted FROM pg_locks
WHERE relation = 'table_1'::regclass OR relation = 'table_2'::regclass
ORDER BY mode, relation, granted;
mode |relation|granted
---------------------------------------------------------------------
AccessExclusiveLock |table_1 |f
AccessShareLock |table_1 |t
ShareUpdateExclusiveLock|table_1 |t
(3 rows)
step s3-release-split-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s1-create-concurrently-table_1: <... completed>
create_distributed_table_concurrently
---------------------------------------------------------------------
(1 row)
step s2-reindex: <... completed>
citus_remove_node
---------------------------------------------------------------------
(1 row)
starting permutation: s2-begin s2-create-concurrently-table_1 s2-commit
step s2-begin:
BEGIN;
step s2-create-concurrently-table_1:
SELECT create_distributed_table_concurrently('table_1', 'id');
ERROR: create_distributed_table_concurrently cannot run inside a transaction block
step s2-commit:
COMMIT;
citus_remove_node
---------------------------------------------------------------------
(1 row)
starting permutation: s3-acquire-split-advisory-lock s1-create-concurrently-table_1 s2-create-concurrently-table_1 s3-release-split-advisory-lock
step s3-acquire-split-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-create-concurrently-table_1:
SELECT create_distributed_table_concurrently('table_1', 'id');
<waiting ...>
step s2-create-concurrently-table_1:
SELECT create_distributed_table_concurrently('table_1', 'id');
ERROR: another create_distributed_table_concurrently operation is in progress
step s3-release-split-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s1-create-concurrently-table_1: <... completed>
create_distributed_table_concurrently
---------------------------------------------------------------------
(1 row)
citus_remove_node
---------------------------------------------------------------------
(1 row)
starting permutation: s3-acquire-split-advisory-lock s1-create-concurrently-table_1 s2-create-concurrently-table_2 s3-release-split-advisory-lock
step s3-acquire-split-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-create-concurrently-table_1:
SELECT create_distributed_table_concurrently('table_1', 'id');
<waiting ...>
step s2-create-concurrently-table_2:
SELECT create_distributed_table_concurrently('table_2', 'id');
ERROR: another create_distributed_table_concurrently operation is in progress
step s3-release-split-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s1-create-concurrently-table_1: <... completed>
create_distributed_table_concurrently
---------------------------------------------------------------------
(1 row)
citus_remove_node
---------------------------------------------------------------------
(1 row)
starting permutation: s3-acquire-split-advisory-lock s1-create-concurrently-table_1 s2-create-table_1 s3-release-split-advisory-lock
step s3-acquire-split-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-create-concurrently-table_1:
SELECT create_distributed_table_concurrently('table_1', 'id');
<waiting ...>
step s2-create-table_1:
SELECT create_distributed_table('table_1', 'id');
<waiting ...>
step s3-release-split-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s1-create-concurrently-table_1: <... completed>
create_distributed_table_concurrently
---------------------------------------------------------------------
(1 row)
step s2-create-table_1: <... completed>
ERROR: table "table_1" is already distributed
citus_remove_node
---------------------------------------------------------------------
(1 row)
starting permutation: s3-acquire-split-advisory-lock s1-create-concurrently-table_1 s2-create-table_2 s3-release-split-advisory-lock
step s3-acquire-split-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-create-concurrently-table_1:
SELECT create_distributed_table_concurrently('table_1', 'id');
<waiting ...>
step s2-create-table_2:
SELECT create_distributed_table('table_2', 'id');
<waiting ...>
step s3-release-split-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s1-create-concurrently-table_1: <... completed>
create_distributed_table_concurrently
---------------------------------------------------------------------
(1 row)
step s2-create-table_2: <... completed>
create_distributed_table
---------------------------------------------------------------------
(1 row)
citus_remove_node
---------------------------------------------------------------------
(1 row)
starting permutation: s2-begin s2-create-table_2 s1-create-concurrently-table_default_colocated s4-print-waiting-advisory-locks s2-commit s4-print-colocations
step s2-begin:
BEGIN;
step s2-create-table_2:
SELECT create_distributed_table('table_2', 'id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-create-concurrently-table_default_colocated:
SELECT create_distributed_table_concurrently('table_default_colocated', 'id');
<waiting ...>
step s4-print-waiting-advisory-locks:
SELECT mode, classid, objid, objsubid, granted FROM pg_locks
WHERE locktype = 'advisory' AND classid = 0 AND objid = 3 AND objsubid = 9
ORDER BY granted;
mode |classid|objid|objsubid|granted
---------------------------------------------------------------------
ExclusiveLock| 0| 3| 9|f
ExclusiveLock| 0| 3| 9|t
(2 rows)
step s2-commit:
COMMIT;
step s1-create-concurrently-table_default_colocated: <... completed>
create_distributed_table_concurrently
---------------------------------------------------------------------
(1 row)
step s4-print-colocations:
SELECT * FROM pg_dist_colocation ORDER BY colocationid;
colocationid|shardcount|replicationfactor|distributioncolumntype|distributioncolumncollation
---------------------------------------------------------------------
123173| 4| 1| 21| 0
123174| 4| 1| 23| 0
(2 rows)
citus_remove_node
---------------------------------------------------------------------
(1 row)
starting permutation: s1-create-concurrently-table_default_colocated s3-acquire-split-advisory-lock s1-create-concurrently-table_1 s2-create-table_2 s4-print-waiting-advisory-locks s3-release-split-advisory-lock s4-print-colocations
step s1-create-concurrently-table_default_colocated:
SELECT create_distributed_table_concurrently('table_default_colocated', 'id');
create_distributed_table_concurrently
---------------------------------------------------------------------
(1 row)
step s3-acquire-split-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-create-concurrently-table_1:
SELECT create_distributed_table_concurrently('table_1', 'id');
<waiting ...>
step s2-create-table_2:
SELECT create_distributed_table('table_2', 'id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s4-print-waiting-advisory-locks:
SELECT mode, classid, objid, objsubid, granted FROM pg_locks
WHERE locktype = 'advisory' AND classid = 0 AND objid = 3 AND objsubid = 9
ORDER BY granted;
mode|classid|objid|objsubid|granted
---------------------------------------------------------------------
(0 rows)
step s3-release-split-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s1-create-concurrently-table_1: <... completed>
create_distributed_table_concurrently
---------------------------------------------------------------------
(1 row)
step s4-print-colocations:
SELECT * FROM pg_dist_colocation ORDER BY colocationid;
colocationid|shardcount|replicationfactor|distributioncolumntype|distributioncolumncollation
---------------------------------------------------------------------
123175| 4| 1| 23| 0
123176| 4| 1| 21| 0
(2 rows)
citus_remove_node
---------------------------------------------------------------------
(1 row)
starting permutation: s2-begin s2-create-table_2 s1-create-concurrently-table_none_colocated s4-print-waiting-advisory-locks s2-commit s4-print-colocations
step s2-begin:
BEGIN;
step s2-create-table_2:
SELECT create_distributed_table('table_2', 'id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-create-concurrently-table_none_colocated:
SELECT create_distributed_table_concurrently('table_none_colocated', 'id', colocate_with => 'none');
<waiting ...>
step s4-print-waiting-advisory-locks:
SELECT mode, classid, objid, objsubid, granted FROM pg_locks
WHERE locktype = 'advisory' AND classid = 0 AND objid = 3 AND objsubid = 9
ORDER BY granted;
mode |classid|objid|objsubid|granted
---------------------------------------------------------------------
ExclusiveLock| 0| 3| 9|t
(1 row)
step s2-commit:
COMMIT;
step s1-create-concurrently-table_none_colocated: <... completed>
create_distributed_table_concurrently
---------------------------------------------------------------------
(1 row)
step s4-print-colocations:
SELECT * FROM pg_dist_colocation ORDER BY colocationid;
colocationid|shardcount|replicationfactor|distributioncolumntype|distributioncolumncollation
---------------------------------------------------------------------
123177| 4| 1| 21| 0
123178| 4| 1| 23| 0
(2 rows)
citus_remove_node
---------------------------------------------------------------------
(1 row)
starting permutation: s2-begin s2-create-table_2-none s1-create-concurrently-table_none_colocated s4-print-waiting-advisory-locks s2-commit s4-print-colocations
step s2-begin:
BEGIN;
step s2-create-table_2-none:
SELECT create_distributed_table('table_2', 'id', colocate_with => 'none');
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-create-concurrently-table_none_colocated:
SELECT create_distributed_table_concurrently('table_none_colocated', 'id', colocate_with => 'none');
<waiting ...>
step s4-print-waiting-advisory-locks:
SELECT mode, classid, objid, objsubid, granted FROM pg_locks
WHERE locktype = 'advisory' AND classid = 0 AND objid = 3 AND objsubid = 9
ORDER BY granted;
mode|classid|objid|objsubid|granted
---------------------------------------------------------------------
(0 rows)
step s2-commit:
COMMIT;
step s1-create-concurrently-table_none_colocated: <... completed>
create_distributed_table_concurrently
---------------------------------------------------------------------
(1 row)
step s4-print-colocations:
SELECT * FROM pg_dist_colocation ORDER BY colocationid;
colocationid|shardcount|replicationfactor|distributioncolumntype|distributioncolumncollation
---------------------------------------------------------------------
123179| 4| 1| 21| 0
123180| 4| 1| 23| 0
(2 rows)
citus_remove_node
---------------------------------------------------------------------
(1 row)

View File

@ -247,7 +247,7 @@ step s1-drop: DROP TABLE drop_hash;
step s2-distribute-table: SELECT create_distributed_table('drop_hash', 'id'); <waiting ...> step s2-distribute-table: SELECT create_distributed_table('drop_hash', 'id'); <waiting ...>
step s1-commit: COMMIT; step s1-commit: COMMIT;
step s2-distribute-table: <... completed> step s2-distribute-table: <... completed>
ERROR: could not create distributed table: relation does not exist ERROR: relation with OID XXXX does not exist
step s2-commit: COMMIT; step s2-commit: COMMIT;
step s1-select-count: SELECT COUNT(*) FROM drop_hash; step s1-select-count: SELECT COUNT(*) FROM drop_hash;
ERROR: relation "drop_hash" does not exist ERROR: relation "drop_hash" does not exist

View File

@ -8,9 +8,11 @@ create_distributed_table
step s1-load-cache: step s1-load-cache:
TRUNCATE isolation_table; TRUNCATE isolation_table;
TRUNCATE isolation_table2;
step s1-insert: step s1-insert:
INSERT INTO isolation_table VALUES (5, 10); INSERT INTO isolation_table VALUES (5, 10);
INSERT INTO isolation_table2 VALUES (5, 10);
step s3-acquire-advisory-lock: step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152); SELECT pg_advisory_lock(44000, 55152);
@ -57,7 +59,7 @@ t
step s2-isolate-tenant: <... completed> step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500076 1500078
(1 row) (1 row)
step s2-commit: step s2-commit:
@ -76,9 +78,9 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500075|t | 0
57637|1500076|t | 1
57637|1500077|t | 0 57637|1500077|t | 0
57637|1500078|t | 1
57637|1500079|t | 0
57638|1500074|t | 0 57638|1500074|t | 0
(4 rows) (4 rows)
@ -96,9 +98,11 @@ create_distributed_table
step s1-load-cache: step s1-load-cache:
TRUNCATE isolation_table; TRUNCATE isolation_table;
TRUNCATE isolation_table2;
step s1-insert: step s1-insert:
INSERT INTO isolation_table VALUES (5, 10); INSERT INTO isolation_table VALUES (5, 10);
INSERT INTO isolation_table2 VALUES (5, 10);
step s3-acquire-advisory-lock: step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152); SELECT pg_advisory_lock(44000, 55152);
@ -145,7 +149,7 @@ t
step s2-isolate-tenant: <... completed> step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500082 1500086
(1 row) (1 row)
step s2-commit: step s2-commit:
@ -164,10 +168,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500081|t | 0 57637|1500085|t | 0
57637|1500082|t | 0 57637|1500086|t | 0
57637|1500083|t | 0 57637|1500087|t | 0
57638|1500080|t | 0 57638|1500082|t | 0
(4 rows) (4 rows)
id|value id|value
@ -183,6 +187,7 @@ create_distributed_table
step s1-load-cache: step s1-load-cache:
TRUNCATE isolation_table; TRUNCATE isolation_table;
TRUNCATE isolation_table2;
step s3-acquire-advisory-lock: step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152); SELECT pg_advisory_lock(44000, 55152);
@ -214,6 +219,7 @@ step s2-isolate-tenant:
<waiting ...> <waiting ...>
step s1-insert: step s1-insert:
INSERT INTO isolation_table VALUES (5, 10); INSERT INTO isolation_table VALUES (5, 10);
INSERT INTO isolation_table2 VALUES (5, 10);
step s1-commit: step s1-commit:
COMMIT; COMMIT;
@ -229,7 +235,7 @@ t
step s2-isolate-tenant: <... completed> step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500088 1500094
(1 row) (1 row)
step s2-commit: step s2-commit:
@ -248,10 +254,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500087|t | 0 57637|1500093|t | 0
57637|1500088|t | 1 57637|1500094|t | 1
57637|1500089|t | 0 57637|1500095|t | 0
57638|1500086|t | 0 57638|1500090|t | 0
(4 rows) (4 rows)
id|value id|value
@ -268,6 +274,7 @@ create_distributed_table
step s1-load-cache: step s1-load-cache:
TRUNCATE isolation_table; TRUNCATE isolation_table;
TRUNCATE isolation_table2;
step s3-acquire-advisory-lock: step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152); SELECT pg_advisory_lock(44000, 55152);
@ -314,7 +321,7 @@ t
step s2-isolate-tenant: <... completed> step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500094 1500102
(1 row) (1 row)
step s2-commit: step s2-commit:
@ -333,10 +340,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500093|t | 1 57637|1500101|t | 1
57637|1500094|t | 1 57637|1500102|t | 1
57637|1500095|t | 2 57637|1500103|t | 2
57638|1500092|t | 1 57638|1500098|t | 1
(4 rows) (4 rows)
id|value id|value
@ -357,6 +364,7 @@ create_distributed_table
step s1-insert: step s1-insert:
INSERT INTO isolation_table VALUES (5, 10); INSERT INTO isolation_table VALUES (5, 10);
INSERT INTO isolation_table2 VALUES (5, 10);
step s3-acquire-advisory-lock: step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152); SELECT pg_advisory_lock(44000, 55152);
@ -403,7 +411,7 @@ t
step s2-isolate-tenant: <... completed> step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500100 1500110
(1 row) (1 row)
step s2-commit: step s2-commit:
@ -422,10 +430,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500099|t | 0 57637|1500109|t | 0
57637|1500100|t | 1 57637|1500110|t | 1
57637|1500101|t | 0 57637|1500111|t | 0
57638|1500098|t | 0 57638|1500106|t | 0
(4 rows) (4 rows)
id|value id|value
@ -442,6 +450,7 @@ create_distributed_table
step s1-insert: step s1-insert:
INSERT INTO isolation_table VALUES (5, 10); INSERT INTO isolation_table VALUES (5, 10);
INSERT INTO isolation_table2 VALUES (5, 10);
step s3-acquire-advisory-lock: step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152); SELECT pg_advisory_lock(44000, 55152);
@ -488,7 +497,7 @@ t
step s2-isolate-tenant: <... completed> step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500106 1500118
(1 row) (1 row)
step s2-commit: step s2-commit:
@ -507,10 +516,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500105|t | 0 57637|1500117|t | 0
57637|1500106|t | 0 57637|1500118|t | 0
57637|1500107|t | 0 57637|1500119|t | 0
57638|1500104|t | 0 57638|1500114|t | 0
(4 rows) (4 rows)
id|value id|value
@ -554,6 +563,7 @@ step s2-isolate-tenant:
<waiting ...> <waiting ...>
step s1-insert: step s1-insert:
INSERT INTO isolation_table VALUES (5, 10); INSERT INTO isolation_table VALUES (5, 10);
INSERT INTO isolation_table2 VALUES (5, 10);
step s1-commit: step s1-commit:
COMMIT; COMMIT;
@ -569,7 +579,7 @@ t
step s2-isolate-tenant: <... completed> step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500112 1500126
(1 row) (1 row)
step s2-commit: step s2-commit:
@ -588,10 +598,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500111|t | 0 57637|1500125|t | 0
57637|1500112|t | 1 57637|1500126|t | 1
57637|1500113|t | 0 57637|1500127|t | 0
57638|1500110|t | 0 57638|1500122|t | 0
(4 rows) (4 rows)
id|value id|value
@ -651,7 +661,7 @@ t
step s2-isolate-tenant: <... completed> step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500118 1500134
(1 row) (1 row)
step s2-commit: step s2-commit:
@ -670,10 +680,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500117|t | 1 57637|1500133|t | 1
57637|1500118|t | 1 57637|1500134|t | 1
57637|1500119|t | 2 57637|1500135|t | 2
57638|1500116|t | 1 57638|1500130|t | 1
(4 rows) (4 rows)
id|value id|value
@ -686,7 +696,7 @@ id|value
(5 rows) (5 rows)
starting permutation: s1-load-cache s1-insert s3-acquire-advisory-lock s1-begin s1-isolate-tenant s2-isolate-tenant s3-release-advisory-lock s1-commit s2-print-cluster starting permutation: s1-load-cache s1-insert s3-acquire-advisory-lock s2-isolate-tenant s1-isolate-tenant-same-coloc s3-release-advisory-lock s2-print-cluster
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -694,9 +704,11 @@ create_distributed_table
step s1-load-cache: step s1-load-cache:
TRUNCATE isolation_table; TRUNCATE isolation_table;
TRUNCATE isolation_table2;
step s1-insert: step s1-insert:
INSERT INTO isolation_table VALUES (5, 10); INSERT INTO isolation_table VALUES (5, 10);
INSERT INTO isolation_table2 VALUES (5, 10);
step s3-acquire-advisory-lock: step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152); SELECT pg_advisory_lock(44000, 55152);
@ -706,17 +718,11 @@ pg_advisory_lock
(1 row) (1 row)
step s1-begin: step s2-isolate-tenant:
BEGIN;
-- the tests are written with the logic where single shard SELECTs
-- do not to open transaction blocks
SET citus.select_opens_transaction_block TO false;
step s1-isolate-tenant:
SELECT isolate_tenant_to_new_shard('isolation_table', 2, shard_transfer_mode => 'force_logical');
<waiting ...>
step s2-isolate-tenant:
SELECT isolate_tenant_to_new_shard('isolation_table', 5, shard_transfer_mode => 'force_logical'); SELECT isolate_tenant_to_new_shard('isolation_table', 5, shard_transfer_mode => 'force_logical');
<waiting ...>
step s1-isolate-tenant-same-coloc:
SELECT isolate_tenant_to_new_shard('isolation_table', 2, shard_transfer_mode => 'force_logical');
ERROR: could not acquire the lock required to split public.isolation_table ERROR: could not acquire the lock required to split public.isolation_table
step s3-release-advisory-lock: step s3-release-advisory-lock:
@ -727,15 +733,12 @@ pg_advisory_unlock
t t
(1 row) (1 row)
step s1-isolate-tenant: <... completed> step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500124 1500142
(1 row) (1 row)
step s1-commit:
COMMIT;
step s2-print-cluster: step s2-print-cluster:
-- row count per shard -- row count per shard
SELECT SELECT
@ -749,10 +752,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500121|t | 1 57637|1500141|t | 0
57638|1500123|t | 0 57637|1500142|t | 1
57638|1500124|t | 0 57637|1500143|t | 0
57638|1500125|t | 0 57638|1500138|t | 0
(4 rows) (4 rows)
id|value id|value
@ -761,14 +764,19 @@ id|value
(1 row) (1 row)
starting permutation: s1-insert s3-acquire-advisory-lock s1-begin s1-isolate-tenant s2-isolate-tenant s3-release-advisory-lock s1-commit s2-print-cluster starting permutation: s1-load-cache s1-insert s3-acquire-advisory-lock s2-isolate-tenant s1-isolate-tenant-same-coloc-blocking s3-release-advisory-lock s2-print-cluster
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
step s1-load-cache:
TRUNCATE isolation_table;
TRUNCATE isolation_table2;
step s1-insert: step s1-insert:
INSERT INTO isolation_table VALUES (5, 10); INSERT INTO isolation_table VALUES (5, 10);
INSERT INTO isolation_table2 VALUES (5, 10);
step s3-acquire-advisory-lock: step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152); SELECT pg_advisory_lock(44000, 55152);
@ -778,17 +786,11 @@ pg_advisory_lock
(1 row) (1 row)
step s1-begin: step s2-isolate-tenant:
BEGIN;
-- the tests are written with the logic where single shard SELECTs
-- do not to open transaction blocks
SET citus.select_opens_transaction_block TO false;
step s1-isolate-tenant:
SELECT isolate_tenant_to_new_shard('isolation_table', 2, shard_transfer_mode => 'force_logical');
<waiting ...>
step s2-isolate-tenant:
SELECT isolate_tenant_to_new_shard('isolation_table', 5, shard_transfer_mode => 'force_logical'); SELECT isolate_tenant_to_new_shard('isolation_table', 5, shard_transfer_mode => 'force_logical');
<waiting ...>
step s1-isolate-tenant-same-coloc-blocking:
SELECT isolate_tenant_to_new_shard('isolation_table', 2, shard_transfer_mode => 'block_writes');
ERROR: could not acquire the lock required to split public.isolation_table ERROR: could not acquire the lock required to split public.isolation_table
step s3-release-advisory-lock: step s3-release-advisory-lock:
@ -799,13 +801,152 @@ pg_advisory_unlock
t t
(1 row) (1 row)
step s1-isolate-tenant: <... completed> step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500130 1500150
(1 row) (1 row)
step s1-commit: step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('isolation_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM isolation_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500149|t | 0
57637|1500150|t | 1
57637|1500151|t | 0
57638|1500146|t | 0
(4 rows)
id|value
---------------------------------------------------------------------
5| 10
(1 row)
starting permutation: s1-load-cache s1-insert s3-acquire-advisory-lock s2-isolate-tenant s1-isolate-tenant-no-same-coloc s3-release-advisory-lock s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-load-cache:
TRUNCATE isolation_table;
TRUNCATE isolation_table2;
step s1-insert:
INSERT INTO isolation_table VALUES (5, 10);
INSERT INTO isolation_table2 VALUES (5, 10);
step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s2-isolate-tenant:
SELECT isolate_tenant_to_new_shard('isolation_table', 5, shard_transfer_mode => 'force_logical');
<waiting ...>
step s1-isolate-tenant-no-same-coloc:
SELECT isolate_tenant_to_new_shard('isolation_table2', 2, shard_transfer_mode => 'force_logical');
ERROR: could not acquire the lock required to split concurrently public.isolation_table2.
step s3-release-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard
---------------------------------------------------------------------
1500158
(1 row)
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('isolation_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM isolation_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500157|t | 0
57637|1500158|t | 1
57637|1500159|t | 0
57638|1500154|t | 0
(4 rows)
id|value
---------------------------------------------------------------------
5| 10
(1 row)
starting permutation: s1-load-cache s1-insert s3-acquire-advisory-lock s2-begin s2-isolate-tenant s1-isolate-tenant-no-same-coloc s3-release-advisory-lock s2-commit s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-load-cache:
TRUNCATE isolation_table;
TRUNCATE isolation_table2;
step s1-insert:
INSERT INTO isolation_table VALUES (5, 10);
INSERT INTO isolation_table2 VALUES (5, 10);
step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s2-begin:
BEGIN;
step s2-isolate-tenant:
SELECT isolate_tenant_to_new_shard('isolation_table', 5, shard_transfer_mode => 'force_logical');
<waiting ...>
step s1-isolate-tenant-no-same-coloc:
SELECT isolate_tenant_to_new_shard('isolation_table2', 2, shard_transfer_mode => 'force_logical');
ERROR: could not acquire the lock required to split concurrently public.isolation_table2.
step s3-release-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard
---------------------------------------------------------------------
1500169
(1 row)
step s2-commit:
COMMIT; COMMIT;
step s2-print-cluster: step s2-print-cluster:
@ -821,10 +962,154 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500127|t | 1 57637|1500168|t | 0
57638|1500129|t | 0 57637|1500169|t | 1
57638|1500130|t | 0 57637|1500170|t | 0
57638|1500131|t | 0 57638|1500165|t | 0
(4 rows)
id|value
---------------------------------------------------------------------
5| 10
(1 row)
starting permutation: s1-load-cache s1-insert s3-acquire-advisory-lock s2-isolate-tenant s1-isolate-tenant-no-same-coloc-blocking s3-release-advisory-lock s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-load-cache:
TRUNCATE isolation_table;
TRUNCATE isolation_table2;
step s1-insert:
INSERT INTO isolation_table VALUES (5, 10);
INSERT INTO isolation_table2 VALUES (5, 10);
step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s2-isolate-tenant:
SELECT isolate_tenant_to_new_shard('isolation_table', 5, shard_transfer_mode => 'force_logical');
<waiting ...>
step s1-isolate-tenant-no-same-coloc-blocking:
SELECT isolate_tenant_to_new_shard('isolation_table2', 2, shard_transfer_mode => 'block_writes');
isolate_tenant_to_new_shard
---------------------------------------------------------------------
1500183
(1 row)
step s3-release-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard
---------------------------------------------------------------------
1500180
(1 row)
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('isolation_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM isolation_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500179|t | 0
57637|1500180|t | 1
57637|1500181|t | 0
57638|1500176|t | 0
(4 rows)
id|value
---------------------------------------------------------------------
5| 10
(1 row)
starting permutation: s1-load-cache s1-insert s3-acquire-advisory-lock s2-isolate-tenant s1-isolate-tenant-no-same-coloc-blocking s3-release-advisory-lock s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-load-cache:
TRUNCATE isolation_table;
TRUNCATE isolation_table2;
step s1-insert:
INSERT INTO isolation_table VALUES (5, 10);
INSERT INTO isolation_table2 VALUES (5, 10);
step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s2-isolate-tenant:
SELECT isolate_tenant_to_new_shard('isolation_table', 5, shard_transfer_mode => 'force_logical');
<waiting ...>
step s1-isolate-tenant-no-same-coloc-blocking:
SELECT isolate_tenant_to_new_shard('isolation_table2', 2, shard_transfer_mode => 'block_writes');
isolate_tenant_to_new_shard
---------------------------------------------------------------------
1500194
(1 row)
step s3-release-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard
---------------------------------------------------------------------
1500191
(1 row)
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('isolation_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM isolation_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500190|t | 0
57637|1500191|t | 1
57637|1500192|t | 0
57638|1500187|t | 0
(4 rows) (4 rows)
id|value id|value

View File

@ -1111,7 +1111,7 @@ ERROR: extension "citus" already exists
-- Snapshot of state at 11.1-1 -- Snapshot of state at 11.1-1
ALTER EXTENSION citus UPDATE TO '11.1-1'; ALTER EXTENSION citus UPDATE TO '11.1-1';
SELECT * FROM multi_extension.print_extension_changes(); SELECT * FROM multi_extension.print_extension_changes();
previous_object | current_object previous_object | current_object
--------------------------------------------------------------------- ---------------------------------------------------------------------
access method columnar | access method columnar |
function alter_columnar_table_reset(regclass,boolean,boolean,boolean,boolean) void | function alter_columnar_table_reset(regclass,boolean,boolean,boolean,boolean) void |
@ -1136,19 +1136,21 @@ SELECT * FROM multi_extension.print_extension_changes();
table columnar.chunk_group | table columnar.chunk_group |
table columnar.options | table columnar.options |
table columnar.stripe | table columnar.stripe |
| function citus_internal_delete_partition_metadata(regclass) void
| function citus_locks() SETOF record | function citus_locks() SETOF record
| function citus_split_shard_by_split_points(bigint,text[],integer[],citus.shard_transfer_mode) void | function citus_split_shard_by_split_points(bigint,text[],integer[],citus.shard_transfer_mode) void
| function create_distributed_table_concurrently(regclass,text,citus.distribution_type,text,integer) void
| function isolate_tenant_to_new_shard(regclass,"any",text,citus.shard_transfer_mode) bigint | function isolate_tenant_to_new_shard(regclass,"any",text,citus.shard_transfer_mode) bigint
| function replicate_reference_tables(citus.shard_transfer_mode) void | function replicate_reference_tables(citus.shard_transfer_mode) void
| function worker_copy_table_to_node(regclass,integer) void | function worker_copy_table_to_node(regclass,integer) void
| function worker_split_copy(bigint,split_copy_info[]) void | function worker_split_copy(bigint,text,split_copy_info[]) void
| function worker_split_shard_release_dsm() void | function worker_split_shard_release_dsm() void
| function worker_split_shard_replication_setup(split_shard_info[]) SETOF replication_slot_info | function worker_split_shard_replication_setup(split_shard_info[]) SETOF replication_slot_info
| type replication_slot_info | type replication_slot_info
| type split_copy_info | type split_copy_info
| type split_shard_info | type split_shard_info
| view citus_locks | view citus_locks
(35 rows) (37 rows)
DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff; DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff;
-- show running version -- show running version

View File

@ -663,7 +663,7 @@ SELECT count(*) FROM lineitem_date WHERE l_shipdate = '1997-08-08';
SET search_path to "Tenant Isolation"; SET search_path to "Tenant Isolation";
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE nodeport = :worker_1_port; UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE nodeport = :worker_1_port;
SELECT isolate_tenant_to_new_shard('lineitem_date', '1997-08-08', shard_transfer_mode => 'block_writes'); SELECT isolate_tenant_to_new_shard('lineitem_date', '1997-08-08', shard_transfer_mode => 'block_writes');
ERROR: cannot split shard because relation "lineitem_date" has an inactive shard placement for the shard xxxxx ERROR: cannot isolate tenant because relation "lineitem_date" has an inactive shard placement for the shard xxxxx
HINT: Use master_copy_shard_placement UDF to repair the inactive shard placement. HINT: Use master_copy_shard_placement UDF to repair the inactive shard placement.
UPDATE pg_dist_shard_placement SET shardstate = 1 WHERE nodeport = :worker_1_port; UPDATE pg_dist_shard_placement SET shardstate = 1 WHERE nodeport = :worker_1_port;
\c - mx_isolation_role_ent - :master_port \c - mx_isolation_role_ent - :master_port

View File

@ -663,7 +663,7 @@ SELECT count(*) FROM lineitem_date WHERE l_shipdate = '1997-08-08';
SET search_path to "Tenant Isolation"; SET search_path to "Tenant Isolation";
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE nodeport = :worker_1_port; UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE nodeport = :worker_1_port;
SELECT isolate_tenant_to_new_shard('lineitem_date', '1997-08-08', shard_transfer_mode => 'force_logical'); SELECT isolate_tenant_to_new_shard('lineitem_date', '1997-08-08', shard_transfer_mode => 'force_logical');
ERROR: cannot split shard because relation "lineitem_date" has an inactive shard placement for the shard xxxxx ERROR: cannot isolate tenant because relation "lineitem_date" has an inactive shard placement for the shard xxxxx
HINT: Use master_copy_shard_placement UDF to repair the inactive shard placement. HINT: Use master_copy_shard_placement UDF to repair the inactive shard placement.
UPDATE pg_dist_shard_placement SET shardstate = 1 WHERE nodeport = :worker_1_port; UPDATE pg_dist_shard_placement SET shardstate = 1 WHERE nodeport = :worker_1_port;
\c - mx_isolation_role_ent - :master_port \c - mx_isolation_role_ent - :master_port

View File

@ -65,6 +65,7 @@ ORDER BY 1;
function citus_internal_add_placement_metadata(bigint,integer,bigint,integer,bigint) function citus_internal_add_placement_metadata(bigint,integer,bigint,integer,bigint)
function citus_internal_add_shard_metadata(regclass,bigint,"char",text,text) function citus_internal_add_shard_metadata(regclass,bigint,"char",text,text)
function citus_internal_delete_colocation_metadata(integer) function citus_internal_delete_colocation_metadata(integer)
function citus_internal_delete_partition_metadata(regclass)
function citus_internal_delete_shard_metadata(bigint) function citus_internal_delete_shard_metadata(bigint)
function citus_internal_global_blocked_processes() function citus_internal_global_blocked_processes()
function citus_internal_local_blocked_processes() function citus_internal_local_blocked_processes()
@ -122,6 +123,7 @@ ORDER BY 1;
function coord_combine_agg_sfunc(internal,oid,cstring,anyelement) function coord_combine_agg_sfunc(internal,oid,cstring,anyelement)
function create_distributed_function(regprocedure,text,text,boolean) function create_distributed_function(regprocedure,text,text,boolean)
function create_distributed_table(regclass,text,citus.distribution_type,text,integer) function create_distributed_table(regclass,text,citus.distribution_type,text,integer)
function create_distributed_table_concurrently(regclass,text,citus.distribution_type,text,integer)
function create_intermediate_result(text,text) function create_intermediate_result(text,text)
function create_reference_table(regclass) function create_reference_table(regclass)
function create_time_partitions(regclass,interval,timestamp with time zone,timestamp with time zone) function create_time_partitions(regclass,interval,timestamp with time zone,timestamp with time zone)
@ -228,7 +230,7 @@ ORDER BY 1;
function worker_partitioned_table_size(regclass) function worker_partitioned_table_size(regclass)
function worker_record_sequence_dependency(regclass,regclass,name) function worker_record_sequence_dependency(regclass,regclass,name)
function worker_save_query_explain_analyze(text,jsonb) function worker_save_query_explain_analyze(text,jsonb)
function worker_split_copy(bigint,split_copy_info[]) function worker_split_copy(bigint,text,split_copy_info[])
function worker_split_shard_release_dsm() function worker_split_shard_release_dsm()
function worker_split_shard_replication_setup(split_shard_info[]) function worker_split_shard_replication_setup(split_shard_info[])
schema citus schema citus
@ -268,5 +270,5 @@ ORDER BY 1;
view citus_stat_statements view citus_stat_statements
view pg_dist_shard_placement view pg_dist_shard_placement
view time_partitions view time_partitions
(260 rows) (262 rows)

View File

@ -186,6 +186,7 @@ SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \
SET citus.enable_binary_protocol = true; SET citus.enable_binary_protocol = true;
SELECT * from worker_split_copy( SELECT * from worker_split_copy(
81060000, -- source shard id to copy 81060000, -- source shard id to copy
'l_orderkey',
ARRAY[ ARRAY[
-- split copy info for split children 1 -- split copy info for split children 1
ROW(81060015, -- destination shard id ROW(81060015, -- destination shard id
@ -208,6 +209,7 @@ SELECT * from worker_split_copy(
-- BEGIN: Trigger 2-way remote shard split copy. -- BEGIN: Trigger 2-way remote shard split copy.
SELECT * from worker_split_copy( SELECT * from worker_split_copy(
81060000, -- source shard id to copy 81060000, -- source shard id to copy
'l_orderkey',
ARRAY[ ARRAY[
-- split copy info for split children 1 -- split copy info for split children 1
ROW(81060015, -- destination shard id ROW(81060015, -- destination shard id

View File

@ -54,6 +54,7 @@ SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \
-- BEGIN: Test Negative scenario -- BEGIN: Test Negative scenario
SELECT * from worker_split_copy( SELECT * from worker_split_copy(
101, -- Invalid source shard id. 101, -- Invalid source shard id.
'id',
ARRAY[ ARRAY[
-- split copy info for split children 1 -- split copy info for split children 1
ROW(81070015, -- destination shard id ROW(81070015, -- destination shard id
@ -70,29 +71,34 @@ SELECT * from worker_split_copy(
ERROR: could not find valid entry for shard xxxxx ERROR: could not find valid entry for shard xxxxx
SELECT * from worker_split_copy( SELECT * from worker_split_copy(
81070000, -- source shard id to copy 81070000, -- source shard id to copy
'id',
ARRAY[] -- empty array ARRAY[] -- empty array
); );
ERROR: cannot determine type of empty array ERROR: cannot determine type of empty array
HINT: Explicitly cast to the desired type, for example ARRAY[]::integer[]. HINT: Explicitly cast to the desired type, for example ARRAY[]::integer[].
SELECT * from worker_split_copy( SELECT * from worker_split_copy(
81070000, -- source shard id to copy 81070000, -- source shard id to copy
'id',
ARRAY[NULL] -- empty array ARRAY[NULL] -- empty array
); );
ERROR: function worker_split_copy(integer, text[]) does not exist ERROR: function worker_split_copy(integer, unknown, text[]) does not exist
HINT: No function matches the given name and argument types. You might need to add explicit type casts. HINT: No function matches the given name and argument types. You might need to add explicit type casts.
SELECT * from worker_split_copy( SELECT * from worker_split_copy(
81070000, -- source shard id to copy 81070000, -- source shard id to copy
'id',
ARRAY[NULL::pg_catalog.split_copy_info]-- empty array ARRAY[NULL::pg_catalog.split_copy_info]-- empty array
); );
ERROR: pg_catalog.split_copy_info array cannot contain null values ERROR: pg_catalog.split_copy_info array cannot contain null values
SELECT * from worker_split_copy( SELECT * from worker_split_copy(
81070000, -- source shard id to copy 81070000, -- source shard id to copy
'id',
ARRAY[ROW(NULL)]-- empty array ARRAY[ROW(NULL)]-- empty array
); );
ERROR: function worker_split_copy(integer, record[]) does not exist ERROR: function worker_split_copy(integer, unknown, record[]) does not exist
HINT: No function matches the given name and argument types. You might need to add explicit type casts. HINT: No function matches the given name and argument types. You might need to add explicit type casts.
SELECT * from worker_split_copy( SELECT * from worker_split_copy(
81070000, -- source shard id to copy 81070000, -- source shard id to copy
'id',
ARRAY[ROW(NULL, NULL, NULL, NULL)::pg_catalog.split_copy_info] -- empty array ARRAY[ROW(NULL, NULL, NULL, NULL)::pg_catalog.split_copy_info] -- empty array
); );
ERROR: destination_shard_id for pg_catalog.split_copy_info cannot be null. ERROR: destination_shard_id for pg_catalog.split_copy_info cannot be null.
@ -102,6 +108,7 @@ ERROR: destination_shard_id for pg_catalog.split_copy_info cannot be null.
SET citus.enable_binary_protocol = false; SET citus.enable_binary_protocol = false;
SELECT * from worker_split_copy( SELECT * from worker_split_copy(
81070000, -- source shard id to copy 81070000, -- source shard id to copy
'id',
ARRAY[ ARRAY[
-- split copy info for split children 1 -- split copy info for split children 1
ROW(81070015, -- destination shard id ROW(81070015, -- destination shard id

View File

@ -149,6 +149,7 @@ SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \
SET citus.enable_binary_protocol = false; SET citus.enable_binary_protocol = false;
SELECT * from worker_split_copy( SELECT * from worker_split_copy(
81070000, -- source shard id to copy 81070000, -- source shard id to copy
'l_orderkey',
ARRAY[ ARRAY[
-- split copy info for split children 1 -- split copy info for split children 1
ROW(81070015, -- destination shard id ROW(81070015, -- destination shard id
@ -171,6 +172,7 @@ SELECT * from worker_split_copy(
-- BEGIN: Trigger 2-way remote shard split copy. -- BEGIN: Trigger 2-way remote shard split copy.
SELECT * from worker_split_copy( SELECT * from worker_split_copy(
81070000, -- source shard id to copy 81070000, -- source shard id to copy
'l_orderkey',
ARRAY[ ARRAY[
-- split copy info for split children 1 -- split copy info for split children 1
ROW(81070015, -- destination shard id ROW(81070015, -- destination shard id

View File

@ -18,6 +18,7 @@ test: failure_copy_on_hash
test: failure_create_reference_table test: failure_create_reference_table
test: failure_create_distributed_table_non_empty test: failure_create_distributed_table_non_empty
test: failure_create_table test: failure_create_table
test: failure_create_distributed_table_concurrently
test: failure_single_select test: failure_single_select
test: failure_multi_shard_update_delete test: failure_multi_shard_update_delete

View File

@ -48,6 +48,7 @@ test: isolation_create_citus_local_table
test: isolation_create_restore_point test: isolation_create_restore_point
test: isolation_create_distributed_table test: isolation_create_distributed_table
test: isolation_create_distributed_table_concurrently
test: isolation_multi_shard_modify_vs_all test: isolation_multi_shard_modify_vs_all
test: isolation_modify_with_subquery_vs_dml test: isolation_modify_with_subquery_vs_dml
test: isolation_hash_copy_vs_all test: isolation_hash_copy_vs_all

View File

@ -277,7 +277,7 @@ test: add_coordinator
test: replicate_reference_tables_to_coordinator test: replicate_reference_tables_to_coordinator
test: citus_local_tables test: citus_local_tables
test: mixed_relkind_tests test: mixed_relkind_tests
test: multi_row_router_insert test: multi_row_router_insert create_distributed_table_concurrently
test: multi_reference_table citus_local_tables_queries test: multi_reference_table citus_local_tables_queries
test: citus_local_table_triggers test: citus_local_table_triggers
test: coordinator_shouldhaveshards test: coordinator_shouldhaveshards

View File

@ -0,0 +1,221 @@
setup
{
-- make sure coordinator is in metadata
SELECT citus_set_coordinator_host('localhost', 57636);
CREATE TABLE table_1(id int PRIMARY KEY);
CREATE TABLE table_2(id smallint PRIMARY KEY);
CREATE TABLE table_default_colocated(id int PRIMARY KEY);
CREATE TABLE table_none_colocated(id int PRIMARY KEY);
}
teardown
{
DROP TABLE table_1 CASCADE;
DROP TABLE table_2 CASCADE;
DROP TABLE table_default_colocated CASCADE;
DROP TABLE table_none_colocated CASCADE;
SELECT citus_remove_node('localhost', 57636);
}
session "s1"
step "s1-create-concurrently-table_1"
{
SELECT create_distributed_table_concurrently('table_1', 'id');
}
step "s1-create-concurrently-table_2"
{
SELECT create_distributed_table_concurrently('table_2', 'id');
}
step "s1-create-concurrently-table_default_colocated"
{
SELECT create_distributed_table_concurrently('table_default_colocated', 'id');
}
step "s1-create-concurrently-table_none_colocated"
{
SELECT create_distributed_table_concurrently('table_none_colocated', 'id', colocate_with => 'none');
}
step "s1-settings"
{
-- session needs to have replication factor set to 1, can't do in setup
SET citus.shard_count TO 4;
SET citus.shard_replication_factor TO 1;
}
step "s1-truncate"
{
TRUNCATE table_1;
}
session "s2"
step "s2-begin"
{
BEGIN;
}
step "s2-settings"
{
-- session needs to have replication factor set to 1, can't do in setup
SET citus.shard_count TO 4;
SET citus.shard_replication_factor TO 1;
}
step "s2-insert"
{
INSERT INTO table_1 SELECT s FROM generate_series(1,20) s;
}
step "s2-update"
{
UPDATE table_1 SET id = 21 WHERE id = 20;
}
step "s2-delete"
{
DELETE FROM table_1 WHERE id = 11;
}
step "s2-copy"
{
COPY table_1 FROM PROGRAM 'echo 30 && echo 31 && echo 32 && echo 33 && echo 34 && echo 35 && echo 36 && echo 37 && echo 38';
}
step "s2-reindex"
{
REINDEX TABLE table_1;
}
step "s2-reindex-concurrently"
{
REINDEX TABLE CONCURRENTLY table_1;
}
step "s2-create-concurrently-table_1"
{
SELECT create_distributed_table_concurrently('table_1', 'id');
}
step "s2-create-table_1"
{
SELECT create_distributed_table('table_1', 'id');
}
step "s2-create-concurrently-table_2"
{
SELECT create_distributed_table_concurrently('table_2', 'id');
}
step "s2-create-table_2"
{
SELECT create_distributed_table('table_2', 'id');
}
step "s2-create-table_2-none"
{
SELECT create_distributed_table('table_2', 'id', colocate_with => 'none');
}
step "s2-print-status"
{
-- sanity check on partitions
SELECT * FROM pg_dist_shard
WHERE logicalrelid = 'table_1'::regclass OR logicalrelid = 'table_2'::regclass
ORDER BY shardminvalue::BIGINT, logicalrelid;
-- sanity check on total elements in the table
SELECT COUNT(*) FROM table_1;
}
step "s2-commit"
{
COMMIT;
}
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-split-advisory-lock"
{
SELECT pg_advisory_lock(44000, 55152);
}
step "s3-release-split-advisory-lock"
{
SELECT pg_advisory_unlock(44000, 55152);
}
session "s4"
step "s4-print-waiting-locks"
{
SELECT mode, relation::regclass, granted FROM pg_locks
WHERE relation = 'table_1'::regclass OR relation = 'table_2'::regclass
ORDER BY mode, relation, granted;
}
step "s4-print-waiting-advisory-locks"
{
SELECT mode, classid, objid, objsubid, granted FROM pg_locks
WHERE locktype = 'advisory' AND classid = 0 AND objid = 3 AND objsubid = 9
ORDER BY granted;
}
step "s4-print-colocations"
{
SELECT * FROM pg_dist_colocation ORDER BY colocationid;
}
// show concurrent insert is NOT blocked by create_distributed_table_concurrently
permutation "s1-truncate" "s3-acquire-split-advisory-lock" "s1-settings" "s2-settings" "s1-create-concurrently-table_1" "s2-begin" "s2-insert" "s2-commit" "s3-release-split-advisory-lock" "s2-print-status"
// show concurrent update is NOT blocked by create_distributed_table_concurrently
permutation "s1-truncate" "s3-acquire-split-advisory-lock" "s1-create-concurrently-table_1" "s2-begin" "s2-insert" "s2-update" "s2-commit" "s3-release-split-advisory-lock" "s2-print-status"
// show concurrent delete is NOT blocked by create_distributed_table_concurrently
permutation "s1-truncate" "s3-acquire-split-advisory-lock" "s1-create-concurrently-table_1" "s2-begin" "s2-insert" "s2-delete" "s2-commit" "s3-release-split-advisory-lock" "s2-print-status"
// show concurrent copy is NOT blocked by create_distributed_table_concurrently
permutation "s1-truncate" "s3-acquire-split-advisory-lock" "s1-create-concurrently-table_1" "s2-begin" "s2-insert" "s2-copy" "s2-commit" "s3-release-split-advisory-lock" "s2-print-status"
// show concurrent reindex concurrently is blocked by create_distributed_table_concurrently
// both tries to acquire SHARE UPDATE EXCLUSIVE on the table
permutation "s3-acquire-split-advisory-lock" "s1-create-concurrently-table_1" "s2-insert" "s2-reindex-concurrently" "s4-print-waiting-locks" "s3-release-split-advisory-lock"
// show concurrent reindex is blocked by create_distributed_table_concurrently
// reindex tries to acquire ACCESS EXCLUSIVE lock while create-concurrently tries to acquire SHARE UPDATE EXCLUSIVE on the table
permutation "s3-acquire-split-advisory-lock" "s1-create-concurrently-table_1" "s2-insert" "s2-reindex" "s4-print-waiting-locks" "s3-release-split-advisory-lock"
// show create_distributed_table_concurrently operation inside a transaction are NOT allowed
permutation "s2-begin" "s2-create-concurrently-table_1" "s2-commit"
// show concurrent create_distributed_table_concurrently operations with the same table are NOT allowed
permutation "s3-acquire-split-advisory-lock" "s1-create-concurrently-table_1" "s2-create-concurrently-table_1" "s3-release-split-advisory-lock"
// show concurrent create_distributed_table_concurrently operations with different tables are NOT allowed
permutation "s3-acquire-split-advisory-lock" "s1-create-concurrently-table_1" "s2-create-concurrently-table_2" "s3-release-split-advisory-lock"
// show concurrent create_distributed_table_concurrently and create_distribute_table operations with the same table are NOT allowed
permutation "s3-acquire-split-advisory-lock" "s1-create-concurrently-table_1" "s2-create-table_1" "s3-release-split-advisory-lock"
// show concurrent create_distributed_table_concurrently and create_distribute_table operations with different tables are allowed
permutation "s3-acquire-split-advisory-lock" "s1-create-concurrently-table_1" "s2-create-table_2" "s3-release-split-advisory-lock"
// tests with colocated_with combinations
// show concurrent colocate_with => 'default' and colocate_with => 'default' are NOT allowed if there is no default colocation entry yet.
permutation "s2-begin" "s2-create-table_2" "s1-create-concurrently-table_default_colocated" "s4-print-waiting-advisory-locks" "s2-commit" "s4-print-colocations"
// show concurrent colocate_with => 'default' and colocate_with => 'default' are allowed if there is already a default colocation entry.
permutation "s1-create-concurrently-table_default_colocated" "s3-acquire-split-advisory-lock" "s1-create-concurrently-table_1" "s2-create-table_2" "s4-print-waiting-advisory-locks" "s3-release-split-advisory-lock" "s4-print-colocations"
// show concurrent colocate_with => 'default' and colocate_with => 'none' are allowed.
permutation "s2-begin" "s2-create-table_2" "s1-create-concurrently-table_none_colocated" "s4-print-waiting-advisory-locks" "s2-commit" "s4-print-colocations"
// show concurrent colocate_with => 'none' and colocate_with => 'none' are allowed.
permutation "s2-begin" "s2-create-table_2-none" "s1-create-concurrently-table_none_colocated" "s4-print-waiting-advisory-locks" "s2-commit" "s4-print-colocations"

View File

@ -10,11 +10,16 @@ setup
CREATE TABLE isolation_table (id int PRIMARY KEY, value int); CREATE TABLE isolation_table (id int PRIMARY KEY, value int);
SELECT create_distributed_table('isolation_table', 'id'); SELECT create_distributed_table('isolation_table', 'id');
-- different colocation id
CREATE TABLE isolation_table2 (id smallint PRIMARY KEY, value int);
SELECT create_distributed_table('isolation_table2', 'id');
} }
teardown teardown
{ {
DROP TABLE isolation_table; DROP TABLE isolation_table;
DROP TABLE isolation_table2;
} }
session "s1" session "s1"
@ -32,11 +37,13 @@ step "s1-begin"
step "s1-load-cache" step "s1-load-cache"
{ {
TRUNCATE isolation_table; TRUNCATE isolation_table;
TRUNCATE isolation_table2;
} }
step "s1-insert" step "s1-insert"
{ {
INSERT INTO isolation_table VALUES (5, 10); INSERT INTO isolation_table VALUES (5, 10);
INSERT INTO isolation_table2 VALUES (5, 10);
} }
step "s1-update" step "s1-update"
@ -59,11 +66,26 @@ step "s1-copy"
COPY isolation_table FROM PROGRAM 'echo "1,1\n2,2\n3,3\n4,4\n5,5"' WITH CSV; COPY isolation_table FROM PROGRAM 'echo "1,1\n2,2\n3,3\n4,4\n5,5"' WITH CSV;
} }
step "s1-isolate-tenant" step "s1-isolate-tenant-same-coloc"
{ {
SELECT isolate_tenant_to_new_shard('isolation_table', 2, shard_transfer_mode => 'force_logical'); SELECT isolate_tenant_to_new_shard('isolation_table', 2, shard_transfer_mode => 'force_logical');
} }
step "s1-isolate-tenant-same-coloc-blocking"
{
SELECT isolate_tenant_to_new_shard('isolation_table', 2, shard_transfer_mode => 'block_writes');
}
step "s1-isolate-tenant-no-same-coloc"
{
SELECT isolate_tenant_to_new_shard('isolation_table2', 2, shard_transfer_mode => 'force_logical');
}
step "s1-isolate-tenant-no-same-coloc-blocking"
{
SELECT isolate_tenant_to_new_shard('isolation_table2', 2, shard_transfer_mode => 'block_writes');
}
step "s1-commit" step "s1-commit"
{ {
COMMIT; COMMIT;
@ -122,7 +144,7 @@ step "s3-release-advisory-lock"
// s1 can execute its DML command concurrently with s2 shard isolation => // s1 can execute its DML command concurrently with s2 shard isolation =>
// s3 releases the advisory lock so that s2 can finish the transaction // s3 releases the advisory lock so that s2 can finish the transaction
// run tenant isolation while concurrently performing an DML and index creation // run tenant isolation while concurrently performing an DML
// we expect DML queries of s2 to succeed without being blocked. // we expect DML queries of s2 to succeed without being blocked.
permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-update" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster" permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-update" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"
permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-delete" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster" permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-delete" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"
@ -135,8 +157,20 @@ permutation "s1-insert" "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-be
permutation "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-insert" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster" permutation "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-insert" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"
permutation "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-copy" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster" permutation "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-copy" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"
// concurrent tenant isolation blocks on different shards of the same table (or any colocated table) // concurrent nonblocking tenant isolations with the same colocation id are not allowed
permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s1-begin" "s1-isolate-tenant" "s2-isolate-tenant" "s3-release-advisory-lock" "s1-commit" "s2-print-cluster" permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s2-isolate-tenant" "s1-isolate-tenant-same-coloc" "s3-release-advisory-lock" "s2-print-cluster"
// the same test above without loading the cache at first // concurrent blocking and nonblocking tenant isolations with the same colocation id are not allowed
permutation "s1-insert" "s3-acquire-advisory-lock" "s1-begin" "s1-isolate-tenant" "s2-isolate-tenant" "s3-release-advisory-lock" "s1-commit" "s2-print-cluster" permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s2-isolate-tenant" "s1-isolate-tenant-same-coloc-blocking" "s3-release-advisory-lock" "s2-print-cluster"
// concurrent nonblocking tenant isolations in different transactions are not allowed
permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s2-isolate-tenant" "s1-isolate-tenant-no-same-coloc" "s3-release-advisory-lock" "s2-print-cluster"
// concurrent nonblocking tenant isolations in the same transaction are not allowed
permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s2-begin" "s2-isolate-tenant" "s1-isolate-tenant-no-same-coloc" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"
// concurrent blocking and nonblocking tenant isolations with different colocation ids in different transactions are allowed
permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s2-isolate-tenant" "s1-isolate-tenant-no-same-coloc-blocking" "s3-release-advisory-lock" "s2-print-cluster"
// concurrent blocking and nonblocking tenant isolations with different colocation ids in the same transaction are allowed
permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s2-isolate-tenant" "s1-isolate-tenant-no-same-coloc-blocking" "s3-release-advisory-lock" "s2-print-cluster"

View File

@ -0,0 +1,141 @@
create schema create_distributed_table_concurrently;
set search_path to create_distributed_table_concurrently;
set citus.shard_replication_factor to 1;
-- make sure we have the coordinator in the metadata
SELECT 1 FROM citus_set_coordinator_host('localhost', :master_port);
create table ref (id int primary key);
select create_reference_table('ref');
insert into ref select s from generate_series(0,9) s;
create table test (key text, id int references ref (id) on delete cascade, t timestamptz default now()) partition by range (t);
create table test_1 partition of test for values from ('2022-01-01') to ('2022-12-31');
create table test_2 partition of test for values from ('2023-01-01') to ('2023-12-31');
insert into test (key,id,t) select s,s%10, '2022-01-01'::date + interval '1 year' * (s%2) from generate_series(1,100) s;
create table nocolo (x int, y int);
-- test error conditions
select create_distributed_table_concurrently('test','key', 'append');
select create_distributed_table_concurrently('test','key', 'range');
select create_distributed_table_concurrently('test','noexists', 'hash');
select create_distributed_table_concurrently(0,'key');
select create_distributed_table_concurrently('ref','id');
set citus.shard_replication_factor to 2;
select create_distributed_table_concurrently('test','key', 'hash');
set citus.shard_replication_factor to 1;
begin;
select create_distributed_table_concurrently('test','key');
rollback;
select create_distributed_table_concurrently('test','key'), create_distributed_table_concurrently('test','key');
select create_distributed_table_concurrently('nocolo','x');
select create_distributed_table_concurrently('test','key', colocate_with := 'nocolo');
select create_distributed_table_concurrently('test','key', colocate_with := 'noexists');
-- use colocate_with "default"
select create_distributed_table_concurrently('test','key', shard_count := 11);
select shardcount from pg_dist_partition p join pg_dist_colocation c using (colocationid) where logicalrelid = 'test'::regclass;
select count(*) from pg_dist_shard where logicalrelid = 'test'::regclass;
-- verify queries still work
select count(*) from test;
select key, id from test where key = '1';
select count(*) from test_1;
-- verify that the foreign key to reference table was created
begin;
delete from ref;
select count(*) from test;
rollback;
-- verify that we can undistribute the table
begin;
select undistribute_table('test', cascade_via_foreign_keys := true);
rollback;
-- verify that we can co-locate with create_distributed_table_concurrently
create table test2 (x text primary key, y text);
insert into test2 (x,y) select s,s from generate_series(1,100) s;
select create_distributed_table_concurrently('test2','x', colocate_with := 'test');
-- verify co-located joins work
select count(*) from test join test2 on (key = x);
select id, y from test join test2 on (key = x) where key = '1';
-- verify co-locaed foreign keys work
alter table test add constraint fk foreign key (key) references test2 (x);
-------foreign key tests among different table types--------
-- verify we do not allow foreign keys from reference table to distributed table concurrently
create table ref_table1(id int);
create table dist_table1(id int primary key);
select create_reference_table('ref_table1');
alter table ref_table1 add constraint fkey foreign key (id) references dist_table1(id);
select create_distributed_table_concurrently('dist_table1', 'id');
-- verify we do not allow foreign keys from citus local table to distributed table concurrently
create table citus_local_table1(id int);
select citus_add_local_table_to_metadata('citus_local_table1');
create table dist_table2(id int primary key);
alter table citus_local_table1 add constraint fkey foreign key (id) references dist_table2(id);
select create_distributed_table_concurrently('dist_table2', 'id');
-- verify we do not allow foreign keys from regular table to distributed table concurrently
create table local_table1(id int);
create table dist_table3(id int primary key);
alter table local_table1 add constraint fkey foreign key (id) references dist_table3(id);
select create_distributed_table_concurrently('dist_table3', 'id');
-- verify we allow foreign keys from distributed table to reference table concurrently
create table ref_table2(id int primary key);
select create_reference_table('ref_table2');
create table dist_table4(id int references ref_table2(id));
select create_distributed_table_concurrently('dist_table4', 'id');
insert into ref_table2 select s from generate_series(1,100) s;
insert into dist_table4 select s from generate_series(1,100) s;
select count(*) as total from dist_table4;
-- verify we do not allow foreign keys from distributed table to citus local table concurrently
create table citus_local_table2(id int primary key);
select citus_add_local_table_to_metadata('citus_local_table2');
create table dist_table5(id int references citus_local_table2(id));
select create_distributed_table_concurrently('dist_table5', 'id');
-- verify we do not allow foreign keys from distributed table to regular table concurrently
create table local_table2(id int primary key);
create table dist_table6(id int references local_table2(id));
select create_distributed_table_concurrently('dist_table6', 'id');
-------foreign key tests among different table types--------
-- columnar tests --
-- create table with partitions
create table test_columnar (id int) partition by range (id);
create table test_columnar_1 partition of test_columnar for values from (1) to (51);
create table test_columnar_2 partition of test_columnar for values from (51) to (101) using columnar;
-- load some data
insert into test_columnar (id) select s from generate_series(1,100) s;
-- distribute table
select create_distributed_table_concurrently('test_columnar','id');
-- verify queries still work
select count(*) from test_columnar;
select id from test_columnar where id = 1;
select id from test_columnar where id = 51;
select count(*) from test_columnar_1;
select count(*) from test_columnar_2;
-- columnar tests --
set client_min_messages to warning;
drop schema create_distributed_table_concurrently cascade;

View File

@ -0,0 +1,110 @@
--
-- failure_create_distributed_table_concurrently adds failure tests for creating distributed table concurrently without data.
--
-- due to different libpq versions
-- some warning messages differ
-- between local and CI
SET client_min_messages TO ERROR;
-- setup db
CREATE SCHEMA IF NOT EXISTS create_dist_tbl_con;
SET SEARCH_PATH = create_dist_tbl_con;
SET citus.shard_count TO 2;
SET citus.shard_replication_factor TO 1;
SET citus.max_adaptive_executor_pool_size TO 1;
SELECT pg_backend_pid() as pid \gset
-- make sure coordinator is in the metadata
SELECT citus_set_coordinator_host('localhost', 57636);
-- create table that will be distributed concurrently
CREATE TABLE table_1 (id int PRIMARY KEY);
-- START OF TESTS
SELECT citus.mitmproxy('conn.allow()');
-- failure on shard table creation
SELECT citus.mitmproxy('conn.onQuery(query="CREATE TABLE create_dist_tbl_con.table_1").kill()');
SELECT create_distributed_table_concurrently('table_1', 'id');
-- cancellation on shard table creation
SELECT citus.mitmproxy('conn.onQuery(query="CREATE TABLE create_dist_tbl_con.table_1").cancel(' || :pid || ')');
SELECT create_distributed_table_concurrently('table_1', 'id');
-- failure on table constraints on replica identity creation
SELECT citus.mitmproxy('conn.onQuery(query="ALTER TABLE create_dist_tbl_con.table_1 ADD CONSTRAINT").kill()');
SELECT create_distributed_table_concurrently('table_1', 'id');
-- cancellation on table constraints on replica identity creation
SELECT citus.mitmproxy('conn.onQuery(query="ALTER TABLE create_dist_tbl_con.table_1 ADD CONSTRAINT").cancel(' || :pid || ')');
SELECT create_distributed_table_concurrently('table_1', 'id');
-- failure on subscription creation
SELECT citus.mitmproxy('conn.onQuery(query="CREATE SUBSCRIPTION").kill()');
SELECT create_distributed_table_concurrently('table_1', 'id');
-- cancellation on subscription creation
SELECT citus.mitmproxy('conn.onQuery(query="CREATE SUBSCRIPTION").cancel(' || :pid || ')');
SELECT create_distributed_table_concurrently('table_1', 'id');
-- failure on catching up LSN
SELECT citus.mitmproxy('conn.onQuery(query="SELECT min\(latest_end_lsn\) FROM pg_stat_subscription").kill()');
SELECT create_distributed_table_concurrently('table_1', 'id');
-- cancellation on catching up LSN
SELECT citus.mitmproxy('conn.onQuery(query="SELECT min\(latest_end_lsn\) FROM pg_stat_subscription").cancel(' || :pid || ')');
SELECT create_distributed_table_concurrently('table_1', 'id');
-- Comment out below flaky tests. It is caused by shard split cleanup which does not work properly yet.
-- -- failure on dropping subscription
-- SELECT citus.mitmproxy('conn.onQuery(query="DROP SUBSCRIPTION").kill()');
-- SELECT create_distributed_table_concurrently('table_1', 'id');
-- -- cancellation on dropping subscription
-- SELECT citus.mitmproxy('conn.onQuery(query="DROP SUBSCRIPTION").cancel(' || :pid || ')');
-- SELECT create_distributed_table_concurrently('table_1', 'id');
-- -- failure on dropping old shard
-- SELECT citus.mitmproxy('conn.onQuery(query="DROP TABLE IF EXISTS create_dist_tbl_con.table_1").kill()');
-- SELECT create_distributed_table_concurrently('table_1', 'id');
-- -- cancellation on dropping old shard
-- SELECT citus.mitmproxy('conn.onQuery(query="DROP TABLE IF EXISTS create_dist_tbl_con.table_1").cancel(' || :pid || ')');
-- SELECT create_distributed_table_concurrently('table_1', 'id');
-- failure on transaction begin
SELECT citus.mitmproxy('conn.onQuery(query="BEGIN").kill()');
SELECT create_distributed_table_concurrently('table_1', 'id');
-- failure on transaction begin
SELECT citus.mitmproxy('conn.onQuery(query="BEGIN").cancel(' || :pid || ')');
SELECT create_distributed_table_concurrently('table_1', 'id');
-- failure on transaction commit
SELECT citus.mitmproxy('conn.onQuery(query="COMMIT").kill()');
SELECT create_distributed_table_concurrently('table_1', 'id');
-- failure on transaction commit
SELECT citus.mitmproxy('conn.onQuery(query="COMMIT").cancel(' || :pid || ')');
SELECT create_distributed_table_concurrently('table_1', 'id');
-- failure on prepare transaction
SELECT citus.mitmproxy('conn.onQuery(query="PREPARE TRANSACTION").kill()');
SELECT create_distributed_table_concurrently('table_1', 'id');
-- failure on prepare transaction
SELECT citus.mitmproxy('conn.onQuery(query="PREPARE TRANSACTION").cancel(' || :pid || ')');
SELECT create_distributed_table_concurrently('table_1', 'id');
-- END OF TESTS
SELECT citus.mitmproxy('conn.allow()');
-- Verify that the table can be distributed concurrently after unsuccessful attempts
SELECT create_distributed_table_concurrently('table_1', 'id');
SELECT * FROM pg_dist_shard WHERE logicalrelid = 'table_1'::regclass;
DROP SCHEMA create_dist_tbl_con CASCADE;
SET search_path TO default;
SELECT citus_remove_node('localhost', 57636);

View File

@ -160,6 +160,7 @@ SET citus.enable_binary_protocol = true;
SELECT * from worker_split_copy( SELECT * from worker_split_copy(
81060000, -- source shard id to copy 81060000, -- source shard id to copy
'l_orderkey',
ARRAY[ ARRAY[
-- split copy info for split children 1 -- split copy info for split children 1
ROW(81060015, -- destination shard id ROW(81060015, -- destination shard id
@ -178,6 +179,7 @@ SELECT * from worker_split_copy(
-- BEGIN: Trigger 2-way remote shard split copy. -- BEGIN: Trigger 2-way remote shard split copy.
SELECT * from worker_split_copy( SELECT * from worker_split_copy(
81060000, -- source shard id to copy 81060000, -- source shard id to copy
'l_orderkey',
ARRAY[ ARRAY[
-- split copy info for split children 1 -- split copy info for split children 1
ROW(81060015, -- destination shard id ROW(81060015, -- destination shard id

View File

@ -38,6 +38,7 @@ SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \
-- BEGIN: Test Negative scenario -- BEGIN: Test Negative scenario
SELECT * from worker_split_copy( SELECT * from worker_split_copy(
101, -- Invalid source shard id. 101, -- Invalid source shard id.
'id',
ARRAY[ ARRAY[
-- split copy info for split children 1 -- split copy info for split children 1
ROW(81070015, -- destination shard id ROW(81070015, -- destination shard id
@ -54,26 +55,31 @@ SELECT * from worker_split_copy(
SELECT * from worker_split_copy( SELECT * from worker_split_copy(
81070000, -- source shard id to copy 81070000, -- source shard id to copy
'id',
ARRAY[] -- empty array ARRAY[] -- empty array
); );
SELECT * from worker_split_copy( SELECT * from worker_split_copy(
81070000, -- source shard id to copy 81070000, -- source shard id to copy
'id',
ARRAY[NULL] -- empty array ARRAY[NULL] -- empty array
); );
SELECT * from worker_split_copy( SELECT * from worker_split_copy(
81070000, -- source shard id to copy 81070000, -- source shard id to copy
'id',
ARRAY[NULL::pg_catalog.split_copy_info]-- empty array ARRAY[NULL::pg_catalog.split_copy_info]-- empty array
); );
SELECT * from worker_split_copy( SELECT * from worker_split_copy(
81070000, -- source shard id to copy 81070000, -- source shard id to copy
'id',
ARRAY[ROW(NULL)]-- empty array ARRAY[ROW(NULL)]-- empty array
); );
SELECT * from worker_split_copy( SELECT * from worker_split_copy(
81070000, -- source shard id to copy 81070000, -- source shard id to copy
'id',
ARRAY[ROW(NULL, NULL, NULL, NULL)::pg_catalog.split_copy_info] -- empty array ARRAY[ROW(NULL, NULL, NULL, NULL)::pg_catalog.split_copy_info] -- empty array
); );
-- END: Test Negative scenario -- END: Test Negative scenario
@ -83,6 +89,7 @@ SELECT * from worker_split_copy(
SET citus.enable_binary_protocol = false; SET citus.enable_binary_protocol = false;
SELECT * from worker_split_copy( SELECT * from worker_split_copy(
81070000, -- source shard id to copy 81070000, -- source shard id to copy
'id',
ARRAY[ ARRAY[
-- split copy info for split children 1 -- split copy info for split children 1
ROW(81070015, -- destination shard id ROW(81070015, -- destination shard id

View File

@ -152,6 +152,7 @@ SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \
SET citus.enable_binary_protocol = false; SET citus.enable_binary_protocol = false;
SELECT * from worker_split_copy( SELECT * from worker_split_copy(
81070000, -- source shard id to copy 81070000, -- source shard id to copy
'l_orderkey',
ARRAY[ ARRAY[
-- split copy info for split children 1 -- split copy info for split children 1
ROW(81070015, -- destination shard id ROW(81070015, -- destination shard id
@ -170,6 +171,7 @@ SELECT * from worker_split_copy(
-- BEGIN: Trigger 2-way remote shard split copy. -- BEGIN: Trigger 2-way remote shard split copy.
SELECT * from worker_split_copy( SELECT * from worker_split_copy(
81070000, -- source shard id to copy 81070000, -- source shard id to copy
'l_orderkey',
ARRAY[ ARRAY[
-- split copy info for split children 1 -- split copy info for split children 1
ROW(81070015, -- destination shard id ROW(81070015, -- destination shard id