From 6bb31c5d75a042856e998da2cbee0922047f06ed Mon Sep 17 00:00:00 2001 From: Marco Slot Date: Tue, 30 Aug 2022 14:35:40 +0200 Subject: [PATCH] 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 Co-authored-by: aykutbozkurt --- .../commands/create_distributed_table.c | 836 +++++++++++++++--- .../distributed/commands/foreign_constraint.c | 60 ++ .../distributed/metadata/metadata_cache.c | 43 + .../distributed/metadata/metadata_sync.c | 48 + .../distributed/metadata/metadata_utility.c | 91 ++ .../distributed/metadata/node_metadata.c | 18 + .../citus_split_shard_by_split_points.c | 8 +- .../distributed/operations/isolate_shards.c | 8 +- .../distributed/operations/repair_shards.c | 3 +- .../distributed/operations/shard_split.c | 520 +++++++---- .../operations/worker_split_copy_udf.c | 36 +- .../shardsplit/shardsplit_decoder.c | 6 +- .../distributed/sql/citus--11.0-4--11.1-1.sql | 2 + .../sql/downgrades/citus--11.1-1--11.0-4.sql | 3 + .../11.1-1.sql | 7 + .../latest.sql | 7 + .../11.1-1.sql | 14 + .../latest.sql | 14 + .../sql/udfs/worker_split_copy/11.1-1.sql | 3 +- .../sql/udfs/worker_split_copy/latest.sql | 3 +- src/backend/distributed/utils/array_type.c | 3 +- .../distributed/utils/colocation_utils.c | 154 +++- .../distributed/utils/distribution_column.c | 73 +- .../utils/distribution_column_map.c | 139 +++ .../utils/multi_partitioning_utils.c | 36 +- src/include/distributed/colocation_utils.h | 11 + src/include/distributed/commands.h | 2 + src/include/distributed/distribution_column.h | 2 + src/include/distributed/metadata_cache.h | 1 + src/include/distributed/metadata_sync.h | 1 + src/include/distributed/metadata_utility.h | 4 + .../distributed/multi_partitioning_utils.h | 1 + src/include/distributed/repair_shards.h | 1 + src/include/distributed/resource_lock.h | 7 +- src/include/distributed/shard_split.h | 13 +- .../shardsplit_logical_replication.h | 1 + .../utils/distribution_column_map.h | 32 + src/include/distributed/worker_manager.h | 1 + .../create_distributed_table_concurrently.out | 288 ++++++ ..._create_distributed_table_concurrently.out | 203 +++++ ..._create_distributed_table_concurrently.out | 763 ++++++++++++++++ .../expected/isolation_drop_vs_all.out | 2 +- ...isolation_tenant_isolation_nonblocking.out | 437 +++++++-- src/test/regress/expected/multi_extension.out | 8 +- .../expected/multi_tenant_isolation.out | 2 +- .../multi_tenant_isolation_nonblocking.out | 2 +- .../expected/upgrade_list_citus_objects.out | 6 +- .../worker_split_binary_copy_test.out | 2 + .../expected/worker_split_copy_test.out | 11 +- .../expected/worker_split_text_copy_test.out | 2 + src/test/regress/failure_schedule | 1 + src/test/regress/isolation_schedule | 1 + src/test/regress/multi_1_schedule | 2 +- ...create_distributed_table_concurrently.spec | 221 +++++ ...solation_tenant_isolation_nonblocking.spec | 46 +- .../create_distributed_table_concurrently.sql | 141 +++ ..._create_distributed_table_concurrently.sql | 110 +++ .../sql/worker_split_binary_copy_test.sql | 2 + .../regress/sql/worker_split_copy_test.sql | 7 + .../sql/worker_split_text_copy_test.sql | 2 + 60 files changed, 4062 insertions(+), 409 deletions(-) create mode 100644 src/backend/distributed/sql/udfs/citus_internal_delete_partition_metadata/11.1-1.sql create mode 100644 src/backend/distributed/sql/udfs/citus_internal_delete_partition_metadata/latest.sql create mode 100644 src/backend/distributed/sql/udfs/create_distributed_table_concurrently/11.1-1.sql create mode 100644 src/backend/distributed/sql/udfs/create_distributed_table_concurrently/latest.sql create mode 100644 src/backend/distributed/utils/distribution_column_map.c create mode 100644 src/include/distributed/utils/distribution_column_map.h create mode 100644 src/test/regress/expected/create_distributed_table_concurrently.out create mode 100644 src/test/regress/expected/failure_create_distributed_table_concurrently.out create mode 100644 src/test/regress/expected/isolation_create_distributed_table_concurrently.out create mode 100644 src/test/regress/spec/isolation_create_distributed_table_concurrently.spec create mode 100644 src/test/regress/sql/create_distributed_table_concurrently.sql create mode 100644 src/test/regress/sql/failure_create_distributed_table_concurrently.sql diff --git a/src/backend/distributed/commands/create_distributed_table.c b/src/backend/distributed/commands/create_distributed_table.c index 537f68a73..fd35acce2 100644 --- a/src/backend/distributed/commands/create_distributed_table.c +++ b/src/backend/distributed/commands/create_distributed_table.c @@ -42,6 +42,7 @@ #include "distributed/colocation_utils.h" #include "distributed/commands.h" #include "distributed/deparser.h" +#include "distributed/distributed_execution_locks.h" #include "distributed/distribution_column.h" #include "distributed/listutils.h" #include "distributed/local_executor.h" @@ -59,12 +60,16 @@ #include "distributed/reference_table_utils.h" #include "distributed/relation_access_tracking.h" #include "distributed/remote_commands.h" +#include "distributed/repair_shards.h" #include "distributed/resource_lock.h" +#include "distributed/shard_rebalancer.h" +#include "distributed/shard_split.h" #include "distributed/shared_library_init.h" #include "distributed/shard_rebalancer.h" #include "distributed/worker_protocol.h" #include "distributed/worker_shard_visibility.h" #include "distributed/worker_transaction.h" +#include "distributed/utils/distribution_column_map.h" #include "distributed/version_compat.h" #include "executor/executor.h" #include "executor/spi.h" @@ -76,6 +81,7 @@ #include "parser/parse_node.h" #include "parser/parse_relation.h" #include "parser/parser.h" +#include "postmaster/postmaster.h" #include "storage/lmgr.h" #include "tcop/pquery.h" #include "tcop/tcopprot.h" @@ -93,8 +99,18 @@ #define LOG_PER_TUPLE_AMOUNT 1000000 /* 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, 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, Oid colocatedTableId, bool localTableEmpty); 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, char distributionMethod, uint32 colocationId, char replicationModel, bool viaDeprecatedAPI); -static void EnsureTableCanBeColocatedWith(Oid relationId, char replicationModel, - Oid distributionColumnType, - Oid sourceRelationId); static void EnsureLocalTableEmpty(Oid relationId); static void EnsureRelationHasNoTriggers(Oid relationId); 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 viaDeprecatedAPI); static void EnsureCitusTableCanBeCreated(Oid relationOid); +static void PropagatePrerequisiteObjectsForDistributedTable(Oid relationId); static void EnsureDistributedSequencesHaveOneType(Oid relationId, List *seqInfoList); static List * GetFKeyCreationCommandsRelationInvolvedWithTableType(Oid relationId, @@ -134,9 +148,17 @@ static void DoCopyFromLocalTableIntoShards(Relation distributedRelation, EState *estate); static void ErrorIfTemporaryTable(Oid relationId); 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 */ 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_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 * 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); relationId = DropFKeysAndUndistributeTable(relationId); } - /* * To support foreign keys between reference tables and local tables, * 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); - /* - * Ensure that the sequences used in column defaults of the table - * have proper types - */ - EnsureRelationHasCompatibleSequenceTypes(relationId); + EnsureTableNotDistributed(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)); + PropagatePrerequisiteObjectsForDistributedTable(relationId); char replicationModel = DecideReplicationModel(distributionMethod, colocateWithTableName, @@ -453,7 +1045,7 @@ CreateDistributedTable(Oid relationId, char *distributionColumnName, Var *distributionColumn = BuildDistributionKeyFromColumnName(relationId, distributionColumnName, - ExclusiveLock); + NoLock); /* * 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 * 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); - Relation pgDistColocation = table_open(DistColocationRelationId(), ExclusiveLock); - Oid distributionColumnType = distributionColumn->vartype; 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 */ - colocationId = ColocationId(shardCount, ShardReplicationFactor, - distributionColumnType, - distributionColumnCollation); + AcquireColocationDefaultLock(); + } + 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 - * of the colocated table is actually shardCount + * 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. */ - 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; - } - } - } - - if (colocationId == INVALID_COLOCATION_ID) + ReleaseColocationDefaultLock(); + } + + if (colocationId == INVALID_COLOCATION_ID) + { + if (IsColocateWithDefault(colocateWithTableName)) { + /* + * 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, distributionColumnType, 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; - EnsureTableNotDistributed(relationId); EnsureLocalTableEmptyIfNecessary(relationId, distributionMethod, viaDeprecatedAPI); /* user really wants triggers? */ @@ -1131,13 +1721,13 @@ EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn, } } - if (PartitionTable(relationId)) + if (PartitionTableNoLock(relationId)) { parentRelationId = PartitionParentOid(relationId); } /* 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); @@ -1155,7 +1745,7 @@ EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn, * reach this point because, we call CreateDistributedTable for partitions if their * parent table is distributed. */ - if (PartitionedTable(relationId)) + if (PartitionedTableNoLock(relationId)) { /* we cannot distribute partitioned tables with master_create_distributed_table */ if (viaDeprecatedAPI) @@ -1174,7 +1764,7 @@ EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn, } /* we don't support distributing tables with multi-level partitioning */ - if (PartitionTable(relationId)) + if (PartitionTableNoLock(relationId)) { 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 * 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. */ diff --git a/src/backend/distributed/commands/foreign_constraint.c b/src/backend/distributed/commands/foreign_constraint.c index 62d004792..1cf6a0956 100644 --- a/src/backend/distributed/commands/foreign_constraint.c +++ b/src/backend/distributed/commands/foreign_constraint.c @@ -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 * constraints and errors out if it is not possible to create one of the diff --git a/src/backend/distributed/metadata/metadata_cache.c b/src/backend/distributed/metadata/metadata_cache.c index 39041f897..afce210c8 100644 --- a/src/backend/distributed/metadata/metadata_cache.c +++ b/src/backend/distributed/metadata/metadata_cache.c @@ -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 * not. diff --git a/src/backend/distributed/metadata/metadata_sync.c b/src/backend/distributed/metadata/metadata_sync.c index f7ab4376e..44c5cded2 100644 --- a/src/backend/distributed/metadata/metadata_sync.c +++ b/src/backend/distributed/metadata/metadata_sync.c @@ -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. */ 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_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 * 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 * add a row to pg_dist_shard. diff --git a/src/backend/distributed/metadata/metadata_utility.c b/src/backend/distributed/metadata/metadata_utility.c index 3781eac50..ee7be267e 100644 --- a/src/backend/distributed/metadata/metadata_utility.c +++ b/src/backend/distributed/metadata/metadata_utility.c @@ -39,6 +39,7 @@ #include "distributed/metadata_utility.h" #include "distributed/coordinator_protocol.h" #include "distributed/metadata_cache.h" +#include "distributed/metadata_sync.h" #include "distributed/multi_join_order.h" #include "distributed/multi_logical_optimizer.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 * if not. Superusers always have such permissions. diff --git a/src/backend/distributed/metadata/node_metadata.c b/src/backend/distributed/metadata/node_metadata.c index 5603189c5..3f729eccf 100644 --- a/src/backend/distributed/metadata/node_metadata.c +++ b/src/backend/distributed/metadata/node_metadata.c @@ -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('') " + "to configure the coordinator hostname"))); + } +} + + /* * InsertCoordinatorIfClusterEmpty can be used to ensure Citus tables can be * created even on a node that has just performed CREATE EXTENSION citus; diff --git a/src/backend/distributed/operations/citus_split_shard_by_split_points.c b/src/backend/distributed/operations/citus_split_shard_by_split_points.c index 28093f336..5bdbaf576 100644 --- a/src/backend/distributed/operations/citus_split_shard_by_split_points.c +++ b/src/backend/distributed/operations/citus_split_shard_by_split_points.c @@ -23,6 +23,7 @@ #include "distributed/connection_management.h" #include "distributed/remote_commands.h" #include "distributed/shard_split.h" +#include "distributed/utils/distribution_column_map.h" /* declarations for dynamic loading */ 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); SplitMode shardSplitMode = LookupSplitMode(shardTransferModeOid); + DistributionColumnMap *distributionColumnOverrides = NULL; + List *sourceColocatedShardIntervalList = NIL; SplitShard( shardSplitMode, SHARD_SPLIT_API, shardIdToSplit, shardSplitPointsList, - nodeIdsForPlacementList); + nodeIdsForPlacementList, + distributionColumnOverrides, + sourceColocatedShardIntervalList, + INVALID_COLOCATION_ID); PG_RETURN_VOID(); } diff --git a/src/backend/distributed/operations/isolate_shards.c b/src/backend/distributed/operations/isolate_shards.c index d37ff3bee..c0f7739b8 100644 --- a/src/backend/distributed/operations/isolate_shards.c +++ b/src/backend/distributed/operations/isolate_shards.c @@ -33,6 +33,7 @@ #include "distributed/worker_transaction.h" #include "distributed/version_compat.h" #include "distributed/shard_split.h" +#include "distributed/utils/distribution_column_map.h" #include "nodes/pg_list.h" #include "storage/lock.h" #include "utils/builtins.h" @@ -163,12 +164,17 @@ isolate_tenant_to_new_shard(PG_FUNCTION_ARGS) nodeIdsForPlacementList = lappend_int(nodeIdsForPlacementList, sourceNodeId); } + DistributionColumnMap *distributionColumnOverrides = NULL; + List *sourceColocatedShardIntervalList = NIL; SplitMode splitMode = LookupSplitMode(shardTransferModeOid); SplitShard(splitMode, ISOLATE_TENANT_TO_NEW_SHARD, sourceShard->shardId, shardSplitPointsList, - nodeIdsForPlacementList); + nodeIdsForPlacementList, + distributionColumnOverrides, + sourceColocatedShardIntervalList, + INVALID_COLOCATION_ID); cacheEntry = GetCitusTableCacheEntry(relationId); ShardInterval *newShard = FindShardInterval(tenantIdDatum, cacheEntry); diff --git a/src/backend/distributed/operations/repair_shards.c b/src/backend/distributed/operations/repair_shards.c index a4457d691..193797384 100644 --- a/src/backend/distributed/operations/repair_shards.c +++ b/src/backend/distributed/operations/repair_shards.c @@ -71,7 +71,6 @@ typedef struct ShardCommandList } ShardCommandList; /* local function forward declarations */ -static bool RelationCanPublishAllModifications(Oid relationId); static bool CanUseLogicalReplication(Oid relationId, char shardReplicationMode); static void ErrorIfTableCannotBeReplicated(Oid relationId); static void ErrorIfTargetNodeIsNotSafeToCopyTo(const char *targetNodeName, @@ -635,7 +634,7 @@ VerifyTablesHaveReplicaIdentity(List *colocatedTableList) * RelationCanPublishAllModifications returns true if the relation is safe to publish * all modification while being replicated via logical replication. */ -static bool +bool RelationCanPublishAllModifications(Oid relationId) { Relation relation = RelationIdGetRelation(relationId); diff --git a/src/backend/distributed/operations/shard_split.c b/src/backend/distributed/operations/shard_split.c index f5fdf62ca..bc4ad208b 100644 --- a/src/backend/distributed/operations/shard_split.c +++ b/src/backend/distributed/operations/shard_split.c @@ -30,6 +30,7 @@ #include "distributed/shard_split.h" #include "distributed/reference_table_utils.h" #include "distributed/repair_shards.h" +#include "distributed/resource_lock.h" #include "distributed/multi_partitioning_utils.h" #include "distributed/worker_manager.h" #include "distributed/worker_transaction.h" @@ -37,6 +38,7 @@ #include "distributed/pg_dist_shard.h" #include "distributed/metadata_sync.h" #include "distributed/multi_physical_planner.h" +#include "distributed/utils/distribution_column_map.h" #include "commands/dbcommands.h" #include "distributed/shardsplit_logical_replication.h" #include "distributed/deparse_shard_query.h" @@ -71,12 +73,8 @@ static void ErrorIfCannotSplitShardExtended(SplitOperation splitOperation, ShardInterval *shardIntervalToSplit, List *shardSplitPointsList, List *nodeIdsForPlacementList); -static void CreateAndCopySplitShardsForShardGroup( - HTAB *mapOfShardToPlacementCreatedByWorkflow, - WorkerNode *sourceShardNode, - List *sourceColocatedShardIntervalList, - List *shardGroupSplitIntervalListList, - List *workersForPlacementList); +static void ErrorIfModificationAndSplitInTheSameTransaction(SplitOperation + splitOperation); static void CreateSplitShardsForShardGroup(HTAB *mapOfShardToPlacementCreatedByWorkflow, List *shardGroupSplitIntervalListList, List *workersForPlacementList); @@ -98,23 +96,35 @@ static void CreateSplitIntervalsForShard(ShardInterval *sourceShard, List *splitPointsForShard, List **shardSplitChildrenIntervalList); static void BlockingShardSplit(SplitOperation splitOperation, - ShardInterval *shardIntervalToSplit, + uint64 splitWorkflowId, + List *sourceColocatedShardIntervalList, List *shardSplitPointsList, - List *workersForPlacementList); + List *workersForPlacementList, + DistributionColumnMap *distributionColumnOverrides); static void NonBlockingShardSplit(SplitOperation splitOperation, - ShardInterval *shardIntervalToSplit, + uint64 splitWorkflowId, + List *sourceColocatedShardIntervalList, List *shardSplitPointsList, - List *workersForPlacementList); + List *workersForPlacementList, + DistributionColumnMap *distributionColumnOverrides, + uint32 targetColocationId); static void DoSplitCopy(WorkerNode *sourceShardNode, List *sourceColocatedShardIntervalList, List *shardGroupSplitIntervalListList, List *workersForPlacementList, - char *snapShotName); + char *snapShotName, + DistributionColumnMap *distributionColumnOverrides); static StringInfo CreateSplitCopyCommand(ShardInterval *sourceShardSplitInterval, + char *distributionColumnName, List *splitChildrenShardIntervalList, List *workersForPlacementList); static Task * CreateSplitCopyTask(StringInfo splitCopyUdfCommand, char *snapshotName, int taskId, uint64 jobId); +static void UpdateDistributionColumnsForShardGroup(List *colocatedShardList, + DistributionColumnMap *distCols, + char distributionMethod, + int shardCount, + uint32 colocationId); static void InsertSplitChildrenShardMetadata(List *shardGroupSplitIntervalListList, List *workersForPlacementList); static void CreatePartitioningHierarchy(List *shardGroupSplitIntervalListList, @@ -126,30 +136,38 @@ static HTAB * CreateEmptyMapForShardsCreatedByWorkflow(); static Task * CreateTaskForDDLCommandList(List *ddlCommandList, WorkerNode *workerNode); static StringInfo CreateSplitShardReplicationSetupUDF( List *sourceColocatedShardIntervalList, List *shardGroupSplitIntervalListList, - List *destinationWorkerNodesList); + List *destinationWorkerNodesList, + DistributionColumnMap * + distributionColumnOverrides); static List * ParseReplicationSlotInfoFromResult(PGresult *result); static List * ExecuteSplitShardReplicationSetupUDF(WorkerNode *sourceWorkerNode, List *sourceColocatedShardIntervalList, List *shardGroupSplitIntervalListList, - List *destinationWorkerNodesList); + List *destinationWorkerNodesList, + DistributionColumnMap * + distributionColumnOverrides); static void ExecuteSplitShardReleaseSharedMemory(WorkerNode *sourceWorkerNode); static void AddDummyShardEntryInMap(HTAB *mapOfDummyShards, uint32 targetNodeId, ShardInterval *shardInterval); static void DropDummyShards(HTAB *mapOfDummyShardToPlacement); static void DropDummyShard(MultiConnection *connection, ShardInterval *shardInterval); static uint64 GetNextShardIdForSplitChild(void); +static void AcquireNonblockingSplitLock(Oid relationId); +static List * GetWorkerNodesFromWorkerIds(List *nodeIdsForPlacementList); /* Customize error message strings based on operation type */ static const char *const SplitOperationName[] = { [SHARD_SPLIT_API] = "split", [ISOLATE_TENANT_TO_NEW_SHARD] = "isolate", + [CREATE_DISTRIBUTED_TABLE] = "create" }; static const char *const SplitTargetName[] = { [SHARD_SPLIT_API] = "shard", [ISOLATE_TENANT_TO_NEW_SHARD] = "tenant", + [CREATE_DISTRIBUTED_TABLE] = "distributed table" }; /* Function definitions */ @@ -230,6 +248,12 @@ ErrorIfCannotSplitShardExtended(SplitOperation splitOperation, List *shardSplitPointsList, List *nodeIdsForPlacementList) { + /* we should not perform checks for create distributed table operation */ + if (splitOperation == CREATE_DISTRIBUTED_TABLE) + { + return; + } + CitusTableCacheEntry *cachedTableEntry = GetCitusTableCacheEntry( shardIntervalToSplit->relationId); @@ -352,20 +376,11 @@ ErrorIfCannotSplitShardExtended(SplitOperation splitOperation, /* - * 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. + * ErrorIfModificationAndSplitInTheSameTransaction will error if we detect split operation + * in the same transaction which has modification before. */ -void -SplitShard(SplitMode splitMode, - SplitOperation splitOperation, - uint64 shardIdToSplit, - List *shardSplitPointsList, - List *nodeIdsForPlacementList) +static void +ErrorIfModificationAndSplitInTheSameTransaction(SplitOperation splitOperation) { if (XactModificationLevel > XACT_MODIFICATION_NONE) { @@ -375,13 +390,82 @@ SplitShard(SplitMode splitMode, SplitOperationName[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), errmsg("multiple shard movements/splits via logical " "replication in the same transaction is currently " "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); List *colocatedTableList = ColocatedTableList(shardIntervalToSplit->relationId); @@ -391,6 +475,7 @@ SplitShard(SplitMode splitMode, VerifyTablesHaveReplicaIdentity(colocatedTableList); } + /* Acquire global lock to prevent concurrent split on the same colocation group or relation */ Oid relationId = RelationIdForShard(shardIdToSplit); AcquirePlacementColocationLock(relationId, ExclusiveLock, "split"); @@ -407,47 +492,50 @@ SplitShard(SplitMode splitMode, LockRelationOid(colocatedTableId, ShareUpdateExclusiveLock); } - ErrorIfCannotSplitShard(SHARD_SPLIT_API, shardIntervalToSplit); + ErrorIfCannotSplitShard(splitOperation, shardIntervalToSplit); ErrorIfCannotSplitShardExtended( - SHARD_SPLIT_API, + splitOperation, shardIntervalToSplit, shardSplitPointsList, nodeIdsForPlacementList); - List *workersForPlacementList = NIL; - Datum nodeId; - foreach_int(nodeId, nodeIdsForPlacementList) + List *workersForPlacementList = GetWorkerNodesFromWorkerIds(nodeIdsForPlacementList); + + List *sourceColocatedShardIntervalList = NIL; + if (colocatedShardIntervalList == NIL) { - uint32 nodeIdValue = DatumGetUInt32(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); + sourceColocatedShardIntervalList = ColocatedShardIntervalList( + shardIntervalToSplit); } + else + { + sourceColocatedShardIntervalList = colocatedShardIntervalList; + } + + /* use the user-specified shard ID as the split workflow ID */ + uint64 splitWorkflowId = shardIntervalToSplit->shardId; if (splitMode == BLOCKING_SPLIT) { EnsureReferenceTablesExistOnAllNodesExtended(TRANSFER_MODE_BLOCK_WRITES); BlockingShardSplit( splitOperation, - shardIntervalToSplit, + splitWorkflowId, + sourceColocatedShardIntervalList, shardSplitPointsList, - workersForPlacementList); + workersForPlacementList, + distributionColumnOverrides); } else { NonBlockingShardSplit( splitOperation, - shardIntervalToSplit, + splitWorkflowId, + sourceColocatedShardIntervalList, shardSplitPointsList, - workersForPlacementList); + workersForPlacementList, + distributionColumnOverrides, + targetColocationId); PlacementMovedUsingLogicalReplicationInTX = true; } @@ -521,20 +609,20 @@ CreateEmptyMapForShardsCreatedByWorkflow() /* * SplitShard API to split a given shard (or shard group) in blocking fashion * based on specified split points to a set of destination nodes. - * 'splitOperation' : Customer operation that triggered split. - * 'shardIntervalToSplit' : Source shard interval to be split. - * 'shardSplitPointsList' : Split Points list for the source 'shardInterval'. - * 'workersForPlacementList' : Placement list corresponding to split children. + * 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. */ static void BlockingShardSplit(SplitOperation splitOperation, - ShardInterval *shardIntervalToSplit, + uint64 splitWorkflowId, + List *sourceColocatedShardIntervalList, List *shardSplitPointsList, - List *workersForPlacementList) + List *workersForPlacementList, + DistributionColumnMap *distributionColumnOverrides) { - List *sourceColocatedShardIntervalList = ColocatedShardIntervalList( - shardIntervalToSplit); - BlockWritesToShardList(sourceColocatedShardIntervalList); /* First create shard interval metadata for split children */ @@ -543,29 +631,29 @@ BlockingShardSplit(SplitOperation splitOperation, shardSplitPointsList); /* Only single placement allowed (already validated RelationReplicationFactor = 1) */ - List *sourcePlacementList = ActiveShardPlacementList(shardIntervalToSplit->shardId); - Assert(sourcePlacementList->length == 1); - ShardPlacement *sourceShardPlacement = (ShardPlacement *) linitial( - sourcePlacementList); - WorkerNode *sourceShardToCopyNode = FindNodeWithNodeId(sourceShardPlacement->nodeId, - false /* missingOk */); - + ShardInterval *firstShard = linitial(sourceColocatedShardIntervalList); + WorkerNode *sourceShardNode = + ActiveShardPlacementWorkerNode(firstShard->shardId); HTAB *mapOfShardToPlacementCreatedByWorkflow = CreateEmptyMapForShardsCreatedByWorkflow(); PG_TRY(); { - /* - * Physically create split children, perform split copy and create auxiliary structures. - * This includes: indexes, replicaIdentity. triggers and statistics. - * Foreign key constraints are created after Metadata changes (see CreateForeignKeyConstraints). - */ - CreateAndCopySplitShardsForShardGroup( - mapOfShardToPlacementCreatedByWorkflow, - sourceShardToCopyNode, - sourceColocatedShardIntervalList, - shardGroupSplitIntervalListList, - workersForPlacementList); + /* Physically create split children. */ + CreateSplitShardsForShardGroup(mapOfShardToPlacementCreatedByWorkflow, + shardGroupSplitIntervalListList, + workersForPlacementList); + + /* For Blocking split, copy isn't snapshotted */ + char *snapshotName = NULL; + DoSplitCopy(sourceShardNode, sourceColocatedShardIntervalList, + shardGroupSplitIntervalListList, workersForPlacementList, + snapshotName, distributionColumnOverrides); + + /* Create auxiliary structures (indexes, stats, replicaindentities, triggers) */ + CreateAuxiliaryStructuresForShardGroup(shardGroupSplitIntervalListList, + workersForPlacementList, + true /* includeReplicaIdentity*/); /* * Up to this point, we performed various subtransactions that may @@ -608,6 +696,7 @@ BlockingShardSplit(SplitOperation splitOperation, } PG_END_TRY(); + 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. * 'sourceShardNode' : Source shard worker node. @@ -768,7 +830,7 @@ CreateAndCopySplitShardsForShardGroup(HTAB *mapOfShardToPlacementCreatedByWorkfl static void DoSplitCopy(WorkerNode *sourceShardNode, List *sourceColocatedShardIntervalList, List *shardGroupSplitIntervalListList, List *destinationWorkerNodesList, - char *snapShotName) + char *snapShotName, DistributionColumnMap *distributionColumnOverrides) { ShardInterval *sourceShardIntervalToCopy = NULL; List *splitShardIntervalList = NIL; @@ -783,25 +845,40 @@ DoSplitCopy(WorkerNode *sourceShardNode, List *sourceColocatedShardIntervalList, * data themselves. Their partitions do contain data, but those are * different colocated shards that will be copied seperately. */ - if (!PartitionedTable(sourceShardIntervalToCopy->relationId)) + if (PartitionedTable(sourceShardIntervalToCopy->relationId)) { - StringInfo splitCopyUdfCommand = CreateSplitCopyCommand( - 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++; + continue; } + + 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, @@ -834,6 +911,7 @@ DoSplitCopy(WorkerNode *sourceShardNode, List *sourceColocatedShardIntervalList, */ static StringInfo CreateSplitCopyCommand(ShardInterval *sourceShardSplitInterval, + char *distributionColumnName, List *splitChildrenShardIntervalList, List *destinationWorkerNodesList) { @@ -865,8 +943,9 @@ CreateSplitCopyCommand(ShardInterval *sourceShardSplitInterval, appendStringInfo(splitCopyInfoArray, "]"); 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, + quote_literal_cstr(distributionColumnName), splitCopyInfoArray->data); return splitCopyUdf; @@ -976,8 +1055,19 @@ CreateSplitIntervalsForShard(ShardInterval *sourceShard, int shardIntervalCount = list_length(splitPointsForShard) + 1; ListCell *splitPointCell = list_head(splitPointsForShard); int32 splitParentMaxValue = DatumGetInt32(sourceShard->maxValue); - 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++) { 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. * 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 - * based on specified split points to a set of destination nodes. - * splitOperation : Customer operation that triggered split. - * shardIntervalToSplit : Source shard interval to be split. - * shardSplitPointsList : Split Points list for the source 'shardInterval'. - * workersForPlacementList : Placement list corresponding to split children. + * AcquireNonblockingSplitLock does not allow concurrent nonblocking splits, because we share memory and + * replication slots. */ static void -NonBlockingShardSplit(SplitOperation splitOperation, - ShardInterval *shardIntervalToSplit, - List *shardSplitPointsList, - List *workersForPlacementList) +AcquireNonblockingSplitLock(Oid relationId) { + 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 *databaseName = get_database_name(MyDatabaseId); - List *sourceColocatedShardIntervalList = ColocatedShardIntervalList( - shardIntervalToSplit); - /* First create shard interval metadata for split children */ List *shardGroupSplitIntervalListList = CreateSplitIntervalsForShardGroup( sourceColocatedShardIntervalList, shardSplitPointsList); - WorkerNode *sourceShardToCopyNode = ActiveShardPlacementWorkerNode( - shardIntervalToSplit->shardId); + ShardInterval *firstShard = linitial(sourceColocatedShardIntervalList); + + /* 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 */ HTAB *publicationInfoHash = CreateShardSplitInfoMapForPublication( @@ -1352,13 +1531,13 @@ NonBlockingShardSplit(SplitOperation splitOperation, /* 4) Create Publications. */ CreatePublications(sourceConnection, publicationInfoHash); - /* 5) Execute 'worker_split_shard_replication_setup UDF */ List *replicationSlotInfoList = ExecuteSplitShardReplicationSetupUDF( sourceShardToCopyNode, sourceColocatedShardIntervalList, shardGroupSplitIntervalListList, - workersForPlacementList); + workersForPlacementList, + distributionColumnOverrides); /* * Subscriber flow starts from here. @@ -1400,7 +1579,7 @@ NonBlockingShardSplit(SplitOperation splitOperation, /* 8) Do snapshotted Copy */ DoSplitCopy(sourceShardToCopyNode, sourceColocatedShardIntervalList, shardGroupSplitIntervalListList, workersForPlacementList, - snapshot); + snapshot, distributionColumnOverrides); /* * 9) Create replica identities, this needs to be done before enabling @@ -1446,7 +1625,6 @@ NonBlockingShardSplit(SplitOperation splitOperation, /* 19) Drop Publications */ DropPublications(sourceConnection, publicationInfoHash); - /* * 20) Drop old shards and delete related metadata. Have to do that before * creating the new shard metadata, because there's cross-checks @@ -1454,7 +1632,40 @@ NonBlockingShardSplit(SplitOperation splitOperation, */ 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, workersForPlacementList); @@ -1462,7 +1673,7 @@ NonBlockingShardSplit(SplitOperation splitOperation, 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 * key creation depends on the new metadata. */ @@ -1470,7 +1681,7 @@ NonBlockingShardSplit(SplitOperation splitOperation, workersForPlacementList); /* - * 23) Drop dummy shards. + * 24) Drop dummy shards. */ DropDummyShards(mapOfDummyShardToPlacement); @@ -1654,12 +1865,14 @@ static List * ExecuteSplitShardReplicationSetupUDF(WorkerNode *sourceWorkerNode, List *sourceColocatedShardIntervalList, List *shardGroupSplitIntervalListList, - List *destinationWorkerNodesList) + List *destinationWorkerNodesList, + DistributionColumnMap *distributionColumnOverrides) { StringInfo splitShardReplicationUDF = CreateSplitShardReplicationSetupUDF( sourceColocatedShardIntervalList, shardGroupSplitIntervalListList, - destinationWorkerNodesList); + destinationWorkerNodesList, + distributionColumnOverrides); /* Force a new connection to execute the UDF */ int connectionFlags = 0; @@ -1756,7 +1969,8 @@ ExecuteSplitShardReleaseSharedMemory(WorkerNode *sourceWorkerNode) StringInfo CreateSplitShardReplicationSetupUDF(List *sourceColocatedShardIntervalList, List *shardGroupSplitIntervalListList, - List *destinationWorkerNodesList) + List *destinationWorkerNodesList, + DistributionColumnMap *distributionColumnOverrides) { StringInfo splitChildrenRows = makeStringInfo(); @@ -1768,11 +1982,15 @@ CreateSplitShardReplicationSetupUDF(List *sourceColocatedShardIntervalList, { int64 sourceShardId = sourceShardIntervalToCopy->shardId; Oid relationId = sourceShardIntervalToCopy->relationId; - Var *partitionColumn = DistPartitionKey(relationId); + + Var *distributionColumn = + GetDistributionColumnWithOverrides(relationId, + distributionColumnOverrides); bool missingOK = false; - char *partitionColumnName = - get_attname(relationId, partitionColumn->varattno, missingOK); + char *distributionColumnName = + get_attname(relationId, distributionColumn->varattno, + missingOK); ShardInterval *splitChildShardInterval = NULL; WorkerNode *destinationWorkerNode = NULL; @@ -1795,7 +2013,7 @@ CreateSplitShardReplicationSetupUDF(List *sourceColocatedShardIntervalList, appendStringInfo(splitChildrenRows, "ROW(%lu, %s, %lu, %s, %s, %u)::pg_catalog.split_shard_info", sourceShardId, - quote_literal_cstr(partitionColumnName), + quote_literal_cstr(distributionColumnName), splitChildShardInterval->shardId, quote_literal_cstr(minValueString->data), quote_literal_cstr(maxValueString->data), diff --git a/src/backend/distributed/operations/worker_split_copy_udf.c b/src/backend/distributed/operations/worker_split_copy_udf.c index 2b33654f9..fe20aeca2 100644 --- a/src/backend/distributed/operations/worker_split_copy_udf.c +++ b/src/backend/distributed/operations/worker_split_copy_udf.c @@ -9,15 +9,16 @@ #include "postgres.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/array.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); @@ -38,6 +39,7 @@ static DestReceiver ** CreateShardCopyDestReceivers(EState *estate, static DestReceiver * CreatePartitionedSplitCopyDestReceiver(EState *executor, ShardInterval * shardIntervalToSplitCopy, + char *partitionColumnName, List *splitCopyInfoList); static void BuildMinMaxRangeArrays(List *splitCopyInfoList, ArrayType **minValueArray, ArrayType **maxValueArray); @@ -54,7 +56,10 @@ worker_split_copy(PG_FUNCTION_ARGS) uint64 shardIdToSplitCopy = DatumGetUInt64(PG_GETARG_DATUM(0)); 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); if (arrayHasNull) { @@ -82,6 +87,7 @@ worker_split_copy(PG_FUNCTION_ARGS) EState *executor = CreateExecutorState(); DestReceiver *splitCopyDestReceiver = CreatePartitionedSplitCopyDestReceiver(executor, shardIntervalToSplitCopy, + partitionColumnName, splitCopyInfoList); Oid sourceShardToCopySchemaOId = get_rel_namespace( @@ -228,6 +234,7 @@ CreateShardCopyDestReceivers(EState *estate, ShardInterval *shardIntervalToSplit static DestReceiver * CreatePartitionedSplitCopyDestReceiver(EState *estate, ShardInterval *shardIntervalToSplitCopy, + char *partitionColumnName, List *splitCopyInfoList) { /* Create underlying ShardCopyDestReceivers */ @@ -240,10 +247,17 @@ CreatePartitionedSplitCopyDestReceiver(EState *estate, ArrayType *minValuesArray = NULL; ArrayType *maxValuesArray = NULL; BuildMinMaxRangeArrays(splitCopyInfoList, &minValuesArray, &maxValuesArray); - CitusTableCacheEntry *cacheEntry = GetCitusTableCacheEntry( - shardIntervalToSplitCopy->relationId); - char partitionMethod = cacheEntry->partitionMethod; - Var *partitionColumn = cacheEntry->partitionColumn; + + /* we currently only support hash-distribution */ + char partitionMethod = DISTRIBUTE_BY_HASH; + + /* 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 = QueryTupleShardSearchInfo(minValuesArray, maxValuesArray, diff --git a/src/backend/distributed/shardsplit/shardsplit_decoder.c b/src/backend/distributed/shardsplit/shardsplit_decoder.c index f6de28ca3..59fb3118f 100644 --- a/src/backend/distributed/shardsplit/shardsplit_decoder.c +++ b/src/backend/distributed/shardsplit/shardsplit_decoder.c @@ -14,6 +14,7 @@ #include "replication/logical.h" #include "utils/typcache.h" + extern void _PG_output_plugin_init(OutputPluginCallbacks *cb); static LogicalDecodeChangeCB pgoutputChangeCB; @@ -216,8 +217,9 @@ GetHashValueForIncomingTuple(Relation sourceShardRelation, TYPECACHE_HASH_PROC_FINFO); /* get hashed value of the distribution value */ - Datum hashedValueDatum = FunctionCall1(&(typeEntry->hash_proc_finfo), - partitionColumnValue); + Datum hashedValueDatum = FunctionCall1Coll(&(typeEntry->hash_proc_finfo), + typeEntry->typcollation, + partitionColumnValue); return DatumGetInt32(hashedValueDatum); } diff --git a/src/backend/distributed/sql/citus--11.0-4--11.1-1.sql b/src/backend/distributed/sql/citus--11.0-4--11.1-1.sql index 98469b614..27c1e69d8 100644 --- a/src/backend/distributed/sql/citus--11.0-4--11.1-1.sql +++ b/src/backend/distributed/sql/citus--11.0-4--11.1-1.sql @@ -1,4 +1,6 @@ #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_cleanup_job_schema_cache(); diff --git a/src/backend/distributed/sql/downgrades/citus--11.1-1--11.0-4.sql b/src/backend/distributed/sql/downgrades/citus--11.1-1--11.0-4.sql index d37451583..c2ec9c482 100644 --- a/src/backend/distributed/sql/downgrades/citus--11.1-1--11.0-4.sql +++ b/src/backend/distributed/sql/downgrades/citus--11.1-1--11.0-4.sql @@ -70,6 +70,7 @@ DROP FUNCTION pg_catalog.citus_split_shard_by_split_points( shard_transfer_mode citus.shard_transfer_mode); DROP FUNCTION pg_catalog.worker_split_copy( source_shard_id bigint, + distribution_column text, splitCopyInfos 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); #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); diff --git a/src/backend/distributed/sql/udfs/citus_internal_delete_partition_metadata/11.1-1.sql b/src/backend/distributed/sql/udfs/citus_internal_delete_partition_metadata/11.1-1.sql new file mode 100644 index 000000000..c7cb5455d --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_internal_delete_partition_metadata/11.1-1.sql @@ -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'; + diff --git a/src/backend/distributed/sql/udfs/citus_internal_delete_partition_metadata/latest.sql b/src/backend/distributed/sql/udfs/citus_internal_delete_partition_metadata/latest.sql new file mode 100644 index 000000000..c7cb5455d --- /dev/null +++ b/src/backend/distributed/sql/udfs/citus_internal_delete_partition_metadata/latest.sql @@ -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'; + diff --git a/src/backend/distributed/sql/udfs/create_distributed_table_concurrently/11.1-1.sql b/src/backend/distributed/sql/udfs/create_distributed_table_concurrently/11.1-1.sql new file mode 100644 index 000000000..721f44990 --- /dev/null +++ b/src/backend/distributed/sql/udfs/create_distributed_table_concurrently/11.1-1.sql @@ -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'; diff --git a/src/backend/distributed/sql/udfs/create_distributed_table_concurrently/latest.sql b/src/backend/distributed/sql/udfs/create_distributed_table_concurrently/latest.sql new file mode 100644 index 000000000..721f44990 --- /dev/null +++ b/src/backend/distributed/sql/udfs/create_distributed_table_concurrently/latest.sql @@ -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'; diff --git a/src/backend/distributed/sql/udfs/worker_split_copy/11.1-1.sql b/src/backend/distributed/sql/udfs/worker_split_copy/11.1-1.sql index 0ecad4a07..d375af964 100644 --- a/src/backend/distributed/sql/udfs/worker_split_copy/11.1-1.sql +++ b/src/backend/distributed/sql/udfs/worker_split_copy/11.1-1.sql @@ -14,9 +14,10 @@ ALTER TYPE citus.split_copy_info SET SCHEMA pg_catalog; CREATE OR REPLACE FUNCTION pg_catalog.worker_split_copy( source_shard_id bigint, + distribution_column text, splitCopyInfos pg_catalog.split_copy_info[]) RETURNS void LANGUAGE C STRICT 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'; diff --git a/src/backend/distributed/sql/udfs/worker_split_copy/latest.sql b/src/backend/distributed/sql/udfs/worker_split_copy/latest.sql index 0ecad4a07..d375af964 100644 --- a/src/backend/distributed/sql/udfs/worker_split_copy/latest.sql +++ b/src/backend/distributed/sql/udfs/worker_split_copy/latest.sql @@ -14,9 +14,10 @@ ALTER TYPE citus.split_copy_info SET SCHEMA pg_catalog; CREATE OR REPLACE FUNCTION pg_catalog.worker_split_copy( source_shard_id bigint, + distribution_column text, splitCopyInfos pg_catalog.split_copy_info[]) RETURNS void LANGUAGE C STRICT 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'; diff --git a/src/backend/distributed/utils/array_type.c b/src/backend/distributed/utils/array_type.c index b214d2ee7..348b25b4a 100644 --- a/src/backend/distributed/utils/array_type.c +++ b/src/backend/distributed/utils/array_type.c @@ -114,7 +114,8 @@ IntegerArrayTypeToList(ArrayType *arrayObject) 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; diff --git a/src/backend/distributed/utils/colocation_utils.c b/src/backend/distributed/utils/colocation_utils.c index 005f3aa62..bb9488af1 100644 --- a/src/backend/distributed/utils/colocation_utils.c +++ b/src/backend/distributed/utils/colocation_utils.c @@ -53,6 +53,7 @@ static void DeleteColocationGroup(uint32 colocationId); static uint32 CreateColocationGroupForRelation(Oid sourceRelationId); static void BreakColocation(Oid sourceRelationId); + /* exports for SQL callable functions */ PG_FUNCTION_INFO_V1(mark_tables_colocated); 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. * 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 * 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 */ CommandCounterIncrement(); - table_close(pgDistColocation, RowExclusiveLock); + table_close(pgDistColocation, NoLock); } @@ -1271,5 +1316,110 @@ DeleteColocationGroupLocally(uint32 colocationId) } 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))); + } } diff --git a/src/backend/distributed/utils/distribution_column.c b/src/backend/distributed/utils/distribution_column.c index 5b59bd529..7fbab98eb 100644 --- a/src/backend/distributed/utils/distribution_column.c +++ b/src/backend/distributed/utils/distribution_column.c @@ -24,6 +24,7 @@ #include "nodes/nodes.h" #include "nodes/primnodes.h" #include "parser/scansup.h" +#include "parser/parse_relation.h" #include "utils/builtins.h" #include "utils/elog.h" #include "utils/errcodes.h" @@ -123,7 +124,7 @@ column_to_column_name(PG_FUNCTION_ARGS) Var * BuildDistributionKeyFromColumnName(Oid relationId, char *columnName, LOCKMODE lockMode) { - Relation relation = try_relation_open(relationId, ExclusiveLock); + Relation relation = try_relation_open(relationId, lockMode); 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 * relation identifier and the column's internal (Var) representation. diff --git a/src/backend/distributed/utils/distribution_column_map.c b/src/backend/distributed/utils/distribution_column_map.c new file mode 100644 index 000000000..c3c0db01f --- /dev/null +++ b/src/backend/distributed/utils/distribution_column_map.c @@ -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); +} diff --git a/src/backend/distributed/utils/multi_partitioning_utils.c b/src/backend/distributed/utils/multi_partitioning_utils.c index 3ec02da48..c5fcd2377 100644 --- a/src/backend/distributed/utils/multi_partitioning_utils.c +++ b/src/backend/distributed/utils/multi_partitioning_utils.c @@ -996,11 +996,19 @@ IsParentTable(Oid relationId) systable_endscan(scan); 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; } + relation_close(relation, AccessShareLock); + return tableInherited; } @@ -1291,3 +1299,29 @@ PartitionBound(Oid partitionId) 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; +} diff --git a/src/include/distributed/colocation_utils.h b/src/include/distributed/colocation_utils.h index 0095ac427..9e6641cd3 100644 --- a/src/include/distributed/colocation_utils.h +++ b/src/include/distributed/colocation_utils.h @@ -36,6 +36,7 @@ extern void InsertColocationGroupLocally(uint32 colocationId, int shardCount, Oid distributionColumnType, Oid distributionColumnCollation); extern bool IsColocateWithNone(char *colocateWithTableName); +extern bool IsColocateWithDefault(char *colocateWithTableName); extern uint32 GetNextColocationId(void); extern void ErrorIfShardPlacementsNotColocated(Oid leftRelationId, Oid rightRelationId); extern void CheckReplicationModel(Oid sourceRelationId, Oid targetRelationId); @@ -48,5 +49,15 @@ extern void UpdateRelationColocationGroup(Oid distributedRelationId, uint32 colo extern void DeleteColocationGroupIfNoTablesBelong(uint32 colocationId); extern List * ColocationGroupTableList(uint32 colocationId, uint32 count); 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_ */ diff --git a/src/include/distributed/commands.h b/src/include/distributed/commands.h index b3ddba31d..b7030adee 100644 --- a/src/include/distributed/commands.h +++ b/src/include/distributed/commands.h @@ -258,6 +258,8 @@ extern void ErrorIfUnsupportedForeignConstraintExists(Relation relation, char distributionMethod, Var *distributionColumn, uint32 colocationId); +extern void EnsureNoFKeyFromTableType(Oid relationId, int tableTypeFlag); +extern void EnsureNoFKeyToTableType(Oid relationId, int tableTypeFlag); extern void ErrorOutForFKeyBetweenPostgresAndCitusLocalTable(Oid localTableId); extern bool ColumnReferencedByAnyForeignKey(char *columnName, Oid relationId); extern bool ColumnAppearsInForeignKey(char *columnName, Oid relationId); diff --git a/src/include/distributed/distribution_column.h b/src/include/distributed/distribution_column.h index ced1be9a7..a7ec6a593 100644 --- a/src/include/distributed/distribution_column.h +++ b/src/include/distributed/distribution_column.h @@ -23,5 +23,7 @@ extern Var * BuildDistributionKeyFromColumnName(Oid relationId, char *columnName, LOCKMODE lockMode); 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 */ diff --git a/src/include/distributed/metadata_cache.h b/src/include/distributed/metadata_cache.h index 9a573a7c9..1a5939879 100644 --- a/src/include/distributed/metadata_cache.h +++ b/src/include/distributed/metadata_cache.h @@ -143,6 +143,7 @@ extern List * AllCitusTableIds(void); extern bool IsCitusTableType(Oid relationId, CitusTableType tableType); extern bool IsCitusTableTypeCacheEntry(CitusTableCacheEntry *tableEtnry, CitusTableType tableType); +extern char * GetTableTypeName(Oid tableId); extern void SetCreateCitusTransactionLevel(int val); extern int GetCitusCreationLevel(void); diff --git a/src/include/distributed/metadata_sync.h b/src/include/distributed/metadata_sync.h index e539f5f61..eb64f14fa 100644 --- a/src/include/distributed/metadata_sync.h +++ b/src/include/distributed/metadata_sync.h @@ -69,6 +69,7 @@ extern char * MarkObjectsDistributedCreateCommand(List *addresses, extern char * DistributionCreateCommand(CitusTableCacheEntry *cacheEntry); extern char * DistributionDeleteCommand(const char *schemaName, const char *tableName); +extern char * DistributionDeleteMetadataCommand(Oid relationId); extern char * TableOwnerResetCommand(Oid distributedRelationId); extern char * NodeListInsertCommand(List *workerNodeList); extern List * ShardListInsertCommand(List *shardIntervalList); diff --git a/src/include/distributed/metadata_utility.h b/src/include/distributed/metadata_utility.h index 7d9d49646..793790fe0 100644 --- a/src/include/distributed/metadata_utility.h +++ b/src/include/distributed/metadata_utility.h @@ -248,6 +248,10 @@ extern void InsertIntoPgDistPartition(Oid relationId, char distributionMethod, Var *distributionColumn, uint32 colocationId, char replicationModel, 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 DeleteShardRow(uint64 shardId); extern void UpdateShardPlacementState(uint64 placementId, char shardState); diff --git a/src/include/distributed/multi_partitioning_utils.h b/src/include/distributed/multi_partitioning_utils.h index 9f905b19d..b8cfe38c0 100644 --- a/src/include/distributed/multi_partitioning_utils.h +++ b/src/include/distributed/multi_partitioning_utils.h @@ -30,5 +30,6 @@ extern char * GeneratePartitioningInformation(Oid tableId); extern void FixPartitionConstraintsOnWorkers(Oid relationId); extern void FixLocalPartitionConstraints(Oid relationId, int64 shardId); extern void FixPartitionShardIndexNames(Oid relationId, Oid parentIndexOid); +extern List * ListShardsUnderParentRelation(Oid relationId); #endif /* MULTI_PARTITIONING_UTILS_H_ */ diff --git a/src/include/distributed/repair_shards.h b/src/include/distributed/repair_shards.h index fa0d76190..eb845adc2 100644 --- a/src/include/distributed/repair_shards.h +++ b/src/include/distributed/repair_shards.h @@ -17,3 +17,4 @@ extern void ErrorIfMoveUnsupportedTableType(Oid relationId); extern void CopyShardsToNode(WorkerNode *sourceNode, WorkerNode *targetNode, List *shardIntervalList, char *snapshotName); extern void VerifyTablesHaveReplicaIdentity(List *colocatedTableList); +extern bool RelationCanPublishAllModifications(Oid relationId); diff --git a/src/include/distributed/resource_lock.h b/src/include/distributed/resource_lock.h index c808e9157..f2a331451 100644 --- a/src/include/distributed/resource_lock.h +++ b/src/include/distributed/resource_lock.h @@ -48,7 +48,10 @@ typedef enum AdvisoryLocktagClass /* CitusOperations has constants for citus operations */ 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; /* 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 List * GetSortedReferenceShardIntervals(List *relationList); +void AcquireCreateDistributedTableConcurrentlyLock(Oid relationId); + /* Lock parent table's colocated shard resource */ extern void LockParentShardResourceIfPartition(List *shardIntervalList, LOCKMODE lockMode); diff --git a/src/include/distributed/shard_split.h b/src/include/distributed/shard_split.h index 7464534b7..2154ff446 100644 --- a/src/include/distributed/shard_split.h +++ b/src/include/distributed/shard_split.h @@ -12,6 +12,8 @@ #ifndef SHARDSPLIT_H_ #define SHARDSPLIT_H_ +#include "distributed/utils/distribution_column_map.h" + /* Split Modes supported by Shard Split API */ typedef enum SplitMode { @@ -28,10 +30,10 @@ typedef enum SplitMode typedef enum SplitOperation { SHARD_SPLIT_API = 0, - ISOLATE_TENANT_TO_NEW_SHARD + ISOLATE_TENANT_TO_NEW_SHARD, + CREATE_DISTRIBUTED_TABLE } SplitOperation; - /* * SplitShard API to split a given shard (or shard group) using split mode and * specified split points to a set of destination nodes. @@ -40,10 +42,15 @@ extern void SplitShard(SplitMode splitMode, SplitOperation splitOperation, uint64 shardIdToSplit, List *shardSplitPointsList, - List *nodeIdsForPlacementList); + List *nodeIdsForPlacementList, + DistributionColumnMap *distributionColumnOverrides, + List *colocatedShardIntervalList, + uint32 targetColocationId); extern void DropShardList(List *shardIntervalList); extern SplitMode LookupSplitMode(Oid shardTransferModeOid); +extern void ErrorIfMultipleNonblockingMoveSplitInTheSameTransaction(void); + #endif /* SHARDSPLIT_H_ */ diff --git a/src/include/distributed/shardsplit_logical_replication.h b/src/include/distributed/shardsplit_logical_replication.h index b47861369..a7dc3485e 100644 --- a/src/include/distributed/shardsplit_logical_replication.h +++ b/src/include/distributed/shardsplit_logical_replication.h @@ -47,4 +47,5 @@ extern void DropShardSplitPublications(MultiConnection *sourceConnection, extern void DropShardSplitSubsriptions(List *shardSplitSubscribersMetadataList); extern void DropShardSplitReplicationSlots(MultiConnection *sourceConnection, List *replicationSlotInfoList); + #endif /* SHARDSPLIT_LOGICAL_REPLICATION_H */ diff --git a/src/include/distributed/utils/distribution_column_map.h b/src/include/distributed/utils/distribution_column_map.h new file mode 100644 index 000000000..1b671c88b --- /dev/null +++ b/src/include/distributed/utils/distribution_column_map.h @@ -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 */ diff --git a/src/include/distributed/worker_manager.h b/src/include/distributed/worker_manager.h index 1c5b27f1f..4cc3e5c6f 100644 --- a/src/include/distributed/worker_manager.h +++ b/src/include/distributed/worker_manager.h @@ -90,6 +90,7 @@ extern WorkerNode * FindWorkerNodeAnyCluster(const char *nodeName, int32 nodePor extern WorkerNode * FindNodeWithNodeId(int nodeId, bool missingOk); extern List * ReadDistNode(bool includeNodesFromOtherClusters); extern void EnsureCoordinator(void); +extern void EnsureCoordinatorIsInMetadata(void); extern void InsertCoordinatorIfClusterEmpty(void); extern uint32 GroupForNode(char *nodeName, int32 nodePort); extern WorkerNode * PrimaryNodeForGroup(int32 groupId, bool *groupContainsNodes); diff --git a/src/test/regress/expected/create_distributed_table_concurrently.out b/src/test/regress/expected/create_distributed_table_concurrently.out new file mode 100644 index 000000000..19298a39d --- /dev/null +++ b/src/test/regress/expected/create_distributed_table_concurrently.out @@ -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; diff --git a/src/test/regress/expected/failure_create_distributed_table_concurrently.out b/src/test/regress/expected/failure_create_distributed_table_concurrently.out new file mode 100644 index 000000000..9c5d211e9 --- /dev/null +++ b/src/test/regress/expected/failure_create_distributed_table_concurrently.out @@ -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) + diff --git a/src/test/regress/expected/isolation_create_distributed_table_concurrently.out b/src/test/regress/expected/isolation_create_distributed_table_concurrently.out new file mode 100644 index 000000000..bf092851e --- /dev/null +++ b/src/test/regress/expected/isolation_create_distributed_table_concurrently.out @@ -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'); + +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'); + +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'); + +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'); + +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'); + +step s2-insert: + INSERT INTO table_1 SELECT s FROM generate_series(1,20) s; + +step s2-reindex-concurrently: + REINDEX TABLE CONCURRENTLY table_1; + +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'); + +step s2-insert: + INSERT INTO table_1 SELECT s FROM generate_series(1,20) s; + +step s2-reindex: + REINDEX TABLE table_1; + +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'); + +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'); + +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'); + +step s2-create-table_1: + SELECT create_distributed_table('table_1', 'id'); + +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'); + +step s2-create-table_2: + SELECT create_distributed_table('table_2', 'id'); + +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'); + +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'); + +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'); + +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'); + +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) + diff --git a/src/test/regress/expected/isolation_drop_vs_all.out b/src/test/regress/expected/isolation_drop_vs_all.out index ec7e4fe24..7dab13615 100644 --- a/src/test/regress/expected/isolation_drop_vs_all.out +++ b/src/test/regress/expected/isolation_drop_vs_all.out @@ -247,7 +247,7 @@ step s1-drop: DROP TABLE drop_hash; step s2-distribute-table: SELECT create_distributed_table('drop_hash', 'id'); step s1-commit: COMMIT; 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 s1-select-count: SELECT COUNT(*) FROM drop_hash; ERROR: relation "drop_hash" does not exist diff --git a/src/test/regress/expected/isolation_tenant_isolation_nonblocking.out b/src/test/regress/expected/isolation_tenant_isolation_nonblocking.out index cd6b9fa22..7d8991615 100644 --- a/src/test/regress/expected/isolation_tenant_isolation_nonblocking.out +++ b/src/test/regress/expected/isolation_tenant_isolation_nonblocking.out @@ -8,9 +8,11 @@ create_distributed_table 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); @@ -57,7 +59,7 @@ t step s2-isolate-tenant: <... completed> isolate_tenant_to_new_shard --------------------------------------------------------------------- - 1500076 + 1500078 (1 row) step s2-commit: @@ -76,9 +78,9 @@ step s2-print-cluster: nodeport|shardid|success|result --------------------------------------------------------------------- - 57637|1500075|t | 0 - 57637|1500076|t | 1 57637|1500077|t | 0 + 57637|1500078|t | 1 + 57637|1500079|t | 0 57638|1500074|t | 0 (4 rows) @@ -96,9 +98,11 @@ create_distributed_table 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); @@ -145,7 +149,7 @@ t step s2-isolate-tenant: <... completed> isolate_tenant_to_new_shard --------------------------------------------------------------------- - 1500082 + 1500086 (1 row) step s2-commit: @@ -164,10 +168,10 @@ step s2-print-cluster: nodeport|shardid|success|result --------------------------------------------------------------------- - 57637|1500081|t | 0 - 57637|1500082|t | 0 - 57637|1500083|t | 0 - 57638|1500080|t | 0 + 57637|1500085|t | 0 + 57637|1500086|t | 0 + 57637|1500087|t | 0 + 57638|1500082|t | 0 (4 rows) id|value @@ -183,6 +187,7 @@ create_distributed_table step s1-load-cache: TRUNCATE isolation_table; + TRUNCATE isolation_table2; step s3-acquire-advisory-lock: SELECT pg_advisory_lock(44000, 55152); @@ -214,6 +219,7 @@ step s2-isolate-tenant: step s1-insert: INSERT INTO isolation_table VALUES (5, 10); + INSERT INTO isolation_table2 VALUES (5, 10); step s1-commit: COMMIT; @@ -229,7 +235,7 @@ t step s2-isolate-tenant: <... completed> isolate_tenant_to_new_shard --------------------------------------------------------------------- - 1500088 + 1500094 (1 row) step s2-commit: @@ -248,10 +254,10 @@ step s2-print-cluster: nodeport|shardid|success|result --------------------------------------------------------------------- - 57637|1500087|t | 0 - 57637|1500088|t | 1 - 57637|1500089|t | 0 - 57638|1500086|t | 0 + 57637|1500093|t | 0 + 57637|1500094|t | 1 + 57637|1500095|t | 0 + 57638|1500090|t | 0 (4 rows) id|value @@ -268,6 +274,7 @@ create_distributed_table step s1-load-cache: TRUNCATE isolation_table; + TRUNCATE isolation_table2; step s3-acquire-advisory-lock: SELECT pg_advisory_lock(44000, 55152); @@ -314,7 +321,7 @@ t step s2-isolate-tenant: <... completed> isolate_tenant_to_new_shard --------------------------------------------------------------------- - 1500094 + 1500102 (1 row) step s2-commit: @@ -333,10 +340,10 @@ step s2-print-cluster: nodeport|shardid|success|result --------------------------------------------------------------------- - 57637|1500093|t | 1 - 57637|1500094|t | 1 - 57637|1500095|t | 2 - 57638|1500092|t | 1 + 57637|1500101|t | 1 + 57637|1500102|t | 1 + 57637|1500103|t | 2 + 57638|1500098|t | 1 (4 rows) id|value @@ -357,6 +364,7 @@ create_distributed_table 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); @@ -403,7 +411,7 @@ t step s2-isolate-tenant: <... completed> isolate_tenant_to_new_shard --------------------------------------------------------------------- - 1500100 + 1500110 (1 row) step s2-commit: @@ -422,10 +430,10 @@ step s2-print-cluster: nodeport|shardid|success|result --------------------------------------------------------------------- - 57637|1500099|t | 0 - 57637|1500100|t | 1 - 57637|1500101|t | 0 - 57638|1500098|t | 0 + 57637|1500109|t | 0 + 57637|1500110|t | 1 + 57637|1500111|t | 0 + 57638|1500106|t | 0 (4 rows) id|value @@ -442,6 +450,7 @@ create_distributed_table 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); @@ -488,7 +497,7 @@ t step s2-isolate-tenant: <... completed> isolate_tenant_to_new_shard --------------------------------------------------------------------- - 1500106 + 1500118 (1 row) step s2-commit: @@ -507,10 +516,10 @@ step s2-print-cluster: nodeport|shardid|success|result --------------------------------------------------------------------- - 57637|1500105|t | 0 - 57637|1500106|t | 0 - 57637|1500107|t | 0 - 57638|1500104|t | 0 + 57637|1500117|t | 0 + 57637|1500118|t | 0 + 57637|1500119|t | 0 + 57638|1500114|t | 0 (4 rows) id|value @@ -554,6 +563,7 @@ step s2-isolate-tenant: step s1-insert: INSERT INTO isolation_table VALUES (5, 10); + INSERT INTO isolation_table2 VALUES (5, 10); step s1-commit: COMMIT; @@ -569,7 +579,7 @@ t step s2-isolate-tenant: <... completed> isolate_tenant_to_new_shard --------------------------------------------------------------------- - 1500112 + 1500126 (1 row) step s2-commit: @@ -588,10 +598,10 @@ step s2-print-cluster: nodeport|shardid|success|result --------------------------------------------------------------------- - 57637|1500111|t | 0 - 57637|1500112|t | 1 - 57637|1500113|t | 0 - 57638|1500110|t | 0 + 57637|1500125|t | 0 + 57637|1500126|t | 1 + 57637|1500127|t | 0 + 57638|1500122|t | 0 (4 rows) id|value @@ -651,7 +661,7 @@ t step s2-isolate-tenant: <... completed> isolate_tenant_to_new_shard --------------------------------------------------------------------- - 1500118 + 1500134 (1 row) step s2-commit: @@ -670,10 +680,10 @@ step s2-print-cluster: nodeport|shardid|success|result --------------------------------------------------------------------- - 57637|1500117|t | 1 - 57637|1500118|t | 1 - 57637|1500119|t | 2 - 57638|1500116|t | 1 + 57637|1500133|t | 1 + 57637|1500134|t | 1 + 57637|1500135|t | 2 + 57638|1500130|t | 1 (4 rows) id|value @@ -686,7 +696,7 @@ id|value (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 --------------------------------------------------------------------- @@ -694,9 +704,11 @@ create_distributed_table 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); @@ -706,17 +718,11 @@ pg_advisory_lock (1 row) -step s1-begin: - 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'); - -step s2-isolate-tenant: +step s2-isolate-tenant: SELECT isolate_tenant_to_new_shard('isolation_table', 5, shard_transfer_mode => 'force_logical'); + +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 step s3-release-advisory-lock: @@ -727,15 +733,12 @@ pg_advisory_unlock t (1 row) -step s1-isolate-tenant: <... completed> +step s2-isolate-tenant: <... completed> isolate_tenant_to_new_shard --------------------------------------------------------------------- - 1500124 + 1500142 (1 row) -step s1-commit: - COMMIT; - step s2-print-cluster: -- row count per shard SELECT @@ -749,10 +752,10 @@ step s2-print-cluster: nodeport|shardid|success|result --------------------------------------------------------------------- - 57637|1500121|t | 1 - 57638|1500123|t | 0 - 57638|1500124|t | 0 - 57638|1500125|t | 0 + 57637|1500141|t | 0 + 57637|1500142|t | 1 + 57637|1500143|t | 0 + 57638|1500138|t | 0 (4 rows) id|value @@ -761,14 +764,19 @@ id|value (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 --------------------------------------------------------------------- (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); @@ -778,17 +786,11 @@ pg_advisory_lock (1 row) -step s1-begin: - 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'); - -step s2-isolate-tenant: +step s2-isolate-tenant: SELECT isolate_tenant_to_new_shard('isolation_table', 5, 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'); ERROR: could not acquire the lock required to split public.isolation_table step s3-release-advisory-lock: @@ -799,13 +801,152 @@ pg_advisory_unlock t (1 row) -step s1-isolate-tenant: <... completed> +step s2-isolate-tenant: <... completed> isolate_tenant_to_new_shard --------------------------------------------------------------------- - 1500130 + 1500150 (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'); + +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'); + +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; step s2-print-cluster: @@ -821,10 +962,154 @@ step s2-print-cluster: nodeport|shardid|success|result --------------------------------------------------------------------- - 57637|1500127|t | 1 - 57638|1500129|t | 0 - 57638|1500130|t | 0 - 57638|1500131|t | 0 + 57637|1500168|t | 0 + 57637|1500169|t | 1 + 57637|1500170|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'); + +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'); + +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) id|value diff --git a/src/test/regress/expected/multi_extension.out b/src/test/regress/expected/multi_extension.out index aea663960..c8f6c20b0 100644 --- a/src/test/regress/expected/multi_extension.out +++ b/src/test/regress/expected/multi_extension.out @@ -1111,7 +1111,7 @@ ERROR: extension "citus" already exists -- Snapshot of state at 11.1-1 ALTER EXTENSION citus UPDATE TO '11.1-1'; SELECT * FROM multi_extension.print_extension_changes(); - previous_object | current_object + previous_object | current_object --------------------------------------------------------------------- access method columnar | 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.options | table columnar.stripe | + | function citus_internal_delete_partition_metadata(regclass) void | function citus_locks() SETOF record | 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 replicate_reference_tables(citus.shard_transfer_mode) void | function worker_copy_table_to_node(regclass,integer) void - | function worker_split_copy(bigint,split_copy_info[]) void + | function worker_split_copy(bigint,text,split_copy_info[]) void | function worker_split_shard_release_dsm() void | function worker_split_shard_replication_setup(split_shard_info[]) SETOF replication_slot_info | type replication_slot_info | type split_copy_info | type split_shard_info | view citus_locks -(35 rows) +(37 rows) DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff; -- show running version diff --git a/src/test/regress/expected/multi_tenant_isolation.out b/src/test/regress/expected/multi_tenant_isolation.out index f97dc3fc0..046d814b0 100644 --- a/src/test/regress/expected/multi_tenant_isolation.out +++ b/src/test/regress/expected/multi_tenant_isolation.out @@ -663,7 +663,7 @@ SELECT count(*) FROM lineitem_date WHERE l_shipdate = '1997-08-08'; SET search_path to "Tenant Isolation"; 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'); -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. UPDATE pg_dist_shard_placement SET shardstate = 1 WHERE nodeport = :worker_1_port; \c - mx_isolation_role_ent - :master_port diff --git a/src/test/regress/expected/multi_tenant_isolation_nonblocking.out b/src/test/regress/expected/multi_tenant_isolation_nonblocking.out index 01e3bb88e..1c87236a0 100644 --- a/src/test/regress/expected/multi_tenant_isolation_nonblocking.out +++ b/src/test/regress/expected/multi_tenant_isolation_nonblocking.out @@ -663,7 +663,7 @@ SELECT count(*) FROM lineitem_date WHERE l_shipdate = '1997-08-08'; SET search_path to "Tenant Isolation"; 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'); -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. UPDATE pg_dist_shard_placement SET shardstate = 1 WHERE nodeport = :worker_1_port; \c - mx_isolation_role_ent - :master_port diff --git a/src/test/regress/expected/upgrade_list_citus_objects.out b/src/test/regress/expected/upgrade_list_citus_objects.out index b51d6e15d..98bc47c6d 100644 --- a/src/test/regress/expected/upgrade_list_citus_objects.out +++ b/src/test/regress/expected/upgrade_list_citus_objects.out @@ -65,6 +65,7 @@ ORDER BY 1; 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_delete_colocation_metadata(integer) + function citus_internal_delete_partition_metadata(regclass) function citus_internal_delete_shard_metadata(bigint) function citus_internal_global_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 create_distributed_function(regprocedure,text,text,boolean) 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_reference_table(regclass) 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_record_sequence_dependency(regclass,regclass,name) function worker_save_query_explain_analyze(text,jsonb) - function worker_split_copy(bigint,split_copy_info[]) + function worker_split_copy(bigint,text,split_copy_info[]) function worker_split_shard_release_dsm() function worker_split_shard_replication_setup(split_shard_info[]) schema citus @@ -268,5 +270,5 @@ ORDER BY 1; view citus_stat_statements view pg_dist_shard_placement view time_partitions -(260 rows) +(262 rows) diff --git a/src/test/regress/expected/worker_split_binary_copy_test.out b/src/test/regress/expected/worker_split_binary_copy_test.out index 07dacbdb1..f23dc2043 100644 --- a/src/test/regress/expected/worker_split_binary_copy_test.out +++ b/src/test/regress/expected/worker_split_binary_copy_test.out @@ -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; SELECT * from worker_split_copy( 81060000, -- source shard id to copy + 'l_orderkey', ARRAY[ -- split copy info for split children 1 ROW(81060015, -- destination shard id @@ -208,6 +209,7 @@ SELECT * from worker_split_copy( -- BEGIN: Trigger 2-way remote shard split copy. SELECT * from worker_split_copy( 81060000, -- source shard id to copy + 'l_orderkey', ARRAY[ -- split copy info for split children 1 ROW(81060015, -- destination shard id diff --git a/src/test/regress/expected/worker_split_copy_test.out b/src/test/regress/expected/worker_split_copy_test.out index c17ef5aa4..67d515198 100644 --- a/src/test/regress/expected/worker_split_copy_test.out +++ b/src/test/regress/expected/worker_split_copy_test.out @@ -54,6 +54,7 @@ SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \ -- BEGIN: Test Negative scenario SELECT * from worker_split_copy( 101, -- Invalid source shard id. + 'id', ARRAY[ -- split copy info for split children 1 ROW(81070015, -- destination shard id @@ -70,29 +71,34 @@ SELECT * from worker_split_copy( ERROR: could not find valid entry for shard xxxxx SELECT * from worker_split_copy( 81070000, -- source shard id to copy + 'id', ARRAY[] -- empty array ); ERROR: cannot determine type of empty array HINT: Explicitly cast to the desired type, for example ARRAY[]::integer[]. SELECT * from worker_split_copy( 81070000, -- source shard id to copy + 'id', 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. SELECT * from worker_split_copy( 81070000, -- source shard id to copy + 'id', ARRAY[NULL::pg_catalog.split_copy_info]-- empty array ); ERROR: pg_catalog.split_copy_info array cannot contain null values SELECT * from worker_split_copy( 81070000, -- source shard id to copy + 'id', 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. SELECT * from worker_split_copy( 81070000, -- source shard id to copy + 'id', 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. @@ -102,6 +108,7 @@ ERROR: destination_shard_id for pg_catalog.split_copy_info cannot be null. SET citus.enable_binary_protocol = false; SELECT * from worker_split_copy( 81070000, -- source shard id to copy + 'id', ARRAY[ -- split copy info for split children 1 ROW(81070015, -- destination shard id diff --git a/src/test/regress/expected/worker_split_text_copy_test.out b/src/test/regress/expected/worker_split_text_copy_test.out index 164d3a6d7..cddb6d85e 100644 --- a/src/test/regress/expected/worker_split_text_copy_test.out +++ b/src/test/regress/expected/worker_split_text_copy_test.out @@ -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; SELECT * from worker_split_copy( 81070000, -- source shard id to copy + 'l_orderkey', ARRAY[ -- split copy info for split children 1 ROW(81070015, -- destination shard id @@ -171,6 +172,7 @@ SELECT * from worker_split_copy( -- BEGIN: Trigger 2-way remote shard split copy. SELECT * from worker_split_copy( 81070000, -- source shard id to copy + 'l_orderkey', ARRAY[ -- split copy info for split children 1 ROW(81070015, -- destination shard id diff --git a/src/test/regress/failure_schedule b/src/test/regress/failure_schedule index 18a45fd26..816f9d9e2 100644 --- a/src/test/regress/failure_schedule +++ b/src/test/regress/failure_schedule @@ -18,6 +18,7 @@ test: failure_copy_on_hash test: failure_create_reference_table test: failure_create_distributed_table_non_empty test: failure_create_table +test: failure_create_distributed_table_concurrently test: failure_single_select test: failure_multi_shard_update_delete diff --git a/src/test/regress/isolation_schedule b/src/test/regress/isolation_schedule index 5432895a4..853da116e 100644 --- a/src/test/regress/isolation_schedule +++ b/src/test/regress/isolation_schedule @@ -48,6 +48,7 @@ test: isolation_create_citus_local_table test: isolation_create_restore_point test: isolation_create_distributed_table +test: isolation_create_distributed_table_concurrently test: isolation_multi_shard_modify_vs_all test: isolation_modify_with_subquery_vs_dml test: isolation_hash_copy_vs_all diff --git a/src/test/regress/multi_1_schedule b/src/test/regress/multi_1_schedule index bef14d45a..9b248b5cd 100644 --- a/src/test/regress/multi_1_schedule +++ b/src/test/regress/multi_1_schedule @@ -277,7 +277,7 @@ test: add_coordinator test: replicate_reference_tables_to_coordinator test: citus_local_tables 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: citus_local_table_triggers test: coordinator_shouldhaveshards diff --git a/src/test/regress/spec/isolation_create_distributed_table_concurrently.spec b/src/test/regress/spec/isolation_create_distributed_table_concurrently.spec new file mode 100644 index 000000000..02c31c96b --- /dev/null +++ b/src/test/regress/spec/isolation_create_distributed_table_concurrently.spec @@ -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" diff --git a/src/test/regress/spec/isolation_tenant_isolation_nonblocking.spec b/src/test/regress/spec/isolation_tenant_isolation_nonblocking.spec index 3b22024f4..e7395e631 100644 --- a/src/test/regress/spec/isolation_tenant_isolation_nonblocking.spec +++ b/src/test/regress/spec/isolation_tenant_isolation_nonblocking.spec @@ -10,11 +10,16 @@ setup CREATE TABLE isolation_table (id int PRIMARY KEY, value int); 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 { DROP TABLE isolation_table; + DROP TABLE isolation_table2; } session "s1" @@ -32,11 +37,13 @@ step "s1-begin" 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 "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; } -step "s1-isolate-tenant" +step "s1-isolate-tenant-same-coloc" { 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" { COMMIT; @@ -122,7 +144,7 @@ step "s3-release-advisory-lock" // s1 can execute its DML command concurrently with s2 shard isolation => // 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. 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" @@ -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-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) -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" +// concurrent nonblocking tenant isolations with the same colocation id are not allowed +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 -permutation "s1-insert" "s3-acquire-advisory-lock" "s1-begin" "s1-isolate-tenant" "s2-isolate-tenant" "s3-release-advisory-lock" "s1-commit" "s2-print-cluster" +// concurrent blocking and nonblocking tenant isolations with the same colocation id are not allowed +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" diff --git a/src/test/regress/sql/create_distributed_table_concurrently.sql b/src/test/regress/sql/create_distributed_table_concurrently.sql new file mode 100644 index 000000000..be1984e6d --- /dev/null +++ b/src/test/regress/sql/create_distributed_table_concurrently.sql @@ -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; diff --git a/src/test/regress/sql/failure_create_distributed_table_concurrently.sql b/src/test/regress/sql/failure_create_distributed_table_concurrently.sql new file mode 100644 index 000000000..502c3940a --- /dev/null +++ b/src/test/regress/sql/failure_create_distributed_table_concurrently.sql @@ -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); diff --git a/src/test/regress/sql/worker_split_binary_copy_test.sql b/src/test/regress/sql/worker_split_binary_copy_test.sql index a47e968bd..489ff9dc4 100644 --- a/src/test/regress/sql/worker_split_binary_copy_test.sql +++ b/src/test/regress/sql/worker_split_binary_copy_test.sql @@ -160,6 +160,7 @@ SET citus.enable_binary_protocol = true; SELECT * from worker_split_copy( 81060000, -- source shard id to copy + 'l_orderkey', ARRAY[ -- split copy info for split children 1 ROW(81060015, -- destination shard id @@ -178,6 +179,7 @@ SELECT * from worker_split_copy( -- BEGIN: Trigger 2-way remote shard split copy. SELECT * from worker_split_copy( 81060000, -- source shard id to copy + 'l_orderkey', ARRAY[ -- split copy info for split children 1 ROW(81060015, -- destination shard id diff --git a/src/test/regress/sql/worker_split_copy_test.sql b/src/test/regress/sql/worker_split_copy_test.sql index b799eb305..2fac91c69 100644 --- a/src/test/regress/sql/worker_split_copy_test.sql +++ b/src/test/regress/sql/worker_split_copy_test.sql @@ -38,6 +38,7 @@ SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \ -- BEGIN: Test Negative scenario SELECT * from worker_split_copy( 101, -- Invalid source shard id. + 'id', ARRAY[ -- split copy info for split children 1 ROW(81070015, -- destination shard id @@ -54,26 +55,31 @@ SELECT * from worker_split_copy( SELECT * from worker_split_copy( 81070000, -- source shard id to copy + 'id', ARRAY[] -- empty array ); SELECT * from worker_split_copy( 81070000, -- source shard id to copy + 'id', ARRAY[NULL] -- empty array ); SELECT * from worker_split_copy( 81070000, -- source shard id to copy + 'id', ARRAY[NULL::pg_catalog.split_copy_info]-- empty array ); SELECT * from worker_split_copy( 81070000, -- source shard id to copy + 'id', ARRAY[ROW(NULL)]-- empty array ); SELECT * from worker_split_copy( 81070000, -- source shard id to copy + 'id', ARRAY[ROW(NULL, NULL, NULL, NULL)::pg_catalog.split_copy_info] -- empty array ); -- END: Test Negative scenario @@ -83,6 +89,7 @@ SELECT * from worker_split_copy( SET citus.enable_binary_protocol = false; SELECT * from worker_split_copy( 81070000, -- source shard id to copy + 'id', ARRAY[ -- split copy info for split children 1 ROW(81070015, -- destination shard id diff --git a/src/test/regress/sql/worker_split_text_copy_test.sql b/src/test/regress/sql/worker_split_text_copy_test.sql index 10791a66d..fe2a614b3 100644 --- a/src/test/regress/sql/worker_split_text_copy_test.sql +++ b/src/test/regress/sql/worker_split_text_copy_test.sql @@ -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; SELECT * from worker_split_copy( 81070000, -- source shard id to copy + 'l_orderkey', ARRAY[ -- split copy info for split children 1 ROW(81070015, -- destination shard id @@ -170,6 +171,7 @@ SELECT * from worker_split_copy( -- BEGIN: Trigger 2-way remote shard split copy. SELECT * from worker_split_copy( 81070000, -- source shard id to copy + 'l_orderkey', ARRAY[ -- split copy info for split children 1 ROW(81070015, -- destination shard id