mirror of https://github.com/citusdata/citus.git
Add non-blocking variant of create_distributed_table (#6087)
Added create_distributed_table_concurrently which is nonblocking variant of create_distributed_table. It bases on the split API which takes advantage of logical replication to support nonblocking split operations. Co-authored-by: Marco Slot <marco.slot@gmail.com> Co-authored-by: aykutbozkurt <aykut.bozkurt1995@gmail.com>pull/6273/head
parent
d68654680b
commit
6bb31c5d75
|
@ -42,6 +42,7 @@
|
||||||
#include "distributed/colocation_utils.h"
|
#include "distributed/colocation_utils.h"
|
||||||
#include "distributed/commands.h"
|
#include "distributed/commands.h"
|
||||||
#include "distributed/deparser.h"
|
#include "distributed/deparser.h"
|
||||||
|
#include "distributed/distributed_execution_locks.h"
|
||||||
#include "distributed/distribution_column.h"
|
#include "distributed/distribution_column.h"
|
||||||
#include "distributed/listutils.h"
|
#include "distributed/listutils.h"
|
||||||
#include "distributed/local_executor.h"
|
#include "distributed/local_executor.h"
|
||||||
|
@ -59,12 +60,16 @@
|
||||||
#include "distributed/reference_table_utils.h"
|
#include "distributed/reference_table_utils.h"
|
||||||
#include "distributed/relation_access_tracking.h"
|
#include "distributed/relation_access_tracking.h"
|
||||||
#include "distributed/remote_commands.h"
|
#include "distributed/remote_commands.h"
|
||||||
|
#include "distributed/repair_shards.h"
|
||||||
#include "distributed/resource_lock.h"
|
#include "distributed/resource_lock.h"
|
||||||
|
#include "distributed/shard_rebalancer.h"
|
||||||
|
#include "distributed/shard_split.h"
|
||||||
#include "distributed/shared_library_init.h"
|
#include "distributed/shared_library_init.h"
|
||||||
#include "distributed/shard_rebalancer.h"
|
#include "distributed/shard_rebalancer.h"
|
||||||
#include "distributed/worker_protocol.h"
|
#include "distributed/worker_protocol.h"
|
||||||
#include "distributed/worker_shard_visibility.h"
|
#include "distributed/worker_shard_visibility.h"
|
||||||
#include "distributed/worker_transaction.h"
|
#include "distributed/worker_transaction.h"
|
||||||
|
#include "distributed/utils/distribution_column_map.h"
|
||||||
#include "distributed/version_compat.h"
|
#include "distributed/version_compat.h"
|
||||||
#include "executor/executor.h"
|
#include "executor/executor.h"
|
||||||
#include "executor/spi.h"
|
#include "executor/spi.h"
|
||||||
|
@ -76,6 +81,7 @@
|
||||||
#include "parser/parse_node.h"
|
#include "parser/parse_node.h"
|
||||||
#include "parser/parse_relation.h"
|
#include "parser/parse_relation.h"
|
||||||
#include "parser/parser.h"
|
#include "parser/parser.h"
|
||||||
|
#include "postmaster/postmaster.h"
|
||||||
#include "storage/lmgr.h"
|
#include "storage/lmgr.h"
|
||||||
#include "tcop/pquery.h"
|
#include "tcop/pquery.h"
|
||||||
#include "tcop/tcopprot.h"
|
#include "tcop/tcopprot.h"
|
||||||
|
@ -93,8 +99,18 @@
|
||||||
#define LOG_PER_TUPLE_AMOUNT 1000000
|
#define LOG_PER_TUPLE_AMOUNT 1000000
|
||||||
|
|
||||||
/* local function forward declarations */
|
/* local function forward declarations */
|
||||||
|
static void CreateDistributedTableConcurrently(Oid relationId,
|
||||||
|
char *distributionColumnName,
|
||||||
|
char distributionMethod,
|
||||||
|
char *colocateWithTableName,
|
||||||
|
int shardCount,
|
||||||
|
bool shardCountIsStrict);
|
||||||
static char DecideReplicationModel(char distributionMethod, char *colocateWithTableName,
|
static char DecideReplicationModel(char distributionMethod, char *colocateWithTableName,
|
||||||
bool viaDeprecatedAPI);
|
bool viaDeprecatedAPI);
|
||||||
|
static List * HashSplitPointsForShardList(List *shardList);
|
||||||
|
static List * HashSplitPointsForShardCount(int shardCount);
|
||||||
|
static List * WorkerNodesForShardList(List *shardList);
|
||||||
|
static List * RoundRobinWorkerNodeList(List *workerNodeList, int listLength);
|
||||||
static void CreateHashDistributedTableShards(Oid relationId, int shardCount,
|
static void CreateHashDistributedTableShards(Oid relationId, int shardCount,
|
||||||
Oid colocatedTableId, bool localTableEmpty);
|
Oid colocatedTableId, bool localTableEmpty);
|
||||||
static uint32 ColocationIdForNewTable(Oid relationId, Var *distributionColumn,
|
static uint32 ColocationIdForNewTable(Oid relationId, Var *distributionColumn,
|
||||||
|
@ -105,9 +121,6 @@ static uint32 ColocationIdForNewTable(Oid relationId, Var *distributionColumn,
|
||||||
static void EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn,
|
static void EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn,
|
||||||
char distributionMethod, uint32 colocationId,
|
char distributionMethod, uint32 colocationId,
|
||||||
char replicationModel, bool viaDeprecatedAPI);
|
char replicationModel, bool viaDeprecatedAPI);
|
||||||
static void EnsureTableCanBeColocatedWith(Oid relationId, char replicationModel,
|
|
||||||
Oid distributionColumnType,
|
|
||||||
Oid sourceRelationId);
|
|
||||||
static void EnsureLocalTableEmpty(Oid relationId);
|
static void EnsureLocalTableEmpty(Oid relationId);
|
||||||
static void EnsureRelationHasNoTriggers(Oid relationId);
|
static void EnsureRelationHasNoTriggers(Oid relationId);
|
||||||
static Oid SupportFunctionForColumn(Var *partitionColumn, Oid accessMethodId,
|
static Oid SupportFunctionForColumn(Var *partitionColumn, Oid accessMethodId,
|
||||||
|
@ -117,6 +130,7 @@ static void EnsureLocalTableEmptyIfNecessary(Oid relationId, char distributionMe
|
||||||
static bool ShouldLocalTableBeEmpty(Oid relationId, char distributionMethod, bool
|
static bool ShouldLocalTableBeEmpty(Oid relationId, char distributionMethod, bool
|
||||||
viaDeprecatedAPI);
|
viaDeprecatedAPI);
|
||||||
static void EnsureCitusTableCanBeCreated(Oid relationOid);
|
static void EnsureCitusTableCanBeCreated(Oid relationOid);
|
||||||
|
static void PropagatePrerequisiteObjectsForDistributedTable(Oid relationId);
|
||||||
static void EnsureDistributedSequencesHaveOneType(Oid relationId,
|
static void EnsureDistributedSequencesHaveOneType(Oid relationId,
|
||||||
List *seqInfoList);
|
List *seqInfoList);
|
||||||
static List * GetFKeyCreationCommandsRelationInvolvedWithTableType(Oid relationId,
|
static List * GetFKeyCreationCommandsRelationInvolvedWithTableType(Oid relationId,
|
||||||
|
@ -134,9 +148,17 @@ static void DoCopyFromLocalTableIntoShards(Relation distributedRelation,
|
||||||
EState *estate);
|
EState *estate);
|
||||||
static void ErrorIfTemporaryTable(Oid relationId);
|
static void ErrorIfTemporaryTable(Oid relationId);
|
||||||
static void ErrorIfForeignTable(Oid relationOid);
|
static void ErrorIfForeignTable(Oid relationOid);
|
||||||
|
static void SendAddLocalTableToMetadataCommandOutsideTransaction(Oid relationId);
|
||||||
|
static void EnsureDistributableTable(Oid relationId);
|
||||||
|
static void EnsureForeignKeysForDistributedTableConcurrently(Oid relationId);
|
||||||
|
static void EnsureColocateWithTableIsValid(Oid relationId, char distributionMethod,
|
||||||
|
char *distributionColumnName,
|
||||||
|
char *colocateWithTableName);
|
||||||
|
static void WarnIfTableHaveNoReplicaIdentity(Oid relationId);
|
||||||
|
|
||||||
/* exports for SQL callable functions */
|
/* exports for SQL callable functions */
|
||||||
PG_FUNCTION_INFO_V1(master_create_distributed_table);
|
PG_FUNCTION_INFO_V1(master_create_distributed_table);
|
||||||
|
PG_FUNCTION_INFO_V1(create_distributed_table_concurrently);
|
||||||
PG_FUNCTION_INFO_V1(create_distributed_table);
|
PG_FUNCTION_INFO_V1(create_distributed_table);
|
||||||
PG_FUNCTION_INFO_V1(create_reference_table);
|
PG_FUNCTION_INFO_V1(create_reference_table);
|
||||||
|
|
||||||
|
@ -254,6 +276,589 @@ create_distributed_table(PG_FUNCTION_ARGS)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* create_distributed_concurrently gets a table name, distribution column,
|
||||||
|
* distribution method and colocate_with option, then it creates a
|
||||||
|
* distributed table.
|
||||||
|
*/
|
||||||
|
Datum
|
||||||
|
create_distributed_table_concurrently(PG_FUNCTION_ARGS)
|
||||||
|
{
|
||||||
|
CheckCitusVersion(ERROR);
|
||||||
|
|
||||||
|
if (PG_ARGISNULL(0) || PG_ARGISNULL(1) || PG_ARGISNULL(2) || PG_ARGISNULL(3))
|
||||||
|
{
|
||||||
|
PG_RETURN_VOID();
|
||||||
|
}
|
||||||
|
|
||||||
|
Oid relationId = PG_GETARG_OID(0);
|
||||||
|
text *distributionColumnText = PG_GETARG_TEXT_P(1);
|
||||||
|
char *distributionColumnName = text_to_cstring(distributionColumnText);
|
||||||
|
Oid distributionMethodOid = PG_GETARG_OID(2);
|
||||||
|
char distributionMethod = LookupDistributionMethod(distributionMethodOid);
|
||||||
|
text *colocateWithTableNameText = PG_GETARG_TEXT_P(3);
|
||||||
|
char *colocateWithTableName = text_to_cstring(colocateWithTableNameText);
|
||||||
|
|
||||||
|
bool shardCountIsStrict = false;
|
||||||
|
int shardCount = ShardCount;
|
||||||
|
if (!PG_ARGISNULL(4))
|
||||||
|
{
|
||||||
|
if (pg_strncasecmp(colocateWithTableName, "default", NAMEDATALEN) != 0 &&
|
||||||
|
pg_strncasecmp(colocateWithTableName, "none", NAMEDATALEN) != 0)
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errmsg("Cannot use colocate_with with a table "
|
||||||
|
"and shard_count at the same time")));
|
||||||
|
}
|
||||||
|
|
||||||
|
shardCount = PG_GETARG_INT32(4);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* if shard_count parameter is given than we have to
|
||||||
|
* make sure table has that many shards
|
||||||
|
*/
|
||||||
|
shardCountIsStrict = true;
|
||||||
|
}
|
||||||
|
|
||||||
|
CreateDistributedTableConcurrently(relationId, distributionColumnName,
|
||||||
|
distributionMethod,
|
||||||
|
colocateWithTableName,
|
||||||
|
shardCount,
|
||||||
|
shardCountIsStrict);
|
||||||
|
|
||||||
|
PG_RETURN_VOID();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* CreateDistributedTableConcurrently distributes a table by first converting
|
||||||
|
* it to a Citus local table and then splitting the shard of the Citus local
|
||||||
|
* table.
|
||||||
|
*
|
||||||
|
* If anything goes wrong during the second phase, the table is left as a
|
||||||
|
* Citus local table.
|
||||||
|
*/
|
||||||
|
static void
|
||||||
|
CreateDistributedTableConcurrently(Oid relationId, char *distributionColumnName,
|
||||||
|
char distributionMethod,
|
||||||
|
char *colocateWithTableName,
|
||||||
|
int shardCount,
|
||||||
|
bool shardCountIsStrict)
|
||||||
|
{
|
||||||
|
/*
|
||||||
|
* We disallow create_distributed_table_concurrently in transaction blocks
|
||||||
|
* because we cannot handle preceding writes, and we block writes at the
|
||||||
|
* very end of the operation so the transaction should end immediately after.
|
||||||
|
*/
|
||||||
|
PreventInTransactionBlock(true, "create_distributed_table_concurrently");
|
||||||
|
|
||||||
|
/*
|
||||||
|
* do not allow multiple create_distributed_table_concurrently in the same
|
||||||
|
* transaction. We should do that check just here because concurrent local table
|
||||||
|
* conversion can cause issues.
|
||||||
|
*/
|
||||||
|
ErrorIfMultipleNonblockingMoveSplitInTheSameTransaction();
|
||||||
|
|
||||||
|
/* do not allow concurrent CreateDistributedTableConcurrently operations */
|
||||||
|
AcquireCreateDistributedTableConcurrentlyLock(relationId);
|
||||||
|
|
||||||
|
if (distributionMethod != DISTRIBUTE_BY_HASH)
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
|
||||||
|
errmsg("only hash-distributed tables can be distributed "
|
||||||
|
"without blocking writes")));
|
||||||
|
}
|
||||||
|
|
||||||
|
if (ShardReplicationFactor > 1)
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errmsg("cannot distribute a table concurrently when "
|
||||||
|
"citus.shard_replication_factor > 1")));
|
||||||
|
}
|
||||||
|
|
||||||
|
EnsureCoordinatorIsInMetadata();
|
||||||
|
EnsureCitusTableCanBeCreated(relationId);
|
||||||
|
|
||||||
|
EnsureValidDistributionColumn(relationId, distributionColumnName);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Ensure table type is valid to be distributed. It should be either regular or citus local table.
|
||||||
|
*/
|
||||||
|
EnsureDistributableTable(relationId);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* we rely on citus_add_local_table_to_metadata, so it can generate irrelevant messages.
|
||||||
|
* we want to error with a user friendly message if foreign keys are not supported.
|
||||||
|
* We can miss foreign key violations because we are not holding locks, so relation
|
||||||
|
* can be modified until we acquire the lock for the relation, but we do as much as we can
|
||||||
|
* to be user friendly on foreign key violation messages.
|
||||||
|
*/
|
||||||
|
|
||||||
|
EnsureForeignKeysForDistributedTableConcurrently(relationId);
|
||||||
|
|
||||||
|
bool viaDeprecatedAPI = false;
|
||||||
|
char replicationModel = DecideReplicationModel(distributionMethod,
|
||||||
|
colocateWithTableName,
|
||||||
|
viaDeprecatedAPI);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* we fail transaction before local table conversion if the table could not be colocated with
|
||||||
|
* given table. We should make those checks after local table conversion by acquiring locks to
|
||||||
|
* the relation because the distribution column can be modified in that period.
|
||||||
|
*/
|
||||||
|
if (!IsColocateWithDefault(colocateWithTableName) && !IsColocateWithNone(
|
||||||
|
colocateWithTableName))
|
||||||
|
{
|
||||||
|
EnsureColocateWithTableIsValid(relationId, distributionMethod,
|
||||||
|
distributionColumnName,
|
||||||
|
colocateWithTableName);
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Get name of the table before possibly replacing it in
|
||||||
|
* citus_add_local_table_to_metadata.
|
||||||
|
*/
|
||||||
|
char *tableName = get_rel_name(relationId);
|
||||||
|
Oid schemaId = get_rel_namespace(relationId);
|
||||||
|
char *schemaName = get_namespace_name(schemaId);
|
||||||
|
RangeVar *rangeVar = makeRangeVar(schemaName, tableName, -1);
|
||||||
|
|
||||||
|
/* If table is a regular table, then we need to add it into metadata. */
|
||||||
|
if (!IsCitusTable(relationId))
|
||||||
|
{
|
||||||
|
/*
|
||||||
|
* Before taking locks, convert the table into a Citus local table and commit
|
||||||
|
* to allow shard split to see the shard.
|
||||||
|
*/
|
||||||
|
SendAddLocalTableToMetadataCommandOutsideTransaction(relationId);
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Lock target relation with a shard update exclusive lock to
|
||||||
|
* block DDL, but not writes.
|
||||||
|
*
|
||||||
|
* If there was a concurrent drop/rename, error out by setting missingOK = false.
|
||||||
|
*/
|
||||||
|
bool missingOK = false;
|
||||||
|
relationId = RangeVarGetRelid(rangeVar, ShareUpdateExclusiveLock, missingOK);
|
||||||
|
|
||||||
|
if (PartitionedTableNoLock(relationId))
|
||||||
|
{
|
||||||
|
/* also lock partitions */
|
||||||
|
LockPartitionRelations(relationId, ShareUpdateExclusiveLock);
|
||||||
|
}
|
||||||
|
|
||||||
|
WarnIfTableHaveNoReplicaIdentity(relationId);
|
||||||
|
|
||||||
|
List *shardList = LoadShardIntervalList(relationId);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* It's technically possible for the table to have been concurrently
|
||||||
|
* distributed just after citus_add_local_table_to_metadata and just
|
||||||
|
* before acquiring the lock, so double check.
|
||||||
|
*/
|
||||||
|
if (list_length(shardList) != 1 ||
|
||||||
|
!IsCitusTableType(relationId, CITUS_LOCAL_TABLE))
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||||
|
errmsg("table was concurrently modified")));
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* The table currently has one shard, we will split that shard to match the
|
||||||
|
* target distribution.
|
||||||
|
*/
|
||||||
|
ShardInterval *shardToSplit = (ShardInterval *) linitial(shardList);
|
||||||
|
|
||||||
|
PropagatePrerequisiteObjectsForDistributedTable(relationId);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* we should re-evaluate distribution column values. It may have changed,
|
||||||
|
* because we did not lock the relation at the previous check before local
|
||||||
|
* table conversion.
|
||||||
|
*/
|
||||||
|
Var *distributionColumn = BuildDistributionKeyFromColumnName(relationId,
|
||||||
|
distributionColumnName,
|
||||||
|
NoLock);
|
||||||
|
Oid distributionColumnType = distributionColumn->vartype;
|
||||||
|
Oid distributionColumnCollation = distributionColumn->varcollid;
|
||||||
|
|
||||||
|
/* get an advisory lock to serialize concurrent default group creations */
|
||||||
|
if (IsColocateWithDefault(colocateWithTableName))
|
||||||
|
{
|
||||||
|
AcquireColocationDefaultLock();
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* At this stage, we only want to check for an existing co-location group.
|
||||||
|
* We cannot create a new co-location group until after replication slot
|
||||||
|
* creation in NonBlockingShardSplit.
|
||||||
|
*/
|
||||||
|
uint32 colocationId = FindColocateWithColocationId(relationId,
|
||||||
|
replicationModel,
|
||||||
|
distributionColumnType,
|
||||||
|
distributionColumnCollation,
|
||||||
|
shardCount,
|
||||||
|
shardCountIsStrict,
|
||||||
|
colocateWithTableName);
|
||||||
|
|
||||||
|
if (IsColocateWithDefault(colocateWithTableName) && (colocationId !=
|
||||||
|
INVALID_COLOCATION_ID))
|
||||||
|
{
|
||||||
|
/*
|
||||||
|
* we can release advisory lock if there is already a default entry for given params;
|
||||||
|
* else, we should keep it to prevent different default coloc entry creation by
|
||||||
|
* concurrent operations.
|
||||||
|
*/
|
||||||
|
ReleaseColocationDefaultLock();
|
||||||
|
}
|
||||||
|
|
||||||
|
EnsureRelationCanBeDistributed(relationId, distributionColumn, distributionMethod,
|
||||||
|
colocationId, replicationModel, viaDeprecatedAPI);
|
||||||
|
|
||||||
|
Oid colocatedTableId = InvalidOid;
|
||||||
|
if (colocationId != INVALID_COLOCATION_ID)
|
||||||
|
{
|
||||||
|
colocatedTableId = ColocatedTableId(colocationId);
|
||||||
|
}
|
||||||
|
|
||||||
|
List *workersForPlacementList;
|
||||||
|
List *shardSplitPointsList;
|
||||||
|
|
||||||
|
if (colocatedTableId != InvalidOid)
|
||||||
|
{
|
||||||
|
List *colocatedShardList = LoadShardIntervalList(colocatedTableId);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Match the shard ranges of an existing table.
|
||||||
|
*/
|
||||||
|
shardSplitPointsList = HashSplitPointsForShardList(colocatedShardList);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Find the node IDs of the shard placements.
|
||||||
|
*/
|
||||||
|
workersForPlacementList = WorkerNodesForShardList(colocatedShardList);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
/*
|
||||||
|
* Generate a new set of #shardCount shards.
|
||||||
|
*/
|
||||||
|
shardSplitPointsList = HashSplitPointsForShardCount(shardCount);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Place shards in a round-robin fashion across all data nodes.
|
||||||
|
*/
|
||||||
|
List *workerNodeList = DistributedTablePlacementNodeList(NoLock);
|
||||||
|
workersForPlacementList = RoundRobinWorkerNodeList(workerNodeList, shardCount);
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Make sure that existing reference tables have been replicated to all the nodes
|
||||||
|
* such that we can create foreign keys and joins work immediately after creation.
|
||||||
|
* We do this after applying all essential checks to error out early in case of
|
||||||
|
* user error.
|
||||||
|
*/
|
||||||
|
EnsureReferenceTablesExistOnAllNodes();
|
||||||
|
|
||||||
|
/*
|
||||||
|
* At this point, the table is a Citus local table, which means it does
|
||||||
|
* not have a partition column in the metadata. However, we cannot update
|
||||||
|
* the metadata here because that would prevent us from creating a replication
|
||||||
|
* slot to copy ongoing changes. Instead, we pass a hash that maps relation
|
||||||
|
* IDs to partition column vars.
|
||||||
|
*/
|
||||||
|
DistributionColumnMap *distributionColumnOverrides = CreateDistributionColumnMap();
|
||||||
|
AddDistributionColumnForRelation(distributionColumnOverrides, relationId,
|
||||||
|
distributionColumnName);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* there is no colocation entries yet for local table, so we should
|
||||||
|
* check if table has any partition and add them to same colocation
|
||||||
|
* group
|
||||||
|
*/
|
||||||
|
List *sourceColocatedShardIntervalList = ListShardsUnderParentRelation(relationId);
|
||||||
|
|
||||||
|
SplitMode splitMode = NON_BLOCKING_SPLIT;
|
||||||
|
SplitOperation splitOperation = CREATE_DISTRIBUTED_TABLE;
|
||||||
|
SplitShard(
|
||||||
|
splitMode,
|
||||||
|
splitOperation,
|
||||||
|
shardToSplit->shardId,
|
||||||
|
shardSplitPointsList,
|
||||||
|
workersForPlacementList,
|
||||||
|
distributionColumnOverrides,
|
||||||
|
sourceColocatedShardIntervalList,
|
||||||
|
colocationId
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* EnsureForeignKeysForDistributedTableConcurrently ensures that referenced and referencing foreign
|
||||||
|
* keys for the given table are supported.
|
||||||
|
*
|
||||||
|
* We allow distributed -> reference
|
||||||
|
* distributed -> citus local
|
||||||
|
*
|
||||||
|
* We disallow reference -> distributed
|
||||||
|
* citus local -> distributed
|
||||||
|
* regular -> distributed
|
||||||
|
*
|
||||||
|
* Normally regular -> distributed is allowed but it is not allowed when we create the
|
||||||
|
* distributed table concurrently because we rely on conversion of regular table to citus local table,
|
||||||
|
* which errors with an unfriendly message.
|
||||||
|
*/
|
||||||
|
static void
|
||||||
|
EnsureForeignKeysForDistributedTableConcurrently(Oid relationId)
|
||||||
|
{
|
||||||
|
/*
|
||||||
|
* disallow citus local -> distributed fkeys.
|
||||||
|
* disallow reference -> distributed fkeys.
|
||||||
|
* disallow regular -> distributed fkeys.
|
||||||
|
*/
|
||||||
|
EnsureNoFKeyFromTableType(relationId, INCLUDE_CITUS_LOCAL_TABLES |
|
||||||
|
INCLUDE_REFERENCE_TABLES | INCLUDE_LOCAL_TABLES);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* disallow distributed -> regular fkeys.
|
||||||
|
*/
|
||||||
|
EnsureNoFKeyToTableType(relationId, INCLUDE_LOCAL_TABLES);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* EnsureColocateWithTableIsValid ensures given relation can be colocated with the table of given name.
|
||||||
|
*/
|
||||||
|
static void
|
||||||
|
EnsureColocateWithTableIsValid(Oid relationId, char distributionMethod,
|
||||||
|
char *distributionColumnName, char *colocateWithTableName)
|
||||||
|
{
|
||||||
|
bool viaDeprecatedAPI = false;
|
||||||
|
char replicationModel = DecideReplicationModel(distributionMethod,
|
||||||
|
colocateWithTableName,
|
||||||
|
viaDeprecatedAPI);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* we fail transaction before local table conversion if the table could not be colocated with
|
||||||
|
* given table. We should make those checks after local table conversion by acquiring locks to
|
||||||
|
* the relation because the distribution column can be modified in that period.
|
||||||
|
*/
|
||||||
|
Oid distributionColumnType = ColumnTypeIdForRelationColumnName(relationId,
|
||||||
|
distributionColumnName);
|
||||||
|
|
||||||
|
text *colocateWithTableNameText = cstring_to_text(colocateWithTableName);
|
||||||
|
Oid colocateWithTableId = ResolveRelationId(colocateWithTableNameText, false);
|
||||||
|
EnsureTableCanBeColocatedWith(relationId, replicationModel,
|
||||||
|
distributionColumnType, colocateWithTableId);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* AcquireCreateDistributedTableConcurrentlyLock does not allow concurrent create_distributed_table_concurrently
|
||||||
|
* operations.
|
||||||
|
*/
|
||||||
|
void
|
||||||
|
AcquireCreateDistributedTableConcurrentlyLock(Oid relationId)
|
||||||
|
{
|
||||||
|
LOCKTAG tag;
|
||||||
|
const bool sessionLock = false;
|
||||||
|
const bool dontWait = true;
|
||||||
|
|
||||||
|
SET_LOCKTAG_CITUS_OPERATION(tag, CITUS_CREATE_DISTRIBUTED_TABLE_CONCURRENTLY);
|
||||||
|
|
||||||
|
LockAcquireResult lockAcquired = LockAcquire(&tag, ExclusiveLock, sessionLock,
|
||||||
|
dontWait);
|
||||||
|
if (!lockAcquired)
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errmsg("another create_distributed_table_concurrently "
|
||||||
|
"operation is in progress"),
|
||||||
|
errhint("Make sure that the concurrent operation has "
|
||||||
|
"finished and re-run the command")));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* SendAddLocalTableToMetadataCommandOutsideTransaction executes metadata add local
|
||||||
|
* table command locally to avoid deadlock.
|
||||||
|
*/
|
||||||
|
static void
|
||||||
|
SendAddLocalTableToMetadataCommandOutsideTransaction(Oid relationId)
|
||||||
|
{
|
||||||
|
char *qualifiedRelationName = generate_qualified_relation_name(relationId);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* we need to allow nested distributed execution, because we start a new distributed
|
||||||
|
* execution inside the pushed-down UDF citus_add_local_table_to_metadata. Normally
|
||||||
|
* citus does not allow that because it cannot guarantee correctness.
|
||||||
|
*/
|
||||||
|
StringInfo allowNestedDistributionCommand = makeStringInfo();
|
||||||
|
appendStringInfo(allowNestedDistributionCommand,
|
||||||
|
"SET LOCAL citus.allow_nested_distributed_execution to ON");
|
||||||
|
|
||||||
|
StringInfo addLocalTableToMetadataCommand = makeStringInfo();
|
||||||
|
appendStringInfo(addLocalTableToMetadataCommand,
|
||||||
|
"SELECT pg_catalog.citus_add_local_table_to_metadata(%s)",
|
||||||
|
quote_literal_cstr(qualifiedRelationName));
|
||||||
|
|
||||||
|
List *commands = list_make2(allowNestedDistributionCommand->data,
|
||||||
|
addLocalTableToMetadataCommand->data);
|
||||||
|
char *username = NULL;
|
||||||
|
SendCommandListToWorkerOutsideTransaction(LocalHostName, PostPortNumber, username,
|
||||||
|
commands);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* WarnIfTableHaveNoReplicaIdentity notices user if the given table or its partitions (if any)
|
||||||
|
* do not have a replica identity which is required for logical replication to replicate
|
||||||
|
* UPDATE and DELETE commands during create_distributed_table_concurrently.
|
||||||
|
*/
|
||||||
|
void
|
||||||
|
WarnIfTableHaveNoReplicaIdentity(Oid relationId)
|
||||||
|
{
|
||||||
|
bool foundRelationWithNoReplicaIdentity = false;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Check for source relation's partitions if any. We do not need to check for the source relation
|
||||||
|
* because we can replicate partitioned table even if it does not have replica identity.
|
||||||
|
* Source table will have no data if it has partitions.
|
||||||
|
*/
|
||||||
|
if (PartitionedTable(relationId))
|
||||||
|
{
|
||||||
|
List *partitionList = PartitionList(relationId);
|
||||||
|
ListCell *partitionCell = NULL;
|
||||||
|
|
||||||
|
foreach(partitionCell, partitionList)
|
||||||
|
{
|
||||||
|
Oid partitionTableId = lfirst_oid(partitionCell);
|
||||||
|
|
||||||
|
if (!RelationCanPublishAllModifications(partitionTableId))
|
||||||
|
{
|
||||||
|
foundRelationWithNoReplicaIdentity = true;
|
||||||
|
break;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
/* check for source relation if it is not partitioned */
|
||||||
|
else
|
||||||
|
{
|
||||||
|
if (!RelationCanPublishAllModifications(relationId))
|
||||||
|
{
|
||||||
|
foundRelationWithNoReplicaIdentity = true;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (foundRelationWithNoReplicaIdentity)
|
||||||
|
{
|
||||||
|
char *relationName = get_rel_name(relationId);
|
||||||
|
|
||||||
|
ereport(NOTICE, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
||||||
|
errmsg("relation %s does not have a REPLICA "
|
||||||
|
"IDENTITY or PRIMARY KEY", relationName),
|
||||||
|
errdetail("UPDATE and DELETE commands on the relation will "
|
||||||
|
"error out during create_distributed_table_concurrently unless "
|
||||||
|
"there is a REPLICA IDENTITY or PRIMARY KEY. "
|
||||||
|
"INSERT commands will still work.")));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* HashSplitPointsForShardList returns a list of split points which match
|
||||||
|
* the shard ranges of the given list of shards;
|
||||||
|
*/
|
||||||
|
static List *
|
||||||
|
HashSplitPointsForShardList(List *shardList)
|
||||||
|
{
|
||||||
|
List *splitPointList = NIL;
|
||||||
|
|
||||||
|
ShardInterval *shardInterval = NULL;
|
||||||
|
foreach_ptr(shardInterval, shardList)
|
||||||
|
{
|
||||||
|
int32 shardMaxValue = DatumGetInt32(shardInterval->maxValue);
|
||||||
|
|
||||||
|
splitPointList = lappend_int(splitPointList, shardMaxValue);
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Split point lists only include the upper boundaries.
|
||||||
|
*/
|
||||||
|
splitPointList = list_delete_last(splitPointList);
|
||||||
|
|
||||||
|
return splitPointList;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* HashSplitPointsForShardCount returns a list of split points for a given
|
||||||
|
* shard count with roughly equal hash ranges.
|
||||||
|
*/
|
||||||
|
static List *
|
||||||
|
HashSplitPointsForShardCount(int shardCount)
|
||||||
|
{
|
||||||
|
List *splitPointList = NIL;
|
||||||
|
|
||||||
|
/* calculate the split of the hash space */
|
||||||
|
uint64 hashTokenIncrement = HASH_TOKEN_COUNT / shardCount;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Split points lists only include the upper boundaries, so we only
|
||||||
|
* go up to shardCount - 1 and do not have to apply the correction
|
||||||
|
* for the last shardmaxvalue.
|
||||||
|
*/
|
||||||
|
for (int64 shardIndex = 0; shardIndex < shardCount - 1; shardIndex++)
|
||||||
|
{
|
||||||
|
/* initialize the hash token space for this shard */
|
||||||
|
int32 shardMinValue = PG_INT32_MIN + (shardIndex * hashTokenIncrement);
|
||||||
|
int32 shardMaxValue = shardMinValue + (hashTokenIncrement - 1);
|
||||||
|
|
||||||
|
splitPointList = lappend_int(splitPointList, shardMaxValue);
|
||||||
|
}
|
||||||
|
|
||||||
|
return splitPointList;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* WorkerNodesForShardList returns a list of node ids reflecting the locations of
|
||||||
|
* the given list of shards.
|
||||||
|
*/
|
||||||
|
static List *
|
||||||
|
WorkerNodesForShardList(List *shardList)
|
||||||
|
{
|
||||||
|
List *nodeIdList = NIL;
|
||||||
|
|
||||||
|
ShardInterval *shardInterval = NULL;
|
||||||
|
foreach_ptr(shardInterval, shardList)
|
||||||
|
{
|
||||||
|
WorkerNode *workerNode = ActiveShardPlacementWorkerNode(shardInterval->shardId);
|
||||||
|
nodeIdList = lappend_int(nodeIdList, workerNode->nodeId);
|
||||||
|
}
|
||||||
|
|
||||||
|
return nodeIdList;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* RoundRobinWorkerNodeList round robins over the workers in the worker node list
|
||||||
|
* and adds node ids to a list of length listLength.
|
||||||
|
*/
|
||||||
|
static List *
|
||||||
|
RoundRobinWorkerNodeList(List *workerNodeList, int listLength)
|
||||||
|
{
|
||||||
|
List *nodeIdList = NIL;
|
||||||
|
|
||||||
|
for (int idx = 0; idx < listLength; idx++)
|
||||||
|
{
|
||||||
|
int nodeIdx = idx % list_length(workerNodeList);
|
||||||
|
WorkerNode *workerNode = (WorkerNode *) list_nth(workerNodeList, nodeIdx);
|
||||||
|
nodeIdList = lappend_int(nodeIdList, workerNode->nodeId);
|
||||||
|
}
|
||||||
|
|
||||||
|
return nodeIdList;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* create_reference_table creates a distributed table with the given relationId. The
|
* create_reference_table creates a distributed table with the given relationId. The
|
||||||
* created table has one shard and replication factor is set to the active worker
|
* created table has one shard and replication factor is set to the active worker
|
||||||
|
@ -394,7 +999,6 @@ CreateDistributedTable(Oid relationId, char *distributionColumnName,
|
||||||
INCLUDE_ALL_TABLE_TYPES);
|
INCLUDE_ALL_TABLE_TYPES);
|
||||||
relationId = DropFKeysAndUndistributeTable(relationId);
|
relationId = DropFKeysAndUndistributeTable(relationId);
|
||||||
}
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* To support foreign keys between reference tables and local tables,
|
* To support foreign keys between reference tables and local tables,
|
||||||
* we drop & re-define foreign keys at the end of this function so
|
* we drop & re-define foreign keys at the end of this function so
|
||||||
|
@ -431,21 +1035,9 @@ CreateDistributedTable(Oid relationId, char *distributionColumnName,
|
||||||
|
|
||||||
LockRelationOid(relationId, ExclusiveLock);
|
LockRelationOid(relationId, ExclusiveLock);
|
||||||
|
|
||||||
/*
|
EnsureTableNotDistributed(relationId);
|
||||||
* Ensure that the sequences used in column defaults of the table
|
|
||||||
* have proper types
|
|
||||||
*/
|
|
||||||
EnsureRelationHasCompatibleSequenceTypes(relationId);
|
|
||||||
|
|
||||||
/*
|
PropagatePrerequisiteObjectsForDistributedTable(relationId);
|
||||||
* distributed tables might have dependencies on different objects, since we create
|
|
||||||
* shards for a distributed table via multiple sessions these objects will be created
|
|
||||||
* via their own connection and committed immediately so they become visible to all
|
|
||||||
* sessions creating shards.
|
|
||||||
*/
|
|
||||||
ObjectAddress *tableAddress = palloc0(sizeof(ObjectAddress));
|
|
||||||
ObjectAddressSet(*tableAddress, RelationRelationId, relationId);
|
|
||||||
EnsureAllObjectDependenciesExistOnAllNodes(list_make1(tableAddress));
|
|
||||||
|
|
||||||
char replicationModel = DecideReplicationModel(distributionMethod,
|
char replicationModel = DecideReplicationModel(distributionMethod,
|
||||||
colocateWithTableName,
|
colocateWithTableName,
|
||||||
|
@ -453,7 +1045,7 @@ CreateDistributedTable(Oid relationId, char *distributionColumnName,
|
||||||
|
|
||||||
Var *distributionColumn = BuildDistributionKeyFromColumnName(relationId,
|
Var *distributionColumn = BuildDistributionKeyFromColumnName(relationId,
|
||||||
distributionColumnName,
|
distributionColumnName,
|
||||||
ExclusiveLock);
|
NoLock);
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* ColocationIdForNewTable assumes caller acquires lock on relationId. In our case,
|
* ColocationIdForNewTable assumes caller acquires lock on relationId. In our case,
|
||||||
|
@ -582,6 +1174,31 @@ CreateDistributedTable(Oid relationId, char *distributionColumnName,
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* PropagatePrerequisiteObjectsForDistributedTable ensures we can create shards
|
||||||
|
* on all nodes by ensuring all dependent objects exist on all node.
|
||||||
|
*/
|
||||||
|
static void
|
||||||
|
PropagatePrerequisiteObjectsForDistributedTable(Oid relationId)
|
||||||
|
{
|
||||||
|
/*
|
||||||
|
* Ensure that the sequences used in column defaults of the table
|
||||||
|
* have proper types
|
||||||
|
*/
|
||||||
|
EnsureRelationHasCompatibleSequenceTypes(relationId);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* distributed tables might have dependencies on different objects, since we create
|
||||||
|
* shards for a distributed table via multiple sessions these objects will be created
|
||||||
|
* via their own connection and committed immediately so they become visible to all
|
||||||
|
* sessions creating shards.
|
||||||
|
*/
|
||||||
|
ObjectAddress *tableAddress = palloc0(sizeof(ObjectAddress));
|
||||||
|
ObjectAddressSet(*tableAddress, RelationRelationId, relationId);
|
||||||
|
EnsureAllObjectDependenciesExistOnAllNodes(list_make1(tableAddress));
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* EnsureSequenceTypeSupported ensures that the type of the column that uses
|
* EnsureSequenceTypeSupported ensures that the type of the column that uses
|
||||||
* a sequence on its DEFAULT is consistent with previous uses (if any) of the
|
* a sequence on its DEFAULT is consistent with previous uses (if any) of the
|
||||||
|
@ -956,82 +1573,56 @@ ColocationIdForNewTable(Oid relationId, Var *distributionColumn,
|
||||||
*/
|
*/
|
||||||
Assert(distributionMethod == DISTRIBUTE_BY_HASH);
|
Assert(distributionMethod == DISTRIBUTE_BY_HASH);
|
||||||
|
|
||||||
Relation pgDistColocation = table_open(DistColocationRelationId(), ExclusiveLock);
|
|
||||||
|
|
||||||
Oid distributionColumnType = distributionColumn->vartype;
|
Oid distributionColumnType = distributionColumn->vartype;
|
||||||
Oid distributionColumnCollation = get_typcollation(distributionColumnType);
|
Oid distributionColumnCollation = get_typcollation(distributionColumnType);
|
||||||
bool createdColocationGroup = false;
|
|
||||||
|
|
||||||
if (pg_strncasecmp(colocateWithTableName, "default", NAMEDATALEN) == 0)
|
/* get an advisory lock to serialize concurrent default group creations */
|
||||||
|
if (IsColocateWithDefault(colocateWithTableName))
|
||||||
{
|
{
|
||||||
/* check for default colocation group */
|
AcquireColocationDefaultLock();
|
||||||
colocationId = ColocationId(shardCount, ShardReplicationFactor,
|
}
|
||||||
|
|
||||||
|
colocationId = FindColocateWithColocationId(relationId,
|
||||||
|
replicationModel,
|
||||||
distributionColumnType,
|
distributionColumnType,
|
||||||
distributionColumnCollation);
|
distributionColumnCollation,
|
||||||
|
shardCount,
|
||||||
|
shardCountIsStrict,
|
||||||
|
colocateWithTableName);
|
||||||
|
|
||||||
|
if (IsColocateWithDefault(colocateWithTableName) && (colocationId !=
|
||||||
|
INVALID_COLOCATION_ID))
|
||||||
|
{
|
||||||
/*
|
/*
|
||||||
* if the shardCount is strict then we check if the shard count
|
* we can release advisory lock if there is already a default entry for given params;
|
||||||
* of the colocated table is actually shardCount
|
* else, we should keep it to prevent different default coloc entry creation by
|
||||||
|
* concurrent operations.
|
||||||
*/
|
*/
|
||||||
if (shardCountIsStrict && colocationId != INVALID_COLOCATION_ID)
|
ReleaseColocationDefaultLock();
|
||||||
{
|
|
||||||
Oid colocatedTableId = ColocatedTableId(colocationId);
|
|
||||||
|
|
||||||
if (colocatedTableId != InvalidOid)
|
|
||||||
{
|
|
||||||
CitusTableCacheEntry *cacheEntry =
|
|
||||||
GetCitusTableCacheEntry(colocatedTableId);
|
|
||||||
int colocatedTableShardCount = cacheEntry->shardIntervalArrayLength;
|
|
||||||
|
|
||||||
if (colocatedTableShardCount != shardCount)
|
|
||||||
{
|
|
||||||
colocationId = INVALID_COLOCATION_ID;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if (colocationId == INVALID_COLOCATION_ID)
|
if (colocationId == INVALID_COLOCATION_ID)
|
||||||
{
|
{
|
||||||
|
if (IsColocateWithDefault(colocateWithTableName))
|
||||||
|
{
|
||||||
|
/*
|
||||||
|
* Generate a new colocation ID and insert a pg_dist_colocation
|
||||||
|
* record.
|
||||||
|
*/
|
||||||
colocationId = CreateColocationGroup(shardCount, ShardReplicationFactor,
|
colocationId = CreateColocationGroup(shardCount, ShardReplicationFactor,
|
||||||
distributionColumnType,
|
distributionColumnType,
|
||||||
distributionColumnCollation);
|
distributionColumnCollation);
|
||||||
createdColocationGroup = true;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
else if (IsColocateWithNone(colocateWithTableName))
|
else if (IsColocateWithNone(colocateWithTableName))
|
||||||
{
|
{
|
||||||
|
/*
|
||||||
|
* Generate a new colocation ID and insert a pg_dist_colocation
|
||||||
|
* record.
|
||||||
|
*/
|
||||||
colocationId = CreateColocationGroup(shardCount, ShardReplicationFactor,
|
colocationId = CreateColocationGroup(shardCount, ShardReplicationFactor,
|
||||||
distributionColumnType,
|
distributionColumnType,
|
||||||
distributionColumnCollation);
|
distributionColumnCollation);
|
||||||
|
|
||||||
createdColocationGroup = true;
|
|
||||||
}
|
}
|
||||||
else
|
|
||||||
{
|
|
||||||
text *colocateWithTableNameText = cstring_to_text(colocateWithTableName);
|
|
||||||
Oid sourceRelationId = ResolveRelationId(colocateWithTableNameText, false);
|
|
||||||
|
|
||||||
EnsureTableCanBeColocatedWith(relationId, replicationModel,
|
|
||||||
distributionColumnType, sourceRelationId);
|
|
||||||
|
|
||||||
colocationId = TableColocationId(sourceRelationId);
|
|
||||||
}
|
|
||||||
|
|
||||||
/*
|
|
||||||
* If we created a new colocation group then we need to keep the lock to
|
|
||||||
* prevent a concurrent create_distributed_table call from creating another
|
|
||||||
* colocation group with the same parameters. If we're using an existing
|
|
||||||
* colocation group then other transactions will use the same one.
|
|
||||||
*/
|
|
||||||
if (createdColocationGroup)
|
|
||||||
{
|
|
||||||
/* keep the exclusive lock */
|
|
||||||
table_close(pgDistColocation, NoLock);
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
/* release the exclusive lock */
|
|
||||||
table_close(pgDistColocation, ExclusiveLock);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1055,7 +1646,6 @@ EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn,
|
||||||
{
|
{
|
||||||
Oid parentRelationId = InvalidOid;
|
Oid parentRelationId = InvalidOid;
|
||||||
|
|
||||||
EnsureTableNotDistributed(relationId);
|
|
||||||
EnsureLocalTableEmptyIfNecessary(relationId, distributionMethod, viaDeprecatedAPI);
|
EnsureLocalTableEmptyIfNecessary(relationId, distributionMethod, viaDeprecatedAPI);
|
||||||
|
|
||||||
/* user really wants triggers? */
|
/* user really wants triggers? */
|
||||||
|
@ -1131,13 +1721,13 @@ EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn,
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (PartitionTable(relationId))
|
if (PartitionTableNoLock(relationId))
|
||||||
{
|
{
|
||||||
parentRelationId = PartitionParentOid(relationId);
|
parentRelationId = PartitionParentOid(relationId);
|
||||||
}
|
}
|
||||||
|
|
||||||
/* partitions cannot be distributed if their parent is not distributed */
|
/* partitions cannot be distributed if their parent is not distributed */
|
||||||
if (PartitionTable(relationId) && !IsCitusTable(parentRelationId))
|
if (PartitionTableNoLock(relationId) && !IsCitusTable(parentRelationId))
|
||||||
{
|
{
|
||||||
char *parentRelationName = get_rel_name(parentRelationId);
|
char *parentRelationName = get_rel_name(parentRelationId);
|
||||||
|
|
||||||
|
@ -1155,7 +1745,7 @@ EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn,
|
||||||
* reach this point because, we call CreateDistributedTable for partitions if their
|
* reach this point because, we call CreateDistributedTable for partitions if their
|
||||||
* parent table is distributed.
|
* parent table is distributed.
|
||||||
*/
|
*/
|
||||||
if (PartitionedTable(relationId))
|
if (PartitionedTableNoLock(relationId))
|
||||||
{
|
{
|
||||||
/* we cannot distribute partitioned tables with master_create_distributed_table */
|
/* we cannot distribute partitioned tables with master_create_distributed_table */
|
||||||
if (viaDeprecatedAPI)
|
if (viaDeprecatedAPI)
|
||||||
|
@ -1174,7 +1764,7 @@ EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn,
|
||||||
}
|
}
|
||||||
|
|
||||||
/* we don't support distributing tables with multi-level partitioning */
|
/* we don't support distributing tables with multi-level partitioning */
|
||||||
if (PartitionTable(relationId))
|
if (PartitionTableNoLock(relationId))
|
||||||
{
|
{
|
||||||
char *parentRelationName = get_rel_name(parentRelationId);
|
char *parentRelationName = get_rel_name(parentRelationId);
|
||||||
|
|
||||||
|
@ -1227,55 +1817,6 @@ ErrorIfTableIsACatalogTable(Relation relation)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
|
||||||
* EnsureTableCanBeColocatedWith checks whether a given replication model and
|
|
||||||
* distribution column type is suitable to distribute a table to be colocated
|
|
||||||
* with given source table.
|
|
||||||
*
|
|
||||||
* We only pass relationId to provide meaningful error messages.
|
|
||||||
*/
|
|
||||||
static void
|
|
||||||
EnsureTableCanBeColocatedWith(Oid relationId, char replicationModel,
|
|
||||||
Oid distributionColumnType, Oid sourceRelationId)
|
|
||||||
{
|
|
||||||
CitusTableCacheEntry *sourceTableEntry = GetCitusTableCacheEntry(sourceRelationId);
|
|
||||||
char sourceReplicationModel = sourceTableEntry->replicationModel;
|
|
||||||
Var *sourceDistributionColumn = DistPartitionKeyOrError(sourceRelationId);
|
|
||||||
|
|
||||||
if (!IsCitusTableTypeCacheEntry(sourceTableEntry, HASH_DISTRIBUTED))
|
|
||||||
{
|
|
||||||
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
|
||||||
errmsg("cannot distribute relation"),
|
|
||||||
errdetail("Currently, colocate_with option is only supported "
|
|
||||||
"for hash distributed tables.")));
|
|
||||||
}
|
|
||||||
|
|
||||||
if (sourceReplicationModel != replicationModel)
|
|
||||||
{
|
|
||||||
char *relationName = get_rel_name(relationId);
|
|
||||||
char *sourceRelationName = get_rel_name(sourceRelationId);
|
|
||||||
|
|
||||||
ereport(ERROR, (errmsg("cannot colocate tables %s and %s",
|
|
||||||
sourceRelationName, relationName),
|
|
||||||
errdetail("Replication models don't match for %s and %s.",
|
|
||||||
sourceRelationName, relationName)));
|
|
||||||
}
|
|
||||||
|
|
||||||
Oid sourceDistributionColumnType = sourceDistributionColumn->vartype;
|
|
||||||
if (sourceDistributionColumnType != distributionColumnType)
|
|
||||||
{
|
|
||||||
char *relationName = get_rel_name(relationId);
|
|
||||||
char *sourceRelationName = get_rel_name(sourceRelationId);
|
|
||||||
|
|
||||||
ereport(ERROR, (errmsg("cannot colocate tables %s and %s",
|
|
||||||
sourceRelationName, relationName),
|
|
||||||
errdetail("Distribution column types don't match for "
|
|
||||||
"%s and %s.", sourceRelationName,
|
|
||||||
relationName)));
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* EnsureLocalTableEmptyIfNecessary errors out if the function should be empty
|
* EnsureLocalTableEmptyIfNecessary errors out if the function should be empty
|
||||||
* according to ShouldLocalTableBeEmpty but it is not.
|
* according to ShouldLocalTableBeEmpty but it is not.
|
||||||
|
@ -1350,6 +1891,27 @@ EnsureLocalTableEmpty(Oid relationId)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* EnsureDistributableTable ensures the given table type is appropriate to
|
||||||
|
* be distributed. Table type should be regular or citus local table.
|
||||||
|
*/
|
||||||
|
static void
|
||||||
|
EnsureDistributableTable(Oid relationId)
|
||||||
|
{
|
||||||
|
bool isLocalTable = IsCitusTableType(relationId, CITUS_LOCAL_TABLE);
|
||||||
|
bool isRegularTable = !IsCitusTableType(relationId, ANY_CITUS_TABLE_TYPE);
|
||||||
|
|
||||||
|
if (!isLocalTable && !isRegularTable)
|
||||||
|
{
|
||||||
|
char *relationName = get_rel_name(relationId);
|
||||||
|
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_INVALID_TABLE_DEFINITION),
|
||||||
|
errmsg("table \"%s\" is already distributed",
|
||||||
|
relationName)));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* EnsureTableNotDistributed errors out if the table is distributed.
|
* EnsureTableNotDistributed errors out if the table is distributed.
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -97,6 +97,66 @@ ConstraintIsAForeignKeyToReferenceTable(char *inputConstaintName, Oid relationId
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* EnsureNoFKeyFromTableType ensures that given relation is not referenced by any table specified
|
||||||
|
* by table type flag.
|
||||||
|
*/
|
||||||
|
void
|
||||||
|
EnsureNoFKeyFromTableType(Oid relationId, int tableTypeFlag)
|
||||||
|
{
|
||||||
|
int flags = INCLUDE_REFERENCED_CONSTRAINTS | EXCLUDE_SELF_REFERENCES |
|
||||||
|
tableTypeFlag;
|
||||||
|
List *referencedFKeyOids = GetForeignKeyOids(relationId, flags);
|
||||||
|
|
||||||
|
if (list_length(referencedFKeyOids) > 0)
|
||||||
|
{
|
||||||
|
Oid referencingFKeyOid = linitial_oid(referencedFKeyOids);
|
||||||
|
Oid referencingTableId = GetReferencingTableId(referencingFKeyOid);
|
||||||
|
|
||||||
|
char *referencingRelName = get_rel_name(referencingTableId);
|
||||||
|
char *referencedRelName = get_rel_name(relationId);
|
||||||
|
char *referencingTableTypeName = GetTableTypeName(referencingTableId);
|
||||||
|
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
||||||
|
errmsg("relation %s is referenced by a foreign key from %s",
|
||||||
|
referencedRelName, referencingRelName),
|
||||||
|
errdetail(
|
||||||
|
"foreign keys from a %s to a distributed table are not supported.",
|
||||||
|
referencingTableTypeName)));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* EnsureNoFKeyToTableType ensures that given relation is not referencing by any table specified
|
||||||
|
* by table type flag.
|
||||||
|
*/
|
||||||
|
void
|
||||||
|
EnsureNoFKeyToTableType(Oid relationId, int tableTypeFlag)
|
||||||
|
{
|
||||||
|
int flags = INCLUDE_REFERENCING_CONSTRAINTS | EXCLUDE_SELF_REFERENCES |
|
||||||
|
tableTypeFlag;
|
||||||
|
List *referencingFKeyOids = GetForeignKeyOids(relationId, flags);
|
||||||
|
|
||||||
|
if (list_length(referencingFKeyOids) > 0)
|
||||||
|
{
|
||||||
|
Oid referencedFKeyOid = linitial_oid(referencingFKeyOids);
|
||||||
|
Oid referencedTableId = GetReferencedTableId(referencedFKeyOid);
|
||||||
|
|
||||||
|
char *referencedRelName = get_rel_name(referencedTableId);
|
||||||
|
char *referencingRelName = get_rel_name(relationId);
|
||||||
|
char *referencedTableTypeName = GetTableTypeName(referencedTableId);
|
||||||
|
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
||||||
|
errmsg("relation %s is referenced by a foreign key from %s",
|
||||||
|
referencedRelName, referencingRelName),
|
||||||
|
errdetail(
|
||||||
|
"foreign keys from a distributed table to a %s are not supported.",
|
||||||
|
referencedTableTypeName)));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* ErrorIfUnsupportedForeignConstraintExists runs checks related to foreign
|
* ErrorIfUnsupportedForeignConstraintExists runs checks related to foreign
|
||||||
* constraints and errors out if it is not possible to create one of the
|
* constraints and errors out if it is not possible to create one of the
|
||||||
|
|
|
@ -493,6 +493,49 @@ IsCitusTableTypeInternal(char partitionMethod, char replicationModel,
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* GetTableTypeName returns string representation of the table type.
|
||||||
|
*/
|
||||||
|
char *
|
||||||
|
GetTableTypeName(Oid tableId)
|
||||||
|
{
|
||||||
|
bool regularTable = false;
|
||||||
|
char partitionMethod = ' ';
|
||||||
|
char replicationModel = ' ';
|
||||||
|
if (IsCitusTable(tableId))
|
||||||
|
{
|
||||||
|
CitusTableCacheEntry *referencingCacheEntry = GetCitusTableCacheEntry(tableId);
|
||||||
|
partitionMethod = referencingCacheEntry->partitionMethod;
|
||||||
|
replicationModel = referencingCacheEntry->replicationModel;
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
regularTable = true;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (regularTable)
|
||||||
|
{
|
||||||
|
return "regular table";
|
||||||
|
}
|
||||||
|
else if (partitionMethod == 'h')
|
||||||
|
{
|
||||||
|
return "distributed table";
|
||||||
|
}
|
||||||
|
else if (partitionMethod == 'n' && replicationModel == 't')
|
||||||
|
{
|
||||||
|
return "reference table";
|
||||||
|
}
|
||||||
|
else if (partitionMethod == 'n' && replicationModel != 't')
|
||||||
|
{
|
||||||
|
return "citus local table";
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
return "unknown table";
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* IsCitusTable returns whether relationId is a distributed relation or
|
* IsCitusTable returns whether relationId is a distributed relation or
|
||||||
* not.
|
* not.
|
||||||
|
|
|
@ -159,6 +159,7 @@ PG_FUNCTION_INFO_V1(worker_record_sequence_dependency);
|
||||||
* or regular users as long as the regular user owns the input object.
|
* or regular users as long as the regular user owns the input object.
|
||||||
*/
|
*/
|
||||||
PG_FUNCTION_INFO_V1(citus_internal_add_partition_metadata);
|
PG_FUNCTION_INFO_V1(citus_internal_add_partition_metadata);
|
||||||
|
PG_FUNCTION_INFO_V1(citus_internal_delete_partition_metadata);
|
||||||
PG_FUNCTION_INFO_V1(citus_internal_add_shard_metadata);
|
PG_FUNCTION_INFO_V1(citus_internal_add_shard_metadata);
|
||||||
PG_FUNCTION_INFO_V1(citus_internal_add_placement_metadata);
|
PG_FUNCTION_INFO_V1(citus_internal_add_placement_metadata);
|
||||||
PG_FUNCTION_INFO_V1(citus_internal_update_placement_metadata);
|
PG_FUNCTION_INFO_V1(citus_internal_update_placement_metadata);
|
||||||
|
@ -1211,6 +1212,24 @@ DistributionDeleteCommand(const char *schemaName, const char *tableName)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* DistributionDeleteMetadataCommand returns a query to delete pg_dist_partition
|
||||||
|
* metadata from a worker node for a given table.
|
||||||
|
*/
|
||||||
|
char *
|
||||||
|
DistributionDeleteMetadataCommand(Oid relationId)
|
||||||
|
{
|
||||||
|
StringInfo deleteCommand = makeStringInfo();
|
||||||
|
char *qualifiedRelationName = generate_qualified_relation_name(relationId);
|
||||||
|
|
||||||
|
appendStringInfo(deleteCommand,
|
||||||
|
"SELECT pg_catalog.citus_internal_delete_partition_metadata(%s)",
|
||||||
|
quote_literal_cstr(qualifiedRelationName));
|
||||||
|
|
||||||
|
return deleteCommand->data;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* TableOwnerResetCommand generates a commands that can be executed
|
* TableOwnerResetCommand generates a commands that can be executed
|
||||||
* to reset the table owner.
|
* to reset the table owner.
|
||||||
|
@ -3199,6 +3218,35 @@ EnsurePartitionMetadataIsSane(Oid relationId, char distributionMethod, int coloc
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* citus_internal_delete_partition_metadata is an internal UDF to
|
||||||
|
* delete a row in pg_dist_partition.
|
||||||
|
*/
|
||||||
|
Datum
|
||||||
|
citus_internal_delete_partition_metadata(PG_FUNCTION_ARGS)
|
||||||
|
{
|
||||||
|
CheckCitusVersion(ERROR);
|
||||||
|
|
||||||
|
PG_ENSURE_ARGNOTNULL(0, "relation");
|
||||||
|
Oid relationId = PG_GETARG_OID(0);
|
||||||
|
|
||||||
|
/* only owner of the table (or superuser) is allowed to add the Citus metadata */
|
||||||
|
EnsureTableOwner(relationId);
|
||||||
|
|
||||||
|
/* we want to serialize all the metadata changes to this table */
|
||||||
|
LockRelationOid(relationId, ShareUpdateExclusiveLock);
|
||||||
|
|
||||||
|
if (!ShouldSkipMetadataChecks())
|
||||||
|
{
|
||||||
|
EnsureCoordinatorInitiatedOperation();
|
||||||
|
}
|
||||||
|
|
||||||
|
DeletePartitionRow(relationId);
|
||||||
|
|
||||||
|
PG_RETURN_VOID();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* citus_internal_add_shard_metadata is an internal UDF to
|
* citus_internal_add_shard_metadata is an internal UDF to
|
||||||
* add a row to pg_dist_shard.
|
* add a row to pg_dist_shard.
|
||||||
|
|
|
@ -39,6 +39,7 @@
|
||||||
#include "distributed/metadata_utility.h"
|
#include "distributed/metadata_utility.h"
|
||||||
#include "distributed/coordinator_protocol.h"
|
#include "distributed/coordinator_protocol.h"
|
||||||
#include "distributed/metadata_cache.h"
|
#include "distributed/metadata_cache.h"
|
||||||
|
#include "distributed/metadata_sync.h"
|
||||||
#include "distributed/multi_join_order.h"
|
#include "distributed/multi_join_order.h"
|
||||||
#include "distributed/multi_logical_optimizer.h"
|
#include "distributed/multi_logical_optimizer.h"
|
||||||
#include "distributed/multi_partitioning_utils.h"
|
#include "distributed/multi_partitioning_utils.h"
|
||||||
|
@ -2153,6 +2154,96 @@ UpdatePgDistPartitionAutoConverted(Oid citusTableId, bool autoConverted)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* UpdateDistributionColumnGlobally sets the distribution column and colocation ID
|
||||||
|
* for a table in pg_dist_partition on all nodes
|
||||||
|
*/
|
||||||
|
void
|
||||||
|
UpdateDistributionColumnGlobally(Oid relationId, char distributionMethod,
|
||||||
|
Var *distributionColumn, int colocationId)
|
||||||
|
{
|
||||||
|
UpdateDistributionColumn(relationId, distributionMethod, distributionColumn,
|
||||||
|
colocationId);
|
||||||
|
|
||||||
|
if (ShouldSyncTableMetadata(relationId))
|
||||||
|
{
|
||||||
|
/* we use delete+insert because syncing uses specialized RPCs */
|
||||||
|
char *deleteMetadataCommand = DistributionDeleteMetadataCommand(relationId);
|
||||||
|
SendCommandToWorkersWithMetadata(deleteMetadataCommand);
|
||||||
|
|
||||||
|
/* pick up the new metadata (updated above) */
|
||||||
|
CitusTableCacheEntry *cacheEntry = GetCitusTableCacheEntry(relationId);
|
||||||
|
char *insertMetadataCommand = DistributionCreateCommand(cacheEntry);
|
||||||
|
SendCommandToWorkersWithMetadata(insertMetadataCommand);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* UpdateDistributionColumn sets the distribution column and colocation ID for a table
|
||||||
|
* in pg_dist_partition.
|
||||||
|
*/
|
||||||
|
void
|
||||||
|
UpdateDistributionColumn(Oid relationId, char distributionMethod, Var *distributionColumn,
|
||||||
|
int colocationId)
|
||||||
|
{
|
||||||
|
ScanKeyData scanKey[1];
|
||||||
|
int scanKeyCount = 1;
|
||||||
|
bool indexOK = true;
|
||||||
|
Datum values[Natts_pg_dist_partition];
|
||||||
|
bool isnull[Natts_pg_dist_partition];
|
||||||
|
bool replace[Natts_pg_dist_partition];
|
||||||
|
|
||||||
|
Relation pgDistPartition = table_open(DistPartitionRelationId(), RowExclusiveLock);
|
||||||
|
TupleDesc tupleDescriptor = RelationGetDescr(pgDistPartition);
|
||||||
|
ScanKeyInit(&scanKey[0], Anum_pg_dist_partition_logicalrelid,
|
||||||
|
BTEqualStrategyNumber, F_OIDEQ, ObjectIdGetDatum(relationId));
|
||||||
|
|
||||||
|
SysScanDesc scanDescriptor = systable_beginscan(pgDistPartition,
|
||||||
|
DistPartitionLogicalRelidIndexId(),
|
||||||
|
indexOK,
|
||||||
|
NULL, scanKeyCount, scanKey);
|
||||||
|
|
||||||
|
HeapTuple heapTuple = systable_getnext(scanDescriptor);
|
||||||
|
if (!HeapTupleIsValid(heapTuple))
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errmsg("could not find valid entry for citus table with oid: %u",
|
||||||
|
relationId)));
|
||||||
|
}
|
||||||
|
|
||||||
|
memset(replace, 0, sizeof(replace));
|
||||||
|
|
||||||
|
replace[Anum_pg_dist_partition_partmethod - 1] = true;
|
||||||
|
values[Anum_pg_dist_partition_partmethod - 1] = CharGetDatum(distributionMethod);
|
||||||
|
isnull[Anum_pg_dist_partition_partmethod - 1] = false;
|
||||||
|
|
||||||
|
replace[Anum_pg_dist_partition_colocationid - 1] = true;
|
||||||
|
values[Anum_pg_dist_partition_colocationid - 1] = UInt32GetDatum(colocationId);
|
||||||
|
isnull[Anum_pg_dist_partition_colocationid - 1] = false;
|
||||||
|
|
||||||
|
replace[Anum_pg_dist_partition_autoconverted - 1] = true;
|
||||||
|
values[Anum_pg_dist_partition_autoconverted - 1] = BoolGetDatum(false);
|
||||||
|
isnull[Anum_pg_dist_partition_autoconverted - 1] = false;
|
||||||
|
|
||||||
|
char *distributionColumnString = nodeToString((Node *) distributionColumn);
|
||||||
|
|
||||||
|
replace[Anum_pg_dist_partition_partkey - 1] = true;
|
||||||
|
values[Anum_pg_dist_partition_partkey - 1] =
|
||||||
|
CStringGetTextDatum(distributionColumnString);
|
||||||
|
isnull[Anum_pg_dist_partition_partkey - 1] = false;
|
||||||
|
|
||||||
|
heapTuple = heap_modify_tuple(heapTuple, tupleDescriptor, values, isnull, replace);
|
||||||
|
|
||||||
|
CatalogTupleUpdate(pgDistPartition, &heapTuple->t_self, heapTuple);
|
||||||
|
|
||||||
|
CitusInvalidateRelcacheByRelid(relationId);
|
||||||
|
CommandCounterIncrement();
|
||||||
|
|
||||||
|
systable_endscan(scanDescriptor);
|
||||||
|
table_close(pgDistPartition, NoLock);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Check that the current user has `mode` permissions on relationId, error out
|
* Check that the current user has `mode` permissions on relationId, error out
|
||||||
* if not. Superusers always have such permissions.
|
* if not. Superusers always have such permissions.
|
||||||
|
|
|
@ -2546,6 +2546,24 @@ EnsureCoordinator(void)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* EnsureCoordinatorIsInMetadata checks whether the coordinator is added to the
|
||||||
|
* metadata, which is required for many operations.
|
||||||
|
*/
|
||||||
|
void
|
||||||
|
EnsureCoordinatorIsInMetadata(void)
|
||||||
|
{
|
||||||
|
bool isCoordinatorInMetadata = false;
|
||||||
|
PrimaryNodeForGroup(COORDINATOR_GROUP_ID, &isCoordinatorInMetadata);
|
||||||
|
if (!isCoordinatorInMetadata)
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errmsg("coordinator is not added to the metadata"),
|
||||||
|
errhint("Use SELECT citus_set_coordinator_host('<hostname>') "
|
||||||
|
"to configure the coordinator hostname")));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* InsertCoordinatorIfClusterEmpty can be used to ensure Citus tables can be
|
* InsertCoordinatorIfClusterEmpty can be used to ensure Citus tables can be
|
||||||
* created even on a node that has just performed CREATE EXTENSION citus;
|
* created even on a node that has just performed CREATE EXTENSION citus;
|
||||||
|
|
|
@ -23,6 +23,7 @@
|
||||||
#include "distributed/connection_management.h"
|
#include "distributed/connection_management.h"
|
||||||
#include "distributed/remote_commands.h"
|
#include "distributed/remote_commands.h"
|
||||||
#include "distributed/shard_split.h"
|
#include "distributed/shard_split.h"
|
||||||
|
#include "distributed/utils/distribution_column_map.h"
|
||||||
|
|
||||||
/* declarations for dynamic loading */
|
/* declarations for dynamic loading */
|
||||||
PG_FUNCTION_INFO_V1(citus_split_shard_by_split_points);
|
PG_FUNCTION_INFO_V1(citus_split_shard_by_split_points);
|
||||||
|
@ -52,12 +53,17 @@ citus_split_shard_by_split_points(PG_FUNCTION_ARGS)
|
||||||
Oid shardTransferModeOid = PG_GETARG_OID(3);
|
Oid shardTransferModeOid = PG_GETARG_OID(3);
|
||||||
SplitMode shardSplitMode = LookupSplitMode(shardTransferModeOid);
|
SplitMode shardSplitMode = LookupSplitMode(shardTransferModeOid);
|
||||||
|
|
||||||
|
DistributionColumnMap *distributionColumnOverrides = NULL;
|
||||||
|
List *sourceColocatedShardIntervalList = NIL;
|
||||||
SplitShard(
|
SplitShard(
|
||||||
shardSplitMode,
|
shardSplitMode,
|
||||||
SHARD_SPLIT_API,
|
SHARD_SPLIT_API,
|
||||||
shardIdToSplit,
|
shardIdToSplit,
|
||||||
shardSplitPointsList,
|
shardSplitPointsList,
|
||||||
nodeIdsForPlacementList);
|
nodeIdsForPlacementList,
|
||||||
|
distributionColumnOverrides,
|
||||||
|
sourceColocatedShardIntervalList,
|
||||||
|
INVALID_COLOCATION_ID);
|
||||||
|
|
||||||
PG_RETURN_VOID();
|
PG_RETURN_VOID();
|
||||||
}
|
}
|
||||||
|
|
|
@ -33,6 +33,7 @@
|
||||||
#include "distributed/worker_transaction.h"
|
#include "distributed/worker_transaction.h"
|
||||||
#include "distributed/version_compat.h"
|
#include "distributed/version_compat.h"
|
||||||
#include "distributed/shard_split.h"
|
#include "distributed/shard_split.h"
|
||||||
|
#include "distributed/utils/distribution_column_map.h"
|
||||||
#include "nodes/pg_list.h"
|
#include "nodes/pg_list.h"
|
||||||
#include "storage/lock.h"
|
#include "storage/lock.h"
|
||||||
#include "utils/builtins.h"
|
#include "utils/builtins.h"
|
||||||
|
@ -163,12 +164,17 @@ isolate_tenant_to_new_shard(PG_FUNCTION_ARGS)
|
||||||
nodeIdsForPlacementList = lappend_int(nodeIdsForPlacementList, sourceNodeId);
|
nodeIdsForPlacementList = lappend_int(nodeIdsForPlacementList, sourceNodeId);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
DistributionColumnMap *distributionColumnOverrides = NULL;
|
||||||
|
List *sourceColocatedShardIntervalList = NIL;
|
||||||
SplitMode splitMode = LookupSplitMode(shardTransferModeOid);
|
SplitMode splitMode = LookupSplitMode(shardTransferModeOid);
|
||||||
SplitShard(splitMode,
|
SplitShard(splitMode,
|
||||||
ISOLATE_TENANT_TO_NEW_SHARD,
|
ISOLATE_TENANT_TO_NEW_SHARD,
|
||||||
sourceShard->shardId,
|
sourceShard->shardId,
|
||||||
shardSplitPointsList,
|
shardSplitPointsList,
|
||||||
nodeIdsForPlacementList);
|
nodeIdsForPlacementList,
|
||||||
|
distributionColumnOverrides,
|
||||||
|
sourceColocatedShardIntervalList,
|
||||||
|
INVALID_COLOCATION_ID);
|
||||||
|
|
||||||
cacheEntry = GetCitusTableCacheEntry(relationId);
|
cacheEntry = GetCitusTableCacheEntry(relationId);
|
||||||
ShardInterval *newShard = FindShardInterval(tenantIdDatum, cacheEntry);
|
ShardInterval *newShard = FindShardInterval(tenantIdDatum, cacheEntry);
|
||||||
|
|
|
@ -71,7 +71,6 @@ typedef struct ShardCommandList
|
||||||
} ShardCommandList;
|
} ShardCommandList;
|
||||||
|
|
||||||
/* local function forward declarations */
|
/* local function forward declarations */
|
||||||
static bool RelationCanPublishAllModifications(Oid relationId);
|
|
||||||
static bool CanUseLogicalReplication(Oid relationId, char shardReplicationMode);
|
static bool CanUseLogicalReplication(Oid relationId, char shardReplicationMode);
|
||||||
static void ErrorIfTableCannotBeReplicated(Oid relationId);
|
static void ErrorIfTableCannotBeReplicated(Oid relationId);
|
||||||
static void ErrorIfTargetNodeIsNotSafeToCopyTo(const char *targetNodeName,
|
static void ErrorIfTargetNodeIsNotSafeToCopyTo(const char *targetNodeName,
|
||||||
|
@ -635,7 +634,7 @@ VerifyTablesHaveReplicaIdentity(List *colocatedTableList)
|
||||||
* RelationCanPublishAllModifications returns true if the relation is safe to publish
|
* RelationCanPublishAllModifications returns true if the relation is safe to publish
|
||||||
* all modification while being replicated via logical replication.
|
* all modification while being replicated via logical replication.
|
||||||
*/
|
*/
|
||||||
static bool
|
bool
|
||||||
RelationCanPublishAllModifications(Oid relationId)
|
RelationCanPublishAllModifications(Oid relationId)
|
||||||
{
|
{
|
||||||
Relation relation = RelationIdGetRelation(relationId);
|
Relation relation = RelationIdGetRelation(relationId);
|
||||||
|
|
|
@ -30,6 +30,7 @@
|
||||||
#include "distributed/shard_split.h"
|
#include "distributed/shard_split.h"
|
||||||
#include "distributed/reference_table_utils.h"
|
#include "distributed/reference_table_utils.h"
|
||||||
#include "distributed/repair_shards.h"
|
#include "distributed/repair_shards.h"
|
||||||
|
#include "distributed/resource_lock.h"
|
||||||
#include "distributed/multi_partitioning_utils.h"
|
#include "distributed/multi_partitioning_utils.h"
|
||||||
#include "distributed/worker_manager.h"
|
#include "distributed/worker_manager.h"
|
||||||
#include "distributed/worker_transaction.h"
|
#include "distributed/worker_transaction.h"
|
||||||
|
@ -37,6 +38,7 @@
|
||||||
#include "distributed/pg_dist_shard.h"
|
#include "distributed/pg_dist_shard.h"
|
||||||
#include "distributed/metadata_sync.h"
|
#include "distributed/metadata_sync.h"
|
||||||
#include "distributed/multi_physical_planner.h"
|
#include "distributed/multi_physical_planner.h"
|
||||||
|
#include "distributed/utils/distribution_column_map.h"
|
||||||
#include "commands/dbcommands.h"
|
#include "commands/dbcommands.h"
|
||||||
#include "distributed/shardsplit_logical_replication.h"
|
#include "distributed/shardsplit_logical_replication.h"
|
||||||
#include "distributed/deparse_shard_query.h"
|
#include "distributed/deparse_shard_query.h"
|
||||||
|
@ -71,12 +73,8 @@ static void ErrorIfCannotSplitShardExtended(SplitOperation splitOperation,
|
||||||
ShardInterval *shardIntervalToSplit,
|
ShardInterval *shardIntervalToSplit,
|
||||||
List *shardSplitPointsList,
|
List *shardSplitPointsList,
|
||||||
List *nodeIdsForPlacementList);
|
List *nodeIdsForPlacementList);
|
||||||
static void CreateAndCopySplitShardsForShardGroup(
|
static void ErrorIfModificationAndSplitInTheSameTransaction(SplitOperation
|
||||||
HTAB *mapOfShardToPlacementCreatedByWorkflow,
|
splitOperation);
|
||||||
WorkerNode *sourceShardNode,
|
|
||||||
List *sourceColocatedShardIntervalList,
|
|
||||||
List *shardGroupSplitIntervalListList,
|
|
||||||
List *workersForPlacementList);
|
|
||||||
static void CreateSplitShardsForShardGroup(HTAB *mapOfShardToPlacementCreatedByWorkflow,
|
static void CreateSplitShardsForShardGroup(HTAB *mapOfShardToPlacementCreatedByWorkflow,
|
||||||
List *shardGroupSplitIntervalListList,
|
List *shardGroupSplitIntervalListList,
|
||||||
List *workersForPlacementList);
|
List *workersForPlacementList);
|
||||||
|
@ -98,23 +96,35 @@ static void CreateSplitIntervalsForShard(ShardInterval *sourceShard,
|
||||||
List *splitPointsForShard,
|
List *splitPointsForShard,
|
||||||
List **shardSplitChildrenIntervalList);
|
List **shardSplitChildrenIntervalList);
|
||||||
static void BlockingShardSplit(SplitOperation splitOperation,
|
static void BlockingShardSplit(SplitOperation splitOperation,
|
||||||
ShardInterval *shardIntervalToSplit,
|
uint64 splitWorkflowId,
|
||||||
|
List *sourceColocatedShardIntervalList,
|
||||||
List *shardSplitPointsList,
|
List *shardSplitPointsList,
|
||||||
List *workersForPlacementList);
|
List *workersForPlacementList,
|
||||||
|
DistributionColumnMap *distributionColumnOverrides);
|
||||||
static void NonBlockingShardSplit(SplitOperation splitOperation,
|
static void NonBlockingShardSplit(SplitOperation splitOperation,
|
||||||
ShardInterval *shardIntervalToSplit,
|
uint64 splitWorkflowId,
|
||||||
|
List *sourceColocatedShardIntervalList,
|
||||||
List *shardSplitPointsList,
|
List *shardSplitPointsList,
|
||||||
List *workersForPlacementList);
|
List *workersForPlacementList,
|
||||||
|
DistributionColumnMap *distributionColumnOverrides,
|
||||||
|
uint32 targetColocationId);
|
||||||
static void DoSplitCopy(WorkerNode *sourceShardNode,
|
static void DoSplitCopy(WorkerNode *sourceShardNode,
|
||||||
List *sourceColocatedShardIntervalList,
|
List *sourceColocatedShardIntervalList,
|
||||||
List *shardGroupSplitIntervalListList,
|
List *shardGroupSplitIntervalListList,
|
||||||
List *workersForPlacementList,
|
List *workersForPlacementList,
|
||||||
char *snapShotName);
|
char *snapShotName,
|
||||||
|
DistributionColumnMap *distributionColumnOverrides);
|
||||||
static StringInfo CreateSplitCopyCommand(ShardInterval *sourceShardSplitInterval,
|
static StringInfo CreateSplitCopyCommand(ShardInterval *sourceShardSplitInterval,
|
||||||
|
char *distributionColumnName,
|
||||||
List *splitChildrenShardIntervalList,
|
List *splitChildrenShardIntervalList,
|
||||||
List *workersForPlacementList);
|
List *workersForPlacementList);
|
||||||
static Task * CreateSplitCopyTask(StringInfo splitCopyUdfCommand, char *snapshotName, int
|
static Task * CreateSplitCopyTask(StringInfo splitCopyUdfCommand, char *snapshotName, int
|
||||||
taskId, uint64 jobId);
|
taskId, uint64 jobId);
|
||||||
|
static void UpdateDistributionColumnsForShardGroup(List *colocatedShardList,
|
||||||
|
DistributionColumnMap *distCols,
|
||||||
|
char distributionMethod,
|
||||||
|
int shardCount,
|
||||||
|
uint32 colocationId);
|
||||||
static void InsertSplitChildrenShardMetadata(List *shardGroupSplitIntervalListList,
|
static void InsertSplitChildrenShardMetadata(List *shardGroupSplitIntervalListList,
|
||||||
List *workersForPlacementList);
|
List *workersForPlacementList);
|
||||||
static void CreatePartitioningHierarchy(List *shardGroupSplitIntervalListList,
|
static void CreatePartitioningHierarchy(List *shardGroupSplitIntervalListList,
|
||||||
|
@ -126,30 +136,38 @@ static HTAB * CreateEmptyMapForShardsCreatedByWorkflow();
|
||||||
static Task * CreateTaskForDDLCommandList(List *ddlCommandList, WorkerNode *workerNode);
|
static Task * CreateTaskForDDLCommandList(List *ddlCommandList, WorkerNode *workerNode);
|
||||||
static StringInfo CreateSplitShardReplicationSetupUDF(
|
static StringInfo CreateSplitShardReplicationSetupUDF(
|
||||||
List *sourceColocatedShardIntervalList, List *shardGroupSplitIntervalListList,
|
List *sourceColocatedShardIntervalList, List *shardGroupSplitIntervalListList,
|
||||||
List *destinationWorkerNodesList);
|
List *destinationWorkerNodesList,
|
||||||
|
DistributionColumnMap *
|
||||||
|
distributionColumnOverrides);
|
||||||
static List * ParseReplicationSlotInfoFromResult(PGresult *result);
|
static List * ParseReplicationSlotInfoFromResult(PGresult *result);
|
||||||
|
|
||||||
static List * ExecuteSplitShardReplicationSetupUDF(WorkerNode *sourceWorkerNode,
|
static List * ExecuteSplitShardReplicationSetupUDF(WorkerNode *sourceWorkerNode,
|
||||||
List *sourceColocatedShardIntervalList,
|
List *sourceColocatedShardIntervalList,
|
||||||
List *shardGroupSplitIntervalListList,
|
List *shardGroupSplitIntervalListList,
|
||||||
List *destinationWorkerNodesList);
|
List *destinationWorkerNodesList,
|
||||||
|
DistributionColumnMap *
|
||||||
|
distributionColumnOverrides);
|
||||||
static void ExecuteSplitShardReleaseSharedMemory(WorkerNode *sourceWorkerNode);
|
static void ExecuteSplitShardReleaseSharedMemory(WorkerNode *sourceWorkerNode);
|
||||||
static void AddDummyShardEntryInMap(HTAB *mapOfDummyShards, uint32 targetNodeId,
|
static void AddDummyShardEntryInMap(HTAB *mapOfDummyShards, uint32 targetNodeId,
|
||||||
ShardInterval *shardInterval);
|
ShardInterval *shardInterval);
|
||||||
static void DropDummyShards(HTAB *mapOfDummyShardToPlacement);
|
static void DropDummyShards(HTAB *mapOfDummyShardToPlacement);
|
||||||
static void DropDummyShard(MultiConnection *connection, ShardInterval *shardInterval);
|
static void DropDummyShard(MultiConnection *connection, ShardInterval *shardInterval);
|
||||||
static uint64 GetNextShardIdForSplitChild(void);
|
static uint64 GetNextShardIdForSplitChild(void);
|
||||||
|
static void AcquireNonblockingSplitLock(Oid relationId);
|
||||||
|
static List * GetWorkerNodesFromWorkerIds(List *nodeIdsForPlacementList);
|
||||||
|
|
||||||
/* Customize error message strings based on operation type */
|
/* Customize error message strings based on operation type */
|
||||||
static const char *const SplitOperationName[] =
|
static const char *const SplitOperationName[] =
|
||||||
{
|
{
|
||||||
[SHARD_SPLIT_API] = "split",
|
[SHARD_SPLIT_API] = "split",
|
||||||
[ISOLATE_TENANT_TO_NEW_SHARD] = "isolate",
|
[ISOLATE_TENANT_TO_NEW_SHARD] = "isolate",
|
||||||
|
[CREATE_DISTRIBUTED_TABLE] = "create"
|
||||||
};
|
};
|
||||||
static const char *const SplitTargetName[] =
|
static const char *const SplitTargetName[] =
|
||||||
{
|
{
|
||||||
[SHARD_SPLIT_API] = "shard",
|
[SHARD_SPLIT_API] = "shard",
|
||||||
[ISOLATE_TENANT_TO_NEW_SHARD] = "tenant",
|
[ISOLATE_TENANT_TO_NEW_SHARD] = "tenant",
|
||||||
|
[CREATE_DISTRIBUTED_TABLE] = "distributed table"
|
||||||
};
|
};
|
||||||
|
|
||||||
/* Function definitions */
|
/* Function definitions */
|
||||||
|
@ -230,6 +248,12 @@ ErrorIfCannotSplitShardExtended(SplitOperation splitOperation,
|
||||||
List *shardSplitPointsList,
|
List *shardSplitPointsList,
|
||||||
List *nodeIdsForPlacementList)
|
List *nodeIdsForPlacementList)
|
||||||
{
|
{
|
||||||
|
/* we should not perform checks for create distributed table operation */
|
||||||
|
if (splitOperation == CREATE_DISTRIBUTED_TABLE)
|
||||||
|
{
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
CitusTableCacheEntry *cachedTableEntry = GetCitusTableCacheEntry(
|
CitusTableCacheEntry *cachedTableEntry = GetCitusTableCacheEntry(
|
||||||
shardIntervalToSplit->relationId);
|
shardIntervalToSplit->relationId);
|
||||||
|
|
||||||
|
@ -352,20 +376,11 @@ ErrorIfCannotSplitShardExtended(SplitOperation splitOperation,
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* SplitShard API to split a given shard (or shard group) based on specified split points
|
* ErrorIfModificationAndSplitInTheSameTransaction will error if we detect split operation
|
||||||
* to a set of destination nodes.
|
* in the same transaction which has modification before.
|
||||||
* 'splitMode' : Mode of split operation.
|
|
||||||
* 'splitOperation' : Customer operation that triggered split.
|
|
||||||
* 'shardInterval' : Source shard interval to be split.
|
|
||||||
* 'shardSplitPointsList' : Split Points list for the source 'shardInterval'.
|
|
||||||
* 'nodeIdsForPlacementList' : Placement list corresponding to split children.
|
|
||||||
*/
|
*/
|
||||||
void
|
static void
|
||||||
SplitShard(SplitMode splitMode,
|
ErrorIfModificationAndSplitInTheSameTransaction(SplitOperation splitOperation)
|
||||||
SplitOperation splitOperation,
|
|
||||||
uint64 shardIdToSplit,
|
|
||||||
List *shardSplitPointsList,
|
|
||||||
List *nodeIdsForPlacementList)
|
|
||||||
{
|
{
|
||||||
if (XactModificationLevel > XACT_MODIFICATION_NONE)
|
if (XactModificationLevel > XACT_MODIFICATION_NONE)
|
||||||
{
|
{
|
||||||
|
@ -375,13 +390,82 @@ SplitShard(SplitMode splitMode,
|
||||||
SplitOperationName[splitOperation],
|
SplitOperationName[splitOperation],
|
||||||
SplitTargetName[splitOperation])));
|
SplitTargetName[splitOperation])));
|
||||||
}
|
}
|
||||||
else if (PlacementMovedUsingLogicalReplicationInTX)
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* ErrorIfMultipleNonblockingMoveSplitInTheSameTransaction will error if we detect multiple
|
||||||
|
* nonblocking shard movements/splits in the same transaction.
|
||||||
|
*/
|
||||||
|
void
|
||||||
|
ErrorIfMultipleNonblockingMoveSplitInTheSameTransaction(void)
|
||||||
|
{
|
||||||
|
if (PlacementMovedUsingLogicalReplicationInTX)
|
||||||
{
|
{
|
||||||
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
||||||
errmsg("multiple shard movements/splits via logical "
|
errmsg("multiple shard movements/splits via logical "
|
||||||
"replication in the same transaction is currently "
|
"replication in the same transaction is currently "
|
||||||
"not supported")));
|
"not supported")));
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* GetWorkerNodesFromWorkerIds returns list of worker nodes given a list
|
||||||
|
* of worker ids. It will error if any node id is invalid.
|
||||||
|
*/
|
||||||
|
static List *
|
||||||
|
GetWorkerNodesFromWorkerIds(List *nodeIdsForPlacementList)
|
||||||
|
{
|
||||||
|
List *workersForPlacementList = NIL;
|
||||||
|
int32 nodeId;
|
||||||
|
foreach_int(nodeId, nodeIdsForPlacementList)
|
||||||
|
{
|
||||||
|
uint32 nodeIdValue = (uint32) nodeId;
|
||||||
|
WorkerNode *workerNode = LookupNodeByNodeId(nodeIdValue);
|
||||||
|
|
||||||
|
/* NodeId in Citus are unsigned and range from [1, 4294967296]. */
|
||||||
|
if (nodeIdValue < 1 || workerNode == NULL)
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_ACTIVE_SQL_TRANSACTION),
|
||||||
|
errmsg("Invalid Node Id '%u'.", nodeIdValue)));
|
||||||
|
}
|
||||||
|
|
||||||
|
workersForPlacementList =
|
||||||
|
lappend(workersForPlacementList, (void *) workerNode);
|
||||||
|
}
|
||||||
|
|
||||||
|
return workersForPlacementList;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* SplitShard API to split a given shard (or shard group) based on specified split points
|
||||||
|
* to a set of destination nodes.
|
||||||
|
* 'splitMode' : Mode of split operation.
|
||||||
|
* 'splitOperation' : Customer operation that triggered split.
|
||||||
|
* 'shardInterval' : Source shard interval to be split.
|
||||||
|
* 'shardSplitPointsList' : Split Points list for the source 'shardInterval'.
|
||||||
|
* 'nodeIdsForPlacementList' : Placement list corresponding to split children.
|
||||||
|
* 'distributionColumnList' : Maps relation IDs to distribution columns.
|
||||||
|
* If not specified, the distribution column is read
|
||||||
|
* from the metadata.
|
||||||
|
* 'colocatedShardIntervalList' : Shard interval list for colocation group. (only used for
|
||||||
|
* create_distributed_table_concurrently).
|
||||||
|
* 'targetColocationId' : Specifies the colocation ID (only used for
|
||||||
|
* create_distributed_table_concurrently).
|
||||||
|
*/
|
||||||
|
void
|
||||||
|
SplitShard(SplitMode splitMode,
|
||||||
|
SplitOperation splitOperation,
|
||||||
|
uint64 shardIdToSplit,
|
||||||
|
List *shardSplitPointsList,
|
||||||
|
List *nodeIdsForPlacementList,
|
||||||
|
DistributionColumnMap *distributionColumnOverrides,
|
||||||
|
List *colocatedShardIntervalList,
|
||||||
|
uint32 targetColocationId)
|
||||||
|
{
|
||||||
|
ErrorIfModificationAndSplitInTheSameTransaction(splitOperation);
|
||||||
|
|
||||||
ShardInterval *shardIntervalToSplit = LoadShardInterval(shardIdToSplit);
|
ShardInterval *shardIntervalToSplit = LoadShardInterval(shardIdToSplit);
|
||||||
List *colocatedTableList = ColocatedTableList(shardIntervalToSplit->relationId);
|
List *colocatedTableList = ColocatedTableList(shardIntervalToSplit->relationId);
|
||||||
|
@ -391,6 +475,7 @@ SplitShard(SplitMode splitMode,
|
||||||
VerifyTablesHaveReplicaIdentity(colocatedTableList);
|
VerifyTablesHaveReplicaIdentity(colocatedTableList);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/* Acquire global lock to prevent concurrent split on the same colocation group or relation */
|
||||||
Oid relationId = RelationIdForShard(shardIdToSplit);
|
Oid relationId = RelationIdForShard(shardIdToSplit);
|
||||||
AcquirePlacementColocationLock(relationId, ExclusiveLock, "split");
|
AcquirePlacementColocationLock(relationId, ExclusiveLock, "split");
|
||||||
|
|
||||||
|
@ -407,47 +492,50 @@ SplitShard(SplitMode splitMode,
|
||||||
LockRelationOid(colocatedTableId, ShareUpdateExclusiveLock);
|
LockRelationOid(colocatedTableId, ShareUpdateExclusiveLock);
|
||||||
}
|
}
|
||||||
|
|
||||||
ErrorIfCannotSplitShard(SHARD_SPLIT_API, shardIntervalToSplit);
|
ErrorIfCannotSplitShard(splitOperation, shardIntervalToSplit);
|
||||||
ErrorIfCannotSplitShardExtended(
|
ErrorIfCannotSplitShardExtended(
|
||||||
SHARD_SPLIT_API,
|
splitOperation,
|
||||||
shardIntervalToSplit,
|
shardIntervalToSplit,
|
||||||
shardSplitPointsList,
|
shardSplitPointsList,
|
||||||
nodeIdsForPlacementList);
|
nodeIdsForPlacementList);
|
||||||
|
|
||||||
List *workersForPlacementList = NIL;
|
List *workersForPlacementList = GetWorkerNodesFromWorkerIds(nodeIdsForPlacementList);
|
||||||
Datum nodeId;
|
|
||||||
foreach_int(nodeId, nodeIdsForPlacementList)
|
|
||||||
{
|
|
||||||
uint32 nodeIdValue = DatumGetUInt32(nodeId);
|
|
||||||
WorkerNode *workerNode = LookupNodeByNodeId(nodeIdValue);
|
|
||||||
|
|
||||||
/* NodeId in Citus are unsigned and range from [1, 4294967296]. */
|
List *sourceColocatedShardIntervalList = NIL;
|
||||||
if (nodeIdValue < 1 || workerNode == NULL)
|
if (colocatedShardIntervalList == NIL)
|
||||||
{
|
{
|
||||||
ereport(ERROR, (errcode(ERRCODE_ACTIVE_SQL_TRANSACTION),
|
sourceColocatedShardIntervalList = ColocatedShardIntervalList(
|
||||||
errmsg("Invalid Node Id '%u'.", nodeIdValue)));
|
shardIntervalToSplit);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
sourceColocatedShardIntervalList = colocatedShardIntervalList;
|
||||||
}
|
}
|
||||||
|
|
||||||
workersForPlacementList =
|
/* use the user-specified shard ID as the split workflow ID */
|
||||||
lappend(workersForPlacementList, (void *) workerNode);
|
uint64 splitWorkflowId = shardIntervalToSplit->shardId;
|
||||||
}
|
|
||||||
|
|
||||||
if (splitMode == BLOCKING_SPLIT)
|
if (splitMode == BLOCKING_SPLIT)
|
||||||
{
|
{
|
||||||
EnsureReferenceTablesExistOnAllNodesExtended(TRANSFER_MODE_BLOCK_WRITES);
|
EnsureReferenceTablesExistOnAllNodesExtended(TRANSFER_MODE_BLOCK_WRITES);
|
||||||
BlockingShardSplit(
|
BlockingShardSplit(
|
||||||
splitOperation,
|
splitOperation,
|
||||||
shardIntervalToSplit,
|
splitWorkflowId,
|
||||||
|
sourceColocatedShardIntervalList,
|
||||||
shardSplitPointsList,
|
shardSplitPointsList,
|
||||||
workersForPlacementList);
|
workersForPlacementList,
|
||||||
|
distributionColumnOverrides);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
NonBlockingShardSplit(
|
NonBlockingShardSplit(
|
||||||
splitOperation,
|
splitOperation,
|
||||||
shardIntervalToSplit,
|
splitWorkflowId,
|
||||||
|
sourceColocatedShardIntervalList,
|
||||||
shardSplitPointsList,
|
shardSplitPointsList,
|
||||||
workersForPlacementList);
|
workersForPlacementList,
|
||||||
|
distributionColumnOverrides,
|
||||||
|
targetColocationId);
|
||||||
|
|
||||||
PlacementMovedUsingLogicalReplicationInTX = true;
|
PlacementMovedUsingLogicalReplicationInTX = true;
|
||||||
}
|
}
|
||||||
|
@ -521,20 +609,20 @@ CreateEmptyMapForShardsCreatedByWorkflow()
|
||||||
/*
|
/*
|
||||||
* SplitShard API to split a given shard (or shard group) in blocking fashion
|
* SplitShard API to split a given shard (or shard group) in blocking fashion
|
||||||
* based on specified split points to a set of destination nodes.
|
* based on specified split points to a set of destination nodes.
|
||||||
* 'splitOperation' : Customer operation that triggered split.
|
* splitOperation : Customer operation that triggered split.
|
||||||
* 'shardIntervalToSplit' : Source shard interval to be split.
|
* splitWorkflowId : Number used to identify split workflow in names.
|
||||||
* 'shardSplitPointsList' : Split Points list for the source 'shardInterval'.
|
* sourceColocatedShardIntervalList : Source shard group to be split.
|
||||||
* 'workersForPlacementList' : Placement list corresponding to split children.
|
* shardSplitPointsList : Split Points list for the source 'shardInterval'.
|
||||||
|
* workersForPlacementList : Placement list corresponding to split children.
|
||||||
*/
|
*/
|
||||||
static void
|
static void
|
||||||
BlockingShardSplit(SplitOperation splitOperation,
|
BlockingShardSplit(SplitOperation splitOperation,
|
||||||
ShardInterval *shardIntervalToSplit,
|
uint64 splitWorkflowId,
|
||||||
|
List *sourceColocatedShardIntervalList,
|
||||||
List *shardSplitPointsList,
|
List *shardSplitPointsList,
|
||||||
List *workersForPlacementList)
|
List *workersForPlacementList,
|
||||||
|
DistributionColumnMap *distributionColumnOverrides)
|
||||||
{
|
{
|
||||||
List *sourceColocatedShardIntervalList = ColocatedShardIntervalList(
|
|
||||||
shardIntervalToSplit);
|
|
||||||
|
|
||||||
BlockWritesToShardList(sourceColocatedShardIntervalList);
|
BlockWritesToShardList(sourceColocatedShardIntervalList);
|
||||||
|
|
||||||
/* First create shard interval metadata for split children */
|
/* First create shard interval metadata for split children */
|
||||||
|
@ -543,30 +631,30 @@ BlockingShardSplit(SplitOperation splitOperation,
|
||||||
shardSplitPointsList);
|
shardSplitPointsList);
|
||||||
|
|
||||||
/* Only single placement allowed (already validated RelationReplicationFactor = 1) */
|
/* Only single placement allowed (already validated RelationReplicationFactor = 1) */
|
||||||
List *sourcePlacementList = ActiveShardPlacementList(shardIntervalToSplit->shardId);
|
ShardInterval *firstShard = linitial(sourceColocatedShardIntervalList);
|
||||||
Assert(sourcePlacementList->length == 1);
|
WorkerNode *sourceShardNode =
|
||||||
ShardPlacement *sourceShardPlacement = (ShardPlacement *) linitial(
|
ActiveShardPlacementWorkerNode(firstShard->shardId);
|
||||||
sourcePlacementList);
|
|
||||||
WorkerNode *sourceShardToCopyNode = FindNodeWithNodeId(sourceShardPlacement->nodeId,
|
|
||||||
false /* missingOk */);
|
|
||||||
|
|
||||||
|
|
||||||
HTAB *mapOfShardToPlacementCreatedByWorkflow =
|
HTAB *mapOfShardToPlacementCreatedByWorkflow =
|
||||||
CreateEmptyMapForShardsCreatedByWorkflow();
|
CreateEmptyMapForShardsCreatedByWorkflow();
|
||||||
PG_TRY();
|
PG_TRY();
|
||||||
{
|
{
|
||||||
/*
|
/* Physically create split children. */
|
||||||
* Physically create split children, perform split copy and create auxiliary structures.
|
CreateSplitShardsForShardGroup(mapOfShardToPlacementCreatedByWorkflow,
|
||||||
* This includes: indexes, replicaIdentity. triggers and statistics.
|
|
||||||
* Foreign key constraints are created after Metadata changes (see CreateForeignKeyConstraints).
|
|
||||||
*/
|
|
||||||
CreateAndCopySplitShardsForShardGroup(
|
|
||||||
mapOfShardToPlacementCreatedByWorkflow,
|
|
||||||
sourceShardToCopyNode,
|
|
||||||
sourceColocatedShardIntervalList,
|
|
||||||
shardGroupSplitIntervalListList,
|
shardGroupSplitIntervalListList,
|
||||||
workersForPlacementList);
|
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
|
* Up to this point, we performed various subtransactions that may
|
||||||
* require additional clean-up in case of failure. The remaining operations
|
* require additional clean-up in case of failure. The remaining operations
|
||||||
|
@ -608,6 +696,7 @@ BlockingShardSplit(SplitOperation splitOperation,
|
||||||
}
|
}
|
||||||
PG_END_TRY();
|
PG_END_TRY();
|
||||||
|
|
||||||
|
|
||||||
CitusInvalidateRelcacheByRelid(DistShardRelationId());
|
CitusInvalidateRelcacheByRelid(DistShardRelationId());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -731,33 +820,6 @@ CreateAuxiliaryStructuresForShardGroup(List *shardGroupSplitIntervalListList,
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
|
||||||
* Create ShardGroup split children, perform copy and create auxiliary structures
|
|
||||||
* on a list of corresponding workers.
|
|
||||||
*/
|
|
||||||
static void
|
|
||||||
CreateAndCopySplitShardsForShardGroup(HTAB *mapOfShardToPlacementCreatedByWorkflow,
|
|
||||||
WorkerNode *sourceShardNode,
|
|
||||||
List *sourceColocatedShardIntervalList,
|
|
||||||
List *shardGroupSplitIntervalListList,
|
|
||||||
List *workersForPlacementList)
|
|
||||||
{
|
|
||||||
CreateSplitShardsForShardGroup(mapOfShardToPlacementCreatedByWorkflow,
|
|
||||||
shardGroupSplitIntervalListList,
|
|
||||||
workersForPlacementList);
|
|
||||||
|
|
||||||
/* For Blocking split, copy isn't snapshotted */
|
|
||||||
char *snapshotName = NULL;
|
|
||||||
DoSplitCopy(sourceShardNode, sourceColocatedShardIntervalList,
|
|
||||||
shardGroupSplitIntervalListList, workersForPlacementList, snapshotName);
|
|
||||||
|
|
||||||
/* Create auxiliary structures (indexes, stats, replicaindentities, triggers) */
|
|
||||||
CreateAuxiliaryStructuresForShardGroup(shardGroupSplitIntervalListList,
|
|
||||||
workersForPlacementList,
|
|
||||||
true /* includeReplicaIdentity*/);
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Perform Split Copy from source shard(s) to split children.
|
* Perform Split Copy from source shard(s) to split children.
|
||||||
* 'sourceShardNode' : Source shard worker node.
|
* 'sourceShardNode' : Source shard worker node.
|
||||||
|
@ -768,7 +830,7 @@ CreateAndCopySplitShardsForShardGroup(HTAB *mapOfShardToPlacementCreatedByWorkfl
|
||||||
static void
|
static void
|
||||||
DoSplitCopy(WorkerNode *sourceShardNode, List *sourceColocatedShardIntervalList,
|
DoSplitCopy(WorkerNode *sourceShardNode, List *sourceColocatedShardIntervalList,
|
||||||
List *shardGroupSplitIntervalListList, List *destinationWorkerNodesList,
|
List *shardGroupSplitIntervalListList, List *destinationWorkerNodesList,
|
||||||
char *snapShotName)
|
char *snapShotName, DistributionColumnMap *distributionColumnOverrides)
|
||||||
{
|
{
|
||||||
ShardInterval *sourceShardIntervalToCopy = NULL;
|
ShardInterval *sourceShardIntervalToCopy = NULL;
|
||||||
List *splitShardIntervalList = NIL;
|
List *splitShardIntervalList = NIL;
|
||||||
|
@ -783,10 +845,26 @@ DoSplitCopy(WorkerNode *sourceShardNode, List *sourceColocatedShardIntervalList,
|
||||||
* data themselves. Their partitions do contain data, but those are
|
* data themselves. Their partitions do contain data, but those are
|
||||||
* different colocated shards that will be copied seperately.
|
* different colocated shards that will be copied seperately.
|
||||||
*/
|
*/
|
||||||
if (!PartitionedTable(sourceShardIntervalToCopy->relationId))
|
if (PartitionedTable(sourceShardIntervalToCopy->relationId))
|
||||||
{
|
{
|
||||||
|
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(
|
StringInfo splitCopyUdfCommand = CreateSplitCopyCommand(
|
||||||
sourceShardIntervalToCopy,
|
sourceShardIntervalToCopy,
|
||||||
|
distributionColumnName,
|
||||||
splitShardIntervalList,
|
splitShardIntervalList,
|
||||||
destinationWorkerNodesList);
|
destinationWorkerNodesList);
|
||||||
|
|
||||||
|
@ -802,7 +880,6 @@ DoSplitCopy(WorkerNode *sourceShardNode, List *sourceColocatedShardIntervalList,
|
||||||
splitCopyTaskList = lappend(splitCopyTaskList, splitCopyTask);
|
splitCopyTaskList = lappend(splitCopyTaskList, splitCopyTask);
|
||||||
taskId++;
|
taskId++;
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
ExecuteTaskListOutsideTransaction(ROW_MODIFY_NONE, splitCopyTaskList,
|
ExecuteTaskListOutsideTransaction(ROW_MODIFY_NONE, splitCopyTaskList,
|
||||||
MaxAdaptiveExecutorPoolSize,
|
MaxAdaptiveExecutorPoolSize,
|
||||||
|
@ -834,6 +911,7 @@ DoSplitCopy(WorkerNode *sourceShardNode, List *sourceColocatedShardIntervalList,
|
||||||
*/
|
*/
|
||||||
static StringInfo
|
static StringInfo
|
||||||
CreateSplitCopyCommand(ShardInterval *sourceShardSplitInterval,
|
CreateSplitCopyCommand(ShardInterval *sourceShardSplitInterval,
|
||||||
|
char *distributionColumnName,
|
||||||
List *splitChildrenShardIntervalList,
|
List *splitChildrenShardIntervalList,
|
||||||
List *destinationWorkerNodesList)
|
List *destinationWorkerNodesList)
|
||||||
{
|
{
|
||||||
|
@ -865,8 +943,9 @@ CreateSplitCopyCommand(ShardInterval *sourceShardSplitInterval,
|
||||||
appendStringInfo(splitCopyInfoArray, "]");
|
appendStringInfo(splitCopyInfoArray, "]");
|
||||||
|
|
||||||
StringInfo splitCopyUdf = makeStringInfo();
|
StringInfo splitCopyUdf = makeStringInfo();
|
||||||
appendStringInfo(splitCopyUdf, "SELECT pg_catalog.worker_split_copy(%lu, %s);",
|
appendStringInfo(splitCopyUdf, "SELECT pg_catalog.worker_split_copy(%lu, %s, %s);",
|
||||||
sourceShardSplitInterval->shardId,
|
sourceShardSplitInterval->shardId,
|
||||||
|
quote_literal_cstr(distributionColumnName),
|
||||||
splitCopyInfoArray->data);
|
splitCopyInfoArray->data);
|
||||||
|
|
||||||
return splitCopyUdf;
|
return splitCopyUdf;
|
||||||
|
@ -976,8 +1055,19 @@ CreateSplitIntervalsForShard(ShardInterval *sourceShard,
|
||||||
int shardIntervalCount = list_length(splitPointsForShard) + 1;
|
int shardIntervalCount = list_length(splitPointsForShard) + 1;
|
||||||
ListCell *splitPointCell = list_head(splitPointsForShard);
|
ListCell *splitPointCell = list_head(splitPointsForShard);
|
||||||
int32 splitParentMaxValue = DatumGetInt32(sourceShard->maxValue);
|
int32 splitParentMaxValue = DatumGetInt32(sourceShard->maxValue);
|
||||||
|
|
||||||
int32 currentSplitChildMinValue = DatumGetInt32(sourceShard->minValue);
|
int32 currentSplitChildMinValue = DatumGetInt32(sourceShard->minValue);
|
||||||
|
|
||||||
|
/* if we are splitting a Citus local table, assume whole shard range */
|
||||||
|
if (!sourceShard->maxValueExists)
|
||||||
|
{
|
||||||
|
splitParentMaxValue = PG_INT32_MAX;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!sourceShard->minValueExists)
|
||||||
|
{
|
||||||
|
currentSplitChildMinValue = PG_INT32_MIN;
|
||||||
|
}
|
||||||
|
|
||||||
for (int index = 0; index < shardIntervalCount; index++)
|
for (int index = 0; index < shardIntervalCount; index++)
|
||||||
{
|
{
|
||||||
ShardInterval *splitChildShardInterval = CopyShardInterval(sourceShard);
|
ShardInterval *splitChildShardInterval = CopyShardInterval(sourceShard);
|
||||||
|
@ -1008,6 +1098,54 @@ CreateSplitIntervalsForShard(ShardInterval *sourceShard,
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* UpdateDistributionColumnsForShardGroup globally updates the pg_dist_partition metadata
|
||||||
|
* for each relation that has a shard in colocatedShardList.
|
||||||
|
*
|
||||||
|
* This is used primarily for Citus local -> distributed table conversion
|
||||||
|
* in create_distributed_table_concurrently.
|
||||||
|
*
|
||||||
|
* It would be nicer to keep this separate from shard split, but we need to do the
|
||||||
|
* update at exactly the right point in the shard split process, namely after
|
||||||
|
* replication slot creation and before inserting shard metadata, which itself
|
||||||
|
* needs to happen before foreign key creation (mainly because the foreign key
|
||||||
|
* functions depend on metadata).
|
||||||
|
*/
|
||||||
|
static void
|
||||||
|
UpdateDistributionColumnsForShardGroup(List *colocatedShardList,
|
||||||
|
DistributionColumnMap *distributionColumnMap,
|
||||||
|
char distributionMethod,
|
||||||
|
int shardCount,
|
||||||
|
uint32 colocationId)
|
||||||
|
{
|
||||||
|
ShardInterval *shardInterval = NULL;
|
||||||
|
foreach_ptr(shardInterval, colocatedShardList)
|
||||||
|
{
|
||||||
|
Oid relationId = shardInterval->relationId;
|
||||||
|
Var *distributionColumn = GetDistributionColumnFromMap(distributionColumnMap,
|
||||||
|
relationId);
|
||||||
|
|
||||||
|
/* we should have an entry for every relation ID in the colocation group */
|
||||||
|
Assert(distributionColumn != NULL);
|
||||||
|
|
||||||
|
if (colocationId == INVALID_COLOCATION_ID)
|
||||||
|
{
|
||||||
|
/*
|
||||||
|
* Getting here with an invalid co-location ID means that no
|
||||||
|
* appropriate co-location group exists yet.
|
||||||
|
*/
|
||||||
|
colocationId = CreateColocationGroup(shardCount,
|
||||||
|
ShardReplicationFactor,
|
||||||
|
distributionColumn->vartype,
|
||||||
|
distributionColumn->varcollid);
|
||||||
|
}
|
||||||
|
|
||||||
|
UpdateDistributionColumnGlobally(relationId, distributionMethod,
|
||||||
|
distributionColumn, colocationId);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Insert new shard and placement metadata.
|
* Insert new shard and placement metadata.
|
||||||
* Sync the Metadata with all nodes if enabled.
|
* Sync the Metadata with all nodes if enabled.
|
||||||
|
@ -1264,33 +1402,74 @@ TryDropSplitShardsOnFailure(HTAB *mapOfShardToPlacementCreatedByWorkflow)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* AcquireNonblockingSplitLock does not allow concurrent nonblocking splits, because we share memory and
|
||||||
|
* replication slots.
|
||||||
|
*/
|
||||||
|
static void
|
||||||
|
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
|
* 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.
|
* based on specified split points to a set of destination nodes.
|
||||||
* splitOperation : Customer operation that triggered split.
|
* splitOperation : Customer operation that triggered split.
|
||||||
* shardIntervalToSplit : Source shard interval to be split.
|
* splitWorkflowId : Number used to identify split workflow in names.
|
||||||
|
* sourceColocatedShardIntervalList : Source shard group to be split.
|
||||||
* shardSplitPointsList : Split Points list for the source 'shardInterval'.
|
* shardSplitPointsList : Split Points list for the source 'shardInterval'.
|
||||||
* workersForPlacementList : Placement list corresponding to split children.
|
* 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).
|
||||||
*/
|
*/
|
||||||
static void
|
void
|
||||||
NonBlockingShardSplit(SplitOperation splitOperation,
|
NonBlockingShardSplit(SplitOperation splitOperation,
|
||||||
ShardInterval *shardIntervalToSplit,
|
uint64 splitWorkflowId,
|
||||||
|
List *sourceColocatedShardIntervalList,
|
||||||
List *shardSplitPointsList,
|
List *shardSplitPointsList,
|
||||||
List *workersForPlacementList)
|
List *workersForPlacementList,
|
||||||
|
DistributionColumnMap *distributionColumnOverrides,
|
||||||
|
uint32 targetColocationId)
|
||||||
{
|
{
|
||||||
|
ErrorIfMultipleNonblockingMoveSplitInTheSameTransaction();
|
||||||
|
|
||||||
char *superUser = CitusExtensionOwnerName();
|
char *superUser = CitusExtensionOwnerName();
|
||||||
char *databaseName = get_database_name(MyDatabaseId);
|
char *databaseName = get_database_name(MyDatabaseId);
|
||||||
|
|
||||||
List *sourceColocatedShardIntervalList = ColocatedShardIntervalList(
|
|
||||||
shardIntervalToSplit);
|
|
||||||
|
|
||||||
/* First create shard interval metadata for split children */
|
/* First create shard interval metadata for split children */
|
||||||
List *shardGroupSplitIntervalListList = CreateSplitIntervalsForShardGroup(
|
List *shardGroupSplitIntervalListList = CreateSplitIntervalsForShardGroup(
|
||||||
sourceColocatedShardIntervalList,
|
sourceColocatedShardIntervalList,
|
||||||
shardSplitPointsList);
|
shardSplitPointsList);
|
||||||
|
|
||||||
WorkerNode *sourceShardToCopyNode = ActiveShardPlacementWorkerNode(
|
ShardInterval *firstShard = linitial(sourceColocatedShardIntervalList);
|
||||||
shardIntervalToSplit->shardId);
|
|
||||||
|
/* Acquire global lock to prevent concurrent nonblocking splits */
|
||||||
|
AcquireNonblockingSplitLock(firstShard->relationId);
|
||||||
|
|
||||||
|
WorkerNode *sourceShardToCopyNode =
|
||||||
|
ActiveShardPlacementWorkerNode(firstShard->shardId);
|
||||||
|
|
||||||
/* Create hashmap to group shards for publication-subscription management */
|
/* Create hashmap to group shards for publication-subscription management */
|
||||||
HTAB *publicationInfoHash = CreateShardSplitInfoMapForPublication(
|
HTAB *publicationInfoHash = CreateShardSplitInfoMapForPublication(
|
||||||
|
@ -1352,13 +1531,13 @@ NonBlockingShardSplit(SplitOperation splitOperation,
|
||||||
/* 4) Create Publications. */
|
/* 4) Create Publications. */
|
||||||
CreatePublications(sourceConnection, publicationInfoHash);
|
CreatePublications(sourceConnection, publicationInfoHash);
|
||||||
|
|
||||||
|
|
||||||
/* 5) Execute 'worker_split_shard_replication_setup UDF */
|
/* 5) Execute 'worker_split_shard_replication_setup UDF */
|
||||||
List *replicationSlotInfoList = ExecuteSplitShardReplicationSetupUDF(
|
List *replicationSlotInfoList = ExecuteSplitShardReplicationSetupUDF(
|
||||||
sourceShardToCopyNode,
|
sourceShardToCopyNode,
|
||||||
sourceColocatedShardIntervalList,
|
sourceColocatedShardIntervalList,
|
||||||
shardGroupSplitIntervalListList,
|
shardGroupSplitIntervalListList,
|
||||||
workersForPlacementList);
|
workersForPlacementList,
|
||||||
|
distributionColumnOverrides);
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Subscriber flow starts from here.
|
* Subscriber flow starts from here.
|
||||||
|
@ -1400,7 +1579,7 @@ NonBlockingShardSplit(SplitOperation splitOperation,
|
||||||
/* 8) Do snapshotted Copy */
|
/* 8) Do snapshotted Copy */
|
||||||
DoSplitCopy(sourceShardToCopyNode, sourceColocatedShardIntervalList,
|
DoSplitCopy(sourceShardToCopyNode, sourceColocatedShardIntervalList,
|
||||||
shardGroupSplitIntervalListList, workersForPlacementList,
|
shardGroupSplitIntervalListList, workersForPlacementList,
|
||||||
snapshot);
|
snapshot, distributionColumnOverrides);
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* 9) Create replica identities, this needs to be done before enabling
|
* 9) Create replica identities, this needs to be done before enabling
|
||||||
|
@ -1446,7 +1625,6 @@ NonBlockingShardSplit(SplitOperation splitOperation,
|
||||||
/* 19) Drop Publications */
|
/* 19) Drop Publications */
|
||||||
DropPublications(sourceConnection, publicationInfoHash);
|
DropPublications(sourceConnection, publicationInfoHash);
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* 20) Drop old shards and delete related metadata. Have to do that before
|
* 20) Drop old shards and delete related metadata. Have to do that before
|
||||||
* creating the new shard metadata, because there's cross-checks
|
* creating the new shard metadata, because there's cross-checks
|
||||||
|
@ -1454,7 +1632,40 @@ NonBlockingShardSplit(SplitOperation splitOperation,
|
||||||
*/
|
*/
|
||||||
DropShardList(sourceColocatedShardIntervalList);
|
DropShardList(sourceColocatedShardIntervalList);
|
||||||
|
|
||||||
/* 21) Insert new shard and placement metdata */
|
/*
|
||||||
|
* 21) In case of create_distributed_table_concurrently, which converts
|
||||||
|
* a Citus local table to a distributed table, update the distributed
|
||||||
|
* table metadata now.
|
||||||
|
*
|
||||||
|
* We would rather have this be outside of the scope of NonBlockingShardSplit,
|
||||||
|
* but we cannot make metadata changes before replication slot creation, and
|
||||||
|
* we cannot create the replication slot before creating new shards and
|
||||||
|
* corresponding publications, because the decoder uses a catalog snapshot
|
||||||
|
* from the time of the slot creation, which means it would not be able to see
|
||||||
|
* the shards or publications when replication starts if it was created before.
|
||||||
|
*
|
||||||
|
* We also cannot easily move metadata changes to be after this function,
|
||||||
|
* because CreateForeignKeyConstraints relies on accurate metadata and
|
||||||
|
* we also want to perform the clean-up logic in PG_CATCH in case of
|
||||||
|
* failure.
|
||||||
|
*
|
||||||
|
* Hence, this appears to be the only suitable spot for updating
|
||||||
|
* pg_dist_partition and pg_dist_colocation.
|
||||||
|
*/
|
||||||
|
if (splitOperation == CREATE_DISTRIBUTED_TABLE)
|
||||||
|
{
|
||||||
|
/* we currently only use split for hash-distributed tables */
|
||||||
|
char distributionMethod = DISTRIBUTE_BY_HASH;
|
||||||
|
int shardCount = list_length(shardSplitPointsList) + 1;
|
||||||
|
|
||||||
|
UpdateDistributionColumnsForShardGroup(sourceColocatedShardIntervalList,
|
||||||
|
distributionColumnOverrides,
|
||||||
|
distributionMethod,
|
||||||
|
shardCount,
|
||||||
|
targetColocationId);
|
||||||
|
}
|
||||||
|
|
||||||
|
/* 22) Insert new shard and placement metdata */
|
||||||
InsertSplitChildrenShardMetadata(shardGroupSplitIntervalListList,
|
InsertSplitChildrenShardMetadata(shardGroupSplitIntervalListList,
|
||||||
workersForPlacementList);
|
workersForPlacementList);
|
||||||
|
|
||||||
|
@ -1462,7 +1673,7 @@ NonBlockingShardSplit(SplitOperation splitOperation,
|
||||||
workersForPlacementList);
|
workersForPlacementList);
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* 22) Create foreign keys if exists after the metadata changes happening in
|
* 23) Create foreign keys if exists after the metadata changes happening in
|
||||||
* DropShardList() and InsertSplitChildrenShardMetadata() because the foreign
|
* DropShardList() and InsertSplitChildrenShardMetadata() because the foreign
|
||||||
* key creation depends on the new metadata.
|
* key creation depends on the new metadata.
|
||||||
*/
|
*/
|
||||||
|
@ -1470,7 +1681,7 @@ NonBlockingShardSplit(SplitOperation splitOperation,
|
||||||
workersForPlacementList);
|
workersForPlacementList);
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* 23) Drop dummy shards.
|
* 24) Drop dummy shards.
|
||||||
*/
|
*/
|
||||||
DropDummyShards(mapOfDummyShardToPlacement);
|
DropDummyShards(mapOfDummyShardToPlacement);
|
||||||
|
|
||||||
|
@ -1654,12 +1865,14 @@ static List *
|
||||||
ExecuteSplitShardReplicationSetupUDF(WorkerNode *sourceWorkerNode,
|
ExecuteSplitShardReplicationSetupUDF(WorkerNode *sourceWorkerNode,
|
||||||
List *sourceColocatedShardIntervalList,
|
List *sourceColocatedShardIntervalList,
|
||||||
List *shardGroupSplitIntervalListList,
|
List *shardGroupSplitIntervalListList,
|
||||||
List *destinationWorkerNodesList)
|
List *destinationWorkerNodesList,
|
||||||
|
DistributionColumnMap *distributionColumnOverrides)
|
||||||
{
|
{
|
||||||
StringInfo splitShardReplicationUDF = CreateSplitShardReplicationSetupUDF(
|
StringInfo splitShardReplicationUDF = CreateSplitShardReplicationSetupUDF(
|
||||||
sourceColocatedShardIntervalList,
|
sourceColocatedShardIntervalList,
|
||||||
shardGroupSplitIntervalListList,
|
shardGroupSplitIntervalListList,
|
||||||
destinationWorkerNodesList);
|
destinationWorkerNodesList,
|
||||||
|
distributionColumnOverrides);
|
||||||
|
|
||||||
/* Force a new connection to execute the UDF */
|
/* Force a new connection to execute the UDF */
|
||||||
int connectionFlags = 0;
|
int connectionFlags = 0;
|
||||||
|
@ -1756,7 +1969,8 @@ ExecuteSplitShardReleaseSharedMemory(WorkerNode *sourceWorkerNode)
|
||||||
StringInfo
|
StringInfo
|
||||||
CreateSplitShardReplicationSetupUDF(List *sourceColocatedShardIntervalList,
|
CreateSplitShardReplicationSetupUDF(List *sourceColocatedShardIntervalList,
|
||||||
List *shardGroupSplitIntervalListList,
|
List *shardGroupSplitIntervalListList,
|
||||||
List *destinationWorkerNodesList)
|
List *destinationWorkerNodesList,
|
||||||
|
DistributionColumnMap *distributionColumnOverrides)
|
||||||
{
|
{
|
||||||
StringInfo splitChildrenRows = makeStringInfo();
|
StringInfo splitChildrenRows = makeStringInfo();
|
||||||
|
|
||||||
|
@ -1768,11 +1982,15 @@ CreateSplitShardReplicationSetupUDF(List *sourceColocatedShardIntervalList,
|
||||||
{
|
{
|
||||||
int64 sourceShardId = sourceShardIntervalToCopy->shardId;
|
int64 sourceShardId = sourceShardIntervalToCopy->shardId;
|
||||||
Oid relationId = sourceShardIntervalToCopy->relationId;
|
Oid relationId = sourceShardIntervalToCopy->relationId;
|
||||||
Var *partitionColumn = DistPartitionKey(relationId);
|
|
||||||
|
Var *distributionColumn =
|
||||||
|
GetDistributionColumnWithOverrides(relationId,
|
||||||
|
distributionColumnOverrides);
|
||||||
|
|
||||||
bool missingOK = false;
|
bool missingOK = false;
|
||||||
char *partitionColumnName =
|
char *distributionColumnName =
|
||||||
get_attname(relationId, partitionColumn->varattno, missingOK);
|
get_attname(relationId, distributionColumn->varattno,
|
||||||
|
missingOK);
|
||||||
|
|
||||||
ShardInterval *splitChildShardInterval = NULL;
|
ShardInterval *splitChildShardInterval = NULL;
|
||||||
WorkerNode *destinationWorkerNode = NULL;
|
WorkerNode *destinationWorkerNode = NULL;
|
||||||
|
@ -1795,7 +2013,7 @@ CreateSplitShardReplicationSetupUDF(List *sourceColocatedShardIntervalList,
|
||||||
appendStringInfo(splitChildrenRows,
|
appendStringInfo(splitChildrenRows,
|
||||||
"ROW(%lu, %s, %lu, %s, %s, %u)::pg_catalog.split_shard_info",
|
"ROW(%lu, %s, %lu, %s, %s, %u)::pg_catalog.split_shard_info",
|
||||||
sourceShardId,
|
sourceShardId,
|
||||||
quote_literal_cstr(partitionColumnName),
|
quote_literal_cstr(distributionColumnName),
|
||||||
splitChildShardInterval->shardId,
|
splitChildShardInterval->shardId,
|
||||||
quote_literal_cstr(minValueString->data),
|
quote_literal_cstr(minValueString->data),
|
||||||
quote_literal_cstr(maxValueString->data),
|
quote_literal_cstr(maxValueString->data),
|
||||||
|
|
|
@ -9,15 +9,16 @@
|
||||||
|
|
||||||
#include "postgres.h"
|
#include "postgres.h"
|
||||||
#include "pg_version_compat.h"
|
#include "pg_version_compat.h"
|
||||||
|
#include "distributed/citus_ruleutils.h"
|
||||||
|
#include "distributed/distribution_column.h"
|
||||||
|
#include "distributed/intermediate_results.h"
|
||||||
|
#include "distributed/listutils.h"
|
||||||
|
#include "distributed/multi_executor.h"
|
||||||
|
#include "distributed/utils/array_type.h"
|
||||||
|
#include "distributed/worker_shard_copy.h"
|
||||||
#include "utils/lsyscache.h"
|
#include "utils/lsyscache.h"
|
||||||
#include "utils/array.h"
|
#include "utils/array.h"
|
||||||
#include "utils/builtins.h"
|
#include "utils/builtins.h"
|
||||||
#include "distributed/utils/array_type.h"
|
|
||||||
#include "distributed/listutils.h"
|
|
||||||
#include "distributed/multi_executor.h"
|
|
||||||
#include "distributed/worker_shard_copy.h"
|
|
||||||
#include "distributed/intermediate_results.h"
|
|
||||||
#include "distributed/citus_ruleutils.h"
|
|
||||||
|
|
||||||
PG_FUNCTION_INFO_V1(worker_split_copy);
|
PG_FUNCTION_INFO_V1(worker_split_copy);
|
||||||
|
|
||||||
|
@ -38,6 +39,7 @@ static DestReceiver ** CreateShardCopyDestReceivers(EState *estate,
|
||||||
static DestReceiver * CreatePartitionedSplitCopyDestReceiver(EState *executor,
|
static DestReceiver * CreatePartitionedSplitCopyDestReceiver(EState *executor,
|
||||||
ShardInterval *
|
ShardInterval *
|
||||||
shardIntervalToSplitCopy,
|
shardIntervalToSplitCopy,
|
||||||
|
char *partitionColumnName,
|
||||||
List *splitCopyInfoList);
|
List *splitCopyInfoList);
|
||||||
static void BuildMinMaxRangeArrays(List *splitCopyInfoList, ArrayType **minValueArray,
|
static void BuildMinMaxRangeArrays(List *splitCopyInfoList, ArrayType **minValueArray,
|
||||||
ArrayType **maxValueArray);
|
ArrayType **maxValueArray);
|
||||||
|
@ -54,7 +56,10 @@ worker_split_copy(PG_FUNCTION_ARGS)
|
||||||
uint64 shardIdToSplitCopy = DatumGetUInt64(PG_GETARG_DATUM(0));
|
uint64 shardIdToSplitCopy = DatumGetUInt64(PG_GETARG_DATUM(0));
|
||||||
ShardInterval *shardIntervalToSplitCopy = LoadShardInterval(shardIdToSplitCopy);
|
ShardInterval *shardIntervalToSplitCopy = LoadShardInterval(shardIdToSplitCopy);
|
||||||
|
|
||||||
ArrayType *splitCopyInfoArrayObject = PG_GETARG_ARRAYTYPE_P(1);
|
text *partitionColumnText = PG_GETARG_TEXT_P(1);
|
||||||
|
char *partitionColumnName = text_to_cstring(partitionColumnText);
|
||||||
|
|
||||||
|
ArrayType *splitCopyInfoArrayObject = PG_GETARG_ARRAYTYPE_P(2);
|
||||||
bool arrayHasNull = ARR_HASNULL(splitCopyInfoArrayObject);
|
bool arrayHasNull = ARR_HASNULL(splitCopyInfoArrayObject);
|
||||||
if (arrayHasNull)
|
if (arrayHasNull)
|
||||||
{
|
{
|
||||||
|
@ -82,6 +87,7 @@ worker_split_copy(PG_FUNCTION_ARGS)
|
||||||
EState *executor = CreateExecutorState();
|
EState *executor = CreateExecutorState();
|
||||||
DestReceiver *splitCopyDestReceiver = CreatePartitionedSplitCopyDestReceiver(executor,
|
DestReceiver *splitCopyDestReceiver = CreatePartitionedSplitCopyDestReceiver(executor,
|
||||||
shardIntervalToSplitCopy,
|
shardIntervalToSplitCopy,
|
||||||
|
partitionColumnName,
|
||||||
splitCopyInfoList);
|
splitCopyInfoList);
|
||||||
|
|
||||||
Oid sourceShardToCopySchemaOId = get_rel_namespace(
|
Oid sourceShardToCopySchemaOId = get_rel_namespace(
|
||||||
|
@ -228,6 +234,7 @@ CreateShardCopyDestReceivers(EState *estate, ShardInterval *shardIntervalToSplit
|
||||||
static DestReceiver *
|
static DestReceiver *
|
||||||
CreatePartitionedSplitCopyDestReceiver(EState *estate,
|
CreatePartitionedSplitCopyDestReceiver(EState *estate,
|
||||||
ShardInterval *shardIntervalToSplitCopy,
|
ShardInterval *shardIntervalToSplitCopy,
|
||||||
|
char *partitionColumnName,
|
||||||
List *splitCopyInfoList)
|
List *splitCopyInfoList)
|
||||||
{
|
{
|
||||||
/* Create underlying ShardCopyDestReceivers */
|
/* Create underlying ShardCopyDestReceivers */
|
||||||
|
@ -240,10 +247,17 @@ CreatePartitionedSplitCopyDestReceiver(EState *estate,
|
||||||
ArrayType *minValuesArray = NULL;
|
ArrayType *minValuesArray = NULL;
|
||||||
ArrayType *maxValuesArray = NULL;
|
ArrayType *maxValuesArray = NULL;
|
||||||
BuildMinMaxRangeArrays(splitCopyInfoList, &minValuesArray, &maxValuesArray);
|
BuildMinMaxRangeArrays(splitCopyInfoList, &minValuesArray, &maxValuesArray);
|
||||||
CitusTableCacheEntry *cacheEntry = GetCitusTableCacheEntry(
|
|
||||||
shardIntervalToSplitCopy->relationId);
|
/* we currently only support hash-distribution */
|
||||||
char partitionMethod = cacheEntry->partitionMethod;
|
char partitionMethod = DISTRIBUTE_BY_HASH;
|
||||||
Var *partitionColumn = cacheEntry->partitionColumn;
|
|
||||||
|
/* synthetically build the partition column by looking at shard columns */
|
||||||
|
uint64 shardId = shardIntervalToSplitCopy->shardId;
|
||||||
|
bool missingOK = false;
|
||||||
|
Oid shardRelationId = LookupShardRelationFromCatalog(shardId, missingOK);
|
||||||
|
Var *partitionColumn = BuildDistributionKeyFromColumnName(shardRelationId,
|
||||||
|
partitionColumnName,
|
||||||
|
AccessShareLock);
|
||||||
|
|
||||||
CitusTableCacheEntry *shardSearchInfo =
|
CitusTableCacheEntry *shardSearchInfo =
|
||||||
QueryTupleShardSearchInfo(minValuesArray, maxValuesArray,
|
QueryTupleShardSearchInfo(minValuesArray, maxValuesArray,
|
||||||
|
|
|
@ -14,6 +14,7 @@
|
||||||
#include "replication/logical.h"
|
#include "replication/logical.h"
|
||||||
#include "utils/typcache.h"
|
#include "utils/typcache.h"
|
||||||
|
|
||||||
|
|
||||||
extern void _PG_output_plugin_init(OutputPluginCallbacks *cb);
|
extern void _PG_output_plugin_init(OutputPluginCallbacks *cb);
|
||||||
static LogicalDecodeChangeCB pgoutputChangeCB;
|
static LogicalDecodeChangeCB pgoutputChangeCB;
|
||||||
|
|
||||||
|
@ -216,7 +217,8 @@ GetHashValueForIncomingTuple(Relation sourceShardRelation,
|
||||||
TYPECACHE_HASH_PROC_FINFO);
|
TYPECACHE_HASH_PROC_FINFO);
|
||||||
|
|
||||||
/* get hashed value of the distribution value */
|
/* get hashed value of the distribution value */
|
||||||
Datum hashedValueDatum = FunctionCall1(&(typeEntry->hash_proc_finfo),
|
Datum hashedValueDatum = FunctionCall1Coll(&(typeEntry->hash_proc_finfo),
|
||||||
|
typeEntry->typcollation,
|
||||||
partitionColumnValue);
|
partitionColumnValue);
|
||||||
|
|
||||||
return DatumGetInt32(hashedValueDatum);
|
return DatumGetInt32(hashedValueDatum);
|
||||||
|
|
|
@ -1,4 +1,6 @@
|
||||||
#include "udfs/citus_locks/11.1-1.sql"
|
#include "udfs/citus_locks/11.1-1.sql"
|
||||||
|
#include "udfs/create_distributed_table_concurrently/11.1-1.sql"
|
||||||
|
#include "udfs/citus_internal_delete_partition_metadata/11.1-1.sql"
|
||||||
|
|
||||||
DROP FUNCTION pg_catalog.worker_create_schema(bigint,text);
|
DROP FUNCTION pg_catalog.worker_create_schema(bigint,text);
|
||||||
DROP FUNCTION pg_catalog.worker_cleanup_job_schema_cache();
|
DROP FUNCTION pg_catalog.worker_cleanup_job_schema_cache();
|
||||||
|
|
|
@ -70,6 +70,7 @@ DROP FUNCTION pg_catalog.citus_split_shard_by_split_points(
|
||||||
shard_transfer_mode citus.shard_transfer_mode);
|
shard_transfer_mode citus.shard_transfer_mode);
|
||||||
DROP FUNCTION pg_catalog.worker_split_copy(
|
DROP FUNCTION pg_catalog.worker_split_copy(
|
||||||
source_shard_id bigint,
|
source_shard_id bigint,
|
||||||
|
distribution_column text,
|
||||||
splitCopyInfos pg_catalog.split_copy_info[]);
|
splitCopyInfos pg_catalog.split_copy_info[]);
|
||||||
DROP TYPE pg_catalog.split_copy_info;
|
DROP TYPE pg_catalog.split_copy_info;
|
||||||
|
|
||||||
|
@ -97,3 +98,5 @@ DROP FUNCTION pg_catalog.replicate_reference_tables(citus.shard_transfer_mode);
|
||||||
|
|
||||||
DROP FUNCTION pg_catalog.isolate_tenant_to_new_shard(table_name regclass, tenant_id "any", cascade_option text, shard_transfer_mode citus.shard_transfer_mode);
|
DROP FUNCTION pg_catalog.isolate_tenant_to_new_shard(table_name regclass, tenant_id "any", cascade_option text, shard_transfer_mode citus.shard_transfer_mode);
|
||||||
#include "../udfs/isolate_tenant_to_new_shard/8.0-1.sql"
|
#include "../udfs/isolate_tenant_to_new_shard/8.0-1.sql"
|
||||||
|
DROP FUNCTION pg_catalog.create_distributed_table_concurrently;
|
||||||
|
DROP FUNCTION pg_catalog.citus_internal_delete_partition_metadata(regclass);
|
||||||
|
|
7
src/backend/distributed/sql/udfs/citus_internal_delete_partition_metadata/11.1-1.sql
generated
Normal file
7
src/backend/distributed/sql/udfs/citus_internal_delete_partition_metadata/11.1-1.sql
generated
Normal file
|
@ -0,0 +1,7 @@
|
||||||
|
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_delete_partition_metadata(table_name regclass)
|
||||||
|
RETURNS void
|
||||||
|
LANGUAGE C STRICT
|
||||||
|
AS 'MODULE_PATHNAME';
|
||||||
|
COMMENT ON FUNCTION pg_catalog.citus_internal_delete_partition_metadata(regclass) IS
|
||||||
|
'Deletes a row from pg_dist_partition with table ownership checks';
|
||||||
|
|
|
@ -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';
|
||||||
|
|
14
src/backend/distributed/sql/udfs/create_distributed_table_concurrently/11.1-1.sql
generated
Normal file
14
src/backend/distributed/sql/udfs/create_distributed_table_concurrently/11.1-1.sql
generated
Normal file
|
@ -0,0 +1,14 @@
|
||||||
|
CREATE FUNCTION pg_catalog.create_distributed_table_concurrently(table_name regclass,
|
||||||
|
distribution_column text,
|
||||||
|
distribution_type citus.distribution_type DEFAULT 'hash',
|
||||||
|
colocate_with text DEFAULT 'default',
|
||||||
|
shard_count int DEFAULT NULL)
|
||||||
|
RETURNS void
|
||||||
|
LANGUAGE C
|
||||||
|
AS 'MODULE_PATHNAME', $$create_distributed_table_concurrently$$;
|
||||||
|
COMMENT ON FUNCTION pg_catalog.create_distributed_table_concurrently(table_name regclass,
|
||||||
|
distribution_column text,
|
||||||
|
distribution_type citus.distribution_type,
|
||||||
|
colocate_with text,
|
||||||
|
shard_count int)
|
||||||
|
IS 'creates a distributed table and avoids blocking writes';
|
|
@ -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';
|
|
@ -14,9 +14,10 @@ ALTER TYPE citus.split_copy_info SET SCHEMA pg_catalog;
|
||||||
|
|
||||||
CREATE OR REPLACE FUNCTION pg_catalog.worker_split_copy(
|
CREATE OR REPLACE FUNCTION pg_catalog.worker_split_copy(
|
||||||
source_shard_id bigint,
|
source_shard_id bigint,
|
||||||
|
distribution_column text,
|
||||||
splitCopyInfos pg_catalog.split_copy_info[])
|
splitCopyInfos pg_catalog.split_copy_info[])
|
||||||
RETURNS void
|
RETURNS void
|
||||||
LANGUAGE C STRICT
|
LANGUAGE C STRICT
|
||||||
AS 'MODULE_PATHNAME', $$worker_split_copy$$;
|
AS 'MODULE_PATHNAME', $$worker_split_copy$$;
|
||||||
COMMENT ON FUNCTION pg_catalog.worker_split_copy(source_shard_id bigint, splitCopyInfos pg_catalog.split_copy_info[])
|
COMMENT ON FUNCTION pg_catalog.worker_split_copy(source_shard_id bigint, distribution_column text, splitCopyInfos pg_catalog.split_copy_info[])
|
||||||
IS 'Perform split copy for shard';
|
IS 'Perform split copy for shard';
|
||||||
|
|
|
@ -14,9 +14,10 @@ ALTER TYPE citus.split_copy_info SET SCHEMA pg_catalog;
|
||||||
|
|
||||||
CREATE OR REPLACE FUNCTION pg_catalog.worker_split_copy(
|
CREATE OR REPLACE FUNCTION pg_catalog.worker_split_copy(
|
||||||
source_shard_id bigint,
|
source_shard_id bigint,
|
||||||
|
distribution_column text,
|
||||||
splitCopyInfos pg_catalog.split_copy_info[])
|
splitCopyInfos pg_catalog.split_copy_info[])
|
||||||
RETURNS void
|
RETURNS void
|
||||||
LANGUAGE C STRICT
|
LANGUAGE C STRICT
|
||||||
AS 'MODULE_PATHNAME', $$worker_split_copy$$;
|
AS 'MODULE_PATHNAME', $$worker_split_copy$$;
|
||||||
COMMENT ON FUNCTION pg_catalog.worker_split_copy(source_shard_id bigint, splitCopyInfos pg_catalog.split_copy_info[])
|
COMMENT ON FUNCTION pg_catalog.worker_split_copy(source_shard_id bigint, distribution_column text, splitCopyInfos pg_catalog.split_copy_info[])
|
||||||
IS 'Perform split copy for shard';
|
IS 'Perform split copy for shard';
|
||||||
|
|
|
@ -114,7 +114,8 @@ IntegerArrayTypeToList(ArrayType *arrayObject)
|
||||||
|
|
||||||
for (int index = 0; index < arrayObjectCount; index++)
|
for (int index = 0; index < arrayObjectCount; index++)
|
||||||
{
|
{
|
||||||
list = lappend_int(list, datumObjectArray[index]);
|
int32 intObject = DatumGetInt32(datumObjectArray[index]);
|
||||||
|
list = lappend_int(list, intObject);
|
||||||
}
|
}
|
||||||
|
|
||||||
return list;
|
return list;
|
||||||
|
|
|
@ -53,6 +53,7 @@ static void DeleteColocationGroup(uint32 colocationId);
|
||||||
static uint32 CreateColocationGroupForRelation(Oid sourceRelationId);
|
static uint32 CreateColocationGroupForRelation(Oid sourceRelationId);
|
||||||
static void BreakColocation(Oid sourceRelationId);
|
static void BreakColocation(Oid sourceRelationId);
|
||||||
|
|
||||||
|
|
||||||
/* exports for SQL callable functions */
|
/* exports for SQL callable functions */
|
||||||
PG_FUNCTION_INFO_V1(mark_tables_colocated);
|
PG_FUNCTION_INFO_V1(mark_tables_colocated);
|
||||||
PG_FUNCTION_INFO_V1(get_colocated_shard_array);
|
PG_FUNCTION_INFO_V1(get_colocated_shard_array);
|
||||||
|
@ -142,6 +143,17 @@ IsColocateWithNone(char *colocateWithTableName)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* IsColocateWithDefault returns true if the given table is
|
||||||
|
* the special keyword "default".
|
||||||
|
*/
|
||||||
|
bool
|
||||||
|
IsColocateWithDefault(char *colocateWithTableName)
|
||||||
|
{
|
||||||
|
return pg_strncasecmp(colocateWithTableName, "default", NAMEDATALEN) == 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* BreakColocation breaks the colocations of the given relation id.
|
* BreakColocation breaks the colocations of the given relation id.
|
||||||
* If t1, t2 and t3 are colocated and we call this function with t2,
|
* If t1, t2 and t3 are colocated and we call this function with t2,
|
||||||
|
@ -564,6 +576,39 @@ ColocationId(int shardCount, int replicationFactor, Oid distributionColumnType,
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* AcquireColocationDefaultLock serializes concurrent creation of a colocation entry
|
||||||
|
* for default group.
|
||||||
|
*/
|
||||||
|
void
|
||||||
|
AcquireColocationDefaultLock(void)
|
||||||
|
{
|
||||||
|
LOCKTAG tag;
|
||||||
|
const bool sessionLock = false;
|
||||||
|
const bool dontWait = false;
|
||||||
|
|
||||||
|
SET_LOCKTAG_CITUS_OPERATION(tag, CITUS_CREATE_COLOCATION_DEFAULT);
|
||||||
|
|
||||||
|
(void) LockAcquire(&tag, ExclusiveLock, sessionLock, dontWait);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* ReleaseColocationDefaultLock releases the lock for concurrent creation of a colocation entry
|
||||||
|
* for default group.
|
||||||
|
*/
|
||||||
|
void
|
||||||
|
ReleaseColocationDefaultLock(void)
|
||||||
|
{
|
||||||
|
LOCKTAG tag;
|
||||||
|
const bool sessionLock = false;
|
||||||
|
|
||||||
|
SET_LOCKTAG_CITUS_OPERATION(tag, CITUS_CREATE_COLOCATION_DEFAULT);
|
||||||
|
|
||||||
|
LockRelease(&tag, ExclusiveLock, sessionLock);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* CreateColocationGroup creates a new colocation id and writes it into
|
* CreateColocationGroup creates a new colocation id and writes it into
|
||||||
* pg_dist_colocation with the given configuration. It also returns the created
|
* pg_dist_colocation with the given configuration. It also returns the created
|
||||||
|
@ -619,7 +664,7 @@ InsertColocationGroupLocally(uint32 colocationId, int shardCount, int replicatio
|
||||||
|
|
||||||
/* increment the counter so that next command can see the row */
|
/* increment the counter so that next command can see the row */
|
||||||
CommandCounterIncrement();
|
CommandCounterIncrement();
|
||||||
table_close(pgDistColocation, RowExclusiveLock);
|
table_close(pgDistColocation, NoLock);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -1271,5 +1316,110 @@ DeleteColocationGroupLocally(uint32 colocationId)
|
||||||
}
|
}
|
||||||
|
|
||||||
systable_endscan(scanDescriptor);
|
systable_endscan(scanDescriptor);
|
||||||
table_close(pgDistColocation, RowExclusiveLock);
|
table_close(pgDistColocation, NoLock);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* FindColocateWithColocationId tries to find a colocation ID for a given
|
||||||
|
* colocate_with clause passed to create_distributed_table.
|
||||||
|
*/
|
||||||
|
uint32
|
||||||
|
FindColocateWithColocationId(Oid relationId, char replicationModel,
|
||||||
|
Oid distributionColumnType,
|
||||||
|
Oid distributionColumnCollation,
|
||||||
|
int shardCount, bool shardCountIsStrict,
|
||||||
|
char *colocateWithTableName)
|
||||||
|
{
|
||||||
|
uint32 colocationId = INVALID_COLOCATION_ID;
|
||||||
|
|
||||||
|
if (IsColocateWithDefault(colocateWithTableName))
|
||||||
|
{
|
||||||
|
/* check for default colocation group */
|
||||||
|
colocationId = ColocationId(shardCount, ShardReplicationFactor,
|
||||||
|
distributionColumnType,
|
||||||
|
distributionColumnCollation);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* if the shardCount is strict then we check if the shard count
|
||||||
|
* of the colocated table is actually shardCount
|
||||||
|
*/
|
||||||
|
if (shardCountIsStrict && colocationId != INVALID_COLOCATION_ID)
|
||||||
|
{
|
||||||
|
Oid colocatedTableId = ColocatedTableId(colocationId);
|
||||||
|
|
||||||
|
if (colocatedTableId != InvalidOid)
|
||||||
|
{
|
||||||
|
CitusTableCacheEntry *cacheEntry =
|
||||||
|
GetCitusTableCacheEntry(colocatedTableId);
|
||||||
|
int colocatedTableShardCount = cacheEntry->shardIntervalArrayLength;
|
||||||
|
|
||||||
|
if (colocatedTableShardCount != shardCount)
|
||||||
|
{
|
||||||
|
colocationId = INVALID_COLOCATION_ID;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
else if (!IsColocateWithNone(colocateWithTableName))
|
||||||
|
{
|
||||||
|
text *colocateWithTableNameText = cstring_to_text(colocateWithTableName);
|
||||||
|
Oid sourceRelationId = ResolveRelationId(colocateWithTableNameText, false);
|
||||||
|
|
||||||
|
EnsureTableCanBeColocatedWith(relationId, replicationModel,
|
||||||
|
distributionColumnType, sourceRelationId);
|
||||||
|
|
||||||
|
colocationId = TableColocationId(sourceRelationId);
|
||||||
|
}
|
||||||
|
|
||||||
|
return colocationId;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* EnsureTableCanBeColocatedWith checks whether a given replication model and
|
||||||
|
* distribution column type is suitable to distribute a table to be colocated
|
||||||
|
* with given source table.
|
||||||
|
*
|
||||||
|
* We only pass relationId to provide meaningful error messages.
|
||||||
|
*/
|
||||||
|
void
|
||||||
|
EnsureTableCanBeColocatedWith(Oid relationId, char replicationModel,
|
||||||
|
Oid distributionColumnType, Oid sourceRelationId)
|
||||||
|
{
|
||||||
|
CitusTableCacheEntry *sourceTableEntry = GetCitusTableCacheEntry(sourceRelationId);
|
||||||
|
char sourceReplicationModel = sourceTableEntry->replicationModel;
|
||||||
|
Var *sourceDistributionColumn = DistPartitionKeyOrError(sourceRelationId);
|
||||||
|
|
||||||
|
if (!IsCitusTableTypeCacheEntry(sourceTableEntry, HASH_DISTRIBUTED))
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
||||||
|
errmsg("cannot distribute relation"),
|
||||||
|
errdetail("Currently, colocate_with option is only supported "
|
||||||
|
"for hash distributed tables.")));
|
||||||
|
}
|
||||||
|
|
||||||
|
if (sourceReplicationModel != replicationModel)
|
||||||
|
{
|
||||||
|
char *relationName = get_rel_name(relationId);
|
||||||
|
char *sourceRelationName = get_rel_name(sourceRelationId);
|
||||||
|
|
||||||
|
ereport(ERROR, (errmsg("cannot colocate tables %s and %s",
|
||||||
|
sourceRelationName, relationName),
|
||||||
|
errdetail("Replication models don't match for %s and %s.",
|
||||||
|
sourceRelationName, relationName)));
|
||||||
|
}
|
||||||
|
|
||||||
|
Oid sourceDistributionColumnType = sourceDistributionColumn->vartype;
|
||||||
|
if (sourceDistributionColumnType != distributionColumnType)
|
||||||
|
{
|
||||||
|
char *relationName = get_rel_name(relationId);
|
||||||
|
char *sourceRelationName = get_rel_name(sourceRelationId);
|
||||||
|
|
||||||
|
ereport(ERROR, (errmsg("cannot colocate tables %s and %s",
|
||||||
|
sourceRelationName, relationName),
|
||||||
|
errdetail("Distribution column types don't match for "
|
||||||
|
"%s and %s.", sourceRelationName,
|
||||||
|
relationName)));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -24,6 +24,7 @@
|
||||||
#include "nodes/nodes.h"
|
#include "nodes/nodes.h"
|
||||||
#include "nodes/primnodes.h"
|
#include "nodes/primnodes.h"
|
||||||
#include "parser/scansup.h"
|
#include "parser/scansup.h"
|
||||||
|
#include "parser/parse_relation.h"
|
||||||
#include "utils/builtins.h"
|
#include "utils/builtins.h"
|
||||||
#include "utils/elog.h"
|
#include "utils/elog.h"
|
||||||
#include "utils/errcodes.h"
|
#include "utils/errcodes.h"
|
||||||
|
@ -123,7 +124,7 @@ column_to_column_name(PG_FUNCTION_ARGS)
|
||||||
Var *
|
Var *
|
||||||
BuildDistributionKeyFromColumnName(Oid relationId, char *columnName, LOCKMODE lockMode)
|
BuildDistributionKeyFromColumnName(Oid relationId, char *columnName, LOCKMODE lockMode)
|
||||||
{
|
{
|
||||||
Relation relation = try_relation_open(relationId, ExclusiveLock);
|
Relation relation = try_relation_open(relationId, lockMode);
|
||||||
|
|
||||||
if (relation == NULL)
|
if (relation == NULL)
|
||||||
{
|
{
|
||||||
|
@ -172,6 +173,76 @@ BuildDistributionKeyFromColumnName(Oid relationId, char *columnName, LOCKMODE lo
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* EnsureValidDistributionColumn Errors out if the
|
||||||
|
* specified column does not exist or is not suitable to be used as a
|
||||||
|
* distribution column. It does not hold locks.
|
||||||
|
*/
|
||||||
|
void
|
||||||
|
EnsureValidDistributionColumn(Oid relationId, char *columnName)
|
||||||
|
{
|
||||||
|
Relation relation = try_relation_open(relationId, AccessShareLock);
|
||||||
|
|
||||||
|
if (relation == NULL)
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errmsg("relation does not exist")));
|
||||||
|
}
|
||||||
|
|
||||||
|
char *tableName = get_rel_name(relationId);
|
||||||
|
|
||||||
|
/* it'd probably better to downcase identifiers consistent with SQL case folding */
|
||||||
|
truncate_identifier(columnName, strlen(columnName), true);
|
||||||
|
|
||||||
|
/* lookup column definition */
|
||||||
|
HeapTuple columnTuple = SearchSysCacheAttName(relationId, columnName);
|
||||||
|
if (!HeapTupleIsValid(columnTuple))
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_UNDEFINED_COLUMN),
|
||||||
|
errmsg("column \"%s\" of relation \"%s\" does not exist",
|
||||||
|
columnName, tableName)));
|
||||||
|
}
|
||||||
|
|
||||||
|
Form_pg_attribute columnForm = (Form_pg_attribute) GETSTRUCT(columnTuple);
|
||||||
|
|
||||||
|
/* check if the column may be referenced in the distribution key */
|
||||||
|
if (columnForm->attnum <= 0)
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
||||||
|
errmsg("cannot reference system column \"%s\" in relation \"%s\"",
|
||||||
|
columnName, tableName)));
|
||||||
|
}
|
||||||
|
|
||||||
|
ReleaseSysCache(columnTuple);
|
||||||
|
|
||||||
|
relation_close(relation, AccessShareLock);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* ColumnTypeIdForRelationColumnName returns type id for the given relation's column name.
|
||||||
|
*/
|
||||||
|
Oid
|
||||||
|
ColumnTypeIdForRelationColumnName(Oid relationId, char *columnName)
|
||||||
|
{
|
||||||
|
Assert(columnName != NULL);
|
||||||
|
|
||||||
|
AttrNumber attNum = get_attnum(relationId, columnName);
|
||||||
|
|
||||||
|
if (attNum == InvalidAttrNumber)
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errmsg("invalid attr %s", columnName)));
|
||||||
|
}
|
||||||
|
|
||||||
|
Relation relation = relation_open(relationId, AccessShareLock);
|
||||||
|
|
||||||
|
Oid typeId = attnumTypeId(relation, attNum);
|
||||||
|
|
||||||
|
relation_close(relation, AccessShareLock);
|
||||||
|
|
||||||
|
return typeId;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* ColumnToColumnName returns the human-readable name of a column given a
|
* ColumnToColumnName returns the human-readable name of a column given a
|
||||||
* relation identifier and the column's internal (Var) representation.
|
* relation identifier and the column's internal (Var) representation.
|
||||||
|
|
|
@ -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);
|
||||||
|
}
|
|
@ -996,11 +996,19 @@ IsParentTable(Oid relationId)
|
||||||
systable_endscan(scan);
|
systable_endscan(scan);
|
||||||
table_close(pgInherits, AccessShareLock);
|
table_close(pgInherits, AccessShareLock);
|
||||||
|
|
||||||
if (tableInherited && PartitionedTable(relationId))
|
Relation relation = try_relation_open(relationId, AccessShareLock);
|
||||||
|
if (relation == NULL)
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errmsg("relation with OID %u does not exist", relationId)));
|
||||||
|
}
|
||||||
|
|
||||||
|
if (tableInherited && PartitionedTableNoLock(relationId))
|
||||||
{
|
{
|
||||||
tableInherited = false;
|
tableInherited = false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
relation_close(relation, AccessShareLock);
|
||||||
|
|
||||||
return tableInherited;
|
return tableInherited;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -1291,3 +1299,29 @@ PartitionBound(Oid partitionId)
|
||||||
|
|
||||||
return partitionBoundString;
|
return partitionBoundString;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* ListShardsUnderParentRelation returns a list of ShardInterval for every
|
||||||
|
* shard under a given relation, meaning it includes the shards of child
|
||||||
|
* tables in a partitioning hierarchy.
|
||||||
|
*/
|
||||||
|
List *
|
||||||
|
ListShardsUnderParentRelation(Oid relationId)
|
||||||
|
{
|
||||||
|
List *shardList = LoadShardIntervalList(relationId);
|
||||||
|
|
||||||
|
if (PartitionedTable(relationId))
|
||||||
|
{
|
||||||
|
List *partitionList = PartitionList(relationId);
|
||||||
|
Oid partitionRelationId = InvalidOid;
|
||||||
|
|
||||||
|
foreach_oid(partitionRelationId, partitionList)
|
||||||
|
{
|
||||||
|
List *childShardList = ListShardsUnderParentRelation(partitionRelationId);
|
||||||
|
shardList = list_concat(shardList, childShardList);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return shardList;
|
||||||
|
}
|
||||||
|
|
|
@ -36,6 +36,7 @@ extern void InsertColocationGroupLocally(uint32 colocationId, int shardCount,
|
||||||
Oid distributionColumnType,
|
Oid distributionColumnType,
|
||||||
Oid distributionColumnCollation);
|
Oid distributionColumnCollation);
|
||||||
extern bool IsColocateWithNone(char *colocateWithTableName);
|
extern bool IsColocateWithNone(char *colocateWithTableName);
|
||||||
|
extern bool IsColocateWithDefault(char *colocateWithTableName);
|
||||||
extern uint32 GetNextColocationId(void);
|
extern uint32 GetNextColocationId(void);
|
||||||
extern void ErrorIfShardPlacementsNotColocated(Oid leftRelationId, Oid rightRelationId);
|
extern void ErrorIfShardPlacementsNotColocated(Oid leftRelationId, Oid rightRelationId);
|
||||||
extern void CheckReplicationModel(Oid sourceRelationId, Oid targetRelationId);
|
extern void CheckReplicationModel(Oid sourceRelationId, Oid targetRelationId);
|
||||||
|
@ -48,5 +49,15 @@ extern void UpdateRelationColocationGroup(Oid distributedRelationId, uint32 colo
|
||||||
extern void DeleteColocationGroupIfNoTablesBelong(uint32 colocationId);
|
extern void DeleteColocationGroupIfNoTablesBelong(uint32 colocationId);
|
||||||
extern List * ColocationGroupTableList(uint32 colocationId, uint32 count);
|
extern List * ColocationGroupTableList(uint32 colocationId, uint32 count);
|
||||||
extern void DeleteColocationGroupLocally(uint32 colocationId);
|
extern void DeleteColocationGroupLocally(uint32 colocationId);
|
||||||
|
extern uint32 FindColocateWithColocationId(Oid relationId, char replicationModel,
|
||||||
|
Oid distributionColumnType,
|
||||||
|
Oid distributionColumnCollation,
|
||||||
|
int shardCount, bool shardCountIsStrict,
|
||||||
|
char *colocateWithTableName);
|
||||||
|
extern void EnsureTableCanBeColocatedWith(Oid relationId, char replicationModel,
|
||||||
|
Oid distributionColumnType,
|
||||||
|
Oid sourceRelationId);
|
||||||
|
extern void AcquireColocationDefaultLock(void);
|
||||||
|
extern void ReleaseColocationDefaultLock(void);
|
||||||
|
|
||||||
#endif /* COLOCATION_UTILS_H_ */
|
#endif /* COLOCATION_UTILS_H_ */
|
||||||
|
|
|
@ -258,6 +258,8 @@ extern void ErrorIfUnsupportedForeignConstraintExists(Relation relation,
|
||||||
char distributionMethod,
|
char distributionMethod,
|
||||||
Var *distributionColumn,
|
Var *distributionColumn,
|
||||||
uint32 colocationId);
|
uint32 colocationId);
|
||||||
|
extern void EnsureNoFKeyFromTableType(Oid relationId, int tableTypeFlag);
|
||||||
|
extern void EnsureNoFKeyToTableType(Oid relationId, int tableTypeFlag);
|
||||||
extern void ErrorOutForFKeyBetweenPostgresAndCitusLocalTable(Oid localTableId);
|
extern void ErrorOutForFKeyBetweenPostgresAndCitusLocalTable(Oid localTableId);
|
||||||
extern bool ColumnReferencedByAnyForeignKey(char *columnName, Oid relationId);
|
extern bool ColumnReferencedByAnyForeignKey(char *columnName, Oid relationId);
|
||||||
extern bool ColumnAppearsInForeignKey(char *columnName, Oid relationId);
|
extern bool ColumnAppearsInForeignKey(char *columnName, Oid relationId);
|
||||||
|
|
|
@ -23,5 +23,7 @@ extern Var * BuildDistributionKeyFromColumnName(Oid relationId,
|
||||||
char *columnName,
|
char *columnName,
|
||||||
LOCKMODE lockMode);
|
LOCKMODE lockMode);
|
||||||
extern char * ColumnToColumnName(Oid relationId, Node *columnNode);
|
extern char * ColumnToColumnName(Oid relationId, Node *columnNode);
|
||||||
|
extern Oid ColumnTypeIdForRelationColumnName(Oid relationId, char *columnName);
|
||||||
|
extern void EnsureValidDistributionColumn(Oid relationId, char *columnName);
|
||||||
|
|
||||||
#endif /* DISTRIBUTION_COLUMN_H */
|
#endif /* DISTRIBUTION_COLUMN_H */
|
||||||
|
|
|
@ -143,6 +143,7 @@ extern List * AllCitusTableIds(void);
|
||||||
extern bool IsCitusTableType(Oid relationId, CitusTableType tableType);
|
extern bool IsCitusTableType(Oid relationId, CitusTableType tableType);
|
||||||
extern bool IsCitusTableTypeCacheEntry(CitusTableCacheEntry *tableEtnry,
|
extern bool IsCitusTableTypeCacheEntry(CitusTableCacheEntry *tableEtnry,
|
||||||
CitusTableType tableType);
|
CitusTableType tableType);
|
||||||
|
extern char * GetTableTypeName(Oid tableId);
|
||||||
|
|
||||||
extern void SetCreateCitusTransactionLevel(int val);
|
extern void SetCreateCitusTransactionLevel(int val);
|
||||||
extern int GetCitusCreationLevel(void);
|
extern int GetCitusCreationLevel(void);
|
||||||
|
|
|
@ -69,6 +69,7 @@ extern char * MarkObjectsDistributedCreateCommand(List *addresses,
|
||||||
extern char * DistributionCreateCommand(CitusTableCacheEntry *cacheEntry);
|
extern char * DistributionCreateCommand(CitusTableCacheEntry *cacheEntry);
|
||||||
extern char * DistributionDeleteCommand(const char *schemaName,
|
extern char * DistributionDeleteCommand(const char *schemaName,
|
||||||
const char *tableName);
|
const char *tableName);
|
||||||
|
extern char * DistributionDeleteMetadataCommand(Oid relationId);
|
||||||
extern char * TableOwnerResetCommand(Oid distributedRelationId);
|
extern char * TableOwnerResetCommand(Oid distributedRelationId);
|
||||||
extern char * NodeListInsertCommand(List *workerNodeList);
|
extern char * NodeListInsertCommand(List *workerNodeList);
|
||||||
extern List * ShardListInsertCommand(List *shardIntervalList);
|
extern List * ShardListInsertCommand(List *shardIntervalList);
|
||||||
|
|
|
@ -248,6 +248,10 @@ extern void InsertIntoPgDistPartition(Oid relationId, char distributionMethod,
|
||||||
Var *distributionColumn, uint32 colocationId,
|
Var *distributionColumn, uint32 colocationId,
|
||||||
char replicationModel, bool autoConverted);
|
char replicationModel, bool autoConverted);
|
||||||
extern void UpdatePgDistPartitionAutoConverted(Oid citusTableId, bool autoConverted);
|
extern void UpdatePgDistPartitionAutoConverted(Oid citusTableId, bool autoConverted);
|
||||||
|
extern void UpdateDistributionColumnGlobally(Oid relationId, char distributionMethod,
|
||||||
|
Var *distributionColumn, int colocationId);
|
||||||
|
extern void UpdateDistributionColumn(Oid relationId, char distributionMethod,
|
||||||
|
Var *distributionColumn, int colocationId);
|
||||||
extern void DeletePartitionRow(Oid distributedRelationId);
|
extern void DeletePartitionRow(Oid distributedRelationId);
|
||||||
extern void DeleteShardRow(uint64 shardId);
|
extern void DeleteShardRow(uint64 shardId);
|
||||||
extern void UpdateShardPlacementState(uint64 placementId, char shardState);
|
extern void UpdateShardPlacementState(uint64 placementId, char shardState);
|
||||||
|
|
|
@ -30,5 +30,6 @@ extern char * GeneratePartitioningInformation(Oid tableId);
|
||||||
extern void FixPartitionConstraintsOnWorkers(Oid relationId);
|
extern void FixPartitionConstraintsOnWorkers(Oid relationId);
|
||||||
extern void FixLocalPartitionConstraints(Oid relationId, int64 shardId);
|
extern void FixLocalPartitionConstraints(Oid relationId, int64 shardId);
|
||||||
extern void FixPartitionShardIndexNames(Oid relationId, Oid parentIndexOid);
|
extern void FixPartitionShardIndexNames(Oid relationId, Oid parentIndexOid);
|
||||||
|
extern List * ListShardsUnderParentRelation(Oid relationId);
|
||||||
|
|
||||||
#endif /* MULTI_PARTITIONING_UTILS_H_ */
|
#endif /* MULTI_PARTITIONING_UTILS_H_ */
|
||||||
|
|
|
@ -17,3 +17,4 @@ extern void ErrorIfMoveUnsupportedTableType(Oid relationId);
|
||||||
extern void CopyShardsToNode(WorkerNode *sourceNode, WorkerNode *targetNode,
|
extern void CopyShardsToNode(WorkerNode *sourceNode, WorkerNode *targetNode,
|
||||||
List *shardIntervalList, char *snapshotName);
|
List *shardIntervalList, char *snapshotName);
|
||||||
extern void VerifyTablesHaveReplicaIdentity(List *colocatedTableList);
|
extern void VerifyTablesHaveReplicaIdentity(List *colocatedTableList);
|
||||||
|
extern bool RelationCanPublishAllModifications(Oid relationId);
|
||||||
|
|
|
@ -48,7 +48,10 @@ typedef enum AdvisoryLocktagClass
|
||||||
/* CitusOperations has constants for citus operations */
|
/* CitusOperations has constants for citus operations */
|
||||||
typedef enum CitusOperations
|
typedef enum CitusOperations
|
||||||
{
|
{
|
||||||
CITUS_TRANSACTION_RECOVERY = 0
|
CITUS_TRANSACTION_RECOVERY = 0,
|
||||||
|
CITUS_NONBLOCKING_SPLIT = 1,
|
||||||
|
CITUS_CREATE_DISTRIBUTED_TABLE_CONCURRENTLY = 2,
|
||||||
|
CITUS_CREATE_COLOCATION_DEFAULT = 3
|
||||||
} CitusOperations;
|
} CitusOperations;
|
||||||
|
|
||||||
/* reuse advisory lock, but with different, unused field 4 (4)*/
|
/* reuse advisory lock, but with different, unused field 4 (4)*/
|
||||||
|
@ -177,6 +180,8 @@ extern void SerializeNonCommutativeWrites(List *shardIntervalList, LOCKMODE lock
|
||||||
extern void LockRelationShardResources(List *relationShardList, LOCKMODE lockMode);
|
extern void LockRelationShardResources(List *relationShardList, LOCKMODE lockMode);
|
||||||
extern List * GetSortedReferenceShardIntervals(List *relationList);
|
extern List * GetSortedReferenceShardIntervals(List *relationList);
|
||||||
|
|
||||||
|
void AcquireCreateDistributedTableConcurrentlyLock(Oid relationId);
|
||||||
|
|
||||||
/* Lock parent table's colocated shard resource */
|
/* Lock parent table's colocated shard resource */
|
||||||
extern void LockParentShardResourceIfPartition(List *shardIntervalList,
|
extern void LockParentShardResourceIfPartition(List *shardIntervalList,
|
||||||
LOCKMODE lockMode);
|
LOCKMODE lockMode);
|
||||||
|
|
|
@ -12,6 +12,8 @@
|
||||||
#ifndef SHARDSPLIT_H_
|
#ifndef SHARDSPLIT_H_
|
||||||
#define SHARDSPLIT_H_
|
#define SHARDSPLIT_H_
|
||||||
|
|
||||||
|
#include "distributed/utils/distribution_column_map.h"
|
||||||
|
|
||||||
/* Split Modes supported by Shard Split API */
|
/* Split Modes supported by Shard Split API */
|
||||||
typedef enum SplitMode
|
typedef enum SplitMode
|
||||||
{
|
{
|
||||||
|
@ -28,10 +30,10 @@ typedef enum SplitMode
|
||||||
typedef enum SplitOperation
|
typedef enum SplitOperation
|
||||||
{
|
{
|
||||||
SHARD_SPLIT_API = 0,
|
SHARD_SPLIT_API = 0,
|
||||||
ISOLATE_TENANT_TO_NEW_SHARD
|
ISOLATE_TENANT_TO_NEW_SHARD,
|
||||||
|
CREATE_DISTRIBUTED_TABLE
|
||||||
} SplitOperation;
|
} SplitOperation;
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* SplitShard API to split a given shard (or shard group) using split mode and
|
* SplitShard API to split a given shard (or shard group) using split mode and
|
||||||
* specified split points to a set of destination nodes.
|
* specified split points to a set of destination nodes.
|
||||||
|
@ -40,10 +42,15 @@ extern void SplitShard(SplitMode splitMode,
|
||||||
SplitOperation splitOperation,
|
SplitOperation splitOperation,
|
||||||
uint64 shardIdToSplit,
|
uint64 shardIdToSplit,
|
||||||
List *shardSplitPointsList,
|
List *shardSplitPointsList,
|
||||||
List *nodeIdsForPlacementList);
|
List *nodeIdsForPlacementList,
|
||||||
|
DistributionColumnMap *distributionColumnOverrides,
|
||||||
|
List *colocatedShardIntervalList,
|
||||||
|
uint32 targetColocationId);
|
||||||
|
|
||||||
extern void DropShardList(List *shardIntervalList);
|
extern void DropShardList(List *shardIntervalList);
|
||||||
|
|
||||||
extern SplitMode LookupSplitMode(Oid shardTransferModeOid);
|
extern SplitMode LookupSplitMode(Oid shardTransferModeOid);
|
||||||
|
|
||||||
|
extern void ErrorIfMultipleNonblockingMoveSplitInTheSameTransaction(void);
|
||||||
|
|
||||||
#endif /* SHARDSPLIT_H_ */
|
#endif /* SHARDSPLIT_H_ */
|
||||||
|
|
|
@ -47,4 +47,5 @@ extern void DropShardSplitPublications(MultiConnection *sourceConnection,
|
||||||
extern void DropShardSplitSubsriptions(List *shardSplitSubscribersMetadataList);
|
extern void DropShardSplitSubsriptions(List *shardSplitSubscribersMetadataList);
|
||||||
extern void DropShardSplitReplicationSlots(MultiConnection *sourceConnection,
|
extern void DropShardSplitReplicationSlots(MultiConnection *sourceConnection,
|
||||||
List *replicationSlotInfoList);
|
List *replicationSlotInfoList);
|
||||||
|
|
||||||
#endif /* SHARDSPLIT_LOGICAL_REPLICATION_H */
|
#endif /* SHARDSPLIT_LOGICAL_REPLICATION_H */
|
||||||
|
|
|
@ -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 */
|
|
@ -90,6 +90,7 @@ extern WorkerNode * FindWorkerNodeAnyCluster(const char *nodeName, int32 nodePor
|
||||||
extern WorkerNode * FindNodeWithNodeId(int nodeId, bool missingOk);
|
extern WorkerNode * FindNodeWithNodeId(int nodeId, bool missingOk);
|
||||||
extern List * ReadDistNode(bool includeNodesFromOtherClusters);
|
extern List * ReadDistNode(bool includeNodesFromOtherClusters);
|
||||||
extern void EnsureCoordinator(void);
|
extern void EnsureCoordinator(void);
|
||||||
|
extern void EnsureCoordinatorIsInMetadata(void);
|
||||||
extern void InsertCoordinatorIfClusterEmpty(void);
|
extern void InsertCoordinatorIfClusterEmpty(void);
|
||||||
extern uint32 GroupForNode(char *nodeName, int32 nodePort);
|
extern uint32 GroupForNode(char *nodeName, int32 nodePort);
|
||||||
extern WorkerNode * PrimaryNodeForGroup(int32 groupId, bool *groupContainsNodes);
|
extern WorkerNode * PrimaryNodeForGroup(int32 groupId, bool *groupContainsNodes);
|
||||||
|
|
|
@ -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;
|
|
@ -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)
|
||||||
|
|
|
@ -0,0 +1,763 @@
|
||||||
|
unused step name: s1-create-concurrently-table_2
|
||||||
|
Parsed test spec with 4 sessions
|
||||||
|
|
||||||
|
starting permutation: s1-truncate s3-acquire-split-advisory-lock s1-settings s2-settings s1-create-concurrently-table_1 s2-begin s2-insert s2-commit s3-release-split-advisory-lock s2-print-status
|
||||||
|
step s1-truncate:
|
||||||
|
TRUNCATE table_1;
|
||||||
|
|
||||||
|
step s3-acquire-split-advisory-lock:
|
||||||
|
SELECT pg_advisory_lock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_lock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-settings:
|
||||||
|
-- session needs to have replication factor set to 1, can't do in setup
|
||||||
|
SET citus.shard_count TO 4;
|
||||||
|
SET citus.shard_replication_factor TO 1;
|
||||||
|
|
||||||
|
step s2-settings:
|
||||||
|
-- session needs to have replication factor set to 1, can't do in setup
|
||||||
|
SET citus.shard_count TO 4;
|
||||||
|
SET citus.shard_replication_factor TO 1;
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_1:
|
||||||
|
SELECT create_distributed_table_concurrently('table_1', 'id');
|
||||||
|
<waiting ...>
|
||||||
|
step s2-begin:
|
||||||
|
BEGIN;
|
||||||
|
|
||||||
|
step s2-insert:
|
||||||
|
INSERT INTO table_1 SELECT s FROM generate_series(1,20) s;
|
||||||
|
|
||||||
|
step s2-commit:
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
step s3-release-split-advisory-lock:
|
||||||
|
SELECT pg_advisory_unlock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_unlock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
t
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_1: <... completed>
|
||||||
|
create_distributed_table_concurrently
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s2-print-status:
|
||||||
|
-- sanity check on partitions
|
||||||
|
SELECT * FROM pg_dist_shard
|
||||||
|
WHERE logicalrelid = 'table_1'::regclass OR logicalrelid = 'table_2'::regclass
|
||||||
|
ORDER BY shardminvalue::BIGINT, logicalrelid;
|
||||||
|
-- sanity check on total elements in the table
|
||||||
|
SELECT COUNT(*) FROM table_1;
|
||||||
|
|
||||||
|
logicalrelid|shardid|shardstorage|shardminvalue|shardmaxvalue
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
table_1 |1400294|t | -2147483648| -1073741825
|
||||||
|
table_1 |1400295|t | -1073741824| -1
|
||||||
|
table_1 |1400296|t | 0| 1073741823
|
||||||
|
table_1 |1400297|t | 1073741824| 2147483647
|
||||||
|
(4 rows)
|
||||||
|
|
||||||
|
count
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
20
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
citus_remove_node
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s1-truncate s3-acquire-split-advisory-lock s1-create-concurrently-table_1 s2-begin s2-insert s2-update s2-commit s3-release-split-advisory-lock s2-print-status
|
||||||
|
step s1-truncate:
|
||||||
|
TRUNCATE table_1;
|
||||||
|
|
||||||
|
step s3-acquire-split-advisory-lock:
|
||||||
|
SELECT pg_advisory_lock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_lock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_1:
|
||||||
|
SELECT create_distributed_table_concurrently('table_1', 'id');
|
||||||
|
<waiting ...>
|
||||||
|
step s2-begin:
|
||||||
|
BEGIN;
|
||||||
|
|
||||||
|
step s2-insert:
|
||||||
|
INSERT INTO table_1 SELECT s FROM generate_series(1,20) s;
|
||||||
|
|
||||||
|
step s2-update:
|
||||||
|
UPDATE table_1 SET id = 21 WHERE id = 20;
|
||||||
|
|
||||||
|
step s2-commit:
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
step s3-release-split-advisory-lock:
|
||||||
|
SELECT pg_advisory_unlock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_unlock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
t
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_1: <... completed>
|
||||||
|
create_distributed_table_concurrently
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s2-print-status:
|
||||||
|
-- sanity check on partitions
|
||||||
|
SELECT * FROM pg_dist_shard
|
||||||
|
WHERE logicalrelid = 'table_1'::regclass OR logicalrelid = 'table_2'::regclass
|
||||||
|
ORDER BY shardminvalue::BIGINT, logicalrelid;
|
||||||
|
-- sanity check on total elements in the table
|
||||||
|
SELECT COUNT(*) FROM table_1;
|
||||||
|
|
||||||
|
logicalrelid|shardid|shardstorage|shardminvalue|shardmaxvalue
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
table_1 |1400299|t | -2147483648| -1073741825
|
||||||
|
table_1 |1400300|t | -1073741824| -1
|
||||||
|
table_1 |1400301|t | 0| 1073741823
|
||||||
|
table_1 |1400302|t | 1073741824| 2147483647
|
||||||
|
(4 rows)
|
||||||
|
|
||||||
|
count
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
20
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
citus_remove_node
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s1-truncate s3-acquire-split-advisory-lock s1-create-concurrently-table_1 s2-begin s2-insert s2-delete s2-commit s3-release-split-advisory-lock s2-print-status
|
||||||
|
step s1-truncate:
|
||||||
|
TRUNCATE table_1;
|
||||||
|
|
||||||
|
step s3-acquire-split-advisory-lock:
|
||||||
|
SELECT pg_advisory_lock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_lock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_1:
|
||||||
|
SELECT create_distributed_table_concurrently('table_1', 'id');
|
||||||
|
<waiting ...>
|
||||||
|
step s2-begin:
|
||||||
|
BEGIN;
|
||||||
|
|
||||||
|
step s2-insert:
|
||||||
|
INSERT INTO table_1 SELECT s FROM generate_series(1,20) s;
|
||||||
|
|
||||||
|
step s2-delete:
|
||||||
|
DELETE FROM table_1 WHERE id = 11;
|
||||||
|
|
||||||
|
step s2-commit:
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
step s3-release-split-advisory-lock:
|
||||||
|
SELECT pg_advisory_unlock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_unlock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
t
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_1: <... completed>
|
||||||
|
create_distributed_table_concurrently
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s2-print-status:
|
||||||
|
-- sanity check on partitions
|
||||||
|
SELECT * FROM pg_dist_shard
|
||||||
|
WHERE logicalrelid = 'table_1'::regclass OR logicalrelid = 'table_2'::regclass
|
||||||
|
ORDER BY shardminvalue::BIGINT, logicalrelid;
|
||||||
|
-- sanity check on total elements in the table
|
||||||
|
SELECT COUNT(*) FROM table_1;
|
||||||
|
|
||||||
|
logicalrelid|shardid|shardstorage|shardminvalue|shardmaxvalue
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
table_1 |1400304|t | -2147483648| -1073741825
|
||||||
|
table_1 |1400305|t | -1073741824| -1
|
||||||
|
table_1 |1400306|t | 0| 1073741823
|
||||||
|
table_1 |1400307|t | 1073741824| 2147483647
|
||||||
|
(4 rows)
|
||||||
|
|
||||||
|
count
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
19
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
citus_remove_node
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s1-truncate s3-acquire-split-advisory-lock s1-create-concurrently-table_1 s2-begin s2-insert s2-copy s2-commit s3-release-split-advisory-lock s2-print-status
|
||||||
|
step s1-truncate:
|
||||||
|
TRUNCATE table_1;
|
||||||
|
|
||||||
|
step s3-acquire-split-advisory-lock:
|
||||||
|
SELECT pg_advisory_lock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_lock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_1:
|
||||||
|
SELECT create_distributed_table_concurrently('table_1', 'id');
|
||||||
|
<waiting ...>
|
||||||
|
step s2-begin:
|
||||||
|
BEGIN;
|
||||||
|
|
||||||
|
step s2-insert:
|
||||||
|
INSERT INTO table_1 SELECT s FROM generate_series(1,20) s;
|
||||||
|
|
||||||
|
step s2-copy:
|
||||||
|
COPY table_1 FROM PROGRAM 'echo 30 && echo 31 && echo 32 && echo 33 && echo 34 && echo 35 && echo 36 && echo 37 && echo 38';
|
||||||
|
|
||||||
|
step s2-commit:
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
step s3-release-split-advisory-lock:
|
||||||
|
SELECT pg_advisory_unlock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_unlock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
t
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_1: <... completed>
|
||||||
|
create_distributed_table_concurrently
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s2-print-status:
|
||||||
|
-- sanity check on partitions
|
||||||
|
SELECT * FROM pg_dist_shard
|
||||||
|
WHERE logicalrelid = 'table_1'::regclass OR logicalrelid = 'table_2'::regclass
|
||||||
|
ORDER BY shardminvalue::BIGINT, logicalrelid;
|
||||||
|
-- sanity check on total elements in the table
|
||||||
|
SELECT COUNT(*) FROM table_1;
|
||||||
|
|
||||||
|
logicalrelid|shardid|shardstorage|shardminvalue|shardmaxvalue
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
table_1 |1400309|t | -2147483648| -1073741825
|
||||||
|
table_1 |1400310|t | -1073741824| -1
|
||||||
|
table_1 |1400311|t | 0| 1073741823
|
||||||
|
table_1 |1400312|t | 1073741824| 2147483647
|
||||||
|
(4 rows)
|
||||||
|
|
||||||
|
count
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
29
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
citus_remove_node
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s3-acquire-split-advisory-lock s1-create-concurrently-table_1 s2-insert s2-reindex-concurrently s4-print-waiting-locks s3-release-split-advisory-lock
|
||||||
|
step s3-acquire-split-advisory-lock:
|
||||||
|
SELECT pg_advisory_lock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_lock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_1:
|
||||||
|
SELECT create_distributed_table_concurrently('table_1', 'id');
|
||||||
|
<waiting ...>
|
||||||
|
step s2-insert:
|
||||||
|
INSERT INTO table_1 SELECT s FROM generate_series(1,20) s;
|
||||||
|
|
||||||
|
step s2-reindex-concurrently:
|
||||||
|
REINDEX TABLE CONCURRENTLY table_1;
|
||||||
|
<waiting ...>
|
||||||
|
step s4-print-waiting-locks:
|
||||||
|
SELECT mode, relation::regclass, granted FROM pg_locks
|
||||||
|
WHERE relation = 'table_1'::regclass OR relation = 'table_2'::regclass
|
||||||
|
ORDER BY mode, relation, granted;
|
||||||
|
|
||||||
|
mode |relation|granted
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
AccessShareLock |table_1 |t
|
||||||
|
ShareUpdateExclusiveLock|table_1 |f
|
||||||
|
ShareUpdateExclusiveLock|table_1 |t
|
||||||
|
(3 rows)
|
||||||
|
|
||||||
|
step s3-release-split-advisory-lock:
|
||||||
|
SELECT pg_advisory_unlock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_unlock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
t
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_1: <... completed>
|
||||||
|
create_distributed_table_concurrently
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s2-reindex-concurrently: <... completed>
|
||||||
|
citus_remove_node
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s3-acquire-split-advisory-lock s1-create-concurrently-table_1 s2-insert s2-reindex s4-print-waiting-locks s3-release-split-advisory-lock
|
||||||
|
step s3-acquire-split-advisory-lock:
|
||||||
|
SELECT pg_advisory_lock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_lock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_1:
|
||||||
|
SELECT create_distributed_table_concurrently('table_1', 'id');
|
||||||
|
<waiting ...>
|
||||||
|
step s2-insert:
|
||||||
|
INSERT INTO table_1 SELECT s FROM generate_series(1,20) s;
|
||||||
|
|
||||||
|
step s2-reindex:
|
||||||
|
REINDEX TABLE table_1;
|
||||||
|
<waiting ...>
|
||||||
|
step s4-print-waiting-locks:
|
||||||
|
SELECT mode, relation::regclass, granted FROM pg_locks
|
||||||
|
WHERE relation = 'table_1'::regclass OR relation = 'table_2'::regclass
|
||||||
|
ORDER BY mode, relation, granted;
|
||||||
|
|
||||||
|
mode |relation|granted
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
AccessExclusiveLock |table_1 |f
|
||||||
|
AccessShareLock |table_1 |t
|
||||||
|
ShareUpdateExclusiveLock|table_1 |t
|
||||||
|
(3 rows)
|
||||||
|
|
||||||
|
step s3-release-split-advisory-lock:
|
||||||
|
SELECT pg_advisory_unlock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_unlock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
t
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_1: <... completed>
|
||||||
|
create_distributed_table_concurrently
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s2-reindex: <... completed>
|
||||||
|
citus_remove_node
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s2-begin s2-create-concurrently-table_1 s2-commit
|
||||||
|
step s2-begin:
|
||||||
|
BEGIN;
|
||||||
|
|
||||||
|
step s2-create-concurrently-table_1:
|
||||||
|
SELECT create_distributed_table_concurrently('table_1', 'id');
|
||||||
|
|
||||||
|
ERROR: create_distributed_table_concurrently cannot run inside a transaction block
|
||||||
|
step s2-commit:
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
citus_remove_node
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s3-acquire-split-advisory-lock s1-create-concurrently-table_1 s2-create-concurrently-table_1 s3-release-split-advisory-lock
|
||||||
|
step s3-acquire-split-advisory-lock:
|
||||||
|
SELECT pg_advisory_lock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_lock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_1:
|
||||||
|
SELECT create_distributed_table_concurrently('table_1', 'id');
|
||||||
|
<waiting ...>
|
||||||
|
step s2-create-concurrently-table_1:
|
||||||
|
SELECT create_distributed_table_concurrently('table_1', 'id');
|
||||||
|
|
||||||
|
ERROR: another create_distributed_table_concurrently operation is in progress
|
||||||
|
step s3-release-split-advisory-lock:
|
||||||
|
SELECT pg_advisory_unlock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_unlock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
t
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_1: <... completed>
|
||||||
|
create_distributed_table_concurrently
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
citus_remove_node
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s3-acquire-split-advisory-lock s1-create-concurrently-table_1 s2-create-concurrently-table_2 s3-release-split-advisory-lock
|
||||||
|
step s3-acquire-split-advisory-lock:
|
||||||
|
SELECT pg_advisory_lock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_lock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_1:
|
||||||
|
SELECT create_distributed_table_concurrently('table_1', 'id');
|
||||||
|
<waiting ...>
|
||||||
|
step s2-create-concurrently-table_2:
|
||||||
|
SELECT create_distributed_table_concurrently('table_2', 'id');
|
||||||
|
|
||||||
|
ERROR: another create_distributed_table_concurrently operation is in progress
|
||||||
|
step s3-release-split-advisory-lock:
|
||||||
|
SELECT pg_advisory_unlock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_unlock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
t
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_1: <... completed>
|
||||||
|
create_distributed_table_concurrently
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
citus_remove_node
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s3-acquire-split-advisory-lock s1-create-concurrently-table_1 s2-create-table_1 s3-release-split-advisory-lock
|
||||||
|
step s3-acquire-split-advisory-lock:
|
||||||
|
SELECT pg_advisory_lock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_lock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_1:
|
||||||
|
SELECT create_distributed_table_concurrently('table_1', 'id');
|
||||||
|
<waiting ...>
|
||||||
|
step s2-create-table_1:
|
||||||
|
SELECT create_distributed_table('table_1', 'id');
|
||||||
|
<waiting ...>
|
||||||
|
step s3-release-split-advisory-lock:
|
||||||
|
SELECT pg_advisory_unlock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_unlock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
t
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_1: <... completed>
|
||||||
|
create_distributed_table_concurrently
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s2-create-table_1: <... completed>
|
||||||
|
ERROR: table "table_1" is already distributed
|
||||||
|
citus_remove_node
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s3-acquire-split-advisory-lock s1-create-concurrently-table_1 s2-create-table_2 s3-release-split-advisory-lock
|
||||||
|
step s3-acquire-split-advisory-lock:
|
||||||
|
SELECT pg_advisory_lock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_lock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_1:
|
||||||
|
SELECT create_distributed_table_concurrently('table_1', 'id');
|
||||||
|
<waiting ...>
|
||||||
|
step s2-create-table_2:
|
||||||
|
SELECT create_distributed_table('table_2', 'id');
|
||||||
|
<waiting ...>
|
||||||
|
step s3-release-split-advisory-lock:
|
||||||
|
SELECT pg_advisory_unlock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_unlock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
t
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_1: <... completed>
|
||||||
|
create_distributed_table_concurrently
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s2-create-table_2: <... completed>
|
||||||
|
create_distributed_table
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
citus_remove_node
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s2-begin s2-create-table_2 s1-create-concurrently-table_default_colocated s4-print-waiting-advisory-locks s2-commit s4-print-colocations
|
||||||
|
step s2-begin:
|
||||||
|
BEGIN;
|
||||||
|
|
||||||
|
step s2-create-table_2:
|
||||||
|
SELECT create_distributed_table('table_2', 'id');
|
||||||
|
|
||||||
|
create_distributed_table
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_default_colocated:
|
||||||
|
SELECT create_distributed_table_concurrently('table_default_colocated', 'id');
|
||||||
|
<waiting ...>
|
||||||
|
step s4-print-waiting-advisory-locks:
|
||||||
|
SELECT mode, classid, objid, objsubid, granted FROM pg_locks
|
||||||
|
WHERE locktype = 'advisory' AND classid = 0 AND objid = 3 AND objsubid = 9
|
||||||
|
ORDER BY granted;
|
||||||
|
|
||||||
|
mode |classid|objid|objsubid|granted
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
ExclusiveLock| 0| 3| 9|f
|
||||||
|
ExclusiveLock| 0| 3| 9|t
|
||||||
|
(2 rows)
|
||||||
|
|
||||||
|
step s2-commit:
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_default_colocated: <... completed>
|
||||||
|
create_distributed_table_concurrently
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s4-print-colocations:
|
||||||
|
SELECT * FROM pg_dist_colocation ORDER BY colocationid;
|
||||||
|
|
||||||
|
colocationid|shardcount|replicationfactor|distributioncolumntype|distributioncolumncollation
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
123173| 4| 1| 21| 0
|
||||||
|
123174| 4| 1| 23| 0
|
||||||
|
(2 rows)
|
||||||
|
|
||||||
|
citus_remove_node
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s1-create-concurrently-table_default_colocated s3-acquire-split-advisory-lock s1-create-concurrently-table_1 s2-create-table_2 s4-print-waiting-advisory-locks s3-release-split-advisory-lock s4-print-colocations
|
||||||
|
step s1-create-concurrently-table_default_colocated:
|
||||||
|
SELECT create_distributed_table_concurrently('table_default_colocated', 'id');
|
||||||
|
|
||||||
|
create_distributed_table_concurrently
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s3-acquire-split-advisory-lock:
|
||||||
|
SELECT pg_advisory_lock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_lock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_1:
|
||||||
|
SELECT create_distributed_table_concurrently('table_1', 'id');
|
||||||
|
<waiting ...>
|
||||||
|
step s2-create-table_2:
|
||||||
|
SELECT create_distributed_table('table_2', 'id');
|
||||||
|
|
||||||
|
create_distributed_table
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s4-print-waiting-advisory-locks:
|
||||||
|
SELECT mode, classid, objid, objsubid, granted FROM pg_locks
|
||||||
|
WHERE locktype = 'advisory' AND classid = 0 AND objid = 3 AND objsubid = 9
|
||||||
|
ORDER BY granted;
|
||||||
|
|
||||||
|
mode|classid|objid|objsubid|granted
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
(0 rows)
|
||||||
|
|
||||||
|
step s3-release-split-advisory-lock:
|
||||||
|
SELECT pg_advisory_unlock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_unlock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
t
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_1: <... completed>
|
||||||
|
create_distributed_table_concurrently
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s4-print-colocations:
|
||||||
|
SELECT * FROM pg_dist_colocation ORDER BY colocationid;
|
||||||
|
|
||||||
|
colocationid|shardcount|replicationfactor|distributioncolumntype|distributioncolumncollation
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
123175| 4| 1| 23| 0
|
||||||
|
123176| 4| 1| 21| 0
|
||||||
|
(2 rows)
|
||||||
|
|
||||||
|
citus_remove_node
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s2-begin s2-create-table_2 s1-create-concurrently-table_none_colocated s4-print-waiting-advisory-locks s2-commit s4-print-colocations
|
||||||
|
step s2-begin:
|
||||||
|
BEGIN;
|
||||||
|
|
||||||
|
step s2-create-table_2:
|
||||||
|
SELECT create_distributed_table('table_2', 'id');
|
||||||
|
|
||||||
|
create_distributed_table
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_none_colocated:
|
||||||
|
SELECT create_distributed_table_concurrently('table_none_colocated', 'id', colocate_with => 'none');
|
||||||
|
<waiting ...>
|
||||||
|
step s4-print-waiting-advisory-locks:
|
||||||
|
SELECT mode, classid, objid, objsubid, granted FROM pg_locks
|
||||||
|
WHERE locktype = 'advisory' AND classid = 0 AND objid = 3 AND objsubid = 9
|
||||||
|
ORDER BY granted;
|
||||||
|
|
||||||
|
mode |classid|objid|objsubid|granted
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
ExclusiveLock| 0| 3| 9|t
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s2-commit:
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_none_colocated: <... completed>
|
||||||
|
create_distributed_table_concurrently
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s4-print-colocations:
|
||||||
|
SELECT * FROM pg_dist_colocation ORDER BY colocationid;
|
||||||
|
|
||||||
|
colocationid|shardcount|replicationfactor|distributioncolumntype|distributioncolumncollation
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
123177| 4| 1| 21| 0
|
||||||
|
123178| 4| 1| 23| 0
|
||||||
|
(2 rows)
|
||||||
|
|
||||||
|
citus_remove_node
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s2-begin s2-create-table_2-none s1-create-concurrently-table_none_colocated s4-print-waiting-advisory-locks s2-commit s4-print-colocations
|
||||||
|
step s2-begin:
|
||||||
|
BEGIN;
|
||||||
|
|
||||||
|
step s2-create-table_2-none:
|
||||||
|
SELECT create_distributed_table('table_2', 'id', colocate_with => 'none');
|
||||||
|
|
||||||
|
create_distributed_table
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_none_colocated:
|
||||||
|
SELECT create_distributed_table_concurrently('table_none_colocated', 'id', colocate_with => 'none');
|
||||||
|
<waiting ...>
|
||||||
|
step s4-print-waiting-advisory-locks:
|
||||||
|
SELECT mode, classid, objid, objsubid, granted FROM pg_locks
|
||||||
|
WHERE locktype = 'advisory' AND classid = 0 AND objid = 3 AND objsubid = 9
|
||||||
|
ORDER BY granted;
|
||||||
|
|
||||||
|
mode|classid|objid|objsubid|granted
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
(0 rows)
|
||||||
|
|
||||||
|
step s2-commit:
|
||||||
|
COMMIT;
|
||||||
|
|
||||||
|
step s1-create-concurrently-table_none_colocated: <... completed>
|
||||||
|
create_distributed_table_concurrently
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s4-print-colocations:
|
||||||
|
SELECT * FROM pg_dist_colocation ORDER BY colocationid;
|
||||||
|
|
||||||
|
colocationid|shardcount|replicationfactor|distributioncolumntype|distributioncolumncollation
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
123179| 4| 1| 21| 0
|
||||||
|
123180| 4| 1| 23| 0
|
||||||
|
(2 rows)
|
||||||
|
|
||||||
|
citus_remove_node
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
|
@ -247,7 +247,7 @@ step s1-drop: DROP TABLE drop_hash;
|
||||||
step s2-distribute-table: SELECT create_distributed_table('drop_hash', 'id'); <waiting ...>
|
step s2-distribute-table: SELECT create_distributed_table('drop_hash', 'id'); <waiting ...>
|
||||||
step s1-commit: COMMIT;
|
step s1-commit: COMMIT;
|
||||||
step s2-distribute-table: <... completed>
|
step s2-distribute-table: <... completed>
|
||||||
ERROR: could not create distributed table: relation does not exist
|
ERROR: relation with OID XXXX does not exist
|
||||||
step s2-commit: COMMIT;
|
step s2-commit: COMMIT;
|
||||||
step s1-select-count: SELECT COUNT(*) FROM drop_hash;
|
step s1-select-count: SELECT COUNT(*) FROM drop_hash;
|
||||||
ERROR: relation "drop_hash" does not exist
|
ERROR: relation "drop_hash" does not exist
|
||||||
|
|
|
@ -8,9 +8,11 @@ create_distributed_table
|
||||||
|
|
||||||
step s1-load-cache:
|
step s1-load-cache:
|
||||||
TRUNCATE isolation_table;
|
TRUNCATE isolation_table;
|
||||||
|
TRUNCATE isolation_table2;
|
||||||
|
|
||||||
step s1-insert:
|
step s1-insert:
|
||||||
INSERT INTO isolation_table VALUES (5, 10);
|
INSERT INTO isolation_table VALUES (5, 10);
|
||||||
|
INSERT INTO isolation_table2 VALUES (5, 10);
|
||||||
|
|
||||||
step s3-acquire-advisory-lock:
|
step s3-acquire-advisory-lock:
|
||||||
SELECT pg_advisory_lock(44000, 55152);
|
SELECT pg_advisory_lock(44000, 55152);
|
||||||
|
@ -57,7 +59,7 @@ t
|
||||||
step s2-isolate-tenant: <... completed>
|
step s2-isolate-tenant: <... completed>
|
||||||
isolate_tenant_to_new_shard
|
isolate_tenant_to_new_shard
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1500076
|
1500078
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
step s2-commit:
|
step s2-commit:
|
||||||
|
@ -76,9 +78,9 @@ step s2-print-cluster:
|
||||||
|
|
||||||
nodeport|shardid|success|result
|
nodeport|shardid|success|result
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
57637|1500075|t | 0
|
|
||||||
57637|1500076|t | 1
|
|
||||||
57637|1500077|t | 0
|
57637|1500077|t | 0
|
||||||
|
57637|1500078|t | 1
|
||||||
|
57637|1500079|t | 0
|
||||||
57638|1500074|t | 0
|
57638|1500074|t | 0
|
||||||
(4 rows)
|
(4 rows)
|
||||||
|
|
||||||
|
@ -96,9 +98,11 @@ create_distributed_table
|
||||||
|
|
||||||
step s1-load-cache:
|
step s1-load-cache:
|
||||||
TRUNCATE isolation_table;
|
TRUNCATE isolation_table;
|
||||||
|
TRUNCATE isolation_table2;
|
||||||
|
|
||||||
step s1-insert:
|
step s1-insert:
|
||||||
INSERT INTO isolation_table VALUES (5, 10);
|
INSERT INTO isolation_table VALUES (5, 10);
|
||||||
|
INSERT INTO isolation_table2 VALUES (5, 10);
|
||||||
|
|
||||||
step s3-acquire-advisory-lock:
|
step s3-acquire-advisory-lock:
|
||||||
SELECT pg_advisory_lock(44000, 55152);
|
SELECT pg_advisory_lock(44000, 55152);
|
||||||
|
@ -145,7 +149,7 @@ t
|
||||||
step s2-isolate-tenant: <... completed>
|
step s2-isolate-tenant: <... completed>
|
||||||
isolate_tenant_to_new_shard
|
isolate_tenant_to_new_shard
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1500082
|
1500086
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
step s2-commit:
|
step s2-commit:
|
||||||
|
@ -164,10 +168,10 @@ step s2-print-cluster:
|
||||||
|
|
||||||
nodeport|shardid|success|result
|
nodeport|shardid|success|result
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
57637|1500081|t | 0
|
57637|1500085|t | 0
|
||||||
57637|1500082|t | 0
|
57637|1500086|t | 0
|
||||||
57637|1500083|t | 0
|
57637|1500087|t | 0
|
||||||
57638|1500080|t | 0
|
57638|1500082|t | 0
|
||||||
(4 rows)
|
(4 rows)
|
||||||
|
|
||||||
id|value
|
id|value
|
||||||
|
@ -183,6 +187,7 @@ create_distributed_table
|
||||||
|
|
||||||
step s1-load-cache:
|
step s1-load-cache:
|
||||||
TRUNCATE isolation_table;
|
TRUNCATE isolation_table;
|
||||||
|
TRUNCATE isolation_table2;
|
||||||
|
|
||||||
step s3-acquire-advisory-lock:
|
step s3-acquire-advisory-lock:
|
||||||
SELECT pg_advisory_lock(44000, 55152);
|
SELECT pg_advisory_lock(44000, 55152);
|
||||||
|
@ -214,6 +219,7 @@ step s2-isolate-tenant:
|
||||||
<waiting ...>
|
<waiting ...>
|
||||||
step s1-insert:
|
step s1-insert:
|
||||||
INSERT INTO isolation_table VALUES (5, 10);
|
INSERT INTO isolation_table VALUES (5, 10);
|
||||||
|
INSERT INTO isolation_table2 VALUES (5, 10);
|
||||||
|
|
||||||
step s1-commit:
|
step s1-commit:
|
||||||
COMMIT;
|
COMMIT;
|
||||||
|
@ -229,7 +235,7 @@ t
|
||||||
step s2-isolate-tenant: <... completed>
|
step s2-isolate-tenant: <... completed>
|
||||||
isolate_tenant_to_new_shard
|
isolate_tenant_to_new_shard
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1500088
|
1500094
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
step s2-commit:
|
step s2-commit:
|
||||||
|
@ -248,10 +254,10 @@ step s2-print-cluster:
|
||||||
|
|
||||||
nodeport|shardid|success|result
|
nodeport|shardid|success|result
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
57637|1500087|t | 0
|
57637|1500093|t | 0
|
||||||
57637|1500088|t | 1
|
57637|1500094|t | 1
|
||||||
57637|1500089|t | 0
|
57637|1500095|t | 0
|
||||||
57638|1500086|t | 0
|
57638|1500090|t | 0
|
||||||
(4 rows)
|
(4 rows)
|
||||||
|
|
||||||
id|value
|
id|value
|
||||||
|
@ -268,6 +274,7 @@ create_distributed_table
|
||||||
|
|
||||||
step s1-load-cache:
|
step s1-load-cache:
|
||||||
TRUNCATE isolation_table;
|
TRUNCATE isolation_table;
|
||||||
|
TRUNCATE isolation_table2;
|
||||||
|
|
||||||
step s3-acquire-advisory-lock:
|
step s3-acquire-advisory-lock:
|
||||||
SELECT pg_advisory_lock(44000, 55152);
|
SELECT pg_advisory_lock(44000, 55152);
|
||||||
|
@ -314,7 +321,7 @@ t
|
||||||
step s2-isolate-tenant: <... completed>
|
step s2-isolate-tenant: <... completed>
|
||||||
isolate_tenant_to_new_shard
|
isolate_tenant_to_new_shard
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1500094
|
1500102
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
step s2-commit:
|
step s2-commit:
|
||||||
|
@ -333,10 +340,10 @@ step s2-print-cluster:
|
||||||
|
|
||||||
nodeport|shardid|success|result
|
nodeport|shardid|success|result
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
57637|1500093|t | 1
|
57637|1500101|t | 1
|
||||||
57637|1500094|t | 1
|
57637|1500102|t | 1
|
||||||
57637|1500095|t | 2
|
57637|1500103|t | 2
|
||||||
57638|1500092|t | 1
|
57638|1500098|t | 1
|
||||||
(4 rows)
|
(4 rows)
|
||||||
|
|
||||||
id|value
|
id|value
|
||||||
|
@ -357,6 +364,7 @@ create_distributed_table
|
||||||
|
|
||||||
step s1-insert:
|
step s1-insert:
|
||||||
INSERT INTO isolation_table VALUES (5, 10);
|
INSERT INTO isolation_table VALUES (5, 10);
|
||||||
|
INSERT INTO isolation_table2 VALUES (5, 10);
|
||||||
|
|
||||||
step s3-acquire-advisory-lock:
|
step s3-acquire-advisory-lock:
|
||||||
SELECT pg_advisory_lock(44000, 55152);
|
SELECT pg_advisory_lock(44000, 55152);
|
||||||
|
@ -403,7 +411,7 @@ t
|
||||||
step s2-isolate-tenant: <... completed>
|
step s2-isolate-tenant: <... completed>
|
||||||
isolate_tenant_to_new_shard
|
isolate_tenant_to_new_shard
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1500100
|
1500110
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
step s2-commit:
|
step s2-commit:
|
||||||
|
@ -422,10 +430,10 @@ step s2-print-cluster:
|
||||||
|
|
||||||
nodeport|shardid|success|result
|
nodeport|shardid|success|result
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
57637|1500099|t | 0
|
57637|1500109|t | 0
|
||||||
57637|1500100|t | 1
|
57637|1500110|t | 1
|
||||||
57637|1500101|t | 0
|
57637|1500111|t | 0
|
||||||
57638|1500098|t | 0
|
57638|1500106|t | 0
|
||||||
(4 rows)
|
(4 rows)
|
||||||
|
|
||||||
id|value
|
id|value
|
||||||
|
@ -442,6 +450,7 @@ create_distributed_table
|
||||||
|
|
||||||
step s1-insert:
|
step s1-insert:
|
||||||
INSERT INTO isolation_table VALUES (5, 10);
|
INSERT INTO isolation_table VALUES (5, 10);
|
||||||
|
INSERT INTO isolation_table2 VALUES (5, 10);
|
||||||
|
|
||||||
step s3-acquire-advisory-lock:
|
step s3-acquire-advisory-lock:
|
||||||
SELECT pg_advisory_lock(44000, 55152);
|
SELECT pg_advisory_lock(44000, 55152);
|
||||||
|
@ -488,7 +497,7 @@ t
|
||||||
step s2-isolate-tenant: <... completed>
|
step s2-isolate-tenant: <... completed>
|
||||||
isolate_tenant_to_new_shard
|
isolate_tenant_to_new_shard
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1500106
|
1500118
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
step s2-commit:
|
step s2-commit:
|
||||||
|
@ -507,10 +516,10 @@ step s2-print-cluster:
|
||||||
|
|
||||||
nodeport|shardid|success|result
|
nodeport|shardid|success|result
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
57637|1500105|t | 0
|
57637|1500117|t | 0
|
||||||
57637|1500106|t | 0
|
57637|1500118|t | 0
|
||||||
57637|1500107|t | 0
|
57637|1500119|t | 0
|
||||||
57638|1500104|t | 0
|
57638|1500114|t | 0
|
||||||
(4 rows)
|
(4 rows)
|
||||||
|
|
||||||
id|value
|
id|value
|
||||||
|
@ -554,6 +563,7 @@ step s2-isolate-tenant:
|
||||||
<waiting ...>
|
<waiting ...>
|
||||||
step s1-insert:
|
step s1-insert:
|
||||||
INSERT INTO isolation_table VALUES (5, 10);
|
INSERT INTO isolation_table VALUES (5, 10);
|
||||||
|
INSERT INTO isolation_table2 VALUES (5, 10);
|
||||||
|
|
||||||
step s1-commit:
|
step s1-commit:
|
||||||
COMMIT;
|
COMMIT;
|
||||||
|
@ -569,7 +579,7 @@ t
|
||||||
step s2-isolate-tenant: <... completed>
|
step s2-isolate-tenant: <... completed>
|
||||||
isolate_tenant_to_new_shard
|
isolate_tenant_to_new_shard
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1500112
|
1500126
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
step s2-commit:
|
step s2-commit:
|
||||||
|
@ -588,10 +598,10 @@ step s2-print-cluster:
|
||||||
|
|
||||||
nodeport|shardid|success|result
|
nodeport|shardid|success|result
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
57637|1500111|t | 0
|
57637|1500125|t | 0
|
||||||
57637|1500112|t | 1
|
57637|1500126|t | 1
|
||||||
57637|1500113|t | 0
|
57637|1500127|t | 0
|
||||||
57638|1500110|t | 0
|
57638|1500122|t | 0
|
||||||
(4 rows)
|
(4 rows)
|
||||||
|
|
||||||
id|value
|
id|value
|
||||||
|
@ -651,7 +661,7 @@ t
|
||||||
step s2-isolate-tenant: <... completed>
|
step s2-isolate-tenant: <... completed>
|
||||||
isolate_tenant_to_new_shard
|
isolate_tenant_to_new_shard
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1500118
|
1500134
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
step s2-commit:
|
step s2-commit:
|
||||||
|
@ -670,10 +680,10 @@ step s2-print-cluster:
|
||||||
|
|
||||||
nodeport|shardid|success|result
|
nodeport|shardid|success|result
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
57637|1500117|t | 1
|
57637|1500133|t | 1
|
||||||
57637|1500118|t | 1
|
57637|1500134|t | 1
|
||||||
57637|1500119|t | 2
|
57637|1500135|t | 2
|
||||||
57638|1500116|t | 1
|
57638|1500130|t | 1
|
||||||
(4 rows)
|
(4 rows)
|
||||||
|
|
||||||
id|value
|
id|value
|
||||||
|
@ -686,7 +696,7 @@ id|value
|
||||||
(5 rows)
|
(5 rows)
|
||||||
|
|
||||||
|
|
||||||
starting permutation: s1-load-cache s1-insert s3-acquire-advisory-lock s1-begin s1-isolate-tenant s2-isolate-tenant s3-release-advisory-lock s1-commit s2-print-cluster
|
starting permutation: s1-load-cache s1-insert s3-acquire-advisory-lock s2-isolate-tenant s1-isolate-tenant-same-coloc s3-release-advisory-lock s2-print-cluster
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
@ -694,9 +704,11 @@ create_distributed_table
|
||||||
|
|
||||||
step s1-load-cache:
|
step s1-load-cache:
|
||||||
TRUNCATE isolation_table;
|
TRUNCATE isolation_table;
|
||||||
|
TRUNCATE isolation_table2;
|
||||||
|
|
||||||
step s1-insert:
|
step s1-insert:
|
||||||
INSERT INTO isolation_table VALUES (5, 10);
|
INSERT INTO isolation_table VALUES (5, 10);
|
||||||
|
INSERT INTO isolation_table2 VALUES (5, 10);
|
||||||
|
|
||||||
step s3-acquire-advisory-lock:
|
step s3-acquire-advisory-lock:
|
||||||
SELECT pg_advisory_lock(44000, 55152);
|
SELECT pg_advisory_lock(44000, 55152);
|
||||||
|
@ -706,17 +718,11 @@ pg_advisory_lock
|
||||||
|
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
step s1-begin:
|
|
||||||
BEGIN;
|
|
||||||
-- the tests are written with the logic where single shard SELECTs
|
|
||||||
-- do not to open transaction blocks
|
|
||||||
SET citus.select_opens_transaction_block TO false;
|
|
||||||
|
|
||||||
step s1-isolate-tenant:
|
|
||||||
SELECT isolate_tenant_to_new_shard('isolation_table', 2, shard_transfer_mode => 'force_logical');
|
|
||||||
<waiting ...>
|
|
||||||
step s2-isolate-tenant:
|
step s2-isolate-tenant:
|
||||||
SELECT isolate_tenant_to_new_shard('isolation_table', 5, shard_transfer_mode => 'force_logical');
|
SELECT isolate_tenant_to_new_shard('isolation_table', 5, shard_transfer_mode => 'force_logical');
|
||||||
|
<waiting ...>
|
||||||
|
step s1-isolate-tenant-same-coloc:
|
||||||
|
SELECT isolate_tenant_to_new_shard('isolation_table', 2, shard_transfer_mode => 'force_logical');
|
||||||
|
|
||||||
ERROR: could not acquire the lock required to split public.isolation_table
|
ERROR: could not acquire the lock required to split public.isolation_table
|
||||||
step s3-release-advisory-lock:
|
step s3-release-advisory-lock:
|
||||||
|
@ -727,15 +733,12 @@ pg_advisory_unlock
|
||||||
t
|
t
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
step s1-isolate-tenant: <... completed>
|
step s2-isolate-tenant: <... completed>
|
||||||
isolate_tenant_to_new_shard
|
isolate_tenant_to_new_shard
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1500124
|
1500142
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
step s1-commit:
|
|
||||||
COMMIT;
|
|
||||||
|
|
||||||
step s2-print-cluster:
|
step s2-print-cluster:
|
||||||
-- row count per shard
|
-- row count per shard
|
||||||
SELECT
|
SELECT
|
||||||
|
@ -749,10 +752,10 @@ step s2-print-cluster:
|
||||||
|
|
||||||
nodeport|shardid|success|result
|
nodeport|shardid|success|result
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
57637|1500121|t | 1
|
57637|1500141|t | 0
|
||||||
57638|1500123|t | 0
|
57637|1500142|t | 1
|
||||||
57638|1500124|t | 0
|
57637|1500143|t | 0
|
||||||
57638|1500125|t | 0
|
57638|1500138|t | 0
|
||||||
(4 rows)
|
(4 rows)
|
||||||
|
|
||||||
id|value
|
id|value
|
||||||
|
@ -761,14 +764,19 @@ id|value
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
|
||||||
starting permutation: s1-insert s3-acquire-advisory-lock s1-begin s1-isolate-tenant s2-isolate-tenant s3-release-advisory-lock s1-commit s2-print-cluster
|
starting permutation: s1-load-cache s1-insert s3-acquire-advisory-lock s2-isolate-tenant s1-isolate-tenant-same-coloc-blocking s3-release-advisory-lock s2-print-cluster
|
||||||
create_distributed_table
|
create_distributed_table
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
step s1-load-cache:
|
||||||
|
TRUNCATE isolation_table;
|
||||||
|
TRUNCATE isolation_table2;
|
||||||
|
|
||||||
step s1-insert:
|
step s1-insert:
|
||||||
INSERT INTO isolation_table VALUES (5, 10);
|
INSERT INTO isolation_table VALUES (5, 10);
|
||||||
|
INSERT INTO isolation_table2 VALUES (5, 10);
|
||||||
|
|
||||||
step s3-acquire-advisory-lock:
|
step s3-acquire-advisory-lock:
|
||||||
SELECT pg_advisory_lock(44000, 55152);
|
SELECT pg_advisory_lock(44000, 55152);
|
||||||
|
@ -778,17 +786,11 @@ pg_advisory_lock
|
||||||
|
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
step s1-begin:
|
|
||||||
BEGIN;
|
|
||||||
-- the tests are written with the logic where single shard SELECTs
|
|
||||||
-- do not to open transaction blocks
|
|
||||||
SET citus.select_opens_transaction_block TO false;
|
|
||||||
|
|
||||||
step s1-isolate-tenant:
|
|
||||||
SELECT isolate_tenant_to_new_shard('isolation_table', 2, shard_transfer_mode => 'force_logical');
|
|
||||||
<waiting ...>
|
|
||||||
step s2-isolate-tenant:
|
step s2-isolate-tenant:
|
||||||
SELECT isolate_tenant_to_new_shard('isolation_table', 5, shard_transfer_mode => 'force_logical');
|
SELECT isolate_tenant_to_new_shard('isolation_table', 5, shard_transfer_mode => 'force_logical');
|
||||||
|
<waiting ...>
|
||||||
|
step s1-isolate-tenant-same-coloc-blocking:
|
||||||
|
SELECT isolate_tenant_to_new_shard('isolation_table', 2, shard_transfer_mode => 'block_writes');
|
||||||
|
|
||||||
ERROR: could not acquire the lock required to split public.isolation_table
|
ERROR: could not acquire the lock required to split public.isolation_table
|
||||||
step s3-release-advisory-lock:
|
step s3-release-advisory-lock:
|
||||||
|
@ -799,13 +801,152 @@ pg_advisory_unlock
|
||||||
t
|
t
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
step s1-isolate-tenant: <... completed>
|
step s2-isolate-tenant: <... completed>
|
||||||
isolate_tenant_to_new_shard
|
isolate_tenant_to_new_shard
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
1500130
|
1500150
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
step s1-commit:
|
step s2-print-cluster:
|
||||||
|
-- row count per shard
|
||||||
|
SELECT
|
||||||
|
nodeport, shardid, success, result
|
||||||
|
FROM
|
||||||
|
run_command_on_placements('isolation_table', 'select count(*) from %s')
|
||||||
|
ORDER BY
|
||||||
|
nodeport, shardid;
|
||||||
|
-- rows
|
||||||
|
SELECT id, value FROM isolation_table ORDER BY id, value;
|
||||||
|
|
||||||
|
nodeport|shardid|success|result
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
57637|1500149|t | 0
|
||||||
|
57637|1500150|t | 1
|
||||||
|
57637|1500151|t | 0
|
||||||
|
57638|1500146|t | 0
|
||||||
|
(4 rows)
|
||||||
|
|
||||||
|
id|value
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
5| 10
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s1-load-cache s1-insert s3-acquire-advisory-lock s2-isolate-tenant s1-isolate-tenant-no-same-coloc s3-release-advisory-lock s2-print-cluster
|
||||||
|
create_distributed_table
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-load-cache:
|
||||||
|
TRUNCATE isolation_table;
|
||||||
|
TRUNCATE isolation_table2;
|
||||||
|
|
||||||
|
step s1-insert:
|
||||||
|
INSERT INTO isolation_table VALUES (5, 10);
|
||||||
|
INSERT INTO isolation_table2 VALUES (5, 10);
|
||||||
|
|
||||||
|
step s3-acquire-advisory-lock:
|
||||||
|
SELECT pg_advisory_lock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_lock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s2-isolate-tenant:
|
||||||
|
SELECT isolate_tenant_to_new_shard('isolation_table', 5, shard_transfer_mode => 'force_logical');
|
||||||
|
<waiting ...>
|
||||||
|
step s1-isolate-tenant-no-same-coloc:
|
||||||
|
SELECT isolate_tenant_to_new_shard('isolation_table2', 2, shard_transfer_mode => 'force_logical');
|
||||||
|
|
||||||
|
ERROR: could not acquire the lock required to split concurrently public.isolation_table2.
|
||||||
|
step s3-release-advisory-lock:
|
||||||
|
SELECT pg_advisory_unlock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_unlock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
t
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s2-isolate-tenant: <... completed>
|
||||||
|
isolate_tenant_to_new_shard
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
1500158
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s2-print-cluster:
|
||||||
|
-- row count per shard
|
||||||
|
SELECT
|
||||||
|
nodeport, shardid, success, result
|
||||||
|
FROM
|
||||||
|
run_command_on_placements('isolation_table', 'select count(*) from %s')
|
||||||
|
ORDER BY
|
||||||
|
nodeport, shardid;
|
||||||
|
-- rows
|
||||||
|
SELECT id, value FROM isolation_table ORDER BY id, value;
|
||||||
|
|
||||||
|
nodeport|shardid|success|result
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
57637|1500157|t | 0
|
||||||
|
57637|1500158|t | 1
|
||||||
|
57637|1500159|t | 0
|
||||||
|
57638|1500154|t | 0
|
||||||
|
(4 rows)
|
||||||
|
|
||||||
|
id|value
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
5| 10
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s1-load-cache s1-insert s3-acquire-advisory-lock s2-begin s2-isolate-tenant s1-isolate-tenant-no-same-coloc s3-release-advisory-lock s2-commit s2-print-cluster
|
||||||
|
create_distributed_table
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-load-cache:
|
||||||
|
TRUNCATE isolation_table;
|
||||||
|
TRUNCATE isolation_table2;
|
||||||
|
|
||||||
|
step s1-insert:
|
||||||
|
INSERT INTO isolation_table VALUES (5, 10);
|
||||||
|
INSERT INTO isolation_table2 VALUES (5, 10);
|
||||||
|
|
||||||
|
step s3-acquire-advisory-lock:
|
||||||
|
SELECT pg_advisory_lock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_lock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s2-begin:
|
||||||
|
BEGIN;
|
||||||
|
|
||||||
|
step s2-isolate-tenant:
|
||||||
|
SELECT isolate_tenant_to_new_shard('isolation_table', 5, shard_transfer_mode => 'force_logical');
|
||||||
|
<waiting ...>
|
||||||
|
step s1-isolate-tenant-no-same-coloc:
|
||||||
|
SELECT isolate_tenant_to_new_shard('isolation_table2', 2, shard_transfer_mode => 'force_logical');
|
||||||
|
|
||||||
|
ERROR: could not acquire the lock required to split concurrently public.isolation_table2.
|
||||||
|
step s3-release-advisory-lock:
|
||||||
|
SELECT pg_advisory_unlock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_unlock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
t
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s2-isolate-tenant: <... completed>
|
||||||
|
isolate_tenant_to_new_shard
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
1500169
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s2-commit:
|
||||||
COMMIT;
|
COMMIT;
|
||||||
|
|
||||||
step s2-print-cluster:
|
step s2-print-cluster:
|
||||||
|
@ -821,10 +962,154 @@ step s2-print-cluster:
|
||||||
|
|
||||||
nodeport|shardid|success|result
|
nodeport|shardid|success|result
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
57637|1500127|t | 1
|
57637|1500168|t | 0
|
||||||
57638|1500129|t | 0
|
57637|1500169|t | 1
|
||||||
57638|1500130|t | 0
|
57637|1500170|t | 0
|
||||||
57638|1500131|t | 0
|
57638|1500165|t | 0
|
||||||
|
(4 rows)
|
||||||
|
|
||||||
|
id|value
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
5| 10
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s1-load-cache s1-insert s3-acquire-advisory-lock s2-isolate-tenant s1-isolate-tenant-no-same-coloc-blocking s3-release-advisory-lock s2-print-cluster
|
||||||
|
create_distributed_table
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-load-cache:
|
||||||
|
TRUNCATE isolation_table;
|
||||||
|
TRUNCATE isolation_table2;
|
||||||
|
|
||||||
|
step s1-insert:
|
||||||
|
INSERT INTO isolation_table VALUES (5, 10);
|
||||||
|
INSERT INTO isolation_table2 VALUES (5, 10);
|
||||||
|
|
||||||
|
step s3-acquire-advisory-lock:
|
||||||
|
SELECT pg_advisory_lock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_lock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s2-isolate-tenant:
|
||||||
|
SELECT isolate_tenant_to_new_shard('isolation_table', 5, shard_transfer_mode => 'force_logical');
|
||||||
|
<waiting ...>
|
||||||
|
step s1-isolate-tenant-no-same-coloc-blocking:
|
||||||
|
SELECT isolate_tenant_to_new_shard('isolation_table2', 2, shard_transfer_mode => 'block_writes');
|
||||||
|
|
||||||
|
isolate_tenant_to_new_shard
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
1500183
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s3-release-advisory-lock:
|
||||||
|
SELECT pg_advisory_unlock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_unlock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
t
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s2-isolate-tenant: <... completed>
|
||||||
|
isolate_tenant_to_new_shard
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
1500180
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s2-print-cluster:
|
||||||
|
-- row count per shard
|
||||||
|
SELECT
|
||||||
|
nodeport, shardid, success, result
|
||||||
|
FROM
|
||||||
|
run_command_on_placements('isolation_table', 'select count(*) from %s')
|
||||||
|
ORDER BY
|
||||||
|
nodeport, shardid;
|
||||||
|
-- rows
|
||||||
|
SELECT id, value FROM isolation_table ORDER BY id, value;
|
||||||
|
|
||||||
|
nodeport|shardid|success|result
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
57637|1500179|t | 0
|
||||||
|
57637|1500180|t | 1
|
||||||
|
57637|1500181|t | 0
|
||||||
|
57638|1500176|t | 0
|
||||||
|
(4 rows)
|
||||||
|
|
||||||
|
id|value
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
5| 10
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
|
||||||
|
starting permutation: s1-load-cache s1-insert s3-acquire-advisory-lock s2-isolate-tenant s1-isolate-tenant-no-same-coloc-blocking s3-release-advisory-lock s2-print-cluster
|
||||||
|
create_distributed_table
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s1-load-cache:
|
||||||
|
TRUNCATE isolation_table;
|
||||||
|
TRUNCATE isolation_table2;
|
||||||
|
|
||||||
|
step s1-insert:
|
||||||
|
INSERT INTO isolation_table VALUES (5, 10);
|
||||||
|
INSERT INTO isolation_table2 VALUES (5, 10);
|
||||||
|
|
||||||
|
step s3-acquire-advisory-lock:
|
||||||
|
SELECT pg_advisory_lock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_lock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s2-isolate-tenant:
|
||||||
|
SELECT isolate_tenant_to_new_shard('isolation_table', 5, shard_transfer_mode => 'force_logical');
|
||||||
|
<waiting ...>
|
||||||
|
step s1-isolate-tenant-no-same-coloc-blocking:
|
||||||
|
SELECT isolate_tenant_to_new_shard('isolation_table2', 2, shard_transfer_mode => 'block_writes');
|
||||||
|
|
||||||
|
isolate_tenant_to_new_shard
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
1500194
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s3-release-advisory-lock:
|
||||||
|
SELECT pg_advisory_unlock(44000, 55152);
|
||||||
|
|
||||||
|
pg_advisory_unlock
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
t
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s2-isolate-tenant: <... completed>
|
||||||
|
isolate_tenant_to_new_shard
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
1500191
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
step s2-print-cluster:
|
||||||
|
-- row count per shard
|
||||||
|
SELECT
|
||||||
|
nodeport, shardid, success, result
|
||||||
|
FROM
|
||||||
|
run_command_on_placements('isolation_table', 'select count(*) from %s')
|
||||||
|
ORDER BY
|
||||||
|
nodeport, shardid;
|
||||||
|
-- rows
|
||||||
|
SELECT id, value FROM isolation_table ORDER BY id, value;
|
||||||
|
|
||||||
|
nodeport|shardid|success|result
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
57637|1500190|t | 0
|
||||||
|
57637|1500191|t | 1
|
||||||
|
57637|1500192|t | 0
|
||||||
|
57638|1500187|t | 0
|
||||||
(4 rows)
|
(4 rows)
|
||||||
|
|
||||||
id|value
|
id|value
|
||||||
|
|
|
@ -1136,19 +1136,21 @@ SELECT * FROM multi_extension.print_extension_changes();
|
||||||
table columnar.chunk_group |
|
table columnar.chunk_group |
|
||||||
table columnar.options |
|
table columnar.options |
|
||||||
table columnar.stripe |
|
table columnar.stripe |
|
||||||
|
| function citus_internal_delete_partition_metadata(regclass) void
|
||||||
| function citus_locks() SETOF record
|
| function citus_locks() SETOF record
|
||||||
| function citus_split_shard_by_split_points(bigint,text[],integer[],citus.shard_transfer_mode) void
|
| function citus_split_shard_by_split_points(bigint,text[],integer[],citus.shard_transfer_mode) void
|
||||||
|
| function create_distributed_table_concurrently(regclass,text,citus.distribution_type,text,integer) void
|
||||||
| function isolate_tenant_to_new_shard(regclass,"any",text,citus.shard_transfer_mode) bigint
|
| function isolate_tenant_to_new_shard(regclass,"any",text,citus.shard_transfer_mode) bigint
|
||||||
| function replicate_reference_tables(citus.shard_transfer_mode) void
|
| function replicate_reference_tables(citus.shard_transfer_mode) void
|
||||||
| function worker_copy_table_to_node(regclass,integer) void
|
| function worker_copy_table_to_node(regclass,integer) void
|
||||||
| function worker_split_copy(bigint,split_copy_info[]) void
|
| function worker_split_copy(bigint,text,split_copy_info[]) void
|
||||||
| function worker_split_shard_release_dsm() void
|
| function worker_split_shard_release_dsm() void
|
||||||
| function worker_split_shard_replication_setup(split_shard_info[]) SETOF replication_slot_info
|
| function worker_split_shard_replication_setup(split_shard_info[]) SETOF replication_slot_info
|
||||||
| type replication_slot_info
|
| type replication_slot_info
|
||||||
| type split_copy_info
|
| type split_copy_info
|
||||||
| type split_shard_info
|
| type split_shard_info
|
||||||
| view citus_locks
|
| view citus_locks
|
||||||
(35 rows)
|
(37 rows)
|
||||||
|
|
||||||
DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff;
|
DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff;
|
||||||
-- show running version
|
-- show running version
|
||||||
|
|
|
@ -663,7 +663,7 @@ SELECT count(*) FROM lineitem_date WHERE l_shipdate = '1997-08-08';
|
||||||
SET search_path to "Tenant Isolation";
|
SET search_path to "Tenant Isolation";
|
||||||
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE nodeport = :worker_1_port;
|
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE nodeport = :worker_1_port;
|
||||||
SELECT isolate_tenant_to_new_shard('lineitem_date', '1997-08-08', shard_transfer_mode => 'block_writes');
|
SELECT isolate_tenant_to_new_shard('lineitem_date', '1997-08-08', shard_transfer_mode => 'block_writes');
|
||||||
ERROR: cannot split shard because relation "lineitem_date" has an inactive shard placement for the shard xxxxx
|
ERROR: cannot isolate tenant because relation "lineitem_date" has an inactive shard placement for the shard xxxxx
|
||||||
HINT: Use master_copy_shard_placement UDF to repair the inactive shard placement.
|
HINT: Use master_copy_shard_placement UDF to repair the inactive shard placement.
|
||||||
UPDATE pg_dist_shard_placement SET shardstate = 1 WHERE nodeport = :worker_1_port;
|
UPDATE pg_dist_shard_placement SET shardstate = 1 WHERE nodeport = :worker_1_port;
|
||||||
\c - mx_isolation_role_ent - :master_port
|
\c - mx_isolation_role_ent - :master_port
|
||||||
|
|
|
@ -663,7 +663,7 @@ SELECT count(*) FROM lineitem_date WHERE l_shipdate = '1997-08-08';
|
||||||
SET search_path to "Tenant Isolation";
|
SET search_path to "Tenant Isolation";
|
||||||
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE nodeport = :worker_1_port;
|
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE nodeport = :worker_1_port;
|
||||||
SELECT isolate_tenant_to_new_shard('lineitem_date', '1997-08-08', shard_transfer_mode => 'force_logical');
|
SELECT isolate_tenant_to_new_shard('lineitem_date', '1997-08-08', shard_transfer_mode => 'force_logical');
|
||||||
ERROR: cannot split shard because relation "lineitem_date" has an inactive shard placement for the shard xxxxx
|
ERROR: cannot isolate tenant because relation "lineitem_date" has an inactive shard placement for the shard xxxxx
|
||||||
HINT: Use master_copy_shard_placement UDF to repair the inactive shard placement.
|
HINT: Use master_copy_shard_placement UDF to repair the inactive shard placement.
|
||||||
UPDATE pg_dist_shard_placement SET shardstate = 1 WHERE nodeport = :worker_1_port;
|
UPDATE pg_dist_shard_placement SET shardstate = 1 WHERE nodeport = :worker_1_port;
|
||||||
\c - mx_isolation_role_ent - :master_port
|
\c - mx_isolation_role_ent - :master_port
|
||||||
|
|
|
@ -65,6 +65,7 @@ ORDER BY 1;
|
||||||
function citus_internal_add_placement_metadata(bigint,integer,bigint,integer,bigint)
|
function citus_internal_add_placement_metadata(bigint,integer,bigint,integer,bigint)
|
||||||
function citus_internal_add_shard_metadata(regclass,bigint,"char",text,text)
|
function citus_internal_add_shard_metadata(regclass,bigint,"char",text,text)
|
||||||
function citus_internal_delete_colocation_metadata(integer)
|
function citus_internal_delete_colocation_metadata(integer)
|
||||||
|
function citus_internal_delete_partition_metadata(regclass)
|
||||||
function citus_internal_delete_shard_metadata(bigint)
|
function citus_internal_delete_shard_metadata(bigint)
|
||||||
function citus_internal_global_blocked_processes()
|
function citus_internal_global_blocked_processes()
|
||||||
function citus_internal_local_blocked_processes()
|
function citus_internal_local_blocked_processes()
|
||||||
|
@ -122,6 +123,7 @@ ORDER BY 1;
|
||||||
function coord_combine_agg_sfunc(internal,oid,cstring,anyelement)
|
function coord_combine_agg_sfunc(internal,oid,cstring,anyelement)
|
||||||
function create_distributed_function(regprocedure,text,text,boolean)
|
function create_distributed_function(regprocedure,text,text,boolean)
|
||||||
function create_distributed_table(regclass,text,citus.distribution_type,text,integer)
|
function create_distributed_table(regclass,text,citus.distribution_type,text,integer)
|
||||||
|
function create_distributed_table_concurrently(regclass,text,citus.distribution_type,text,integer)
|
||||||
function create_intermediate_result(text,text)
|
function create_intermediate_result(text,text)
|
||||||
function create_reference_table(regclass)
|
function create_reference_table(regclass)
|
||||||
function create_time_partitions(regclass,interval,timestamp with time zone,timestamp with time zone)
|
function create_time_partitions(regclass,interval,timestamp with time zone,timestamp with time zone)
|
||||||
|
@ -228,7 +230,7 @@ ORDER BY 1;
|
||||||
function worker_partitioned_table_size(regclass)
|
function worker_partitioned_table_size(regclass)
|
||||||
function worker_record_sequence_dependency(regclass,regclass,name)
|
function worker_record_sequence_dependency(regclass,regclass,name)
|
||||||
function worker_save_query_explain_analyze(text,jsonb)
|
function worker_save_query_explain_analyze(text,jsonb)
|
||||||
function worker_split_copy(bigint,split_copy_info[])
|
function worker_split_copy(bigint,text,split_copy_info[])
|
||||||
function worker_split_shard_release_dsm()
|
function worker_split_shard_release_dsm()
|
||||||
function worker_split_shard_replication_setup(split_shard_info[])
|
function worker_split_shard_replication_setup(split_shard_info[])
|
||||||
schema citus
|
schema citus
|
||||||
|
@ -268,5 +270,5 @@ ORDER BY 1;
|
||||||
view citus_stat_statements
|
view citus_stat_statements
|
||||||
view pg_dist_shard_placement
|
view pg_dist_shard_placement
|
||||||
view time_partitions
|
view time_partitions
|
||||||
(260 rows)
|
(262 rows)
|
||||||
|
|
||||||
|
|
|
@ -186,6 +186,7 @@ SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \
|
||||||
SET citus.enable_binary_protocol = true;
|
SET citus.enable_binary_protocol = true;
|
||||||
SELECT * from worker_split_copy(
|
SELECT * from worker_split_copy(
|
||||||
81060000, -- source shard id to copy
|
81060000, -- source shard id to copy
|
||||||
|
'l_orderkey',
|
||||||
ARRAY[
|
ARRAY[
|
||||||
-- split copy info for split children 1
|
-- split copy info for split children 1
|
||||||
ROW(81060015, -- destination shard id
|
ROW(81060015, -- destination shard id
|
||||||
|
@ -208,6 +209,7 @@ SELECT * from worker_split_copy(
|
||||||
-- BEGIN: Trigger 2-way remote shard split copy.
|
-- BEGIN: Trigger 2-way remote shard split copy.
|
||||||
SELECT * from worker_split_copy(
|
SELECT * from worker_split_copy(
|
||||||
81060000, -- source shard id to copy
|
81060000, -- source shard id to copy
|
||||||
|
'l_orderkey',
|
||||||
ARRAY[
|
ARRAY[
|
||||||
-- split copy info for split children 1
|
-- split copy info for split children 1
|
||||||
ROW(81060015, -- destination shard id
|
ROW(81060015, -- destination shard id
|
||||||
|
|
|
@ -54,6 +54,7 @@ SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \
|
||||||
-- BEGIN: Test Negative scenario
|
-- BEGIN: Test Negative scenario
|
||||||
SELECT * from worker_split_copy(
|
SELECT * from worker_split_copy(
|
||||||
101, -- Invalid source shard id.
|
101, -- Invalid source shard id.
|
||||||
|
'id',
|
||||||
ARRAY[
|
ARRAY[
|
||||||
-- split copy info for split children 1
|
-- split copy info for split children 1
|
||||||
ROW(81070015, -- destination shard id
|
ROW(81070015, -- destination shard id
|
||||||
|
@ -70,29 +71,34 @@ SELECT * from worker_split_copy(
|
||||||
ERROR: could not find valid entry for shard xxxxx
|
ERROR: could not find valid entry for shard xxxxx
|
||||||
SELECT * from worker_split_copy(
|
SELECT * from worker_split_copy(
|
||||||
81070000, -- source shard id to copy
|
81070000, -- source shard id to copy
|
||||||
|
'id',
|
||||||
ARRAY[] -- empty array
|
ARRAY[] -- empty array
|
||||||
);
|
);
|
||||||
ERROR: cannot determine type of empty array
|
ERROR: cannot determine type of empty array
|
||||||
HINT: Explicitly cast to the desired type, for example ARRAY[]::integer[].
|
HINT: Explicitly cast to the desired type, for example ARRAY[]::integer[].
|
||||||
SELECT * from worker_split_copy(
|
SELECT * from worker_split_copy(
|
||||||
81070000, -- source shard id to copy
|
81070000, -- source shard id to copy
|
||||||
|
'id',
|
||||||
ARRAY[NULL] -- empty array
|
ARRAY[NULL] -- empty array
|
||||||
);
|
);
|
||||||
ERROR: function worker_split_copy(integer, text[]) does not exist
|
ERROR: function worker_split_copy(integer, unknown, text[]) does not exist
|
||||||
HINT: No function matches the given name and argument types. You might need to add explicit type casts.
|
HINT: No function matches the given name and argument types. You might need to add explicit type casts.
|
||||||
SELECT * from worker_split_copy(
|
SELECT * from worker_split_copy(
|
||||||
81070000, -- source shard id to copy
|
81070000, -- source shard id to copy
|
||||||
|
'id',
|
||||||
ARRAY[NULL::pg_catalog.split_copy_info]-- empty array
|
ARRAY[NULL::pg_catalog.split_copy_info]-- empty array
|
||||||
);
|
);
|
||||||
ERROR: pg_catalog.split_copy_info array cannot contain null values
|
ERROR: pg_catalog.split_copy_info array cannot contain null values
|
||||||
SELECT * from worker_split_copy(
|
SELECT * from worker_split_copy(
|
||||||
81070000, -- source shard id to copy
|
81070000, -- source shard id to copy
|
||||||
|
'id',
|
||||||
ARRAY[ROW(NULL)]-- empty array
|
ARRAY[ROW(NULL)]-- empty array
|
||||||
);
|
);
|
||||||
ERROR: function worker_split_copy(integer, record[]) does not exist
|
ERROR: function worker_split_copy(integer, unknown, record[]) does not exist
|
||||||
HINT: No function matches the given name and argument types. You might need to add explicit type casts.
|
HINT: No function matches the given name and argument types. You might need to add explicit type casts.
|
||||||
SELECT * from worker_split_copy(
|
SELECT * from worker_split_copy(
|
||||||
81070000, -- source shard id to copy
|
81070000, -- source shard id to copy
|
||||||
|
'id',
|
||||||
ARRAY[ROW(NULL, NULL, NULL, NULL)::pg_catalog.split_copy_info] -- empty array
|
ARRAY[ROW(NULL, NULL, NULL, NULL)::pg_catalog.split_copy_info] -- empty array
|
||||||
);
|
);
|
||||||
ERROR: destination_shard_id for pg_catalog.split_copy_info cannot be null.
|
ERROR: destination_shard_id for pg_catalog.split_copy_info cannot be null.
|
||||||
|
@ -102,6 +108,7 @@ ERROR: destination_shard_id for pg_catalog.split_copy_info cannot be null.
|
||||||
SET citus.enable_binary_protocol = false;
|
SET citus.enable_binary_protocol = false;
|
||||||
SELECT * from worker_split_copy(
|
SELECT * from worker_split_copy(
|
||||||
81070000, -- source shard id to copy
|
81070000, -- source shard id to copy
|
||||||
|
'id',
|
||||||
ARRAY[
|
ARRAY[
|
||||||
-- split copy info for split children 1
|
-- split copy info for split children 1
|
||||||
ROW(81070015, -- destination shard id
|
ROW(81070015, -- destination shard id
|
||||||
|
|
|
@ -149,6 +149,7 @@ SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \
|
||||||
SET citus.enable_binary_protocol = false;
|
SET citus.enable_binary_protocol = false;
|
||||||
SELECT * from worker_split_copy(
|
SELECT * from worker_split_copy(
|
||||||
81070000, -- source shard id to copy
|
81070000, -- source shard id to copy
|
||||||
|
'l_orderkey',
|
||||||
ARRAY[
|
ARRAY[
|
||||||
-- split copy info for split children 1
|
-- split copy info for split children 1
|
||||||
ROW(81070015, -- destination shard id
|
ROW(81070015, -- destination shard id
|
||||||
|
@ -171,6 +172,7 @@ SELECT * from worker_split_copy(
|
||||||
-- BEGIN: Trigger 2-way remote shard split copy.
|
-- BEGIN: Trigger 2-way remote shard split copy.
|
||||||
SELECT * from worker_split_copy(
|
SELECT * from worker_split_copy(
|
||||||
81070000, -- source shard id to copy
|
81070000, -- source shard id to copy
|
||||||
|
'l_orderkey',
|
||||||
ARRAY[
|
ARRAY[
|
||||||
-- split copy info for split children 1
|
-- split copy info for split children 1
|
||||||
ROW(81070015, -- destination shard id
|
ROW(81070015, -- destination shard id
|
||||||
|
|
|
@ -18,6 +18,7 @@ test: failure_copy_on_hash
|
||||||
test: failure_create_reference_table
|
test: failure_create_reference_table
|
||||||
test: failure_create_distributed_table_non_empty
|
test: failure_create_distributed_table_non_empty
|
||||||
test: failure_create_table
|
test: failure_create_table
|
||||||
|
test: failure_create_distributed_table_concurrently
|
||||||
test: failure_single_select
|
test: failure_single_select
|
||||||
|
|
||||||
test: failure_multi_shard_update_delete
|
test: failure_multi_shard_update_delete
|
||||||
|
|
|
@ -48,6 +48,7 @@ test: isolation_create_citus_local_table
|
||||||
test: isolation_create_restore_point
|
test: isolation_create_restore_point
|
||||||
|
|
||||||
test: isolation_create_distributed_table
|
test: isolation_create_distributed_table
|
||||||
|
test: isolation_create_distributed_table_concurrently
|
||||||
test: isolation_multi_shard_modify_vs_all
|
test: isolation_multi_shard_modify_vs_all
|
||||||
test: isolation_modify_with_subquery_vs_dml
|
test: isolation_modify_with_subquery_vs_dml
|
||||||
test: isolation_hash_copy_vs_all
|
test: isolation_hash_copy_vs_all
|
||||||
|
|
|
@ -277,7 +277,7 @@ test: add_coordinator
|
||||||
test: replicate_reference_tables_to_coordinator
|
test: replicate_reference_tables_to_coordinator
|
||||||
test: citus_local_tables
|
test: citus_local_tables
|
||||||
test: mixed_relkind_tests
|
test: mixed_relkind_tests
|
||||||
test: multi_row_router_insert
|
test: multi_row_router_insert create_distributed_table_concurrently
|
||||||
test: multi_reference_table citus_local_tables_queries
|
test: multi_reference_table citus_local_tables_queries
|
||||||
test: citus_local_table_triggers
|
test: citus_local_table_triggers
|
||||||
test: coordinator_shouldhaveshards
|
test: coordinator_shouldhaveshards
|
||||||
|
|
|
@ -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"
|
|
@ -10,11 +10,16 @@ setup
|
||||||
|
|
||||||
CREATE TABLE isolation_table (id int PRIMARY KEY, value int);
|
CREATE TABLE isolation_table (id int PRIMARY KEY, value int);
|
||||||
SELECT create_distributed_table('isolation_table', 'id');
|
SELECT create_distributed_table('isolation_table', 'id');
|
||||||
|
|
||||||
|
-- different colocation id
|
||||||
|
CREATE TABLE isolation_table2 (id smallint PRIMARY KEY, value int);
|
||||||
|
SELECT create_distributed_table('isolation_table2', 'id');
|
||||||
}
|
}
|
||||||
|
|
||||||
teardown
|
teardown
|
||||||
{
|
{
|
||||||
DROP TABLE isolation_table;
|
DROP TABLE isolation_table;
|
||||||
|
DROP TABLE isolation_table2;
|
||||||
}
|
}
|
||||||
|
|
||||||
session "s1"
|
session "s1"
|
||||||
|
@ -32,11 +37,13 @@ step "s1-begin"
|
||||||
step "s1-load-cache"
|
step "s1-load-cache"
|
||||||
{
|
{
|
||||||
TRUNCATE isolation_table;
|
TRUNCATE isolation_table;
|
||||||
|
TRUNCATE isolation_table2;
|
||||||
}
|
}
|
||||||
|
|
||||||
step "s1-insert"
|
step "s1-insert"
|
||||||
{
|
{
|
||||||
INSERT INTO isolation_table VALUES (5, 10);
|
INSERT INTO isolation_table VALUES (5, 10);
|
||||||
|
INSERT INTO isolation_table2 VALUES (5, 10);
|
||||||
}
|
}
|
||||||
|
|
||||||
step "s1-update"
|
step "s1-update"
|
||||||
|
@ -59,11 +66,26 @@ step "s1-copy"
|
||||||
COPY isolation_table FROM PROGRAM 'echo "1,1\n2,2\n3,3\n4,4\n5,5"' WITH CSV;
|
COPY isolation_table FROM PROGRAM 'echo "1,1\n2,2\n3,3\n4,4\n5,5"' WITH CSV;
|
||||||
}
|
}
|
||||||
|
|
||||||
step "s1-isolate-tenant"
|
step "s1-isolate-tenant-same-coloc"
|
||||||
{
|
{
|
||||||
SELECT isolate_tenant_to_new_shard('isolation_table', 2, shard_transfer_mode => 'force_logical');
|
SELECT isolate_tenant_to_new_shard('isolation_table', 2, shard_transfer_mode => 'force_logical');
|
||||||
}
|
}
|
||||||
|
|
||||||
|
step "s1-isolate-tenant-same-coloc-blocking"
|
||||||
|
{
|
||||||
|
SELECT isolate_tenant_to_new_shard('isolation_table', 2, shard_transfer_mode => 'block_writes');
|
||||||
|
}
|
||||||
|
|
||||||
|
step "s1-isolate-tenant-no-same-coloc"
|
||||||
|
{
|
||||||
|
SELECT isolate_tenant_to_new_shard('isolation_table2', 2, shard_transfer_mode => 'force_logical');
|
||||||
|
}
|
||||||
|
|
||||||
|
step "s1-isolate-tenant-no-same-coloc-blocking"
|
||||||
|
{
|
||||||
|
SELECT isolate_tenant_to_new_shard('isolation_table2', 2, shard_transfer_mode => 'block_writes');
|
||||||
|
}
|
||||||
|
|
||||||
step "s1-commit"
|
step "s1-commit"
|
||||||
{
|
{
|
||||||
COMMIT;
|
COMMIT;
|
||||||
|
@ -122,7 +144,7 @@ step "s3-release-advisory-lock"
|
||||||
// s1 can execute its DML command concurrently with s2 shard isolation =>
|
// s1 can execute its DML command concurrently with s2 shard isolation =>
|
||||||
// s3 releases the advisory lock so that s2 can finish the transaction
|
// s3 releases the advisory lock so that s2 can finish the transaction
|
||||||
|
|
||||||
// run tenant isolation while concurrently performing an DML and index creation
|
// run tenant isolation while concurrently performing an DML
|
||||||
// we expect DML queries of s2 to succeed without being blocked.
|
// we expect DML queries of s2 to succeed without being blocked.
|
||||||
permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-update" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"
|
permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-update" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"
|
||||||
permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-delete" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"
|
permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-delete" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"
|
||||||
|
@ -135,8 +157,20 @@ permutation "s1-insert" "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-be
|
||||||
permutation "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-insert" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"
|
permutation "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-insert" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"
|
||||||
permutation "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-copy" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"
|
permutation "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-copy" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"
|
||||||
|
|
||||||
// concurrent tenant isolation blocks on different shards of the same table (or any colocated table)
|
// concurrent nonblocking tenant isolations with the same colocation id are not allowed
|
||||||
permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s1-begin" "s1-isolate-tenant" "s2-isolate-tenant" "s3-release-advisory-lock" "s1-commit" "s2-print-cluster"
|
permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s2-isolate-tenant" "s1-isolate-tenant-same-coloc" "s3-release-advisory-lock" "s2-print-cluster"
|
||||||
|
|
||||||
// the same test above without loading the cache at first
|
// concurrent blocking and nonblocking tenant isolations with the same colocation id are not allowed
|
||||||
permutation "s1-insert" "s3-acquire-advisory-lock" "s1-begin" "s1-isolate-tenant" "s2-isolate-tenant" "s3-release-advisory-lock" "s1-commit" "s2-print-cluster"
|
permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s2-isolate-tenant" "s1-isolate-tenant-same-coloc-blocking" "s3-release-advisory-lock" "s2-print-cluster"
|
||||||
|
|
||||||
|
// concurrent nonblocking tenant isolations in different transactions are not allowed
|
||||||
|
permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s2-isolate-tenant" "s1-isolate-tenant-no-same-coloc" "s3-release-advisory-lock" "s2-print-cluster"
|
||||||
|
|
||||||
|
// concurrent nonblocking tenant isolations in the same transaction are not allowed
|
||||||
|
permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s2-begin" "s2-isolate-tenant" "s1-isolate-tenant-no-same-coloc" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"
|
||||||
|
|
||||||
|
// concurrent blocking and nonblocking tenant isolations with different colocation ids in different transactions are allowed
|
||||||
|
permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s2-isolate-tenant" "s1-isolate-tenant-no-same-coloc-blocking" "s3-release-advisory-lock" "s2-print-cluster"
|
||||||
|
|
||||||
|
// concurrent blocking and nonblocking tenant isolations with different colocation ids in the same transaction are allowed
|
||||||
|
permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s2-isolate-tenant" "s1-isolate-tenant-no-same-coloc-blocking" "s3-release-advisory-lock" "s2-print-cluster"
|
||||||
|
|
|
@ -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;
|
|
@ -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);
|
|
@ -160,6 +160,7 @@ SET citus.enable_binary_protocol = true;
|
||||||
|
|
||||||
SELECT * from worker_split_copy(
|
SELECT * from worker_split_copy(
|
||||||
81060000, -- source shard id to copy
|
81060000, -- source shard id to copy
|
||||||
|
'l_orderkey',
|
||||||
ARRAY[
|
ARRAY[
|
||||||
-- split copy info for split children 1
|
-- split copy info for split children 1
|
||||||
ROW(81060015, -- destination shard id
|
ROW(81060015, -- destination shard id
|
||||||
|
@ -178,6 +179,7 @@ SELECT * from worker_split_copy(
|
||||||
-- BEGIN: Trigger 2-way remote shard split copy.
|
-- BEGIN: Trigger 2-way remote shard split copy.
|
||||||
SELECT * from worker_split_copy(
|
SELECT * from worker_split_copy(
|
||||||
81060000, -- source shard id to copy
|
81060000, -- source shard id to copy
|
||||||
|
'l_orderkey',
|
||||||
ARRAY[
|
ARRAY[
|
||||||
-- split copy info for split children 1
|
-- split copy info for split children 1
|
||||||
ROW(81060015, -- destination shard id
|
ROW(81060015, -- destination shard id
|
||||||
|
|
|
@ -38,6 +38,7 @@ SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \
|
||||||
-- BEGIN: Test Negative scenario
|
-- BEGIN: Test Negative scenario
|
||||||
SELECT * from worker_split_copy(
|
SELECT * from worker_split_copy(
|
||||||
101, -- Invalid source shard id.
|
101, -- Invalid source shard id.
|
||||||
|
'id',
|
||||||
ARRAY[
|
ARRAY[
|
||||||
-- split copy info for split children 1
|
-- split copy info for split children 1
|
||||||
ROW(81070015, -- destination shard id
|
ROW(81070015, -- destination shard id
|
||||||
|
@ -54,26 +55,31 @@ SELECT * from worker_split_copy(
|
||||||
|
|
||||||
SELECT * from worker_split_copy(
|
SELECT * from worker_split_copy(
|
||||||
81070000, -- source shard id to copy
|
81070000, -- source shard id to copy
|
||||||
|
'id',
|
||||||
ARRAY[] -- empty array
|
ARRAY[] -- empty array
|
||||||
);
|
);
|
||||||
|
|
||||||
SELECT * from worker_split_copy(
|
SELECT * from worker_split_copy(
|
||||||
81070000, -- source shard id to copy
|
81070000, -- source shard id to copy
|
||||||
|
'id',
|
||||||
ARRAY[NULL] -- empty array
|
ARRAY[NULL] -- empty array
|
||||||
);
|
);
|
||||||
|
|
||||||
SELECT * from worker_split_copy(
|
SELECT * from worker_split_copy(
|
||||||
81070000, -- source shard id to copy
|
81070000, -- source shard id to copy
|
||||||
|
'id',
|
||||||
ARRAY[NULL::pg_catalog.split_copy_info]-- empty array
|
ARRAY[NULL::pg_catalog.split_copy_info]-- empty array
|
||||||
);
|
);
|
||||||
|
|
||||||
SELECT * from worker_split_copy(
|
SELECT * from worker_split_copy(
|
||||||
81070000, -- source shard id to copy
|
81070000, -- source shard id to copy
|
||||||
|
'id',
|
||||||
ARRAY[ROW(NULL)]-- empty array
|
ARRAY[ROW(NULL)]-- empty array
|
||||||
);
|
);
|
||||||
|
|
||||||
SELECT * from worker_split_copy(
|
SELECT * from worker_split_copy(
|
||||||
81070000, -- source shard id to copy
|
81070000, -- source shard id to copy
|
||||||
|
'id',
|
||||||
ARRAY[ROW(NULL, NULL, NULL, NULL)::pg_catalog.split_copy_info] -- empty array
|
ARRAY[ROW(NULL, NULL, NULL, NULL)::pg_catalog.split_copy_info] -- empty array
|
||||||
);
|
);
|
||||||
-- END: Test Negative scenario
|
-- END: Test Negative scenario
|
||||||
|
@ -83,6 +89,7 @@ SELECT * from worker_split_copy(
|
||||||
SET citus.enable_binary_protocol = false;
|
SET citus.enable_binary_protocol = false;
|
||||||
SELECT * from worker_split_copy(
|
SELECT * from worker_split_copy(
|
||||||
81070000, -- source shard id to copy
|
81070000, -- source shard id to copy
|
||||||
|
'id',
|
||||||
ARRAY[
|
ARRAY[
|
||||||
-- split copy info for split children 1
|
-- split copy info for split children 1
|
||||||
ROW(81070015, -- destination shard id
|
ROW(81070015, -- destination shard id
|
||||||
|
|
|
@ -152,6 +152,7 @@ SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \
|
||||||
SET citus.enable_binary_protocol = false;
|
SET citus.enable_binary_protocol = false;
|
||||||
SELECT * from worker_split_copy(
|
SELECT * from worker_split_copy(
|
||||||
81070000, -- source shard id to copy
|
81070000, -- source shard id to copy
|
||||||
|
'l_orderkey',
|
||||||
ARRAY[
|
ARRAY[
|
||||||
-- split copy info for split children 1
|
-- split copy info for split children 1
|
||||||
ROW(81070015, -- destination shard id
|
ROW(81070015, -- destination shard id
|
||||||
|
@ -170,6 +171,7 @@ SELECT * from worker_split_copy(
|
||||||
-- BEGIN: Trigger 2-way remote shard split copy.
|
-- BEGIN: Trigger 2-way remote shard split copy.
|
||||||
SELECT * from worker_split_copy(
|
SELECT * from worker_split_copy(
|
||||||
81070000, -- source shard id to copy
|
81070000, -- source shard id to copy
|
||||||
|
'l_orderkey',
|
||||||
ARRAY[
|
ARRAY[
|
||||||
-- split copy info for split children 1
|
-- split copy info for split children 1
|
||||||
ROW(81070015, -- destination shard id
|
ROW(81070015, -- destination shard id
|
||||||
|
|
Loading…
Reference in New Issue