From fa467e05e79cca5d07a4ae359f825d0c8ac17b6f Mon Sep 17 00:00:00 2001 From: Onur Tirtir Date: Fri, 17 Mar 2023 14:17:36 +0300 Subject: [PATCH 1/7] Add support for creating distributed tables with a null shard key (#6745) With this PR, we allow creating distributed tables with without specifying a shard key via create_distributed_table(). Here are the the important details about those tables: * Specifying `shard_count` is not allowed because it is assumed to be 1. * We mostly call such tables as "null shard-key" table in code / comments. * To avoid doing a breaking layout change in create_distributed_table(); instead of throwing an error, it will inform the user that `distribution_type` param is ignored unless it's explicitly set to NULL or 'h'. * `colocate_with` param allows colocating such null shard-key tables to each other. * We define this table type, i.e., NULL_SHARD_KEY_TABLE, as a subclass of DISTRIBUTED_TABLE because we mostly want to treat them as distributed tables in terms of SQL / DDL / operation support. * Metadata for such tables look like: - distribution method => DISTRIBUTE_BY_NONE - replication model => REPLICATION_MODEL_STREAMING - colocation id => **!=** INVALID_COLOCATION_ID (distinguishes from Citus local tables) * We assign colocation groups for such tables to different nodes in a round-robin fashion based on the modulo of "colocation id". Note that this PR doesn't care about DDL (except CREATE TABLE) / SQL / operation (i.e., Citus UDFs) support for such tables but adds a preliminary API. --- .../commands/create_distributed_table.c | 195 ++- .../distributed/commands/foreign_constraint.c | 12 +- src/backend/distributed/commands/multi_copy.c | 1 + src/backend/distributed/commands/truncate.c | 3 +- .../distributed/metadata/metadata_cache.c | 27 +- .../distributed/metadata/metadata_sync.c | 11 +- .../distributed/operations/create_shards.c | 88 +- .../distributed/operations/stage_protocol.c | 3 +- .../planner/insert_select_planner.c | 7 + .../planner/multi_physical_planner.c | 2 +- .../planner/multi_router_planner.c | 10 +- .../transaction/relation_access_tracking.c | 8 +- .../distributed/utils/colocation_utils.c | 16 +- .../distributed/utils/distribution_column.c | 2 +- .../distributed/utils/shardinterval_utils.c | 7 +- .../distributed/coordinator_protocol.h | 1 + src/include/distributed/metadata_cache.h | 3 + src/test/regress/citus_tests/run_test.py | 1 + .../regress/expected/create_null_dist_key.out | 1432 +++++++++++++++++ .../expected/multi_colocation_utils.out | 4 +- src/test/regress/expected/multi_extension.out | 27 + src/test/regress/expected/single_node.out | 57 +- src/test/regress/expected/single_node_0.out | 57 +- src/test/regress/multi_1_schedule | 1 + src/test/regress/sql/create_null_dist_key.sql | 962 +++++++++++ src/test/regress/sql/multi_extension.sql | 14 + src/test/regress/sql/single_node.sql | 37 +- 27 files changed, 2911 insertions(+), 77 deletions(-) create mode 100644 src/test/regress/expected/create_null_dist_key.out create mode 100644 src/test/regress/sql/create_null_dist_key.sql diff --git a/src/backend/distributed/commands/create_distributed_table.c b/src/backend/distributed/commands/create_distributed_table.c index 7e907c8a8..12bfcf9a5 100644 --- a/src/backend/distributed/commands/create_distributed_table.c +++ b/src/backend/distributed/commands/create_distributed_table.c @@ -134,6 +134,7 @@ static List * HashSplitPointsForShardList(List *shardList); static List * HashSplitPointsForShardCount(int shardCount); static List * WorkerNodesForShardList(List *shardList); static List * RoundRobinWorkerNodeList(List *workerNodeList, int listLength); +static void CreateNullShardKeyDistTable(Oid relationId, char *colocateWithTableName); static CitusTableParams DecideCitusTableParams(CitusTableType tableType, DistributedTableParams * distributedTableParams); @@ -141,6 +142,8 @@ static void CreateCitusTable(Oid relationId, CitusTableType tableType, DistributedTableParams *distributedTableParams); static void CreateHashDistributedTableShards(Oid relationId, int shardCount, Oid colocatedTableId, bool localTableEmpty); +static void CreateNullShardKeyDistTableShard(Oid relationId, Oid colocatedTableId, + uint32 colocationId); static uint32 ColocationIdForNewTable(Oid relationId, CitusTableType tableType, DistributedTableParams *distributedTableParams, Var *distributionColumn); @@ -216,51 +219,86 @@ create_distributed_table(PG_FUNCTION_ARGS) { CheckCitusVersion(ERROR); - if (PG_ARGISNULL(0) || PG_ARGISNULL(1) || PG_ARGISNULL(2) || PG_ARGISNULL(3)) + if (PG_ARGISNULL(0) || PG_ARGISNULL(3)) { PG_RETURN_VOID(); } Oid relationId = PG_GETARG_OID(0); - text *distributionColumnText = PG_GETARG_TEXT_P(1); + text *distributionColumnText = PG_ARGISNULL(1) ? NULL : PG_GETARG_TEXT_P(1); Oid distributionMethodOid = PG_GETARG_OID(2); text *colocateWithTableNameText = PG_GETARG_TEXT_P(3); char *colocateWithTableName = text_to_cstring(colocateWithTableNameText); bool shardCountIsStrict = false; - int shardCount = ShardCount; - if (!PG_ARGISNULL(4)) + if (distributionColumnText) { - if (pg_strncasecmp(colocateWithTableName, "default", NAMEDATALEN) != 0 && - pg_strncasecmp(colocateWithTableName, "none", NAMEDATALEN) != 0) + if (PG_ARGISNULL(2)) { - ereport(ERROR, (errmsg("Cannot use colocate_with with a table " - "and shard_count at the same time"))); + PG_RETURN_VOID(); } - shardCount = PG_GETARG_INT32(4); + int shardCount = ShardCount; + if (!PG_ARGISNULL(4)) + { + if (!IsColocateWithDefault(colocateWithTableName) && + !IsColocateWithNone(colocateWithTableName)) + { + ereport(ERROR, (errmsg("Cannot use colocate_with with a table " + "and shard_count at the same time"))); + } - /* - * if shard_count parameter is given than we have to - * make sure table has that many shards - */ - shardCountIsStrict = true; + shardCount = PG_GETARG_INT32(4); + + /* + * If shard_count parameter is given, then we have to + * make sure table has that many shards. + */ + shardCountIsStrict = true; + } + + char *distributionColumnName = text_to_cstring(distributionColumnText); + Assert(distributionColumnName != NULL); + + char distributionMethod = LookupDistributionMethod(distributionMethodOid); + + if (shardCount < 1 || shardCount > MAX_SHARD_COUNT) + { + ereport(ERROR, (errmsg("%d is outside the valid range for " + "parameter \"shard_count\" (1 .. %d)", + shardCount, MAX_SHARD_COUNT))); + } + + CreateDistributedTable(relationId, distributionColumnName, distributionMethod, + shardCount, shardCountIsStrict, colocateWithTableName); } - - char *distributionColumnName = text_to_cstring(distributionColumnText); - Assert(distributionColumnName != NULL); - - char distributionMethod = LookupDistributionMethod(distributionMethodOid); - - if (shardCount < 1 || shardCount > MAX_SHARD_COUNT) + else { - ereport(ERROR, (errmsg("%d is outside the valid range for " - "parameter \"shard_count\" (1 .. %d)", - shardCount, MAX_SHARD_COUNT))); - } + if (!PG_ARGISNULL(4)) + { + ereport(ERROR, (errmsg("shard_count can't be specified when the " + "distribution column is null because in " + "that case it's automatically set to 1"))); + } - CreateDistributedTable(relationId, distributionColumnName, distributionMethod, - shardCount, shardCountIsStrict, colocateWithTableName); + if (!PG_ARGISNULL(2) && + LookupDistributionMethod(PG_GETARG_OID(2)) != DISTRIBUTE_BY_HASH) + { + /* + * As we do for shard_count parameter, we could throw an error if + * distribution_type is not NULL when creating a null-shard-key table. + * However, this requires changing the default value of distribution_type + * parameter to NULL and this would mean a breaking change for most + * users because they're mostly using this API to create sharded + * tables. For this reason, here we instead do nothing if the distribution + * method is DISTRIBUTE_BY_HASH. + */ + ereport(ERROR, (errmsg("distribution_type can't be specified " + "when the distribution column is null "))); + } + + CreateNullShardKeyDistTable(relationId, colocateWithTableName); + } PG_RETURN_VOID(); } @@ -276,11 +314,18 @@ create_distributed_table_concurrently(PG_FUNCTION_ARGS) { CheckCitusVersion(ERROR); - if (PG_ARGISNULL(0) || PG_ARGISNULL(1) || PG_ARGISNULL(2) || PG_ARGISNULL(3)) + if (PG_ARGISNULL(0) || PG_ARGISNULL(2) || PG_ARGISNULL(3)) { PG_RETURN_VOID(); } + if (PG_ARGISNULL(1)) + { + ereport(ERROR, (errmsg("cannot use create_distributed_table_concurrently " + "to create a distributed table with a null shard " + "key, consider using create_distributed_table()"))); + } + Oid relationId = PG_GETARG_OID(0); text *distributionColumnText = PG_GETARG_TEXT_P(1); char *distributionColumnName = text_to_cstring(distributionColumnText); @@ -982,6 +1027,23 @@ CreateReferenceTable(Oid relationId) } +/* + * CreateNullShardKeyDistTable is a wrapper around CreateCitusTable that creates a + * single shard distributed table that doesn't have a shard key. + */ +static void +CreateNullShardKeyDistTable(Oid relationId, char *colocateWithTableName) +{ + DistributedTableParams distributedTableParams = { + .colocateWithTableName = colocateWithTableName, + .shardCount = 1, + .shardCountIsStrict = true, + .distributionColumnName = NULL + }; + CreateCitusTable(relationId, NULL_KEY_DISTRIBUTED_TABLE, &distributedTableParams); +} + + /* * CreateCitusTable is the internal method that creates a Citus table in * given configuration. @@ -1000,7 +1062,8 @@ CreateCitusTable(Oid relationId, CitusTableType tableType, DistributedTableParams *distributedTableParams) { if ((tableType == HASH_DISTRIBUTED || tableType == APPEND_DISTRIBUTED || - tableType == RANGE_DISTRIBUTED) != (distributedTableParams != NULL)) + tableType == RANGE_DISTRIBUTED || tableType == NULL_KEY_DISTRIBUTED_TABLE) != + (distributedTableParams != NULL)) { ereport(ERROR, (errmsg("distributed table params must be provided " "when creating a distributed table and must " @@ -1078,7 +1141,7 @@ CreateCitusTable(Oid relationId, CitusTableType tableType, PropagatePrerequisiteObjectsForDistributedTable(relationId); Var *distributionColumn = NULL; - if (distributedTableParams) + if (distributedTableParams && distributedTableParams->distributionColumnName) { distributionColumn = BuildDistributionKeyFromColumnName(relationId, distributedTableParams-> @@ -1150,6 +1213,11 @@ CreateCitusTable(Oid relationId, CitusTableType tableType, { CreateReferenceTableShard(relationId); } + else if (tableType == NULL_KEY_DISTRIBUTED_TABLE) + { + CreateNullShardKeyDistTableShard(relationId, colocatedTableId, + colocationId); + } if (ShouldSyncTableMetadata(relationId)) { @@ -1204,7 +1272,8 @@ CreateCitusTable(Oid relationId, CitusTableType tableType, } /* copy over data for hash distributed and reference tables */ - if (tableType == HASH_DISTRIBUTED || tableType == REFERENCE_TABLE) + if (tableType == HASH_DISTRIBUTED || tableType == NULL_KEY_DISTRIBUTED_TABLE || + tableType == REFERENCE_TABLE) { if (RegularTable(relationId)) { @@ -1268,6 +1337,13 @@ DecideCitusTableParams(CitusTableType tableType, break; } + case NULL_KEY_DISTRIBUTED_TABLE: + { + citusTableParams.distributionMethod = DISTRIBUTE_BY_NONE; + citusTableParams.replicationModel = REPLICATION_MODEL_STREAMING; + break; + } + case REFERENCE_TABLE: { citusTableParams.distributionMethod = DISTRIBUTE_BY_NONE; @@ -1630,6 +1706,41 @@ CreateHashDistributedTableShards(Oid relationId, int shardCount, } +/* + * CreateHashDistributedTableShards creates the shard of given null-shard-key + * distributed table. + */ +static void +CreateNullShardKeyDistTableShard(Oid relationId, Oid colocatedTableId, + uint32 colocationId) +{ + if (colocatedTableId != InvalidOid) + { + /* + * We currently allow concurrent distribution of colocated tables (which + * we probably should not be allowing because of foreign keys / + * partitioning etc). + * + * We also prevent concurrent shard moves / copy / splits) while creating + * a colocated table. + */ + AcquirePlacementColocationLock(colocatedTableId, ShareLock, + "colocate distributed table"); + + /* + * We don't need to force using exclusive connections because we're anyway + * creating a single shard. + */ + bool useExclusiveConnection = false; + CreateColocatedShards(relationId, colocatedTableId, useExclusiveConnection); + } + else + { + CreateNullKeyShardWithRoundRobinPolicy(relationId, colocationId); + } +} + + /* * ColocationIdForNewTable returns a colocation id for given table * according to given configuration. If there is no such configuration, it @@ -1662,8 +1773,8 @@ ColocationIdForNewTable(Oid relationId, CitusTableType tableType, { ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), errmsg("cannot distribute relation"), - errdetail("Currently, colocate_with option is only supported " - "for hash distributed tables."))); + errdetail("Currently, colocate_with option is not supported " + "for append / range distributed tables."))); } return colocationId; @@ -1679,10 +1790,11 @@ ColocationIdForNewTable(Oid relationId, CitusTableType tableType, * can be sure that there will no modifications on the colocation table * until this transaction is committed. */ - Assert(citusTableParams.distributionMethod == DISTRIBUTE_BY_HASH); - Oid distributionColumnType = distributionColumn->vartype; - Oid distributionColumnCollation = get_typcollation(distributionColumnType); + Oid distributionColumnType = + distributionColumn ? distributionColumn->vartype : InvalidOid; + Oid distributionColumnCollation = + distributionColumn ? get_typcollation(distributionColumnType) : InvalidOid; /* get an advisory lock to serialize concurrent default group creations */ if (IsColocateWithDefault(distributedTableParams->colocateWithTableName)) @@ -1871,8 +1983,15 @@ EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn, */ if (PartitionedTableNoLock(relationId)) { - /* distributing partitioned tables in only supported for hash-distribution */ - if (distributionMethod != DISTRIBUTE_BY_HASH) + /* + * Distributing partitioned tables is only supported for hash-distribution + * or null-shard-key tables. + */ + bool isNullShardKeyTable = + distributionMethod == DISTRIBUTE_BY_NONE && + replicationModel == REPLICATION_MODEL_STREAMING && + colocationId != INVALID_COLOCATION_ID; + if (distributionMethod != DISTRIBUTE_BY_HASH && !isNullShardKeyTable) { ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), errmsg("distributing partitioned tables in only supported " diff --git a/src/backend/distributed/commands/foreign_constraint.c b/src/backend/distributed/commands/foreign_constraint.c index 6f12db13f..0b2c5573e 100644 --- a/src/backend/distributed/commands/foreign_constraint.c +++ b/src/backend/distributed/commands/foreign_constraint.c @@ -303,6 +303,11 @@ ErrorIfUnsupportedForeignConstraintExists(Relation relation, char referencingDis /* * Foreign keys from citus local tables or reference tables to distributed * tables are not supported. + * + * We could support foreign keys from references tables to null-shard-key + * tables but this doesn't seem useful a lot. However, if we decide supporting + * this, then we need to expand relation access tracking check for the null-shard-key + * tables too. */ if (referencingIsCitusLocalOrRefTable && !referencedIsCitusLocalOrRefTable) { @@ -361,7 +366,12 @@ ErrorIfUnsupportedForeignConstraintExists(Relation relation, char referencingDis * if tables are hash-distributed and colocated, we need to make sure that * the distribution key is included in foreign constraint. */ - if (!referencedIsCitusLocalOrRefTable && !foreignConstraintOnDistKey) + bool referencedIsNullShardKeyTable = + IsNullShardKeyTableByDistParams(referencedDistMethod, + referencedReplicationModel, + referencedColocationId); + if (!referencedIsCitusLocalOrRefTable && !referencedIsNullShardKeyTable && + !foreignConstraintOnDistKey) { ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), errmsg("cannot create foreign key constraint"), diff --git a/src/backend/distributed/commands/multi_copy.c b/src/backend/distributed/commands/multi_copy.c index 6e3d19b68..fdb9552ef 100644 --- a/src/backend/distributed/commands/multi_copy.c +++ b/src/backend/distributed/commands/multi_copy.c @@ -2146,6 +2146,7 @@ CitusCopyDestReceiverStartup(DestReceiver *dest, int operation, } if (IsCitusTableTypeCacheEntry(cacheEntry, DISTRIBUTED_TABLE) && + !IsCitusTableTypeCacheEntry(cacheEntry, NULL_KEY_DISTRIBUTED_TABLE) && copyDest->partitionColumnIndex == INVALID_PARTITION_COLUMN_INDEX) { ereport(ERROR, (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED), diff --git a/src/backend/distributed/commands/truncate.c b/src/backend/distributed/commands/truncate.c index 52f769a11..70fee6bd5 100644 --- a/src/backend/distributed/commands/truncate.c +++ b/src/backend/distributed/commands/truncate.c @@ -324,7 +324,8 @@ ExecuteTruncateStmtSequentialIfNecessary(TruncateStmt *command) { Oid relationId = RangeVarGetRelid(rangeVar, NoLock, failOK); - if (IsCitusTable(relationId) && !HasDistributionKey(relationId) && + if ((IsCitusTableType(relationId, REFERENCE_TABLE) || + IsCitusTableType(relationId, CITUS_LOCAL_TABLE)) && TableReferenced(relationId)) { char *relationName = get_rel_name(relationId); diff --git a/src/backend/distributed/metadata/metadata_cache.c b/src/backend/distributed/metadata/metadata_cache.c index 1e73eef6b..4eab2aeed 100644 --- a/src/backend/distributed/metadata/metadata_cache.c +++ b/src/backend/distributed/metadata/metadata_cache.c @@ -508,11 +508,21 @@ IsCitusTableTypeInternal(char partitionMethod, char replicationModel, return partitionMethod == DISTRIBUTE_BY_RANGE; } + case NULL_KEY_DISTRIBUTED_TABLE: + { + return partitionMethod == DISTRIBUTE_BY_NONE && + replicationModel != REPLICATION_MODEL_2PC && + colocationId != INVALID_COLOCATION_ID; + } + case DISTRIBUTED_TABLE: { return partitionMethod == DISTRIBUTE_BY_HASH || partitionMethod == DISTRIBUTE_BY_RANGE || - partitionMethod == DISTRIBUTE_BY_APPEND; + partitionMethod == DISTRIBUTE_BY_APPEND || + (partitionMethod == DISTRIBUTE_BY_NONE && + replicationModel != REPLICATION_MODEL_2PC && + colocationId != INVALID_COLOCATION_ID); } case STRICTLY_PARTITIONED_DISTRIBUTED_TABLE: @@ -815,6 +825,21 @@ IsCitusLocalTableByDistParams(char partitionMethod, char replicationModel, } +/* + * IsNullShardKeyTableByDistParams returns true if given partitionMethod, + * replicationModel and colocationId would identify a distributed table that + * has a null shard key. + */ +bool +IsNullShardKeyTableByDistParams(char partitionMethod, char replicationModel, + uint32 colocationId) +{ + return partitionMethod == DISTRIBUTE_BY_NONE && + replicationModel != REPLICATION_MODEL_2PC && + colocationId != INVALID_COLOCATION_ID; +} + + /* * CitusTableList returns a list that includes all the valid distributed table * cache entries. diff --git a/src/backend/distributed/metadata/metadata_sync.c b/src/backend/distributed/metadata/metadata_sync.c index 868617ce0..f9c80942a 100644 --- a/src/backend/distributed/metadata/metadata_sync.c +++ b/src/backend/distributed/metadata/metadata_sync.c @@ -515,7 +515,7 @@ ShouldSyncUserCommandForObject(ObjectAddress objectAddress) /* * ShouldSyncTableMetadata checks if the metadata of a distributed table should be * propagated to metadata workers, i.e. the table is a hash distributed table or - * reference/citus local table. + * a Citus table that doesn't have shard key. */ bool ShouldSyncTableMetadata(Oid relationId) @@ -537,10 +537,11 @@ ShouldSyncTableMetadata(Oid relationId) /* - * ShouldSyncTableMetadataViaCatalog checks if the metadata of a distributed table should - * be propagated to metadata workers, i.e. the table is an MX table or reference table. + * ShouldSyncTableMetadataViaCatalog checks if the metadata of a Citus table should + * be propagated to metadata workers, i.e. the table is an MX table or Citus table + * that doesn't have shard key. * Tables with streaming replication model (which means RF=1) and hash distribution are - * considered as MX tables while tables with none distribution are reference tables. + * considered as MX tables. * * ShouldSyncTableMetadataViaCatalog does not use the CitusTableCache and instead reads * from catalog tables directly. @@ -1080,7 +1081,7 @@ EnsureObjectMetadataIsSane(int distributionArgumentIndex, int colocationId) /* * DistributionCreateCommands generates a commands that can be - * executed to replicate the metadata for a distributed table. + * executed to replicate the metadata for a Citus table. */ char * DistributionCreateCommand(CitusTableCacheEntry *cacheEntry) diff --git a/src/backend/distributed/operations/create_shards.c b/src/backend/distributed/operations/create_shards.c index 3edab94e9..1a2ce5f1c 100644 --- a/src/backend/distributed/operations/create_shards.c +++ b/src/backend/distributed/operations/create_shards.c @@ -217,9 +217,9 @@ CreateColocatedShards(Oid targetRelationId, Oid sourceRelationId, bool List *insertedShardPlacements = NIL; List *insertedShardIds = NIL; - /* make sure that tables are hash partitioned */ - CheckHashPartitionedTable(targetRelationId); - CheckHashPartitionedTable(sourceRelationId); + CitusTableCacheEntry *targetCacheEntry = GetCitusTableCacheEntry(targetRelationId); + Assert(targetCacheEntry->partitionMethod == DISTRIBUTE_BY_HASH || + targetCacheEntry->partitionMethod == DISTRIBUTE_BY_NONE); /* * In contrast to append/range partitioned tables it makes more sense to @@ -259,10 +259,20 @@ CreateColocatedShards(Oid targetRelationId, Oid sourceRelationId, bool *newShardIdPtr = GetNextShardId(); insertedShardIds = lappend(insertedShardIds, newShardIdPtr); - int32 shardMinValue = DatumGetInt32(sourceShardInterval->minValue); - int32 shardMaxValue = DatumGetInt32(sourceShardInterval->maxValue); - text *shardMinValueText = IntegerToText(shardMinValue); - text *shardMaxValueText = IntegerToText(shardMaxValue); + text *shardMinValueText = NULL; + text *shardMaxValueText = NULL; + if (targetCacheEntry->partitionMethod == DISTRIBUTE_BY_NONE) + { + Assert(list_length(sourceShardIntervalList) == 1); + } + else + { + int32 shardMinValue = DatumGetInt32(sourceShardInterval->minValue); + int32 shardMaxValue = DatumGetInt32(sourceShardInterval->maxValue); + shardMinValueText = IntegerToText(shardMinValue); + shardMaxValueText = IntegerToText(shardMaxValue); + } + List *sourceShardPlacementList = ShardPlacementListSortedByWorker( sourceShardId); @@ -362,6 +372,70 @@ CreateReferenceTableShard(Oid distributedTableId) } +/* + * CreateNullKeyShardWithRoundRobinPolicy creates a single shard for the given + * distributedTableId. The created shard does not have min/max values. + * Unlike CreateReferenceTableShard, the shard is _not_ replicated to + * all nodes but would have a single placement like Citus local tables. + * However, this placement doesn't necessarily need to be placed on + * coordinator. This is determined based on modulo of the colocation + * id that given table has been associated to. + */ +void +CreateNullKeyShardWithRoundRobinPolicy(Oid relationId, uint32 colocationId) +{ + EnsureTableOwner(relationId); + + /* we plan to add shards: get an exclusive lock on relation oid */ + LockRelationOid(relationId, ExclusiveLock); + + /* + * Load and sort the worker node list for deterministic placement. + * + * Also take a RowShareLock on pg_dist_node to disallow concurrent + * node list changes that require an exclusive lock. + */ + List *workerNodeList = DistributedTablePlacementNodeList(RowShareLock); + workerNodeList = SortList(workerNodeList, CompareWorkerNodes); + + int32 workerNodeCount = list_length(workerNodeList); + if (workerNodeCount == 0) + { + ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("couldn't find any worker nodes"), + errhint("Add more worker nodes"))); + } + + char shardStorageType = ShardStorageType(relationId); + text *minHashTokenText = NULL; + text *maxHashTokenText = NULL; + uint64 shardId = GetNextShardId(); + InsertShardRow(relationId, shardId, shardStorageType, + minHashTokenText, maxHashTokenText); + + /* determine the node index based on colocation id */ + int roundRobinNodeIdx = colocationId % workerNodeCount; + + int replicationFactor = 1; + List *insertedShardPlacements = InsertShardPlacementRows( + relationId, + shardId, + workerNodeList, + roundRobinNodeIdx, + replicationFactor); + + /* + * We don't need to force using exclusive connections because we're anyway + * creating a single shard. + */ + bool useExclusiveConnection = false; + + bool colocatedShard = false; + CreateShardsOnWorkers(relationId, insertedShardPlacements, + useExclusiveConnection, colocatedShard); +} + + /* * CheckHashPartitionedTable looks up the partition information for the given * tableId and checks if the table is hash partitioned. If not, the function diff --git a/src/backend/distributed/operations/stage_protocol.c b/src/backend/distributed/operations/stage_protocol.c index ce9fe3f31..c1031ffdf 100644 --- a/src/backend/distributed/operations/stage_protocol.c +++ b/src/backend/distributed/operations/stage_protocol.c @@ -521,7 +521,8 @@ RelationShardListForShardCreate(ShardInterval *shardInterval) relationShard->shardId = shardInterval->shardId; List *relationShardList = list_make1(relationShard); - if (IsCitusTableTypeCacheEntry(cacheEntry, HASH_DISTRIBUTED) && + if ((IsCitusTableTypeCacheEntry(cacheEntry, HASH_DISTRIBUTED) || + IsCitusTableTypeCacheEntry(cacheEntry, NULL_KEY_DISTRIBUTED_TABLE)) && cacheEntry->colocationId != INVALID_COLOCATION_ID) { shardIndex = ShardIndex(shardInterval); diff --git a/src/backend/distributed/planner/insert_select_planner.c b/src/backend/distributed/planner/insert_select_planner.c index 2eab62fc3..53fe58cdb 100644 --- a/src/backend/distributed/planner/insert_select_planner.c +++ b/src/backend/distributed/planner/insert_select_planner.c @@ -715,6 +715,13 @@ DistributedInsertSelectSupported(Query *queryTree, RangeTblEntry *insertRte, NULL, NULL); } } + else if (IsCitusTableType(targetRelationId, NULL_KEY_DISTRIBUTED_TABLE)) + { + return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "distributed INSERT ... SELECT cannot target a distributed " + "table with a null shard key", + NULL, NULL); + } else { /* diff --git a/src/backend/distributed/planner/multi_physical_planner.c b/src/backend/distributed/planner/multi_physical_planner.c index f488a1cd5..b210da7d7 100644 --- a/src/backend/distributed/planner/multi_physical_planner.c +++ b/src/backend/distributed/planner/multi_physical_planner.c @@ -2487,7 +2487,7 @@ QueryPushdownTaskCreate(Query *originalQuery, int shardIndex, /* non-distributed tables have only one shard */ shardInterval = cacheEntry->sortedShardIntervalArray[0]; - /* only use reference table as anchor shard if none exists yet */ + /* use as anchor shard only if we couldn't find any yet */ if (anchorShardId == INVALID_SHARD_ID) { anchorShardId = shardInterval->shardId; diff --git a/src/backend/distributed/planner/multi_router_planner.c b/src/backend/distributed/planner/multi_router_planner.c index a95be74f8..97c2cecf6 100644 --- a/src/backend/distributed/planner/multi_router_planner.c +++ b/src/backend/distributed/planner/multi_router_planner.c @@ -2684,7 +2684,7 @@ TargetShardIntervalForFastPathQuery(Query *query, bool *isMultiShardQuery, if (!HasDistributionKey(relationId)) { - /* we don't need to do shard pruning for non-distributed tables */ + /* we don't need to do shard pruning for single shard tables */ return list_make1(LoadShardIntervalList(relationId)); } @@ -2974,7 +2974,7 @@ BuildRoutesForInsert(Query *query, DeferredErrorMessage **planningError) Assert(query->commandType == CMD_INSERT); - /* reference tables and citus local tables can only have one shard */ + /* tables that don't have distribution column can only have one shard */ if (!HasDistributionKeyCacheEntry(cacheEntry)) { List *shardIntervalList = LoadShardIntervalList(distributedTableId); @@ -2992,6 +2992,12 @@ BuildRoutesForInsert(Query *query, DeferredErrorMessage **planningError) ereport(ERROR, (errmsg("local table cannot have %d shards", shardCount))); } + else if (IsCitusTableTypeCacheEntry(cacheEntry, NULL_KEY_DISTRIBUTED_TABLE)) + { + ereport(ERROR, (errmsg("distributed tables having a null shard key " + "cannot have %d shards", + shardCount))); + } } ShardInterval *shardInterval = linitial(shardIntervalList); diff --git a/src/backend/distributed/transaction/relation_access_tracking.c b/src/backend/distributed/transaction/relation_access_tracking.c index 2ecbba5b7..b0af4e476 100644 --- a/src/backend/distributed/transaction/relation_access_tracking.c +++ b/src/backend/distributed/transaction/relation_access_tracking.c @@ -195,7 +195,7 @@ RecordRelationAccessIfNonDistTable(Oid relationId, ShardPlacementAccessType acce * recursively calling RecordRelationAccessBase(), so becareful about * removing this check. */ - if (IsCitusTable(relationId) && HasDistributionKey(relationId)) + if (IsCitusTableType(relationId, DISTRIBUTED_TABLE)) { return; } @@ -732,7 +732,7 @@ CheckConflictingRelationAccesses(Oid relationId, ShardPlacementAccessType access CitusTableCacheEntry *cacheEntry = GetCitusTableCacheEntry(relationId); - if (HasDistributionKeyCacheEntry(cacheEntry) || + if (IsCitusTableTypeCacheEntry(cacheEntry, DISTRIBUTED_TABLE) || cacheEntry->referencingRelationsViaForeignKey == NIL) { return; @@ -931,7 +931,7 @@ HoldsConflictingLockWithReferencedRelations(Oid relationId, ShardPlacementAccess * We're only interested in foreign keys to reference tables and citus * local tables. */ - if (IsCitusTable(referencedRelation) && HasDistributionKey(referencedRelation)) + if (IsCitusTableType(referencedRelation, DISTRIBUTED_TABLE)) { continue; } @@ -993,7 +993,7 @@ HoldsConflictingLockWithReferencingRelations(Oid relationId, ShardPlacementAcces CitusTableCacheEntry *cacheEntry = GetCitusTableCacheEntry(relationId); bool holdsConflictingLocks = false; - Assert(!HasDistributionKeyCacheEntry(cacheEntry)); + Assert(!IsCitusTableTypeCacheEntry(cacheEntry, DISTRIBUTED_TABLE)); Oid referencingRelation = InvalidOid; foreach_oid(referencingRelation, cacheEntry->referencingRelationsViaForeignKey) diff --git a/src/backend/distributed/utils/colocation_utils.c b/src/backend/distributed/utils/colocation_utils.c index 985d4c38e..9cc5df8f9 100644 --- a/src/backend/distributed/utils/colocation_utils.c +++ b/src/backend/distributed/utils/colocation_utils.c @@ -1384,17 +1384,19 @@ 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)) + if (IsCitusTableTypeCacheEntry(sourceTableEntry, APPEND_DISTRIBUTED) || + IsCitusTableTypeCacheEntry(sourceTableEntry, RANGE_DISTRIBUTED) || + IsCitusTableTypeCacheEntry(sourceTableEntry, CITUS_LOCAL_TABLE)) { ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), errmsg("cannot distribute relation"), - errdetail("Currently, colocate_with option is only supported " - "for hash distributed tables."))); + errdetail("Currently, colocate_with option is not supported " + "with append / range distributed tables and local " + "tables added to metadata."))); } + char sourceReplicationModel = sourceTableEntry->replicationModel; if (sourceReplicationModel != replicationModel) { char *relationName = get_rel_name(relationId); @@ -1406,7 +1408,9 @@ EnsureTableCanBeColocatedWith(Oid relationId, char replicationModel, sourceRelationName, relationName))); } - Oid sourceDistributionColumnType = sourceDistributionColumn->vartype; + Var *sourceDistributionColumn = DistPartitionKey(sourceRelationId); + Oid sourceDistributionColumnType = !sourceDistributionColumn ? InvalidOid : + sourceDistributionColumn->vartype; if (sourceDistributionColumnType != distributionColumnType) { char *relationName = get_rel_name(relationId); diff --git a/src/backend/distributed/utils/distribution_column.c b/src/backend/distributed/utils/distribution_column.c index 7fbab98eb..1f5ac9ec5 100644 --- a/src/backend/distributed/utils/distribution_column.c +++ b/src/backend/distributed/utils/distribution_column.c @@ -135,7 +135,7 @@ BuildDistributionKeyFromColumnName(Oid relationId, char *columnName, LOCKMODE lo char *tableName = get_rel_name(relationId); - /* short circuit for reference tables */ + /* short circuit for reference tables and null-shard key tables */ if (columnName == NULL) { return NULL; diff --git a/src/backend/distributed/utils/shardinterval_utils.c b/src/backend/distributed/utils/shardinterval_utils.c index 12635f9f4..6c18e201e 100644 --- a/src/backend/distributed/utils/shardinterval_utils.c +++ b/src/backend/distributed/utils/shardinterval_utils.c @@ -206,7 +206,7 @@ CompareRelationShards(const void *leftElement, const void *rightElement) * * For hash partitioned tables, it calculates hash value of a number in its * range (e.g. min value) and finds which shard should contain the hashed - * value. For reference tables and citus local tables, it simply returns 0. + * value. For the tables that don't have a shard key, it simply returns 0. * For the other table types, the function errors out. */ int @@ -231,12 +231,11 @@ ShardIndex(ShardInterval *shardInterval) "tables that are added to citus metadata"))); } - /* short-circuit for reference tables */ + /* short-circuit for the tables that don't have a distribution key */ if (!HasDistributionKeyCacheEntry(cacheEntry)) { /* - * Reference tables and citus local tables have only a single shard, - * so the index is fixed to 0. + * Such tables have only a single shard, so the index is fixed to 0. */ shardIndex = 0; diff --git a/src/include/distributed/coordinator_protocol.h b/src/include/distributed/coordinator_protocol.h index ad8329a6c..351efb790 100644 --- a/src/include/distributed/coordinator_protocol.h +++ b/src/include/distributed/coordinator_protocol.h @@ -262,6 +262,7 @@ extern void CreateShardsWithRoundRobinPolicy(Oid distributedTableId, int32 shard extern void CreateColocatedShards(Oid targetRelationId, Oid sourceRelationId, bool useExclusiveConnections); extern void CreateReferenceTableShard(Oid distributedTableId); +extern void CreateNullKeyShardWithRoundRobinPolicy(Oid relationId, uint32 colocationId); extern List * WorkerCreateShardCommandList(Oid relationId, int shardIndex, uint64 shardId, List *ddlCommandList, List *foreignConstraintCommandList); diff --git a/src/include/distributed/metadata_cache.h b/src/include/distributed/metadata_cache.h index c23a047ec..6f29f9d63 100644 --- a/src/include/distributed/metadata_cache.h +++ b/src/include/distributed/metadata_cache.h @@ -123,6 +123,7 @@ typedef enum HASH_DISTRIBUTED, APPEND_DISTRIBUTED, RANGE_DISTRIBUTED, + NULL_KEY_DISTRIBUTED_TABLE, /* hash, range or append distributed table */ DISTRIBUTED_TABLE, @@ -157,6 +158,8 @@ extern uint32 ColocationIdViaCatalog(Oid relationId); bool IsReferenceTableByDistParams(char partitionMethod, char replicationModel); extern bool IsCitusLocalTableByDistParams(char partitionMethod, char replicationModel, uint32 colocationId); +extern bool IsNullShardKeyTableByDistParams(char partitionMethod, char replicationModel, + uint32 colocationId); extern List * CitusTableList(void); extern ShardInterval * LoadShardInterval(uint64 shardId); extern bool ShardExists(uint64 shardId); diff --git a/src/test/regress/citus_tests/run_test.py b/src/test/regress/citus_tests/run_test.py index 72ce2fb14..cda9fc0ae 100755 --- a/src/test/regress/citus_tests/run_test.py +++ b/src/test/regress/citus_tests/run_test.py @@ -128,6 +128,7 @@ DEPS = { "multi_mx_copy_data": TestDeps(None, ["multi_mx_create_table"]), "multi_mx_schema_support": TestDeps(None, ["multi_mx_copy_data"]), "multi_simple_queries": TestDeps("base_schedule"), + "create_null_dist_key": TestDeps("minimal_schedule"), } diff --git a/src/test/regress/expected/create_null_dist_key.out b/src/test/regress/expected/create_null_dist_key.out new file mode 100644 index 000000000..e24ff1e91 --- /dev/null +++ b/src/test/regress/expected/create_null_dist_key.out @@ -0,0 +1,1432 @@ +CREATE SCHEMA create_null_dist_key; +SET search_path TO create_null_dist_key; +SET citus.next_shard_id TO 1720000; +SET citus.shard_count TO 32; +SET citus.shard_replication_factor TO 1; +SELECT 1 FROM citus_remove_node('localhost', :worker_1_port); + ?column? +--------------------------------------------------------------------- + 1 +(1 row) + +SELECT 1 FROM citus_remove_node('localhost', :worker_2_port); + ?column? +--------------------------------------------------------------------- + 1 +(1 row) + +CREATE TABLE add_node_test(a int, "b" text); +-- add a node before creating the null-shard-key table +SELECT 1 FROM citus_add_node('localhost', :worker_1_port); + ?column? +--------------------------------------------------------------------- + 1 +(1 row) + +SELECT create_distributed_table('add_node_test', null, colocate_with=>'none', distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- add a node after creating the null-shard-key table +SELECT 1 FROM citus_add_node('localhost', :worker_2_port); + ?column? +--------------------------------------------------------------------- + 1 +(1 row) + +-- make sure that table is created on the worker nodes added before/after create_distributed_table +SELECT result FROM run_command_on_workers($$ + SELECT COUNT(*)=1 FROM pg_class WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + relname='add_node_test' +$$); + result +--------------------------------------------------------------------- + t + t +(2 rows) + +-- and check the metadata tables +SELECT result FROM run_command_on_workers($$ + SELECT (partmethod, partkey, repmodel, autoconverted) FROM pg_dist_partition + WHERE logicalrelid = 'create_null_dist_key.add_node_test'::regclass +$$); + result +--------------------------------------------------------------------- + (n,,s,f) + (n,,s,f) +(2 rows) + +SELECT result FROM run_command_on_workers($$ + SELECT (shardstorage, shardminvalue, shardmaxvalue) FROM pg_dist_shard + WHERE logicalrelid = 'create_null_dist_key.add_node_test'::regclass +$$); + result +--------------------------------------------------------------------- + (t,,) + (t,,) +(2 rows) + +SELECT result FROM run_command_on_workers($$ + SELECT COUNT(*)=1 FROM pg_dist_placement + WHERE shardid = ( + SELECT shardid FROM pg_dist_shard + WHERE logicalrelid = 'create_null_dist_key.add_node_test'::regclass + ); +$$); + result +--------------------------------------------------------------------- + t + t +(2 rows) + +SELECT result FROM run_command_on_workers($$ + SELECT (shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation) FROM pg_dist_colocation + WHERE colocationid = ( + SELECT colocationid FROM pg_dist_partition + WHERE logicalrelid = 'create_null_dist_key.add_node_test'::regclass + ); +$$); + result +--------------------------------------------------------------------- + (1,1,0,0) + (1,1,0,0) +(2 rows) + +SET client_min_messages TO WARNING; +SELECT 1 FROM citus_add_node('localhost', :master_port, groupid => 0); + ?column? +--------------------------------------------------------------------- + 1 +(1 row) + +SET client_min_messages TO NOTICE; +CREATE TABLE invalid_configs_1(a int primary key); +SELECT create_distributed_table('invalid_configs_1', null, shard_count=>2); +ERROR: shard_count can't be specified when the distribution column is null because in that case it's automatically set to 1 +SELECT create_distributed_table('invalid_configs_1', null, shard_count=>1); +ERROR: shard_count can't be specified when the distribution column is null because in that case it's automatically set to 1 +CREATE TABLE nullkey_c1_t1(a int, b int); +CREATE TABLE nullkey_c1_t2(a int, b int); +CREATE TABLE nullkey_c1_t3(a int, b int); +SELECT create_distributed_table('nullkey_c1_t1', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT colocationid AS nullkey_c1_t1_colocation_id FROM pg_dist_partition WHERE logicalrelid = 'create_null_dist_key.nullkey_c1_t1'::regclass \gset +BEGIN; + DROP TABLE nullkey_c1_t1; + -- make sure that we delete the colocation group after dropping the last table that belongs to it + SELECT COUNT(*)=0 FROM pg_dist_colocation WHERE colocationid = :'nullkey_c1_t1_colocation_id'; + ?column? +--------------------------------------------------------------------- + t +(1 row) + +ROLLBACK; +SELECT create_distributed_table('nullkey_c1_t2', null, colocate_with=>'nullkey_c1_t1'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('nullkey_c1_t3', null, colocate_with=>'nullkey_c1_t1', distribution_type=>'append'); +ERROR: distribution_type can't be specified when the distribution column is null +SELECT create_distributed_table('nullkey_c1_t3', null, colocate_with=>'nullkey_c1_t1'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE nullkey_c2_t1(a int, b int); +CREATE TABLE nullkey_c2_t2(a int, b int); +CREATE TABLE nullkey_c2_t3(a int, b int); +-- create_distributed_table_concurrently is not yet supported yet +SELECT create_distributed_table_concurrently('nullkey_c2_t1', null); +ERROR: cannot use create_distributed_table_concurrently to create a distributed table with a null shard key, consider using create_distributed_table() +SELECT create_distributed_table_concurrently('nullkey_c2_t1', null, colocate_with=>'none'); +ERROR: cannot use create_distributed_table_concurrently to create a distributed table with a null shard key, consider using create_distributed_table() +SELECT create_distributed_table('nullkey_c2_t1', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('nullkey_c2_t2', null, colocate_with=>'nullkey_c2_t1', distribution_type=>'hash'); -- distribution_type is ignored anyway + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('nullkey_c2_t3', null, colocate_with=>'nullkey_c2_t2', distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- check the metadata for the colocated tables whose names start with nullkey_c1_ +SELECT logicalrelid, partmethod, partkey, repmodel, autoconverted FROM pg_dist_partition +WHERE logicalrelid IN ( + SELECT oid FROM pg_class + WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + relname LIKE 'nullkey_c1_%' +) +ORDER BY 1; + logicalrelid | partmethod | partkey | repmodel | autoconverted +--------------------------------------------------------------------- + nullkey_c1_t1 | n | | s | f + nullkey_c1_t2 | n | | s | f + nullkey_c1_t3 | n | | s | f +(3 rows) + +-- make sure that all those 3 tables belong to same colocation group +SELECT COUNT(*) FROM pg_dist_partition +WHERE logicalrelid IN ( + SELECT oid FROM pg_class + WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + relname LIKE 'nullkey_c1_%' +) +GROUP BY colocationid; + count +--------------------------------------------------------------------- + 3 +(1 row) + +SELECT logicalrelid, shardstorage, shardminvalue, shardmaxvalue FROM pg_dist_shard +WHERE logicalrelid IN ( + SELECT oid FROM pg_class + WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + relname LIKE 'nullkey_c1_%' +) +ORDER BY 1; + logicalrelid | shardstorage | shardminvalue | shardmaxvalue +--------------------------------------------------------------------- + nullkey_c1_t1 | t | | + nullkey_c1_t2 | t | | + nullkey_c1_t3 | t | | +(3 rows) + +-- make sure that all those 3 shards are created on the same node group +SELECT COUNT(*) FROM pg_dist_placement +WHERE shardid IN ( + SELECT shardid FROM pg_dist_shard + WHERE logicalrelid IN ( + SELECT oid FROM pg_class + WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + relname LIKE 'nullkey_c1_%' + ) +) +GROUP BY groupid; + count +--------------------------------------------------------------------- + 3 +(1 row) + +-- check the metadata for the colocated tables whose names start with nullkey_c2_ +SELECT logicalrelid, partmethod, partkey, repmodel, autoconverted FROM pg_dist_partition +WHERE logicalrelid IN ( + SELECT oid FROM pg_class + WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + relname LIKE 'nullkey_c2_%' +) +ORDER BY 1; + logicalrelid | partmethod | partkey | repmodel | autoconverted +--------------------------------------------------------------------- + nullkey_c2_t1 | n | | s | f + nullkey_c2_t2 | n | | s | f + nullkey_c2_t3 | n | | s | f +(3 rows) + +-- make sure that all those 3 tables belong to same colocation group +SELECT COUNT(*) FROM pg_dist_partition +WHERE logicalrelid IN ( + SELECT oid FROM pg_class + WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + relname LIKE 'nullkey_c2_%' +) +GROUP BY colocationid; + count +--------------------------------------------------------------------- + 3 +(1 row) + +SELECT logicalrelid, shardstorage, shardminvalue, shardmaxvalue FROM pg_dist_shard +WHERE logicalrelid IN ( + SELECT oid FROM pg_class + WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + relname LIKE 'nullkey_c2_%' +) +ORDER BY 1; + logicalrelid | shardstorage | shardminvalue | shardmaxvalue +--------------------------------------------------------------------- + nullkey_c2_t1 | t | | + nullkey_c2_t2 | t | | + nullkey_c2_t3 | t | | +(3 rows) + +-- make sure that all those 3 shards created on the same node group +SELECT COUNT(*) FROM pg_dist_placement +WHERE shardid IN ( + SELECT shardid FROM pg_dist_shard + WHERE logicalrelid IN ( + SELECT oid FROM pg_class + WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + relname LIKE 'nullkey_c2_%' + ) +) +GROUP BY groupid; + count +--------------------------------------------------------------------- + 3 +(1 row) + +-- Make sure that the colocated tables whose names start with nullkey_c1_ +-- belong to a different colocation group than the ones whose names start +-- with nullkey_c2_. +-- +-- It's ok to only compare nullkey_c1_t1 and nullkey_c2_t1 because we already +-- verified that null_dist_key.nullkey_c1_t1 is colocated with the other two +-- and null_dist_key.nullkey_c2_t1 is colocated with the other two. +SELECT +( + SELECT colocationid FROM pg_dist_partition + WHERE logicalrelid = 'create_null_dist_key.nullkey_c1_t1'::regclass +) +!= +( + SELECT colocationid FROM pg_dist_partition + WHERE logicalrelid = 'create_null_dist_key.nullkey_c2_t1'::regclass +); + ?column? +--------------------------------------------------------------------- + t +(1 row) + +-- Since we determine node for the placement based on the module of colocation id, +-- we don't expect those two colocation groups to get assigned to same node. +SELECT +( + SELECT groupid FROM pg_dist_placement + WHERE shardid = ( + SELECT shardid FROM pg_dist_shard + WHERE logicalrelid = 'create_null_dist_key.nullkey_c1_t1'::regclass + ) +) +!= +( + SELECT groupid FROM pg_dist_placement + WHERE shardid = ( + SELECT shardid FROM pg_dist_shard + WHERE logicalrelid = 'create_null_dist_key.nullkey_c2_t1'::regclass + ) +); + ?column? +--------------------------------------------------------------------- + t +(1 row) + +-- It's ok to only check nullkey_c1_t1 and nullkey_c2_t1 because we already +-- verified that null_dist_key.nullkey_c1_t1 is colocated with the other two +-- and null_dist_key.nullkey_c2_t1 is colocated with the other two. +SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation +WHERE colocationid = ( + SELECT colocationid FROM pg_dist_partition + WHERE logicalrelid = 'create_null_dist_key.nullkey_c1_t1'::regclass +); + shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation +--------------------------------------------------------------------- + 1 | 1 | 0 | 0 +(1 row) + +SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation +WHERE colocationid = ( + SELECT colocationid FROM pg_dist_partition + WHERE logicalrelid = 'create_null_dist_key.nullkey_c2_t1'::regclass +); + shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation +--------------------------------------------------------------------- + 1 | 1 | 0 | 0 +(1 row) + +CREATE TABLE round_robin_test_c1(a int, b int); +SELECT create_distributed_table('round_robin_test_c1', null, colocate_with=>'none', distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +\c - - - :master_port +SET search_path TO create_null_dist_key; +SET citus.next_shard_id TO 1730000; +SET citus.shard_count TO 32; +SET citus.shard_replication_factor TO 1; +SET client_min_messages TO NOTICE; +CREATE TABLE round_robin_test_c2(a int, b int); +SELECT create_distributed_table('round_robin_test_c2', null, colocate_with=>'none', distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- Since we determine node for the placement based on the module of colocation id, +-- we don't expect those two colocation groups to get assigned to same node even +-- after reconnecting to the coordinator. +SELECT +( + SELECT groupid FROM pg_dist_placement + WHERE shardid = ( + SELECT shardid FROM pg_dist_shard + WHERE logicalrelid = 'create_null_dist_key.round_robin_test_c1'::regclass + ) +) +!= +( + SELECT groupid FROM pg_dist_placement + WHERE shardid = ( + SELECT shardid FROM pg_dist_shard + WHERE logicalrelid = 'create_null_dist_key.round_robin_test_c2'::regclass + ) +); + ?column? +--------------------------------------------------------------------- + t +(1 row) + +CREATE TABLE distributed_table(a int, b int); +-- cannot colocate a sharded table with null shard key table +SELECT create_distributed_table('distributed_table', 'a', colocate_with=>'nullkey_c1_t1'); +ERROR: cannot colocate tables nullkey_c1_t1 and distributed_table +DETAIL: Distribution column types don't match for nullkey_c1_t1 and distributed_table. +CREATE TABLE reference_table(a int, b int); +CREATE TABLE local(a int, b int); +SELECT create_reference_table('reference_table'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('distributed_table', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- cannot colocate null shard key tables with other table types +CREATE TABLE cannot_colocate_with_other_types (a int, b int); +SELECT create_distributed_table('cannot_colocate_with_other_types', null, colocate_with=>'reference_table'); +ERROR: cannot colocate tables reference_table and cannot_colocate_with_other_types +DETAIL: Replication models don't match for reference_table and cannot_colocate_with_other_types. +SELECT create_distributed_table('cannot_colocate_with_other_types', null, colocate_with=>'distributed_table'); +ERROR: cannot colocate tables distributed_table and cannot_colocate_with_other_types +DETAIL: Distribution column types don't match for distributed_table and cannot_colocate_with_other_types. +SELECT create_distributed_table('cannot_colocate_with_other_types', null, colocate_with=>'local'); -- postgres local +ERROR: relation local is not distributed +SELECT citus_add_local_table_to_metadata('local'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +-- cannot colocate null shard key tables with citus local tables +SELECT create_distributed_table('cannot_colocate_with_other_types', null, colocate_with=>'local'); -- citus local +ERROR: cannot distribute relation +DETAIL: Currently, colocate_with option is not supported with append / range distributed tables and local tables added to metadata. +SET client_min_messages TO WARNING; +-- can't create such a distributed table from another Citus table, except Citus local tables +SELECT create_distributed_table('reference_table', null, colocate_with=>'none'); +ERROR: table "reference_table" is already distributed +SELECT create_distributed_table('distributed_table', null, colocate_with=>'none'); +ERROR: table "distributed_table" is already distributed +SELECT create_distributed_table('local', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +BEGIN; + -- creating a null-shard-key table from a temporary table is not supported + CREATE TEMPORARY TABLE temp_table (a int); + SELECT create_distributed_table('temp_table', null, colocate_with=>'none', distribution_type=>null); +ERROR: cannot distribute a temporary table +ROLLBACK; +-- creating a null-shard-key table from a catalog table is not supported +SELECT create_distributed_table('pg_catalog.pg_index', NULL, distribution_type=>null); +ERROR: cannot create a citus table from a catalog table +-- creating a null-shard-key table from an unlogged table is supported +CREATE UNLOGGED TABLE unlogged_table (a int); +SELECT create_distributed_table('unlogged_table', null, colocate_with=>'none', distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- creating a null-shard-key table from a foreign table is not supported +CREATE FOREIGN TABLE foreign_table ( + id bigint not null, + full_name text not null default '' +) SERVER fake_fdw_server OPTIONS (encoding 'utf-8', compression 'true', table_name 'foreign_table'); +SELECT create_distributed_table('foreign_table', null, colocate_with=>'none', distribution_type=>null); +ERROR: foreign tables cannot be distributed +HINT: Can add foreign table "foreign_table" to metadata by running: SELECT citus_add_local_table_to_metadata($$create_null_dist_key.foreign_table$$); +-- create a null dist key table that has no tuples +CREATE TABLE null_dist_key_table_1 (a int primary key); +SELECT create_distributed_table('null_dist_key_table_1', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- create a null dist key table that has some tuples +CREATE TABLE null_dist_key_table_2(a int primary key); +INSERT INTO null_dist_key_table_2 VALUES(1); +SELECT create_distributed_table('null_dist_key_table_2', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT * FROM null_dist_key_table_2 ORDER BY a; + a +--------------------------------------------------------------------- + 1 +(1 row) + +DROP TABLE null_dist_key_table_1, null_dist_key_table_2; +-- create indexes before creating the null dist key tables +-- .. for an initially empty table +CREATE TABLE null_dist_key_table_1(a int); +CREATE INDEX null_dist_key_table_1_idx ON null_dist_key_table_1(a); +SELECT create_distributed_table('null_dist_key_table_1', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- .. and for another table having data in it before creating null dist key table +CREATE TABLE null_dist_key_table_2(a int); +INSERT INTO null_dist_key_table_2 VALUES(1); +CREATE INDEX null_dist_key_table_2_idx ON null_dist_key_table_2(a); +SELECT create_distributed_table('null_dist_key_table_2', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT * FROM null_dist_key_table_2 ORDER BY a; + a +--------------------------------------------------------------------- + 1 +(1 row) + +-- show that we do not support inheritance relationships +CREATE TABLE parent_table (a int, b text); +CREATE TABLE child_table () INHERITS (parent_table); +-- both of below should error out +SELECT create_distributed_table('parent_table', null, colocate_with=>'none'); +ERROR: parent_table is not a regular, foreign or partitioned table +SELECT create_distributed_table('child_table', null, colocate_with=>'none'); +ERROR: child_table is not a regular, foreign or partitioned table +-- show that we support policies +BEGIN; + CREATE TABLE null_dist_key_table_3 (table_user text); + ALTER TABLE null_dist_key_table_3 ENABLE ROW LEVEL SECURITY; + CREATE ROLE table_users; + CREATE POLICY table_policy ON null_dist_key_table_3 TO table_users + USING (table_user = current_user); + SELECT create_distributed_table('null_dist_key_table_3', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +ROLLBACK; +-- drop them for next tests +DROP TABLE null_dist_key_table_1, null_dist_key_table_2, distributed_table; +-- tests for object names that should be escaped properly +CREATE SCHEMA "NULL_!_dist_key"; +CREATE TABLE "NULL_!_dist_key"."my_TABLE.1!?!"(id int, "Second_Id" int); +SELECT create_distributed_table('"NULL_!_dist_key"."my_TABLE.1!?!"', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- drop the table before creating it when the search path is set +SET search_path to "NULL_!_dist_key" ; +DROP TABLE "my_TABLE.1!?!"; +CREATE TYPE int_jsonb_type AS (key int, value jsonb); +CREATE DOMAIN age_with_default AS int CHECK (value >= 0) DEFAULT 0; +CREATE TYPE yes_no_enum AS ENUM ('yes', 'no'); +CREATE EXTENSION btree_gist; +CREATE SEQUENCE my_seq_1 START WITH 10; +CREATE TABLE "Table?!.1Table"( + id int PRIMARY KEY, + "Second_Id" int, + "local_Type" int_jsonb_type, + "jsondata" jsonb NOT NULL, + name text, + price numeric CHECK (price > 0), + age_with_default_col age_with_default, + yes_no_enum_col yes_no_enum, + seq_col_1 bigserial, + seq_col_2 int DEFAULT nextval('my_seq_1'), + generated_column int GENERATED ALWAYS AS (seq_col_1 * seq_col_2 + 4) STORED, + UNIQUE (id, price), + EXCLUDE USING GIST (name WITH =)); +-- create some objects before create_distributed_table +CREATE INDEX "my!Index1" ON "Table?!.1Table"(id) WITH ( fillfactor = 80 ) WHERE id > 10; +CREATE INDEX text_index ON "Table?!.1Table"(name); +CREATE UNIQUE INDEX uniqueIndex ON "Table?!.1Table" (id); +CREATE STATISTICS stats_1 ON id, price FROM "Table?!.1Table"; +CREATE TEXT SEARCH CONFIGURATION text_search_cfg (parser = default); +CREATE INDEX text_search_index ON "Table?!.1Table" +USING gin (to_tsvector('text_search_cfg'::regconfig, (COALESCE(name, ''::character varying))::text)); +-- ingest some data before create_distributed_table +INSERT INTO "Table?!.1Table" VALUES (1, 1, (1, row_to_json(row(1,1)))::int_jsonb_type, row_to_json(row(1,1), true)), + (2, 1, (2, row_to_json(row(2,2)))::int_jsonb_type, row_to_json(row(2,2), 'false')); +-- create a replica identity before create_distributed_table +ALTER TABLE "Table?!.1Table" REPLICA IDENTITY USING INDEX uniqueIndex; +SELECT create_distributed_table('"Table?!.1Table"', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO "Table?!.1Table" VALUES (10, 15, (150, row_to_json(row(4,8)))::int_jsonb_type, '{}', 'text_1', 10, 27, 'yes', 60, 70); +INSERT INTO "Table?!.1Table" VALUES (5, 5, (5, row_to_json(row(5,5)))::int_jsonb_type, row_to_json(row(5,5), true)); +-- tuples that are supposed to violate different data type / check constraints +INSERT INTO "Table?!.1Table"(id, jsondata, name) VALUES (101, '{"a": 1}', 'text_1'); +ERROR: conflicting key value violates exclusion constraint "Table?!.1Table_name_excl_1730043" +DETAIL: Key (name)=(text_1) conflicts with existing key (name)=(text_1). +CONTEXT: while executing command on localhost:xxxxx +INSERT INTO "Table?!.1Table"(id, jsondata, price) VALUES (101, '{"a": 1}', -1); +ERROR: new row for relation "Table?!.1Table_1730043" violates check constraint "Table?!.1Table_price_check" +DETAIL: Failing row contains (101, null, null, {"a": 1}, null, -1, 0, null, 5, 14, 74). +CONTEXT: while executing command on localhost:xxxxx +INSERT INTO "Table?!.1Table"(id, jsondata, age_with_default_col) VALUES (101, '{"a": 1}', -1); +ERROR: value for domain age_with_default violates check constraint "age_with_default_check" +INSERT INTO "Table?!.1Table"(id, jsondata, yes_no_enum_col) VALUES (101, '{"a": 1}', 'what?'); +ERROR: invalid input value for enum yes_no_enum: "what?" +SELECT * FROM "Table?!.1Table" ORDER BY id; + id | Second_Id | local_Type | jsondata | name | price | age_with_default_col | yes_no_enum_col | seq_col_1 | seq_col_2 | generated_column +--------------------------------------------------------------------- + 1 | 1 | (1,"{""f1"": 1, ""f2"": 1}") | {"f1": 1, "f2": 1} | | | 0 | | 1 | 10 | 14 + 2 | 1 | (2,"{""f1"": 2, ""f2"": 2}") | {"f1": 2, "f2": 2} | | | 0 | | 2 | 11 | 26 + 5 | 5 | (5,"{""f1"": 5, ""f2"": 5}") | {"f1": 5, "f2": 5} | | | 0 | | 3 | 12 | 40 + 10 | 15 | (150,"{""f1"": 4, ""f2"": 8}") | {} | text_1 | 10 | 27 | yes | 60 | 70 | 4204 +(4 rows) + +SET search_path TO create_null_dist_key; +-- create a partitioned table with some columns that +-- are going to be dropped within the tests +CREATE TABLE sensors( + col_to_drop_1 text, + measureid integer, + eventdatetime date, + measure_data jsonb, +PRIMARY KEY (measureid, eventdatetime, measure_data)) +PARTITION BY RANGE(eventdatetime); +-- drop column even before attaching any partitions +ALTER TABLE sensors DROP COLUMN col_to_drop_1; +CREATE TABLE sensors_2000 PARTITION OF sensors FOR VALUES FROM ('2000-01-01') TO ('2001-01-01'); +-- cannot distribute child table without distributing the parent +SELECT create_distributed_table('sensors_2000', NULL, distribution_type=>null); +ERROR: cannot distribute relation "sensors_2000" which is partition of "sensors" +DETAIL: Citus does not support distributing partitions if their parent is not distributed table. +HINT: Distribute the partitioned table "sensors" instead. +SELECT create_distributed_table('sensors', NULL, distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE multi_level_partitioning_parent( + measureid integer, + eventdatetime date, + measure_data jsonb) +PARTITION BY RANGE(eventdatetime); +CREATE TABLE multi_level_partitioning_level_1( + measureid integer, + eventdatetime date, + measure_data jsonb) +PARTITION BY RANGE(eventdatetime); +ALTER TABLE multi_level_partitioning_parent ATTACH PARTITION multi_level_partitioning_level_1 +FOR VALUES FROM ('2000-01-01') TO ('2001-01-01'); +CREATE TABLE multi_level_partitioning_level_2 PARTITION OF multi_level_partitioning_level_1 +FOR VALUES FROM ('2000-01-01') TO ('2000-06-06'); +-- multi-level partitioning is not supported +SELECT create_distributed_table('multi_level_partitioning_parent', NULL, distribution_type=>null); +ERROR: distributing multi-level partitioned tables is not supported +DETAIL: Relation "multi_level_partitioning_level_1" is partitioned table itself and it is also partition of relation "multi_level_partitioning_parent". +CREATE FUNCTION normalize_generate_always_as_error(query text) RETURNS void AS $$ +BEGIN + EXECUTE query; + EXCEPTION WHEN OTHERS THEN + IF SQLERRM LIKE 'cannot insert into column %' OR + SQLERRM LIKE 'cannot insert a non-DEFAULT value into column %' + THEN + RAISE 'cannot insert a non-DEFAULT value into column'; + ELSE + RAISE 'unknown error'; + END IF; +END; +$$LANGUAGE plpgsql; +CREATE TABLE identity_test ( + a int GENERATED BY DEFAULT AS IDENTITY (START WITH 10 INCREMENT BY 10), + b bigint GENERATED ALWAYS AS IDENTITY (START WITH 100 INCREMENT BY 100), + c bigint GENERATED BY DEFAULT AS IDENTITY (START WITH 1000 INCREMENT BY 1000) +); +SELECT create_distributed_table('identity_test', NULL, distribution_type=>null); +ERROR: cannot complete operation on create_null_dist_key.identity_test with smallint/int identity column +HINT: Use bigint identity column instead. +DROP TABLE identity_test; +-- Above failed because we don't support using a data type other than BIGINT +-- for identity columns, so drop the table and create a new one with BIGINT +-- identity columns. +CREATE TABLE identity_test ( + a bigint GENERATED BY DEFAULT AS IDENTITY (START WITH 10 INCREMENT BY 10), + b bigint GENERATED ALWAYS AS IDENTITY (START WITH 100 INCREMENT BY 100), + c bigint GENERATED BY DEFAULT AS IDENTITY (START WITH 1000 INCREMENT BY 1000) +); +SELECT create_distributed_table('identity_test', NULL, distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO identity_test (a) VALUES (5); +SELECT normalize_generate_always_as_error($$INSERT INTO identity_test (b) VALUES (5)$$); -- fails due to missing OVERRIDING SYSTEM VALUE +ERROR: cannot insert a non-DEFAULT value into column +CONTEXT: PL/pgSQL function normalize_generate_always_as_error(text) line XX at RAISE +INSERT INTO identity_test (b) OVERRIDING SYSTEM VALUE VALUES (5); +INSERT INTO identity_test (c) VALUES (5); +SELECT result, success FROM run_command_on_workers($$ + INSERT INTO create_null_dist_key.identity_test (a) VALUES (6) +$$); + result | success +--------------------------------------------------------------------- + INSERT 0 1 | t + INSERT 0 1 | t +(2 rows) + +SELECT result, success FROM run_command_on_workers($$ + SELECT create_null_dist_key.normalize_generate_always_as_error('INSERT INTO create_null_dist_key.identity_test (b) VALUES (1)') +$$); + result | success +--------------------------------------------------------------------- + ERROR: cannot insert a non-DEFAULT value into column | f + ERROR: cannot insert a non-DEFAULT value into column | f +(2 rows) + +-- This should fail due to missing OVERRIDING SYSTEM VALUE. +SELECT result, success FROM run_command_on_workers($$ + SELECT create_null_dist_key.normalize_generate_always_as_error('INSERT INTO create_null_dist_key.identity_test (a, b) VALUES (1, 1)') +$$); + result | success +--------------------------------------------------------------------- + ERROR: cannot insert a non-DEFAULT value into column | f + ERROR: cannot insert a non-DEFAULT value into column | f +(2 rows) + +SELECT result, success FROM run_command_on_workers($$ + INSERT INTO create_null_dist_key.identity_test (a, b) OVERRIDING SYSTEM VALUE VALUES (7, 7) +$$); + result | success +--------------------------------------------------------------------- + INSERT 0 1 | t + INSERT 0 1 | t +(2 rows) + +SELECT result, success FROM run_command_on_workers($$ + INSERT INTO create_null_dist_key.identity_test (c, a) OVERRIDING SYSTEM VALUE VALUES (8, 8) +$$); + result | success +--------------------------------------------------------------------- + INSERT 0 1 | t + INSERT 0 1 | t +(2 rows) + +-- test foreign keys +CREATE TABLE referenced_table(a int UNIQUE, b int); +CREATE TABLE referencing_table(a int, b int, + FOREIGN KEY (a) REFERENCES referenced_table(a)); +-- to a colocated null dist key table +BEGIN; + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'referenced_table'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + INSERT INTO referenced_table VALUES (1, 1); + INSERT INTO referencing_table VALUES (1, 2); + -- fails + INSERT INTO referencing_table VALUES (2, 2); +ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730049" +DETAIL: Key (a)=(2) is not present in table "referenced_table_xxxxxxx". +CONTEXT: while executing command on localhost:xxxxx +ROLLBACK; +-- to a non-colocated null dist key table +BEGIN; + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'none'); +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 +ROLLBACK; +-- to a sharded table +BEGIN; + SELECT create_distributed_table('referenced_table', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null); +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 +ROLLBACK; +-- to a reference table +BEGIN; + SELECT create_reference_table('referenced_table'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + INSERT INTO referenced_table VALUES (1, 1); + INSERT INTO referencing_table VALUES (1, 2); + -- fails + INSERT INTO referencing_table VALUES (2, 2); +ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730085" +DETAIL: Key (a)=(2) is not present in table "referenced_table_xxxxxxx". +CONTEXT: while executing command on localhost:xxxxx +ROLLBACK; +-- to a citus local table +BEGIN; + SELECT citus_add_local_table_to_metadata('referenced_table', cascade_via_foreign_keys=>true); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null); +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 +ROLLBACK; +-- to a postgres table +SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null); +ERROR: referenced table "referenced_table" must be a distributed table or a reference table +DETAIL: To enforce foreign keys, the referencing and referenced rows need to be stored on the same node. +HINT: You could use SELECT create_reference_table('referenced_table') to replicate the referenced table to all nodes or consider dropping the foreign key +-- from a reference table +BEGIN; + SELECT create_reference_table('referencing_table'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); +ERROR: cannot create foreign key constraint since foreign keys from reference tables and local tables to distributed tables are not supported +DETAIL: Reference tables and local tables can only have foreign keys to reference tables and local tables +ROLLBACK; +BEGIN; + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + SELECT create_reference_table('referencing_table'); +ERROR: cannot create foreign key constraint since foreign keys from reference tables and local tables to distributed tables are not supported +DETAIL: Reference tables and local tables can only have foreign keys to reference tables and local tables +ROLLBACK; +-- from a sharded table +BEGIN; + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + SELECT create_distributed_table('referencing_table', 'a'); +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 +ROLLBACK; +-- from a citus local table +BEGIN; + SELECT citus_add_local_table_to_metadata('referencing_table', cascade_via_foreign_keys=>true); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); +ERROR: cannot create foreign key constraint since foreign keys from reference tables and local tables to distributed tables are not supported +DETAIL: Reference tables and local tables can only have foreign keys to reference tables and local tables +ROLLBACK; +BEGIN; + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + SELECT citus_add_local_table_to_metadata('referencing_table', cascade_via_foreign_keys=>true); +ERROR: cannot create foreign key constraint since foreign keys from reference tables and local tables to distributed tables are not supported +DETAIL: Reference tables and local tables can only have foreign keys to reference tables and local tables +ROLLBACK; +-- from a postgres table (only useful to preserve legacy behavior) +BEGIN; + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +ROLLBACK; +-- make sure that we enforce the foreign key constraint when inserting from workers too +SELECT create_reference_table('referenced_table'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO referenced_table VALUES (1, 1); +-- ok +SELECT result, success FROM run_command_on_workers($$ + INSERT INTO create_null_dist_key.referencing_table VALUES (1, 2) +$$); + result | success +--------------------------------------------------------------------- + INSERT 0 1 | t + INSERT 0 1 | t +(2 rows) + +-- fails +SELECT result, success FROM run_command_on_workers($$ + INSERT INTO create_null_dist_key.referencing_table VALUES (2, 2) +$$); + result | success +--------------------------------------------------------------------- + ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730102" | f + ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730102" | f +(2 rows) + +DROP TABLE referencing_table, referenced_table; +CREATE TABLE self_fkey_test(a int UNIQUE, b int, + FOREIGN KEY (b) REFERENCES self_fkey_test(a), + FOREIGN KEY (a) REFERENCES self_fkey_test(a)); +SELECT create_distributed_table('self_fkey_test', NULL, distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO self_fkey_test VALUES (1, 1); -- ok +INSERT INTO self_fkey_test VALUES (2, 3); -- fails +ERROR: insert or update on table "self_fkey_test_1730103" violates foreign key constraint "self_fkey_test_b_fkey_1730103" +DETAIL: Key (b)=(3) is not present in table "self_fkey_test_1730103". +CONTEXT: while executing command on localhost:xxxxx +-- similar foreign key tests but this time create the referencing table later on +-- referencing table is a null shard key table +-- to a colocated null dist key table +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a) REFERENCES referenced_table(a)); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'referenced_table'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + INSERT INTO referenced_table VALUES (1, 1); + INSERT INTO referencing_table VALUES (1, 2); + -- fails + INSERT INTO referencing_table VALUES (2, 2); +ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730105" +DETAIL: Key (a)=(2) is not present in table "referenced_table_xxxxxxx". +CONTEXT: while executing command on localhost:xxxxx +ROLLBACK; +BEGIN; + CREATE TABLE referenced_table(a int, b int, UNIQUE(b, a)); + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a, b) REFERENCES referenced_table(b, a)); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'referenced_table'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + INSERT INTO referenced_table VALUES (1, 2); + INSERT INTO referencing_table VALUES (2, 1); + -- fails + INSERT INTO referencing_table VALUES (1, 2); +ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_b_fkey_1730107" +DETAIL: Key (a, b)=(1, 2) is not present in table "referenced_table_xxxxxxx". +CONTEXT: while executing command on localhost:xxxxx +ROLLBACK; +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a) REFERENCES referenced_table(a) ON UPDATE SET NULL); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'referenced_table'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + INSERT INTO referenced_table VALUES (1, 1); + INSERT INTO referencing_table VALUES (1, 2); + UPDATE referenced_table SET a = 5; + SELECT * FROM referencing_table; + a | b +--------------------------------------------------------------------- + | 2 +(1 row) + +ROLLBACK; +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + CREATE TABLE referencing_table(a serial, b int, FOREIGN KEY (a) REFERENCES referenced_table(a) ON UPDATE SET DEFAULT); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'referenced_table'); +ERROR: cannot create foreign key constraint since Citus does not support ON DELETE / UPDATE SET DEFAULT actions on the columns that default to sequences +ROLLBACK; +-- to a non-colocated null dist key table +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a) REFERENCES referenced_table(a)); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'none'); +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 +ROLLBACK; +-- to a sharded table +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT create_distributed_table('referenced_table', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a) REFERENCES referenced_table(a)); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'none'); +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 +ROLLBACK; +-- to a reference table +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT create_reference_table('referenced_table'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + + CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a) REFERENCES referenced_table(a)); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + INSERT INTO referenced_table VALUES (1, 1); + INSERT INTO referencing_table VALUES (1, 2); + -- fails + INSERT INTO referencing_table VALUES (2, 2); +ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730146" +DETAIL: Key (a)=(2) is not present in table "referenced_table_xxxxxxx". +CONTEXT: while executing command on localhost:xxxxx +ROLLBACK; +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT create_reference_table('referenced_table'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + + CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a) REFERENCES referenced_table(a) ON DELETE CASCADE); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + INSERT INTO referenced_table VALUES (1, 1); + INSERT INTO referencing_table VALUES (1, 2); + DELETE FROM referenced_table CASCADE; + SELECT * FROM referencing_table; + a | b +--------------------------------------------------------------------- +(0 rows) + +ROLLBACK; +-- to a citus local table +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT citus_add_local_table_to_metadata('referenced_table'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + + CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a) REFERENCES referenced_table(a)); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'none'); +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 +ROLLBACK; +-- to a postgres table +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a) REFERENCES referenced_table(a)); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'none'); +ERROR: referenced table "referenced_table" must be a distributed table or a reference table +DETAIL: To enforce foreign keys, the referencing and referenced rows need to be stored on the same node. +HINT: You could use SELECT create_reference_table('referenced_table') to replicate the referenced table to all nodes or consider dropping the foreign key +ROLLBACK; +-- referenced table is a null shard key table +-- from a sharded table +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a) REFERENCES referenced_table(a)); + SELECT create_distributed_table('referencing_table', 'a'); +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 +ROLLBACK; +-- from a reference table +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a) REFERENCES referenced_table(a)); + SELECT create_reference_table('referencing_table'); +ERROR: cannot create foreign key constraint since foreign keys from reference tables and local tables to distributed tables are not supported +DETAIL: Reference tables and local tables can only have foreign keys to reference tables and local tables +ROLLBACK; +-- from a citus local table +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a) REFERENCES referenced_table(a)); + SELECT citus_add_local_table_to_metadata('referencing_table', cascade_via_foreign_keys=>true); +ERROR: cannot create foreign key constraint since foreign keys from reference tables and local tables to distributed tables are not supported +DETAIL: Reference tables and local tables can only have foreign keys to reference tables and local tables +ROLLBACK; +-- from a postgres table (only useful to preserve legacy behavior) +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a) REFERENCES referenced_table(a)); +ROLLBACK; +-- Test whether we switch to sequential execution to enforce foreign +-- key restrictions. +CREATE TABLE referenced_table(id int PRIMARY KEY, value_1 int); +SELECT create_reference_table('referenced_table'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE referencing_table(id int PRIMARY KEY, value_1 int, CONSTRAINT fkey FOREIGN KEY(value_1) REFERENCES referenced_table(id) ON UPDATE CASCADE); +SELECT create_distributed_table('referencing_table', null, colocate_with=>'none', distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO DEBUG1; +BEGIN; + -- Switches to sequential execution because referenced_table is a reference table + -- and referenced by a null-shard-key distributed table. + -- + -- Given that we cannot do parallel access on null-shard-key, this is not useful. + -- However, this is already what we're doing for, e.g., a foreign key from a + -- reference table to another reference table. + TRUNCATE referenced_table CASCADE; +DEBUG: switching to sequential query execution mode +DETAIL: Table "referenced_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode +NOTICE: truncate cascades to table "referencing_table" +DEBUG: truncate cascades to table "referencing_table_xxxxxxx" +DETAIL: from localhost:xxxxx + SELECT COUNT(*) FROM referencing_table; + count +--------------------------------------------------------------------- + 0 +(1 row) + +COMMIT; +BEGIN; + SELECT COUNT(*) FROM referencing_table; + count +--------------------------------------------------------------------- + 0 +(1 row) + + -- Doesn't fail because the SELECT didn't perform parallel execution. + TRUNCATE referenced_table CASCADE; +DEBUG: switching to sequential query execution mode +DETAIL: Table "referenced_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode +NOTICE: truncate cascades to table "referencing_table" +DEBUG: truncate cascades to table "referencing_table_xxxxxxx" +DETAIL: from localhost:xxxxx +COMMIT; +BEGIN; + UPDATE referencing_table SET value_1 = 15; + -- Doesn't fail because the UPDATE didn't perform parallel execution. + TRUNCATE referenced_table CASCADE; +DEBUG: switching to sequential query execution mode +DETAIL: Table "referenced_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode +NOTICE: truncate cascades to table "referencing_table" +DEBUG: truncate cascades to table "referencing_table_xxxxxxx" +DETAIL: from localhost:xxxxx +COMMIT; +BEGIN; + SELECT COUNT(*) FROM referenced_table; + count +--------------------------------------------------------------------- + 0 +(1 row) + + -- doesn't switch to sequential execution + ALTER TABLE referencing_table ADD COLUMN X INT; +ROLLBACK; +BEGIN; + -- Switches to sequential execution because referenced_table is a reference table + -- and referenced by a null-shard-key distributed table. + -- + -- Given that we cannot do parallel access on null-shard-key, this is not useful. + -- However, this is already what we're doing for, e.g., a foreign key from a + -- reference table to another reference table. + UPDATE referenced_table SET id = 101 WHERE id = 99; +DEBUG: switching to sequential query execution mode +DETAIL: Table "referenced_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode + UPDATE referencing_table SET value_1 = 15; +ROLLBACK; +BEGIN; + UPDATE referencing_table SET value_1 = 15; + -- Doesn't fail because prior UPDATE didn't perform parallel execution. + UPDATE referenced_table SET id = 101 WHERE id = 99; +DEBUG: switching to sequential query execution mode +DETAIL: Table "referenced_table" is modified, which might lead to data inconsistencies or distributed deadlocks via parallel accesses to hash distributed tables due to foreign keys. Any parallel modification to those hash distributed tables in the same transaction can only be executed in sequential query execution mode +ROLLBACK; +SET client_min_messages TO WARNING; +DROP TABLE referenced_table, referencing_table; +-- Test whether we unnecessarily switch to sequential execution +-- when the referenced relation is a null-shard-key table. +CREATE TABLE referenced_table(id int PRIMARY KEY, value_1 int); +SELECT create_distributed_table('referenced_table', null, colocate_with=>'none', distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE referencing_table(id int PRIMARY KEY, value_1 int, CONSTRAINT fkey FOREIGN KEY(value_1) REFERENCES referenced_table(id) ON UPDATE CASCADE); +SELECT create_distributed_table('referencing_table', null, colocate_with=>'referenced_table', distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO DEBUG1; +BEGIN; + SELECT COUNT(*) FROM referenced_table; + count +--------------------------------------------------------------------- + 0 +(1 row) + + -- Doesn't switch to sequential execution because the referenced_table is + -- a null-shard-key distributed table. + ALTER TABLE referencing_table ADD COLUMN X INT; +ROLLBACK; +BEGIN; + -- Doesn't switch to sequential execution because the referenced_table is + -- a null-shard-key distributed table. + TRUNCATE referenced_table CASCADE; +NOTICE: truncate cascades to table "referencing_table" +DEBUG: truncate cascades to table "referencing_table_xxxxxxx" +DETAIL: from localhost:xxxxx + SELECT COUNT(*) FROM referencing_table; + count +--------------------------------------------------------------------- + 0 +(1 row) + +COMMIT; +SET client_min_messages TO WARNING; +CREATE FUNCTION increment_value() RETURNS trigger AS $increment_value$ +BEGIN + NEW.value := NEW.value+1; + RETURN NEW; +END; +$increment_value$ LANGUAGE plpgsql; +CREATE TABLE trigger_table_1 (value int); +CREATE TRIGGER trigger_1 +BEFORE INSERT ON trigger_table_1 +FOR EACH ROW EXECUTE FUNCTION increment_value(); +SELECT create_distributed_table('trigger_table_1', NULL, distribution_type=>null); +ERROR: cannot distribute relation "trigger_table_1" because it has triggers +HINT: Consider dropping all the triggers on "trigger_table_1" and retry. +SET citus.enable_unsafe_triggers TO ON; +SELECT create_distributed_table('trigger_table_1', NULL, distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO trigger_table_1 VALUES(1), (2); +SELECT * FROM trigger_table_1 ORDER BY value; + value +--------------------------------------------------------------------- + 2 + 3 +(2 rows) + +CREATE FUNCTION insert_some() RETURNS trigger AS $insert_some$ +BEGIN + RAISE NOTICE 'inserted some rows'; + RETURN NEW; +END; +$insert_some$ LANGUAGE plpgsql; +CREATE TABLE trigger_table_2 (value int); +CREATE TRIGGER trigger_2 +AFTER INSERT ON trigger_table_2 +FOR EACH STATEMENT EXECUTE FUNCTION insert_some(); +ALTER TABLE trigger_table_2 DISABLE TRIGGER trigger_2; +SELECT create_distributed_table('trigger_table_2', NULL, distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SET client_min_messages TO NOTICE; +INSERT INTO trigger_table_2 VALUES(3), (4); +SET client_min_messages TO WARNING; +SELECT * FROM trigger_table_2 ORDER BY value; + value +--------------------------------------------------------------------- + 3 + 4 +(2 rows) + +CREATE FUNCTION combine_old_new_val() RETURNS trigger AS $combine_old_new_val$ +BEGIN + NEW.value = NEW.value * 10 + OLD.value; + RETURN NEW; +END; +$combine_old_new_val$ LANGUAGE plpgsql; +CREATE FUNCTION notice_truncate() RETURNS trigger AS $notice_truncate$ +BEGIN + RAISE NOTICE 'notice_truncate()'; + RETURN NEW; +END; +$notice_truncate$ LANGUAGE plpgsql; +CREATE TABLE trigger_table_3 (value int); +CREATE TRIGGER trigger_3 +BEFORE UPDATE ON trigger_table_3 +FOR EACH ROW EXECUTE FUNCTION combine_old_new_val(); +CREATE TRIGGER trigger_4 +AFTER TRUNCATE ON trigger_table_3 +FOR EACH STATEMENT EXECUTE FUNCTION notice_truncate(); +INSERT INTO trigger_table_3 VALUES(3), (4); +SELECT create_distributed_table('trigger_table_3', NULL, distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +UPDATE trigger_table_3 SET value = 5; +SELECT * FROM trigger_table_3 ORDER BY value; + value +--------------------------------------------------------------------- + 53 + 54 +(2 rows) + +SET client_min_messages TO NOTICE; +TRUNCATE trigger_table_3; +NOTICE: notice_truncate() +CONTEXT: PL/pgSQL function notice_truncate() line XX at RAISE +SET client_min_messages TO WARNING; +-- try a few simple queries at least to make sure that we don't crash +BEGIN; + INSERT INTO nullkey_c1_t1 SELECT * FROM nullkey_c2_t1; +ROLLBACK; +-- cleanup at exit +SET client_min_messages TO ERROR; +DROP SCHEMA create_null_dist_key, "NULL_!_dist_key" CASCADE; diff --git a/src/test/regress/expected/multi_colocation_utils.out b/src/test/regress/expected/multi_colocation_utils.out index 93596a1b4..219327dc1 100644 --- a/src/test/regress/expected/multi_colocation_utils.out +++ b/src/test/regress/expected/multi_colocation_utils.out @@ -612,10 +612,10 @@ CREATE TABLE table_postgresql( id int ); CREATE TABLE table_failing ( id int ); SELECT create_distributed_table('table_failing', 'id', colocate_with => 'table_append'); ERROR: cannot distribute relation -DETAIL: Currently, colocate_with option is only supported for hash distributed tables. +DETAIL: Currently, colocate_with option is not supported with append / range distributed tables and local tables added to metadata. SELECT create_distributed_table('table_failing', 'id', 'append', 'table1_groupE'); ERROR: cannot distribute relation -DETAIL: Currently, colocate_with option is only supported for hash distributed tables. +DETAIL: Currently, colocate_with option is not supported for append / range distributed tables. SELECT create_distributed_table('table_failing', 'id', colocate_with => 'table_postgresql'); ERROR: relation table_postgresql is not distributed SELECT create_distributed_table('table_failing', 'id', colocate_with => 'no_table'); diff --git a/src/test/regress/expected/multi_extension.out b/src/test/regress/expected/multi_extension.out index 0dd52e1ca..56de93802 100644 --- a/src/test/regress/expected/multi_extension.out +++ b/src/test/regress/expected/multi_extension.out @@ -1735,6 +1735,33 @@ HINT: To remove the local data, run: SELECT truncate_local_data_after_distribut DROP TABLE test; TRUNCATE pg_dist_node; +-- confirm that we can create a null shard key table on an empty node +CREATE TABLE test (x int, y int); +INSERT INTO test VALUES (1,2); +SET citus.shard_replication_factor TO 1; +SELECT create_distributed_table('test', null, colocate_with=>'none', distribution_type=>null); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$public.test$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- and make sure that we can't remove the coordinator due to "test" +SELECT citus_remove_node('localhost', :master_port); +ERROR: cannot remove or disable the node localhost:xxxxx because because it contains the only shard placement for shard xxxxx +DETAIL: One of the table(s) that prevents the operation complete successfully is public.test +HINT: To proceed, either drop the tables or use undistribute_table() function to convert them to local tables +DROP TABLE test; +-- and now we should be able to remove the coordinator +SELECT citus_remove_node('localhost', :master_port); + citus_remove_node +--------------------------------------------------------------------- + +(1 row) + -- confirm that we can create a reference table on an empty node CREATE TABLE test (x int, y int); INSERT INTO test VALUES (1,2); diff --git a/src/test/regress/expected/single_node.out b/src/test/regress/expected/single_node.out index 7f152decd..ecb652931 100644 --- a/src/test/regress/expected/single_node.out +++ b/src/test/regress/expected/single_node.out @@ -101,8 +101,63 @@ SELECT pg_reload_conf(); t (1 row) +CREATE TABLE single_node_nullkey_c1(a int, b int); +SELECT create_distributed_table('single_node_nullkey_c1', null, colocate_with=>'none', distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE single_node_nullkey_c2(a int, b int); +SELECT create_distributed_table('single_node_nullkey_c2', null, colocate_with=>'none', distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- created on different colocation groups .. +SELECT +( + SELECT colocationid FROM pg_dist_partition + WHERE logicalrelid = 'single_node.single_node_nullkey_c1'::regclass +) +!= +( + SELECT colocationid FROM pg_dist_partition + WHERE logicalrelid = 'single_node.single_node_nullkey_c2'::regclass +); + ?column? +--------------------------------------------------------------------- + t +(1 row) + +-- .. but both are associated to coordinator +SELECT groupid = 0 FROM pg_dist_placement +WHERE shardid = ( + SELECT shardid FROM pg_dist_shard + WHERE logicalrelid = 'single_node.single_node_nullkey_c1'::regclass +); + ?column? +--------------------------------------------------------------------- + t +(1 row) + +SELECT groupid = 0 FROM pg_dist_placement +WHERE shardid = ( + SELECT shardid FROM pg_dist_shard + WHERE logicalrelid = 'single_node.single_node_nullkey_c2'::regclass +); + ?column? +--------------------------------------------------------------------- + t +(1 row) + +-- try creating a null-shard-key distributed table from a shard relation +SELECT shardid AS round_robin_test_c1_shard_id FROM pg_dist_shard WHERE logicalrelid = 'single_node.single_node_nullkey_c1'::regclass \gset +SELECT create_distributed_table('single_node_nullkey_c1_' || :round_robin_test_c1_shard_id , null, colocate_with=>'none', distribution_type=>null); +ERROR: relation "single_node_nullkey_c1_90630532" is a shard relation SET client_min_messages TO WARNING; -DROP TABLE failover_to_local; +DROP TABLE failover_to_local, single_node_nullkey_c1, single_node_nullkey_c2; RESET client_min_messages; -- so that we don't have to update rest of the test output SET citus.next_shard_id TO 90630500; diff --git a/src/test/regress/expected/single_node_0.out b/src/test/regress/expected/single_node_0.out index a21cdd28a..86b4982e6 100644 --- a/src/test/regress/expected/single_node_0.out +++ b/src/test/regress/expected/single_node_0.out @@ -101,8 +101,63 @@ SELECT pg_reload_conf(); t (1 row) +CREATE TABLE single_node_nullkey_c1(a int, b int); +SELECT create_distributed_table('single_node_nullkey_c1', null, colocate_with=>'none', distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE single_node_nullkey_c2(a int, b int); +SELECT create_distributed_table('single_node_nullkey_c2', null, colocate_with=>'none', distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- created on different colocation groups .. +SELECT +( + SELECT colocationid FROM pg_dist_partition + WHERE logicalrelid = 'single_node.single_node_nullkey_c1'::regclass +) +!= +( + SELECT colocationid FROM pg_dist_partition + WHERE logicalrelid = 'single_node.single_node_nullkey_c2'::regclass +); + ?column? +--------------------------------------------------------------------- + t +(1 row) + +-- .. but both are associated to coordinator +SELECT groupid = 0 FROM pg_dist_placement +WHERE shardid = ( + SELECT shardid FROM pg_dist_shard + WHERE logicalrelid = 'single_node.single_node_nullkey_c1'::regclass +); + ?column? +--------------------------------------------------------------------- + t +(1 row) + +SELECT groupid = 0 FROM pg_dist_placement +WHERE shardid = ( + SELECT shardid FROM pg_dist_shard + WHERE logicalrelid = 'single_node.single_node_nullkey_c2'::regclass +); + ?column? +--------------------------------------------------------------------- + t +(1 row) + +-- try creating a null-shard-key distributed table from a shard relation +SELECT shardid AS round_robin_test_c1_shard_id FROM pg_dist_shard WHERE logicalrelid = 'single_node.single_node_nullkey_c1'::regclass \gset +SELECT create_distributed_table('single_node_nullkey_c1_' || :round_robin_test_c1_shard_id , null, colocate_with=>'none', distribution_type=>null); +ERROR: relation "single_node_nullkey_c1_90630532" is a shard relation SET client_min_messages TO WARNING; -DROP TABLE failover_to_local; +DROP TABLE failover_to_local, single_node_nullkey_c1, single_node_nullkey_c2; RESET client_min_messages; -- so that we don't have to update rest of the test output SET citus.next_shard_id TO 90630500; diff --git a/src/test/regress/multi_1_schedule b/src/test/regress/multi_1_schedule index cefb1777f..a673a71d0 100644 --- a/src/test/regress/multi_1_schedule +++ b/src/test/regress/multi_1_schedule @@ -32,6 +32,7 @@ test: escape_extension_name test: ref_citus_local_fkeys test: alter_database_owner test: distributed_triggers +test: create_null_dist_key test: multi_test_catalog_views test: multi_table_ddl diff --git a/src/test/regress/sql/create_null_dist_key.sql b/src/test/regress/sql/create_null_dist_key.sql new file mode 100644 index 000000000..b03cdde4d --- /dev/null +++ b/src/test/regress/sql/create_null_dist_key.sql @@ -0,0 +1,962 @@ +CREATE SCHEMA create_null_dist_key; +SET search_path TO create_null_dist_key; + +SET citus.next_shard_id TO 1720000; +SET citus.shard_count TO 32; +SET citus.shard_replication_factor TO 1; + +SELECT 1 FROM citus_remove_node('localhost', :worker_1_port); +SELECT 1 FROM citus_remove_node('localhost', :worker_2_port); + +CREATE TABLE add_node_test(a int, "b" text); + +-- add a node before creating the null-shard-key table +SELECT 1 FROM citus_add_node('localhost', :worker_1_port); + +SELECT create_distributed_table('add_node_test', null, colocate_with=>'none', distribution_type=>null); + +-- add a node after creating the null-shard-key table +SELECT 1 FROM citus_add_node('localhost', :worker_2_port); + +-- make sure that table is created on the worker nodes added before/after create_distributed_table +SELECT result FROM run_command_on_workers($$ + SELECT COUNT(*)=1 FROM pg_class WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + relname='add_node_test' +$$); + +-- and check the metadata tables + +SELECT result FROM run_command_on_workers($$ + SELECT (partmethod, partkey, repmodel, autoconverted) FROM pg_dist_partition + WHERE logicalrelid = 'create_null_dist_key.add_node_test'::regclass +$$); + +SELECT result FROM run_command_on_workers($$ + SELECT (shardstorage, shardminvalue, shardmaxvalue) FROM pg_dist_shard + WHERE logicalrelid = 'create_null_dist_key.add_node_test'::regclass +$$); + +SELECT result FROM run_command_on_workers($$ + SELECT COUNT(*)=1 FROM pg_dist_placement + WHERE shardid = ( + SELECT shardid FROM pg_dist_shard + WHERE logicalrelid = 'create_null_dist_key.add_node_test'::regclass + ); +$$); + +SELECT result FROM run_command_on_workers($$ + SELECT (shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation) FROM pg_dist_colocation + WHERE colocationid = ( + SELECT colocationid FROM pg_dist_partition + WHERE logicalrelid = 'create_null_dist_key.add_node_test'::regclass + ); +$$); + +SET client_min_messages TO WARNING; +SELECT 1 FROM citus_add_node('localhost', :master_port, groupid => 0); + +SET client_min_messages TO NOTICE; + +CREATE TABLE invalid_configs_1(a int primary key); +SELECT create_distributed_table('invalid_configs_1', null, shard_count=>2); +SELECT create_distributed_table('invalid_configs_1', null, shard_count=>1); + +CREATE TABLE nullkey_c1_t1(a int, b int); +CREATE TABLE nullkey_c1_t2(a int, b int); +CREATE TABLE nullkey_c1_t3(a int, b int); +SELECT create_distributed_table('nullkey_c1_t1', null, colocate_with=>'none'); + +SELECT colocationid AS nullkey_c1_t1_colocation_id FROM pg_dist_partition WHERE logicalrelid = 'create_null_dist_key.nullkey_c1_t1'::regclass \gset + +BEGIN; + DROP TABLE nullkey_c1_t1; + -- make sure that we delete the colocation group after dropping the last table that belongs to it + SELECT COUNT(*)=0 FROM pg_dist_colocation WHERE colocationid = :'nullkey_c1_t1_colocation_id'; +ROLLBACK; + +SELECT create_distributed_table('nullkey_c1_t2', null, colocate_with=>'nullkey_c1_t1'); +SELECT create_distributed_table('nullkey_c1_t3', null, colocate_with=>'nullkey_c1_t1', distribution_type=>'append'); +SELECT create_distributed_table('nullkey_c1_t3', null, colocate_with=>'nullkey_c1_t1'); + +CREATE TABLE nullkey_c2_t1(a int, b int); +CREATE TABLE nullkey_c2_t2(a int, b int); +CREATE TABLE nullkey_c2_t3(a int, b int); + +-- create_distributed_table_concurrently is not yet supported yet +SELECT create_distributed_table_concurrently('nullkey_c2_t1', null); +SELECT create_distributed_table_concurrently('nullkey_c2_t1', null, colocate_with=>'none'); + +SELECT create_distributed_table('nullkey_c2_t1', null, colocate_with=>'none'); +SELECT create_distributed_table('nullkey_c2_t2', null, colocate_with=>'nullkey_c2_t1', distribution_type=>'hash'); -- distribution_type is ignored anyway +SELECT create_distributed_table('nullkey_c2_t3', null, colocate_with=>'nullkey_c2_t2', distribution_type=>null); + +-- check the metadata for the colocated tables whose names start with nullkey_c1_ + +SELECT logicalrelid, partmethod, partkey, repmodel, autoconverted FROM pg_dist_partition +WHERE logicalrelid IN ( + SELECT oid FROM pg_class + WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + relname LIKE 'nullkey_c1_%' +) +ORDER BY 1; + +-- make sure that all those 3 tables belong to same colocation group +SELECT COUNT(*) FROM pg_dist_partition +WHERE logicalrelid IN ( + SELECT oid FROM pg_class + WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + relname LIKE 'nullkey_c1_%' +) +GROUP BY colocationid; + +SELECT logicalrelid, shardstorage, shardminvalue, shardmaxvalue FROM pg_dist_shard +WHERE logicalrelid IN ( + SELECT oid FROM pg_class + WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + relname LIKE 'nullkey_c1_%' +) +ORDER BY 1; + +-- make sure that all those 3 shards are created on the same node group +SELECT COUNT(*) FROM pg_dist_placement +WHERE shardid IN ( + SELECT shardid FROM pg_dist_shard + WHERE logicalrelid IN ( + SELECT oid FROM pg_class + WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + relname LIKE 'nullkey_c1_%' + ) +) +GROUP BY groupid; + +-- check the metadata for the colocated tables whose names start with nullkey_c2_ + +SELECT logicalrelid, partmethod, partkey, repmodel, autoconverted FROM pg_dist_partition +WHERE logicalrelid IN ( + SELECT oid FROM pg_class + WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + relname LIKE 'nullkey_c2_%' +) +ORDER BY 1; + +-- make sure that all those 3 tables belong to same colocation group +SELECT COUNT(*) FROM pg_dist_partition +WHERE logicalrelid IN ( + SELECT oid FROM pg_class + WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + relname LIKE 'nullkey_c2_%' +) +GROUP BY colocationid; + +SELECT logicalrelid, shardstorage, shardminvalue, shardmaxvalue FROM pg_dist_shard +WHERE logicalrelid IN ( + SELECT oid FROM pg_class + WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + relname LIKE 'nullkey_c2_%' +) +ORDER BY 1; + +-- make sure that all those 3 shards created on the same node group +SELECT COUNT(*) FROM pg_dist_placement +WHERE shardid IN ( + SELECT shardid FROM pg_dist_shard + WHERE logicalrelid IN ( + SELECT oid FROM pg_class + WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + relname LIKE 'nullkey_c2_%' + ) +) +GROUP BY groupid; + +-- Make sure that the colocated tables whose names start with nullkey_c1_ +-- belong to a different colocation group than the ones whose names start +-- with nullkey_c2_. +-- +-- It's ok to only compare nullkey_c1_t1 and nullkey_c2_t1 because we already +-- verified that null_dist_key.nullkey_c1_t1 is colocated with the other two +-- and null_dist_key.nullkey_c2_t1 is colocated with the other two. +SELECT +( + SELECT colocationid FROM pg_dist_partition + WHERE logicalrelid = 'create_null_dist_key.nullkey_c1_t1'::regclass +) +!= +( + SELECT colocationid FROM pg_dist_partition + WHERE logicalrelid = 'create_null_dist_key.nullkey_c2_t1'::regclass +); + +-- Since we determine node for the placement based on the module of colocation id, +-- we don't expect those two colocation groups to get assigned to same node. +SELECT +( + SELECT groupid FROM pg_dist_placement + WHERE shardid = ( + SELECT shardid FROM pg_dist_shard + WHERE logicalrelid = 'create_null_dist_key.nullkey_c1_t1'::regclass + ) +) +!= +( + SELECT groupid FROM pg_dist_placement + WHERE shardid = ( + SELECT shardid FROM pg_dist_shard + WHERE logicalrelid = 'create_null_dist_key.nullkey_c2_t1'::regclass + ) +); + +-- It's ok to only check nullkey_c1_t1 and nullkey_c2_t1 because we already +-- verified that null_dist_key.nullkey_c1_t1 is colocated with the other two +-- and null_dist_key.nullkey_c2_t1 is colocated with the other two. +SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation +WHERE colocationid = ( + SELECT colocationid FROM pg_dist_partition + WHERE logicalrelid = 'create_null_dist_key.nullkey_c1_t1'::regclass +); + +SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation +WHERE colocationid = ( + SELECT colocationid FROM pg_dist_partition + WHERE logicalrelid = 'create_null_dist_key.nullkey_c2_t1'::regclass +); + +CREATE TABLE round_robin_test_c1(a int, b int); +SELECT create_distributed_table('round_robin_test_c1', null, colocate_with=>'none', distribution_type=>null); + +\c - - - :master_port +SET search_path TO create_null_dist_key; +SET citus.next_shard_id TO 1730000; +SET citus.shard_count TO 32; +SET citus.shard_replication_factor TO 1; +SET client_min_messages TO NOTICE; + +CREATE TABLE round_robin_test_c2(a int, b int); +SELECT create_distributed_table('round_robin_test_c2', null, colocate_with=>'none', distribution_type=>null); + +-- Since we determine node for the placement based on the module of colocation id, +-- we don't expect those two colocation groups to get assigned to same node even +-- after reconnecting to the coordinator. +SELECT +( + SELECT groupid FROM pg_dist_placement + WHERE shardid = ( + SELECT shardid FROM pg_dist_shard + WHERE logicalrelid = 'create_null_dist_key.round_robin_test_c1'::regclass + ) +) +!= +( + SELECT groupid FROM pg_dist_placement + WHERE shardid = ( + SELECT shardid FROM pg_dist_shard + WHERE logicalrelid = 'create_null_dist_key.round_robin_test_c2'::regclass + ) +); + +CREATE TABLE distributed_table(a int, b int); + +-- cannot colocate a sharded table with null shard key table +SELECT create_distributed_table('distributed_table', 'a', colocate_with=>'nullkey_c1_t1'); + +CREATE TABLE reference_table(a int, b int); +CREATE TABLE local(a int, b int); +SELECT create_reference_table('reference_table'); +SELECT create_distributed_table('distributed_table', 'a'); + +-- cannot colocate null shard key tables with other table types +CREATE TABLE cannot_colocate_with_other_types (a int, b int); +SELECT create_distributed_table('cannot_colocate_with_other_types', null, colocate_with=>'reference_table'); +SELECT create_distributed_table('cannot_colocate_with_other_types', null, colocate_with=>'distributed_table'); +SELECT create_distributed_table('cannot_colocate_with_other_types', null, colocate_with=>'local'); -- postgres local + +SELECT citus_add_local_table_to_metadata('local'); + +-- cannot colocate null shard key tables with citus local tables +SELECT create_distributed_table('cannot_colocate_with_other_types', null, colocate_with=>'local'); -- citus local + +SET client_min_messages TO WARNING; + +-- can't create such a distributed table from another Citus table, except Citus local tables +SELECT create_distributed_table('reference_table', null, colocate_with=>'none'); +SELECT create_distributed_table('distributed_table', null, colocate_with=>'none'); +SELECT create_distributed_table('local', null, colocate_with=>'none'); + +BEGIN; + -- creating a null-shard-key table from a temporary table is not supported + CREATE TEMPORARY TABLE temp_table (a int); + SELECT create_distributed_table('temp_table', null, colocate_with=>'none', distribution_type=>null); +ROLLBACK; + +-- creating a null-shard-key table from a catalog table is not supported +SELECT create_distributed_table('pg_catalog.pg_index', NULL, distribution_type=>null); + +-- creating a null-shard-key table from an unlogged table is supported +CREATE UNLOGGED TABLE unlogged_table (a int); +SELECT create_distributed_table('unlogged_table', null, colocate_with=>'none', distribution_type=>null); + +-- creating a null-shard-key table from a foreign table is not supported +CREATE FOREIGN TABLE foreign_table ( + id bigint not null, + full_name text not null default '' +) SERVER fake_fdw_server OPTIONS (encoding 'utf-8', compression 'true', table_name 'foreign_table'); +SELECT create_distributed_table('foreign_table', null, colocate_with=>'none', distribution_type=>null); + +-- create a null dist key table that has no tuples +CREATE TABLE null_dist_key_table_1 (a int primary key); +SELECT create_distributed_table('null_dist_key_table_1', null, colocate_with=>'none'); + +-- create a null dist key table that has some tuples +CREATE TABLE null_dist_key_table_2(a int primary key); +INSERT INTO null_dist_key_table_2 VALUES(1); +SELECT create_distributed_table('null_dist_key_table_2', null, colocate_with=>'none'); + +SELECT * FROM null_dist_key_table_2 ORDER BY a; + +DROP TABLE null_dist_key_table_1, null_dist_key_table_2; + +-- create indexes before creating the null dist key tables + +-- .. for an initially empty table +CREATE TABLE null_dist_key_table_1(a int); +CREATE INDEX null_dist_key_table_1_idx ON null_dist_key_table_1(a); +SELECT create_distributed_table('null_dist_key_table_1', null, colocate_with=>'none'); + +-- .. and for another table having data in it before creating null dist key table +CREATE TABLE null_dist_key_table_2(a int); +INSERT INTO null_dist_key_table_2 VALUES(1); +CREATE INDEX null_dist_key_table_2_idx ON null_dist_key_table_2(a); +SELECT create_distributed_table('null_dist_key_table_2', null, colocate_with=>'none'); + +SELECT * FROM null_dist_key_table_2 ORDER BY a; + +-- show that we do not support inheritance relationships +CREATE TABLE parent_table (a int, b text); +CREATE TABLE child_table () INHERITS (parent_table); + +-- both of below should error out +SELECT create_distributed_table('parent_table', null, colocate_with=>'none'); +SELECT create_distributed_table('child_table', null, colocate_with=>'none'); + +-- show that we support policies +BEGIN; + CREATE TABLE null_dist_key_table_3 (table_user text); + + ALTER TABLE null_dist_key_table_3 ENABLE ROW LEVEL SECURITY; + + CREATE ROLE table_users; + CREATE POLICY table_policy ON null_dist_key_table_3 TO table_users + USING (table_user = current_user); + + SELECT create_distributed_table('null_dist_key_table_3', null, colocate_with=>'none'); +ROLLBACK; + +-- drop them for next tests +DROP TABLE null_dist_key_table_1, null_dist_key_table_2, distributed_table; + +-- tests for object names that should be escaped properly + +CREATE SCHEMA "NULL_!_dist_key"; + +CREATE TABLE "NULL_!_dist_key"."my_TABLE.1!?!"(id int, "Second_Id" int); +SELECT create_distributed_table('"NULL_!_dist_key"."my_TABLE.1!?!"', null, colocate_with=>'none'); + +-- drop the table before creating it when the search path is set +SET search_path to "NULL_!_dist_key" ; +DROP TABLE "my_TABLE.1!?!"; + +CREATE TYPE int_jsonb_type AS (key int, value jsonb); +CREATE DOMAIN age_with_default AS int CHECK (value >= 0) DEFAULT 0; +CREATE TYPE yes_no_enum AS ENUM ('yes', 'no'); +CREATE EXTENSION btree_gist; + +CREATE SEQUENCE my_seq_1 START WITH 10; + +CREATE TABLE "Table?!.1Table"( + id int PRIMARY KEY, + "Second_Id" int, + "local_Type" int_jsonb_type, + "jsondata" jsonb NOT NULL, + name text, + price numeric CHECK (price > 0), + age_with_default_col age_with_default, + yes_no_enum_col yes_no_enum, + seq_col_1 bigserial, + seq_col_2 int DEFAULT nextval('my_seq_1'), + generated_column int GENERATED ALWAYS AS (seq_col_1 * seq_col_2 + 4) STORED, + UNIQUE (id, price), + EXCLUDE USING GIST (name WITH =)); + +-- create some objects before create_distributed_table +CREATE INDEX "my!Index1" ON "Table?!.1Table"(id) WITH ( fillfactor = 80 ) WHERE id > 10; +CREATE INDEX text_index ON "Table?!.1Table"(name); +CREATE UNIQUE INDEX uniqueIndex ON "Table?!.1Table" (id); +CREATE STATISTICS stats_1 ON id, price FROM "Table?!.1Table"; + +CREATE TEXT SEARCH CONFIGURATION text_search_cfg (parser = default); +CREATE INDEX text_search_index ON "Table?!.1Table" +USING gin (to_tsvector('text_search_cfg'::regconfig, (COALESCE(name, ''::character varying))::text)); + +-- ingest some data before create_distributed_table +INSERT INTO "Table?!.1Table" VALUES (1, 1, (1, row_to_json(row(1,1)))::int_jsonb_type, row_to_json(row(1,1), true)), + (2, 1, (2, row_to_json(row(2,2)))::int_jsonb_type, row_to_json(row(2,2), 'false')); + +-- create a replica identity before create_distributed_table +ALTER TABLE "Table?!.1Table" REPLICA IDENTITY USING INDEX uniqueIndex; + +SELECT create_distributed_table('"Table?!.1Table"', null, colocate_with=>'none'); + +INSERT INTO "Table?!.1Table" VALUES (10, 15, (150, row_to_json(row(4,8)))::int_jsonb_type, '{}', 'text_1', 10, 27, 'yes', 60, 70); +INSERT INTO "Table?!.1Table" VALUES (5, 5, (5, row_to_json(row(5,5)))::int_jsonb_type, row_to_json(row(5,5), true)); + +-- tuples that are supposed to violate different data type / check constraints +INSERT INTO "Table?!.1Table"(id, jsondata, name) VALUES (101, '{"a": 1}', 'text_1'); +INSERT INTO "Table?!.1Table"(id, jsondata, price) VALUES (101, '{"a": 1}', -1); +INSERT INTO "Table?!.1Table"(id, jsondata, age_with_default_col) VALUES (101, '{"a": 1}', -1); +INSERT INTO "Table?!.1Table"(id, jsondata, yes_no_enum_col) VALUES (101, '{"a": 1}', 'what?'); + +SELECT * FROM "Table?!.1Table" ORDER BY id; + +SET search_path TO create_null_dist_key; + +-- create a partitioned table with some columns that +-- are going to be dropped within the tests +CREATE TABLE sensors( + col_to_drop_1 text, + measureid integer, + eventdatetime date, + measure_data jsonb, +PRIMARY KEY (measureid, eventdatetime, measure_data)) +PARTITION BY RANGE(eventdatetime); + +-- drop column even before attaching any partitions +ALTER TABLE sensors DROP COLUMN col_to_drop_1; + +CREATE TABLE sensors_2000 PARTITION OF sensors FOR VALUES FROM ('2000-01-01') TO ('2001-01-01'); + +-- cannot distribute child table without distributing the parent +SELECT create_distributed_table('sensors_2000', NULL, distribution_type=>null); + +SELECT create_distributed_table('sensors', NULL, distribution_type=>null); + +CREATE TABLE multi_level_partitioning_parent( + measureid integer, + eventdatetime date, + measure_data jsonb) +PARTITION BY RANGE(eventdatetime); + +CREATE TABLE multi_level_partitioning_level_1( + measureid integer, + eventdatetime date, + measure_data jsonb) +PARTITION BY RANGE(eventdatetime); + +ALTER TABLE multi_level_partitioning_parent ATTACH PARTITION multi_level_partitioning_level_1 +FOR VALUES FROM ('2000-01-01') TO ('2001-01-01'); + +CREATE TABLE multi_level_partitioning_level_2 PARTITION OF multi_level_partitioning_level_1 +FOR VALUES FROM ('2000-01-01') TO ('2000-06-06'); + +-- multi-level partitioning is not supported +SELECT create_distributed_table('multi_level_partitioning_parent', NULL, distribution_type=>null); + +CREATE FUNCTION normalize_generate_always_as_error(query text) RETURNS void AS $$ +BEGIN + EXECUTE query; + EXCEPTION WHEN OTHERS THEN + IF SQLERRM LIKE 'cannot insert into column %' OR + SQLERRM LIKE 'cannot insert a non-DEFAULT value into column %' + THEN + RAISE 'cannot insert a non-DEFAULT value into column'; + ELSE + RAISE 'unknown error'; + END IF; +END; +$$LANGUAGE plpgsql; + +CREATE TABLE identity_test ( + a int GENERATED BY DEFAULT AS IDENTITY (START WITH 10 INCREMENT BY 10), + b bigint GENERATED ALWAYS AS IDENTITY (START WITH 100 INCREMENT BY 100), + c bigint GENERATED BY DEFAULT AS IDENTITY (START WITH 1000 INCREMENT BY 1000) +); + +SELECT create_distributed_table('identity_test', NULL, distribution_type=>null); + +DROP TABLE identity_test; + +-- Above failed because we don't support using a data type other than BIGINT +-- for identity columns, so drop the table and create a new one with BIGINT +-- identity columns. +CREATE TABLE identity_test ( + a bigint GENERATED BY DEFAULT AS IDENTITY (START WITH 10 INCREMENT BY 10), + b bigint GENERATED ALWAYS AS IDENTITY (START WITH 100 INCREMENT BY 100), + c bigint GENERATED BY DEFAULT AS IDENTITY (START WITH 1000 INCREMENT BY 1000) +); + +SELECT create_distributed_table('identity_test', NULL, distribution_type=>null); + +INSERT INTO identity_test (a) VALUES (5); + +SELECT normalize_generate_always_as_error($$INSERT INTO identity_test (b) VALUES (5)$$); -- fails due to missing OVERRIDING SYSTEM VALUE +INSERT INTO identity_test (b) OVERRIDING SYSTEM VALUE VALUES (5); + +INSERT INTO identity_test (c) VALUES (5); + +SELECT result, success FROM run_command_on_workers($$ + INSERT INTO create_null_dist_key.identity_test (a) VALUES (6) +$$); + +SELECT result, success FROM run_command_on_workers($$ + SELECT create_null_dist_key.normalize_generate_always_as_error('INSERT INTO create_null_dist_key.identity_test (b) VALUES (1)') +$$); + +-- This should fail due to missing OVERRIDING SYSTEM VALUE. +SELECT result, success FROM run_command_on_workers($$ + SELECT create_null_dist_key.normalize_generate_always_as_error('INSERT INTO create_null_dist_key.identity_test (a, b) VALUES (1, 1)') +$$); + +SELECT result, success FROM run_command_on_workers($$ + INSERT INTO create_null_dist_key.identity_test (a, b) OVERRIDING SYSTEM VALUE VALUES (7, 7) +$$); + +SELECT result, success FROM run_command_on_workers($$ + INSERT INTO create_null_dist_key.identity_test (c, a) OVERRIDING SYSTEM VALUE VALUES (8, 8) +$$); + +-- test foreign keys + +CREATE TABLE referenced_table(a int UNIQUE, b int); +CREATE TABLE referencing_table(a int, b int, + FOREIGN KEY (a) REFERENCES referenced_table(a)); + +-- to a colocated null dist key table +BEGIN; + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'referenced_table'); + + INSERT INTO referenced_table VALUES (1, 1); + INSERT INTO referencing_table VALUES (1, 2); + + -- fails + INSERT INTO referencing_table VALUES (2, 2); +ROLLBACK; + +-- to a non-colocated null dist key table +BEGIN; + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'none'); +ROLLBACK; + +-- to a sharded table +BEGIN; + SELECT create_distributed_table('referenced_table', 'a'); + + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null); +ROLLBACK; + +-- to a reference table +BEGIN; + SELECT create_reference_table('referenced_table'); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null); + + INSERT INTO referenced_table VALUES (1, 1); + INSERT INTO referencing_table VALUES (1, 2); + + -- fails + INSERT INTO referencing_table VALUES (2, 2); +ROLLBACK; + +-- to a citus local table +BEGIN; + SELECT citus_add_local_table_to_metadata('referenced_table', cascade_via_foreign_keys=>true); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null); +ROLLBACK; + +-- to a postgres table +SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null); + +-- from a reference table +BEGIN; + SELECT create_reference_table('referencing_table'); + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); +ROLLBACK; + +BEGIN; + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); + SELECT create_reference_table('referencing_table'); +ROLLBACK; + +-- from a sharded table +BEGIN; + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); + SELECT create_distributed_table('referencing_table', 'a'); +ROLLBACK; + +-- from a citus local table +BEGIN; + SELECT citus_add_local_table_to_metadata('referencing_table', cascade_via_foreign_keys=>true); + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); +ROLLBACK; + +BEGIN; + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); + SELECT citus_add_local_table_to_metadata('referencing_table', cascade_via_foreign_keys=>true); +ROLLBACK; + +-- from a postgres table (only useful to preserve legacy behavior) +BEGIN; + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); +ROLLBACK; + +-- make sure that we enforce the foreign key constraint when inserting from workers too +SELECT create_reference_table('referenced_table'); +SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null); +INSERT INTO referenced_table VALUES (1, 1); +-- ok +SELECT result, success FROM run_command_on_workers($$ + INSERT INTO create_null_dist_key.referencing_table VALUES (1, 2) +$$); +-- fails +SELECT result, success FROM run_command_on_workers($$ + INSERT INTO create_null_dist_key.referencing_table VALUES (2, 2) +$$); + +DROP TABLE referencing_table, referenced_table; + +CREATE TABLE self_fkey_test(a int UNIQUE, b int, + FOREIGN KEY (b) REFERENCES self_fkey_test(a), + FOREIGN KEY (a) REFERENCES self_fkey_test(a)); +SELECT create_distributed_table('self_fkey_test', NULL, distribution_type=>null); + +INSERT INTO self_fkey_test VALUES (1, 1); -- ok +INSERT INTO self_fkey_test VALUES (2, 3); -- fails + +-- similar foreign key tests but this time create the referencing table later on + +-- referencing table is a null shard key table + +-- to a colocated null dist key table +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); + + CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a) REFERENCES referenced_table(a)); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'referenced_table'); + + INSERT INTO referenced_table VALUES (1, 1); + INSERT INTO referencing_table VALUES (1, 2); + + -- fails + INSERT INTO referencing_table VALUES (2, 2); +ROLLBACK; + +BEGIN; + CREATE TABLE referenced_table(a int, b int, UNIQUE(b, a)); + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); + + CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a, b) REFERENCES referenced_table(b, a)); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'referenced_table'); + + INSERT INTO referenced_table VALUES (1, 2); + INSERT INTO referencing_table VALUES (2, 1); + + -- fails + INSERT INTO referencing_table VALUES (1, 2); +ROLLBACK; + +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); + + CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a) REFERENCES referenced_table(a) ON UPDATE SET NULL); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'referenced_table'); + + INSERT INTO referenced_table VALUES (1, 1); + INSERT INTO referencing_table VALUES (1, 2); + + UPDATE referenced_table SET a = 5; + SELECT * FROM referencing_table; +ROLLBACK; + +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); + + CREATE TABLE referencing_table(a serial, b int, FOREIGN KEY (a) REFERENCES referenced_table(a) ON UPDATE SET DEFAULT); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'referenced_table'); +ROLLBACK; + +-- to a non-colocated null dist key table +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); + + CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a) REFERENCES referenced_table(a)); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'none'); +ROLLBACK; + +-- to a sharded table +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT create_distributed_table('referenced_table', 'a'); + + CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a) REFERENCES referenced_table(a)); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'none'); +ROLLBACK; + +-- to a reference table +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT create_reference_table('referenced_table'); + + CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a) REFERENCES referenced_table(a)); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'none'); + + INSERT INTO referenced_table VALUES (1, 1); + INSERT INTO referencing_table VALUES (1, 2); + + -- fails + INSERT INTO referencing_table VALUES (2, 2); +ROLLBACK; + +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT create_reference_table('referenced_table'); + + CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a) REFERENCES referenced_table(a) ON DELETE CASCADE); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'none'); + + INSERT INTO referenced_table VALUES (1, 1); + INSERT INTO referencing_table VALUES (1, 2); + + DELETE FROM referenced_table CASCADE; + SELECT * FROM referencing_table; +ROLLBACK; + +-- to a citus local table +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT citus_add_local_table_to_metadata('referenced_table'); + + CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a) REFERENCES referenced_table(a)); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'none'); +ROLLBACK; + +-- to a postgres table +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a) REFERENCES referenced_table(a)); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'none'); +ROLLBACK; + +-- referenced table is a null shard key table + +-- from a sharded table +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null, colocate_with=>'none'); + + CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a) REFERENCES referenced_table(a)); + SELECT create_distributed_table('referencing_table', 'a'); +ROLLBACK; + +-- from a reference table +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null, colocate_with=>'none'); + + CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a) REFERENCES referenced_table(a)); + SELECT create_reference_table('referencing_table'); +ROLLBACK; + +-- from a citus local table +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null, colocate_with=>'none'); + + CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a) REFERENCES referenced_table(a)); + SELECT citus_add_local_table_to_metadata('referencing_table', cascade_via_foreign_keys=>true); +ROLLBACK; + +-- from a postgres table (only useful to preserve legacy behavior) +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null, colocate_with=>'none'); + + CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a) REFERENCES referenced_table(a)); +ROLLBACK; + +-- Test whether we switch to sequential execution to enforce foreign +-- key restrictions. + +CREATE TABLE referenced_table(id int PRIMARY KEY, value_1 int); +SELECT create_reference_table('referenced_table'); + +CREATE TABLE referencing_table(id int PRIMARY KEY, value_1 int, CONSTRAINT fkey FOREIGN KEY(value_1) REFERENCES referenced_table(id) ON UPDATE CASCADE); +SELECT create_distributed_table('referencing_table', null, colocate_with=>'none', distribution_type=>null); + +SET client_min_messages TO DEBUG1; + +BEGIN; + -- Switches to sequential execution because referenced_table is a reference table + -- and referenced by a null-shard-key distributed table. + -- + -- Given that we cannot do parallel access on null-shard-key, this is not useful. + -- However, this is already what we're doing for, e.g., a foreign key from a + -- reference table to another reference table. + TRUNCATE referenced_table CASCADE; + SELECT COUNT(*) FROM referencing_table; +COMMIT; + +BEGIN; + SELECT COUNT(*) FROM referencing_table; + -- Doesn't fail because the SELECT didn't perform parallel execution. + TRUNCATE referenced_table CASCADE; +COMMIT; + +BEGIN; + UPDATE referencing_table SET value_1 = 15; + -- Doesn't fail because the UPDATE didn't perform parallel execution. + TRUNCATE referenced_table CASCADE; +COMMIT; + +BEGIN; + SELECT COUNT(*) FROM referenced_table; + -- doesn't switch to sequential execution + ALTER TABLE referencing_table ADD COLUMN X INT; +ROLLBACK; + +BEGIN; + -- Switches to sequential execution because referenced_table is a reference table + -- and referenced by a null-shard-key distributed table. + -- + -- Given that we cannot do parallel access on null-shard-key, this is not useful. + -- However, this is already what we're doing for, e.g., a foreign key from a + -- reference table to another reference table. + UPDATE referenced_table SET id = 101 WHERE id = 99; + UPDATE referencing_table SET value_1 = 15; +ROLLBACK; + +BEGIN; + UPDATE referencing_table SET value_1 = 15; + -- Doesn't fail because prior UPDATE didn't perform parallel execution. + UPDATE referenced_table SET id = 101 WHERE id = 99; +ROLLBACK; + +SET client_min_messages TO WARNING; + +DROP TABLE referenced_table, referencing_table; + +-- Test whether we unnecessarily switch to sequential execution +-- when the referenced relation is a null-shard-key table. + +CREATE TABLE referenced_table(id int PRIMARY KEY, value_1 int); +SELECT create_distributed_table('referenced_table', null, colocate_with=>'none', distribution_type=>null); + +CREATE TABLE referencing_table(id int PRIMARY KEY, value_1 int, CONSTRAINT fkey FOREIGN KEY(value_1) REFERENCES referenced_table(id) ON UPDATE CASCADE); +SELECT create_distributed_table('referencing_table', null, colocate_with=>'referenced_table', distribution_type=>null); + +SET client_min_messages TO DEBUG1; + +BEGIN; + SELECT COUNT(*) FROM referenced_table; + -- Doesn't switch to sequential execution because the referenced_table is + -- a null-shard-key distributed table. + ALTER TABLE referencing_table ADD COLUMN X INT; +ROLLBACK; + +BEGIN; + -- Doesn't switch to sequential execution because the referenced_table is + -- a null-shard-key distributed table. + TRUNCATE referenced_table CASCADE; + SELECT COUNT(*) FROM referencing_table; +COMMIT; + +SET client_min_messages TO WARNING; + +CREATE FUNCTION increment_value() RETURNS trigger AS $increment_value$ +BEGIN + NEW.value := NEW.value+1; + RETURN NEW; +END; +$increment_value$ LANGUAGE plpgsql; + +CREATE TABLE trigger_table_1 (value int); + +CREATE TRIGGER trigger_1 +BEFORE INSERT ON trigger_table_1 +FOR EACH ROW EXECUTE FUNCTION increment_value(); + +SELECT create_distributed_table('trigger_table_1', NULL, distribution_type=>null); + +SET citus.enable_unsafe_triggers TO ON; +SELECT create_distributed_table('trigger_table_1', NULL, distribution_type=>null); + +INSERT INTO trigger_table_1 VALUES(1), (2); +SELECT * FROM trigger_table_1 ORDER BY value; + +CREATE FUNCTION insert_some() RETURNS trigger AS $insert_some$ +BEGIN + RAISE NOTICE 'inserted some rows'; + RETURN NEW; +END; +$insert_some$ LANGUAGE plpgsql; + +CREATE TABLE trigger_table_2 (value int); + +CREATE TRIGGER trigger_2 +AFTER INSERT ON trigger_table_2 +FOR EACH STATEMENT EXECUTE FUNCTION insert_some(); + +ALTER TABLE trigger_table_2 DISABLE TRIGGER trigger_2; + +SELECT create_distributed_table('trigger_table_2', NULL, distribution_type=>null); + +SET client_min_messages TO NOTICE; +INSERT INTO trigger_table_2 VALUES(3), (4); +SET client_min_messages TO WARNING; + +SELECT * FROM trigger_table_2 ORDER BY value; + +CREATE FUNCTION combine_old_new_val() RETURNS trigger AS $combine_old_new_val$ +BEGIN + NEW.value = NEW.value * 10 + OLD.value; + RETURN NEW; +END; +$combine_old_new_val$ LANGUAGE plpgsql; + +CREATE FUNCTION notice_truncate() RETURNS trigger AS $notice_truncate$ +BEGIN + RAISE NOTICE 'notice_truncate()'; + RETURN NEW; +END; +$notice_truncate$ LANGUAGE plpgsql; + +CREATE TABLE trigger_table_3 (value int); + +CREATE TRIGGER trigger_3 +BEFORE UPDATE ON trigger_table_3 +FOR EACH ROW EXECUTE FUNCTION combine_old_new_val(); + +CREATE TRIGGER trigger_4 +AFTER TRUNCATE ON trigger_table_3 +FOR EACH STATEMENT EXECUTE FUNCTION notice_truncate(); + +INSERT INTO trigger_table_3 VALUES(3), (4); + +SELECT create_distributed_table('trigger_table_3', NULL, distribution_type=>null); + +UPDATE trigger_table_3 SET value = 5; +SELECT * FROM trigger_table_3 ORDER BY value; + +SET client_min_messages TO NOTICE; +TRUNCATE trigger_table_3; +SET client_min_messages TO WARNING; + +-- try a few simple queries at least to make sure that we don't crash +BEGIN; + INSERT INTO nullkey_c1_t1 SELECT * FROM nullkey_c2_t1; +ROLLBACK; + +-- cleanup at exit +SET client_min_messages TO ERROR; +DROP SCHEMA create_null_dist_key, "NULL_!_dist_key" CASCADE; diff --git a/src/test/regress/sql/multi_extension.sql b/src/test/regress/sql/multi_extension.sql index cbad97524..50b821b0c 100644 --- a/src/test/regress/sql/multi_extension.sql +++ b/src/test/regress/sql/multi_extension.sql @@ -904,6 +904,20 @@ SELECT create_distributed_table('test','x'); DROP TABLE test; TRUNCATE pg_dist_node; +-- confirm that we can create a null shard key table on an empty node +CREATE TABLE test (x int, y int); +INSERT INTO test VALUES (1,2); +SET citus.shard_replication_factor TO 1; +SELECT create_distributed_table('test', null, colocate_with=>'none', distribution_type=>null); + +-- and make sure that we can't remove the coordinator due to "test" +SELECT citus_remove_node('localhost', :master_port); + +DROP TABLE test; + +-- and now we should be able to remove the coordinator +SELECT citus_remove_node('localhost', :master_port); + -- confirm that we can create a reference table on an empty node CREATE TABLE test (x int, y int); INSERT INTO test VALUES (1,2); diff --git a/src/test/regress/sql/single_node.sql b/src/test/regress/sql/single_node.sql index 3419025af..8c612c1bb 100644 --- a/src/test/regress/sql/single_node.sql +++ b/src/test/regress/sql/single_node.sql @@ -63,8 +63,43 @@ ALTER SYSTEM RESET citus.local_shared_pool_size; ALTER SYSTEM RESET citus.max_cached_conns_per_worker; SELECT pg_reload_conf(); +CREATE TABLE single_node_nullkey_c1(a int, b int); +SELECT create_distributed_table('single_node_nullkey_c1', null, colocate_with=>'none', distribution_type=>null); + +CREATE TABLE single_node_nullkey_c2(a int, b int); +SELECT create_distributed_table('single_node_nullkey_c2', null, colocate_with=>'none', distribution_type=>null); + +-- created on different colocation groups .. +SELECT +( + SELECT colocationid FROM pg_dist_partition + WHERE logicalrelid = 'single_node.single_node_nullkey_c1'::regclass +) +!= +( + SELECT colocationid FROM pg_dist_partition + WHERE logicalrelid = 'single_node.single_node_nullkey_c2'::regclass +); + +-- .. but both are associated to coordinator +SELECT groupid = 0 FROM pg_dist_placement +WHERE shardid = ( + SELECT shardid FROM pg_dist_shard + WHERE logicalrelid = 'single_node.single_node_nullkey_c1'::regclass +); + +SELECT groupid = 0 FROM pg_dist_placement +WHERE shardid = ( + SELECT shardid FROM pg_dist_shard + WHERE logicalrelid = 'single_node.single_node_nullkey_c2'::regclass +); + +-- try creating a null-shard-key distributed table from a shard relation +SELECT shardid AS round_robin_test_c1_shard_id FROM pg_dist_shard WHERE logicalrelid = 'single_node.single_node_nullkey_c1'::regclass \gset +SELECT create_distributed_table('single_node_nullkey_c1_' || :round_robin_test_c1_shard_id , null, colocate_with=>'none', distribution_type=>null); + SET client_min_messages TO WARNING; -DROP TABLE failover_to_local; +DROP TABLE failover_to_local, single_node_nullkey_c1, single_node_nullkey_c2; RESET client_min_messages; -- so that we don't have to update rest of the test output From cdf54ff4b1ead8b4a77dc995f0ddd2695a4c2ff1 Mon Sep 17 00:00:00 2001 From: Ahmet Gedemenli Date: Tue, 21 Mar 2023 12:24:16 +0300 Subject: [PATCH 2/7] Add DDL support null-shard-key tables(#6778/#6784/#6787/#6859) Add tests for ddl coverage: * indexes * partitioned tables + indexes with long names * triggers * foreign keys * statistics * grant & revoke statements * truncate & vacuum * create/test/drop view that depends on a dist table with no shard key * policy & rls test * alter table add/drop/alter_type column (using sequences/different data types/identity columns) * alter table add constraint (not null, check, exclusion constraint) * alter table add column with a default value / set default / drop default * alter table set option (autovacuum) * indexes / constraints without names * multiple subcommands Adds support for * Creating new partitions after distributing (with null key) the parent table * Attaching partitions to a distributed table with null distribution key (and automatically distribute the new partition with null key as well) * Detaching partitions from it --- .../commands/create_distributed_table.c | 3 +- src/backend/distributed/commands/table.c | 35 +- .../distributed/planner/multi_join_order.c | 2 +- src/include/distributed/metadata_utility.h | 1 + .../expected/alter_table_null_dist_key.out | 154 +++++++ .../regress/expected/create_null_dist_key.out | 410 +++++++++++++++++- src/test/regress/multi_1_schedule | 2 +- .../regress/sql/alter_table_null_dist_key.sql | 98 +++++ src/test/regress/sql/create_null_dist_key.sql | 280 +++++++++++- 9 files changed, 956 insertions(+), 29 deletions(-) create mode 100644 src/test/regress/expected/alter_table_null_dist_key.out create mode 100644 src/test/regress/sql/alter_table_null_dist_key.sql diff --git a/src/backend/distributed/commands/create_distributed_table.c b/src/backend/distributed/commands/create_distributed_table.c index 12bfcf9a5..544d8f04e 100644 --- a/src/backend/distributed/commands/create_distributed_table.c +++ b/src/backend/distributed/commands/create_distributed_table.c @@ -134,7 +134,6 @@ static List * HashSplitPointsForShardList(List *shardList); static List * HashSplitPointsForShardCount(int shardCount); static List * WorkerNodesForShardList(List *shardList); static List * RoundRobinWorkerNodeList(List *workerNodeList, int listLength); -static void CreateNullShardKeyDistTable(Oid relationId, char *colocateWithTableName); static CitusTableParams DecideCitusTableParams(CitusTableType tableType, DistributedTableParams * distributedTableParams); @@ -1031,7 +1030,7 @@ CreateReferenceTable(Oid relationId) * CreateNullShardKeyDistTable is a wrapper around CreateCitusTable that creates a * single shard distributed table that doesn't have a shard key. */ -static void +void CreateNullShardKeyDistTable(Oid relationId, char *colocateWithTableName) { DistributedTableParams distributedTableParams = { diff --git a/src/backend/distributed/commands/table.c b/src/backend/distributed/commands/table.c index 6d5fcda3f..af3439ab5 100644 --- a/src/backend/distributed/commands/table.c +++ b/src/backend/distributed/commands/table.c @@ -384,6 +384,11 @@ PostprocessCreateTableStmtPartitionOf(CreateStmt *createStatement, const */ if (IsCitusTable(parentRelationId)) { + /* + * We can create Citus local tables and distributed tables with null shard keys + * right away, without switching to sequential mode, because they are going to + * have only one shard. + */ if (IsCitusTableType(parentRelationId, CITUS_LOCAL_TABLE)) { CreateCitusLocalTablePartitionOf(createStatement, relationId, @@ -391,11 +396,18 @@ PostprocessCreateTableStmtPartitionOf(CreateStmt *createStatement, const return; } + char *parentRelationName = generate_qualified_relation_name(parentRelationId); + + if (IsCitusTableType(parentRelationId, NULL_KEY_DISTRIBUTED_TABLE)) + { + CreateNullShardKeyDistTable(relationId, parentRelationName); + return; + } + Var *parentDistributionColumn = DistPartitionKeyOrError(parentRelationId); char *distributionColumnName = ColumnToColumnName(parentRelationId, (Node *) parentDistributionColumn); char parentDistributionMethod = DISTRIBUTE_BY_HASH; - char *parentRelationName = generate_qualified_relation_name(parentRelationId); SwitchToSequentialAndLocalExecutionIfPartitionNameTooLong(parentRelationId, relationId); @@ -589,19 +601,32 @@ PreprocessAttachCitusPartitionToCitusTable(Oid parentCitusRelationId, Oid /* * DistributePartitionUsingParent takes a parent and a partition relation and - * distributes the partition, using the same distribution column as the parent. - * It creates a *hash* distributed table by default, as partitioned tables can only be - * distributed by hash. + * distributes the partition, using the same distribution column as the parent, if the + * parent has a distribution column. It creates a *hash* distributed table by default, as + * partitioned tables can only be distributed by hash, unless it's null key distributed. + * + * If the parent has no distribution key, we distribute the partition with null key too. */ static void DistributePartitionUsingParent(Oid parentCitusRelationId, Oid partitionRelationId) { + char *parentRelationName = generate_qualified_relation_name(parentCitusRelationId); + + if (!HasDistributionKey(parentCitusRelationId)) + { + /* + * If the parent is null key distributed, we should distribute the partition + * with null distribution key as well. + */ + CreateNullShardKeyDistTable(partitionRelationId, parentRelationName); + return; + } + Var *distributionColumn = DistPartitionKeyOrError(parentCitusRelationId); char *distributionColumnName = ColumnToColumnName(parentCitusRelationId, (Node *) distributionColumn); char distributionMethod = DISTRIBUTE_BY_HASH; - char *parentRelationName = generate_qualified_relation_name(parentCitusRelationId); SwitchToSequentialAndLocalExecutionIfPartitionNameTooLong( parentCitusRelationId, partitionRelationId); diff --git a/src/backend/distributed/planner/multi_join_order.c b/src/backend/distributed/planner/multi_join_order.c index b1195c664..0fff79ed8 100644 --- a/src/backend/distributed/planner/multi_join_order.c +++ b/src/backend/distributed/planner/multi_join_order.c @@ -1404,7 +1404,7 @@ DistPartitionKeyOrError(Oid relationId) if (partitionKey == NULL) { ereport(ERROR, (errmsg( - "no distribution column found for relation %d, because it is a reference table", + "no distribution column found for relation %d", relationId))); } diff --git a/src/include/distributed/metadata_utility.h b/src/include/distributed/metadata_utility.h index e27f3df22..fe404acf8 100644 --- a/src/include/distributed/metadata_utility.h +++ b/src/include/distributed/metadata_utility.h @@ -326,6 +326,7 @@ extern void DeletePartitionRow(Oid distributedRelationId); extern void DeleteShardRow(uint64 shardId); extern void UpdatePlacementGroupId(uint64 placementId, int groupId); extern void DeleteShardPlacementRow(uint64 placementId); +extern void CreateNullShardKeyDistTable(Oid relationId, char *colocateWithTableName); extern void CreateDistributedTable(Oid relationId, char *distributionColumnName, char distributionMethod, int shardCount, bool shardCountIsStrict, char *colocateWithTableName); diff --git a/src/test/regress/expected/alter_table_null_dist_key.out b/src/test/regress/expected/alter_table_null_dist_key.out new file mode 100644 index 000000000..1812c33cb --- /dev/null +++ b/src/test/regress/expected/alter_table_null_dist_key.out @@ -0,0 +1,154 @@ +CREATE SCHEMA alter_null_dist_key; +SET search_path TO alter_null_dist_key; +SET citus.next_shard_id TO 1720000; +SET citus.shard_count TO 32; +SET citus.shard_replication_factor TO 1; +CREATE SEQUENCE dist_seq; +CREATE TABLE null_dist_table(a bigint DEFAULT nextval('dist_seq') UNIQUE, "b" text, c bigint GENERATED BY DEFAULT AS IDENTITY); +INSERT INTO null_dist_table("b") VALUES ('test'); +SELECT create_distributed_table('null_dist_table', null, colocate_with=>'none', distribution_type=>null); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$alter_null_dist_key.null_dist_table$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- add column +ALTER TABLE null_dist_table ADD COLUMN d bigint DEFAULT 2; +SELECT * FROM null_dist_table ORDER BY c; + a | b | c | d +--------------------------------------------------------------------- + 1 | test | 1 | 2 +(1 row) + +-- alter default, set to 3 +ALTER TABLE null_dist_table ALTER COLUMN d SET DEFAULT 3; +INSERT INTO null_dist_table("b") VALUES ('test'); +SELECT * FROM null_dist_table ORDER BY c; + a | b | c | d +--------------------------------------------------------------------- + 1 | test | 1 | 2 + 2 | test | 2 | 3 +(2 rows) + +-- drop default, see null +ALTER TABLE null_dist_table ALTER COLUMN d DROP DEFAULT; +INSERT INTO null_dist_table("b") VALUES ('test'); +SELECT * FROM null_dist_table ORDER BY c; + a | b | c | d +--------------------------------------------------------------------- + 1 | test | 1 | 2 + 2 | test | 2 | 3 + 3 | test | 3 | +(3 rows) + +-- cleanup the rows that were added to test the default behavior +DELETE FROM null_dist_table WHERE "b" = 'test' AND a > 1; +-- alter column type +ALTER TABLE null_dist_table ALTER COLUMN d TYPE text; +UPDATE null_dist_table SET d = 'this is a text' WHERE d = '2'; +SELECT * FROM null_dist_table ORDER BY c; + a | b | c | d +--------------------------------------------------------------------- + 1 | test | 1 | this is a text +(1 row) + +-- drop seq column +ALTER TABLE null_dist_table DROP COLUMN a; +SELECT * FROM null_dist_table ORDER BY c; + b | c | d +--------------------------------------------------------------------- + test | 1 | this is a text +(1 row) + +-- add not null constraint +ALTER TABLE null_dist_table ALTER COLUMN b SET NOT NULL; +-- not null constraint violation, error out +INSERT INTO null_dist_table VALUES (NULL, 2, 'test'); +ERROR: null value in column "b" violates not-null constraint +DETAIL: Failing row contains (null, 2, test). +CONTEXT: while executing command on localhost:xxxxx +-- drop not null constraint and try again +ALTER TABLE null_dist_table ALTER COLUMN b DROP NOT NULL; +INSERT INTO null_dist_table VALUES (NULL, 3, 'test'); +SELECT * FROM null_dist_table ORDER BY c; + b | c | d +--------------------------------------------------------------------- + test | 1 | this is a text + | 3 | test +(2 rows) + +-- add exclusion constraint +ALTER TABLE null_dist_table ADD CONSTRAINT exc_b EXCLUDE USING btree (b with =); +-- rename the exclusion constraint, errors out +ALTER TABLE null_dist_table RENAME CONSTRAINT exc_b TO exc_b_1; +ERROR: renaming constraints belonging to distributed tables is currently unsupported +-- create exclusion constraint without a name +ALTER TABLE null_dist_table ADD EXCLUDE USING btree (b with =); +-- test setting autovacuum option +ALTER TABLE null_dist_table SET (autovacuum_enabled = false); +-- test multiple subcommands +ALTER TABLE null_dist_table ADD COLUMN int_column1 INTEGER, + DROP COLUMN d; +SELECT * FROM null_dist_table ORDER BY c; + b | c | int_column1 +--------------------------------------------------------------------- + test | 1 | + | 3 | +(2 rows) + +-- test policy and row level security +CREATE TABLE null_dist_key_with_policy (table_user text); +INSERT INTO null_dist_key_with_policy VALUES ('user_1'); +SELECT create_distributed_table('null_dist_key_with_policy', null); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$alter_null_dist_key.null_dist_key_with_policy$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- enable rls +ALTER TABLE null_dist_key_with_policy ENABLE ROW LEVEL SECURITY; +-- user_1 will be allowed to see the inserted row +CREATE ROLE user_1 WITH LOGIN; +GRANT ALL ON SCHEMA alter_null_dist_key TO user_1; +GRANT ALL ON TABLE alter_null_dist_key.null_dist_key_with_policy TO user_1; +CREATE POLICY table_policy ON null_dist_key_with_policy TO user_1 + USING (table_user = current_user); +-- user_2 will not be allowed to see the inserted row +CREATE ROLE user_2 WITH LOGIN; +GRANT ALL ON SCHEMA alter_null_dist_key TO user_2; +GRANT ALL ON TABLE alter_null_dist_key.null_dist_key_with_policy TO user_2; +CREATE POLICY table_policy_1 ON null_dist_key_with_policy TO user_2 + USING (table_user = current_user); +\c - user_1 - +SELECT * FROM alter_null_dist_key.null_dist_key_with_policy; + table_user +--------------------------------------------------------------------- + user_1 +(1 row) + +\c - user_2 - +SELECT * FROM alter_null_dist_key.null_dist_key_with_policy; + table_user +--------------------------------------------------------------------- +(0 rows) + +-- postgres will always be allowed to see the row as a superuser +\c - postgres - +SELECT * FROM alter_null_dist_key.null_dist_key_with_policy; + table_user +--------------------------------------------------------------------- + user_1 +(1 row) + +-- cleanup +SET client_min_messages TO ERROR; +DROP SCHEMA alter_null_dist_key CASCADE; +DROP ROLE user_1, user_2; diff --git a/src/test/regress/expected/create_null_dist_key.out b/src/test/regress/expected/create_null_dist_key.out index e24ff1e91..af6e66f62 100644 --- a/src/test/regress/expected/create_null_dist_key.out +++ b/src/test/regress/expected/create_null_dist_key.out @@ -497,7 +497,8 @@ SELECT * FROM null_dist_key_table_2 ORDER BY a; DROP TABLE null_dist_key_table_1, null_dist_key_table_2; -- create indexes before creating the null dist key tables -- .. for an initially empty table -CREATE TABLE null_dist_key_table_1(a int); +CREATE TABLE null_dist_key_table_1(a int, b int); +CREATE STATISTICS s1 (dependencies) ON a, b FROM null_dist_key_table_1; CREATE INDEX null_dist_key_table_1_idx ON null_dist_key_table_1(a); SELECT create_distributed_table('null_dist_key_table_1', null, colocate_with=>'none'); create_distributed_table @@ -505,6 +506,7 @@ SELECT create_distributed_table('null_dist_key_table_1', null, colocate_with=>'n (1 row) +CREATE STATISTICS s2 (dependencies) ON a, b FROM null_dist_key_table_1; -- .. and for another table having data in it before creating null dist key table CREATE TABLE null_dist_key_table_2(a int); INSERT INTO null_dist_key_table_2 VALUES(1); @@ -515,6 +517,11 @@ SELECT create_distributed_table('null_dist_key_table_2', null, colocate_with=>'n (1 row) +-- test create index concurrently, then reindex +CREATE INDEX CONCURRENTLY ind_conc ON null_dist_key_table_2(a); +REINDEX INDEX ind_conc; +REINDEX INDEX CONCURRENTLY ind_conc; +DROP INDEX ind_conc; SELECT * FROM null_dist_key_table_2 ORDER BY a; a --------------------------------------------------------------------- @@ -536,15 +543,23 @@ BEGIN; CREATE ROLE table_users; CREATE POLICY table_policy ON null_dist_key_table_3 TO table_users USING (table_user = current_user); + GRANT ALL ON TABLE null_dist_key_table_3 TO table_users; + ALTER TABLE null_dist_key_table_3 OWNER TO table_users; SELECT create_distributed_table('null_dist_key_table_3', null, colocate_with=>'none'); create_distributed_table --------------------------------------------------------------------- (1 row) + REVOKE ALL ON TABLE null_dist_key_table_3 FROM table_users; + ALTER TABLE null_dist_key_table_3 OWNER TO postgres; + GRANT ALL ON TABLE null_dist_key_table_3 TO table_users; ROLLBACK; +ALTER STATISTICS s2 SET STATISTICS 46; +ALTER TABLE null_dist_key_table_1 SET SCHEMA public; +DROP STATISTICS s1, s2; -- drop them for next tests -DROP TABLE null_dist_key_table_1, null_dist_key_table_2, distributed_table; +DROP TABLE public.null_dist_key_table_1, null_dist_key_table_2, distributed_table; -- tests for object names that should be escaped properly CREATE SCHEMA "NULL_!_dist_key"; CREATE TABLE "NULL_!_dist_key"."my_TABLE.1!?!"(id int, "Second_Id" int); @@ -643,6 +658,348 @@ SELECT create_distributed_table('sensors', NULL, distribution_type=>null); (1 row) +-- verify we can create new partitions after distributing the parent table +CREATE TABLE sensors_2001 PARTITION OF sensors FOR VALUES FROM ('2001-01-01') TO ('2002-01-01'); +-- verify we can attach to a null dist key table +CREATE TABLE sensors_2002 (measureid integer, eventdatetime date, measure_data jsonb, PRIMARY KEY (measureid, eventdatetime, measure_data)); +ALTER TABLE sensors ATTACH PARTITION sensors_2002 FOR VALUES FROM ('2002-01-01') TO ('2003-01-01'); +-- verify we can detach from a null dist key table +ALTER TABLE sensors DETACH PARTITION sensors_2001; +-- error out when attaching a noncolocated partition +CREATE TABLE sensors_2003 (measureid integer, eventdatetime date, measure_data jsonb, PRIMARY KEY (measureid, eventdatetime, measure_data)); +SELECT create_distributed_table('sensors_2003', NULL, distribution_type=>null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +ALTER TABLE sensors ATTACH PARTITION sensors_2003 FOR VALUES FROM ('2003-01-01') TO ('2004-01-01'); +ERROR: distributed tables cannot have non-colocated distributed tables as a partition +DROP TABLE sensors_2003; +-- verify we can attach after distributing, if the parent and partition are colocated +CREATE TABLE sensors_2004 (measureid integer, eventdatetime date, measure_data jsonb, PRIMARY KEY (measureid, eventdatetime, measure_data)); +SELECT create_distributed_table('sensors_2004', NULL, distribution_type=>null, colocate_with=>'sensors'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +ALTER TABLE sensors ATTACH PARTITION sensors_2004 FOR VALUES FROM ('2004-01-01') TO ('2005-01-01'); +-- verify we can attach a citus local table +CREATE TABLE sensors_2005 (measureid integer, eventdatetime date, measure_data jsonb, PRIMARY KEY (measureid, eventdatetime, measure_data)); +SELECT citus_add_local_table_to_metadata('sensors_2005'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +ALTER TABLE sensors ATTACH PARTITION sensors_2005 FOR VALUES FROM ('2005-01-01') TO ('2006-01-01'); +-- check metadata +-- check all partitions and the parent on pg_dist_partition +SELECT logicalrelid::text FROM pg_dist_partition WHERE logicalrelid::text IN ('sensors', 'sensors_2000', 'sensors_2001', 'sensors_2002', 'sensors_2004', 'sensors_2005') ORDER BY logicalrelid::text; + logicalrelid +--------------------------------------------------------------------- + sensors + sensors_2000 + sensors_2001 + sensors_2002 + sensors_2004 + sensors_2005 +(6 rows) + +-- verify they are all colocated +SELECT COUNT(DISTINCT(colocationid)) FROM pg_dist_partition WHERE logicalrelid::text IN ('sensors', 'sensors_2000', 'sensors_2001', 'sensors_2002', 'sensors_2004', 'sensors_2005'); + count +--------------------------------------------------------------------- + 1 +(1 row) + +-- verify all partitions are placed on the same node +SELECT COUNT(DISTINCT(groupid)) FROM pg_dist_placement WHERE shardid IN + (SELECT shardid FROM pg_dist_shard WHERE logicalrelid::text IN ('sensors', 'sensors_2000', 'sensors_2001', 'sensors_2002', 'sensors_2004', 'sensors_2005')); + count +--------------------------------------------------------------------- + 1 +(1 row) + +-- verify the shard of sensors_2000 is attached to the parent shard, on the worker node +SELECT COUNT(*) FROM run_command_on_workers($$ + SELECT relpartbound FROM pg_class WHERE relname LIKE 'sensors_2000_1______';$$) + WHERE length(result) > 0; + count +--------------------------------------------------------------------- + 1 +(1 row) + +-- verify the shard of sensors_2001 is detached from the parent shard, on the worker node +SELECT COUNT(*) FROM run_command_on_workers($$ + SELECT relpartbound FROM pg_class WHERE relname LIKE 'sensors_2001_1______';$$) + WHERE length(result) > 0; + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- verify the shard of sensors_2002 is attached to the parent shard, on the worker node +SELECT COUNT(*) FROM run_command_on_workers($$ + SELECT relpartbound FROM pg_class WHERE relname LIKE 'sensors_2002_1______';$$) + WHERE length(result) > 0; + count +--------------------------------------------------------------------- + 1 +(1 row) + +-- create a partitioned citus local table and verify we error out when attaching a partition with null dist key +CREATE TABLE partitioned_citus_local_tbl( + measureid integer, + eventdatetime date, + measure_data jsonb, +PRIMARY KEY (measureid, eventdatetime, measure_data)) +PARTITION BY RANGE(eventdatetime); +SELECT citus_add_local_table_to_metadata('partitioned_citus_local_tbl'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE partition_with_null_key (measureid integer, eventdatetime date, measure_data jsonb, PRIMARY KEY (measureid, eventdatetime, measure_data)); +SELECT create_distributed_table('partition_with_null_key', NULL, distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +ALTER TABLE partitioned_citus_local_tbl ATTACH PARTITION partition_with_null_key FOR VALUES FROM ('2004-01-01') TO ('2005-01-01'); +ERROR: non-distributed partitioned tables cannot have distributed partitions +-- test partitioned tables + indexes with long names +CREATE TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"( + id int PRIMARY KEY, + "TeNANt_Id" int, + "jsondata" jsonb NOT NULL, + name text, + price numeric CHECK (price > 0), + serial_data bigserial, UNIQUE (id, price)) + PARTITION BY LIST(id); +CREATE TABLE "NULL_!_dist_key"."partition1_nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + PARTITION OF "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + FOR VALUES IN (1); +CREATE TABLE "NULL_!_dist_key"."partition2_nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + PARTITION OF "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + FOR VALUES IN (2); +CREATE TABLE "NULL_!_dist_key"."partition100_nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + PARTITION OF "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + FOR VALUES IN (100); +-- create some objects before create_distributed_table +CREATE INDEX "my!Index1New" ON "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id) WITH ( fillfactor = 80 ) WHERE id > 10; +CREATE UNIQUE INDEX uniqueIndexNew ON "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" (id); +-- ingest some data before create_distributed_table +set client_min_messages to ERROR; +INSERT INTO "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" VALUES (1, 1, row_to_json(row(1,1), true)), + (2, 1, row_to_json(row(2,2), 'false')); +reset client_min_messages; +-- create a replica identity before create_distributed_table +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" REPLICA IDENTITY USING INDEX uniqueIndexNew; +NOTICE: identifier "nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" will be truncated to "nullKeyTable.1!?!9012345678901234567890123456789012345678901234" +-- test triggers +SET client_min_messages TO ERROR; +CREATE FUNCTION insert_id_100() RETURNS trigger AS $insert_100$ +BEGIN + INSERT INTO "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" VALUES (100, 1, row_to_json(row(1,1), true)); + RETURN NEW; +END; +$insert_100$ LANGUAGE plpgsql; +CREATE TABLE null_key_table_with_trigger(a INT); +SELECT create_distributed_table('null_key_table_with_trigger', null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- try to add a trigger after distributing the table, fails +CREATE TRIGGER insert_100_trigger + AFTER UPDATE ON null_key_table_with_trigger + FOR EACH STATEMENT EXECUTE FUNCTION insert_id_100(); +ERROR: triggers are not supported on distributed tables +-- now try to distribute a table that already has a trigger on it +CREATE TRIGGER insert_100_trigger + AFTER UPDATE ON "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + FOR EACH STATEMENT EXECUTE FUNCTION insert_id_100(); +-- error out because of the trigger +SELECT create_distributed_table('"NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"', null); +ERROR: cannot distribute relation "nullKeyTable.1!?!9012345678901234567890123456789012345678901234" because it has triggers +HINT: Consider dropping all the triggers on "nullKeyTable.1!?!9012345678901234567890123456789012345678901234" and retry. +SET citus.enable_unsafe_triggers TO ON; +RESET client_min_messages; +-- this shouldn't give any syntax errors +SELECT create_distributed_table('"NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"', null); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$"NULL_!_dist_key"."partition1_nullKeyTable.1!?!90123456789012345678901234567890123"$$) +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$"NULL_!_dist_key"."partition2_nullKeyTable.1!?!90123456789012345678901234567890123"$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- now we can add triggers on distributed tables, because we set the GUC to on +CREATE TRIGGER insert_100_trigger_2 + AFTER UPDATE ON null_key_table_with_trigger + FOR EACH STATEMENT EXECUTE FUNCTION insert_id_100(); +SET client_min_messages TO ERROR; +UPDATE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" SET "TeNANt_Id"="TeNANt_Id"+1; +-- we should see one row with id = 100 +SELECT COUNT(*) FROM "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" WHERE id = 100; + count +--------------------------------------------------------------------- + 1 +(1 row) + +-- create some objects after create_distributed_table +CREATE INDEX "my!Index2New" ON "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id) WITH ( fillfactor = 90 ) WHERE id < 20; +CREATE UNIQUE INDEX uniqueIndex2New ON "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id); +-- error out for already existing, because of the unique index +INSERT INTO "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" VALUES (1, 1, row_to_json(row(1,1), true)); +ERROR: duplicate key value violates unique constraint "partition1_nullKeyTable.1!?!901234567890123456_bf4a8ac1_1730056" +DETAIL: Key (id)=(X) already exists. +CONTEXT: while executing command on localhost:xxxxx +-- verify all 4 shard indexes are created on the same node +SELECT result FROM run_command_on_workers($$ + SELECT COUNT(*) FROM pg_indexes WHERE indexname LIKE '%my!Index_New_1%' OR indexname LIKE '%uniqueindex%new_1%';$$) + ORDER BY nodeport; + result +--------------------------------------------------------------------- + 4 + 0 +(2 rows) + +-- foreign key to a ref table +CREATE TABLE dummy_reference_table (a INT PRIMARY KEY); +SELECT create_reference_table('dummy_reference_table'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +TRUNCATE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"; +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + ADD CONSTRAINT fkey_to_dummy_ref FOREIGN KEY (id) REFERENCES dummy_reference_table(a); +BEGIN; -- try to add the same fkey, reversed + ALTER TABLE dummy_reference_table + ADD CONSTRAINT fkey_to_dummy_ref FOREIGN KEY (a) REFERENCES "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id); +ERROR: cannot create foreign key constraint since foreign keys from reference tables and local tables to distributed tables are not supported +DETAIL: Reference tables and local tables can only have foreign keys to reference tables and local tables +ROLLBACK; +-- errors out because of foreign key violation +INSERT INTO "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" VALUES (100, 1, row_to_json(row(1,1), true)); +ERROR: insert or update on table "partition100_nullKeyTable.1!?!9012345678901234_0aba0bf3_1730058" violates foreign key constraint "fkey_to_dummy_ref_1730055" +DETAIL: Key (id)=(X) is not present in table "dummy_reference_table_1730059". +CONTEXT: while executing command on localhost:xxxxx +-- now inserts successfully +INSERT INTO dummy_reference_table VALUES (100); +INSERT INTO "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" VALUES (100, 1, row_to_json(row(1,1), true)); +DELETE FROM "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" WHERE id = 100; +-- foreign key to a local table, errors out +CREATE TABLE local_table_for_fkey (a INT PRIMARY KEY); +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + ADD CONSTRAINT fkey_to_dummy_local FOREIGN KEY (id) REFERENCES local_table_for_fkey(a); +ERROR: referenced table "local_table_for_fkey" must be a distributed table or a reference table +DETAIL: To enforce foreign keys, the referencing and referenced rows need to be stored on the same node. +HINT: You could use SELECT create_reference_table('local_table_for_fkey') to replicate the referenced table to all nodes or consider dropping the foreign key +-- Normally, we support foreign keys from Postgres tables to distributed +-- tables assuming that the user will soon distribute the local table too +-- anyway. However, this is not the case for null-shard-key tables before +-- we improve SQL support. +ALTER TABLE local_table_for_fkey + ADD CONSTRAINT fkey_from_dummy_local FOREIGN KEY (a) REFERENCES "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +CONTEXT: SQL statement "SELECT fk."a" FROM ONLY "create_null_dist_key"."local_table_for_fkey" fk LEFT OUTER JOIN "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234" pk ON ( pk."id" OPERATOR(pg_catalog.=) fk."a") WHERE pk."id" IS NULL AND (fk."a" IS NOT NULL)" +-- foreign key to a citus local table, errors out +CREATE TABLE citus_local_table_for_fkey (a INT PRIMARY KEY); +SELECT citus_add_local_table_to_metadata('citus_local_table_for_fkey'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + ADD CONSTRAINT fkey_to_dummy_citus_local FOREIGN KEY (id) REFERENCES citus_local_table_for_fkey(a); +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 +-- reversed, still fails +ALTER TABLE citus_local_table_for_fkey + ADD CONSTRAINT fkey_from_dummy_citus_local FOREIGN KEY (a) REFERENCES "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id); +ERROR: cannot create foreign key constraint since foreign keys from reference tables and local tables to distributed tables are not supported +DETAIL: Reference tables and local tables can only have foreign keys to reference tables and local tables +-- foreign key to a distributed table, errors out because not colocated +CREATE TABLE dist_table_for_fkey (a INT PRIMARY KEY); +SELECT create_distributed_table('dist_table_for_fkey', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + ADD CONSTRAINT fkey_to_dummy_dist FOREIGN KEY (id) REFERENCES dist_table_for_fkey(a); +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 +-- reversed, still fails +ALTER TABLE dist_table_for_fkey + ADD CONSTRAINT fkey_to_dummy_dist FOREIGN KEY (a) REFERENCES "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id); +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 +-- create a null key distributed table, not colocated with the partitioned table, and then try to create a fkey +CREATE TABLE null_key_dist_not_colocated (a INT PRIMARY KEY); +SELECT create_distributed_table('null_key_dist_not_colocated', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + ADD CONSTRAINT fkey_to_dummy_dist FOREIGN KEY (id) REFERENCES null_key_dist_not_colocated(a); +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 +-- create a null key distributed table, colocated with the partitioned table, and then create a fkey +CREATE TABLE null_key_dist (a INT PRIMARY KEY); +SELECT create_distributed_table('null_key_dist', null, colocate_with=>'"NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + ADD CONSTRAINT fkey_to_dummy_dist FOREIGN KEY (id) REFERENCES null_key_dist(a); +-- check supported ON DELETE and ON UPDATE commands +ALTER TABLE null_key_dist ADD CONSTRAINT fkey_add_test_1 FOREIGN KEY(a) + REFERENCES "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id) ON DELETE SET DEFAULT; +ALTER TABLE null_key_dist ADD CONSTRAINT fkey_add_test_2 FOREIGN KEY(a) + REFERENCES "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id) ON UPDATE CASCADE; +ALTER TABLE null_key_dist ADD CONSTRAINT fkey_add_test_3 FOREIGN KEY(a) + REFERENCES dummy_reference_table(a) ON DELETE SET DEFAULT; +ALTER TABLE null_key_dist ADD CONSTRAINT fkey_add_test_4 FOREIGN KEY(a) + REFERENCES dummy_reference_table(a) ON UPDATE CASCADE; +ALTER TABLE null_key_dist DROP CONSTRAINT fkey_add_test_1; +ALTER TABLE null_key_dist DROP CONSTRAINT fkey_add_test_2; +ALTER TABLE null_key_dist DROP CONSTRAINT fkey_add_test_3; +ALTER TABLE null_key_dist DROP CONSTRAINT fkey_add_test_4; +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" DROP CONSTRAINT fkey_to_dummy_dist; +-- create a view that depends on the null shard key table +CREATE VIEW public.v1 AS SELECT * FROM null_key_dist; +SELECT * FROM public.v1; + a +--------------------------------------------------------------------- +(0 rows) + +DELETE FROM null_key_dist; +VACUUM null_key_dist; +TRUNCATE null_key_dist; +DROP TABLE null_key_dist CASCADE; +RESET client_min_messages; CREATE TABLE multi_level_partitioning_parent( measureid integer, eventdatetime date, @@ -771,7 +1128,7 @@ BEGIN; INSERT INTO referencing_table VALUES (1, 2); -- fails INSERT INTO referencing_table VALUES (2, 2); -ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730049" +ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730098" DETAIL: Key (a)=(2) is not present in table "referenced_table_xxxxxxx". CONTEXT: while executing command on localhost:xxxxx ROLLBACK; @@ -817,7 +1174,7 @@ BEGIN; INSERT INTO referencing_table VALUES (1, 2); -- fails INSERT INTO referencing_table VALUES (2, 2); -ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730085" +ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730134" DETAIL: Key (a)=(2) is not present in table "referenced_table_xxxxxxx". CONTEXT: while executing command on localhost:xxxxx ROLLBACK; @@ -935,8 +1292,8 @@ SELECT result, success FROM run_command_on_workers($$ $$); result | success --------------------------------------------------------------------- - ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730102" | f - ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730102" | f + ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730151" | f + ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730151" | f (2 rows) DROP TABLE referencing_table, referenced_table; @@ -951,8 +1308,8 @@ SELECT create_distributed_table('self_fkey_test', NULL, distribution_type=>null) INSERT INTO self_fkey_test VALUES (1, 1); -- ok INSERT INTO self_fkey_test VALUES (2, 3); -- fails -ERROR: insert or update on table "self_fkey_test_1730103" violates foreign key constraint "self_fkey_test_b_fkey_1730103" -DETAIL: Key (b)=(3) is not present in table "self_fkey_test_1730103". +ERROR: insert or update on table "self_fkey_test_1730152" violates foreign key constraint "self_fkey_test_b_fkey_1730152" +DETAIL: Key (b)=(3) is not present in table "self_fkey_test_1730152". CONTEXT: while executing command on localhost:xxxxx -- similar foreign key tests but this time create the referencing table later on -- referencing table is a null shard key table @@ -976,7 +1333,7 @@ BEGIN; INSERT INTO referencing_table VALUES (1, 2); -- fails INSERT INTO referencing_table VALUES (2, 2); -ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730105" +ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730154" DETAIL: Key (a)=(2) is not present in table "referenced_table_xxxxxxx". CONTEXT: while executing command on localhost:xxxxx ROLLBACK; @@ -999,7 +1356,7 @@ BEGIN; INSERT INTO referencing_table VALUES (2, 1); -- fails INSERT INTO referencing_table VALUES (1, 2); -ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_b_fkey_1730107" +ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_b_fkey_1730156" DETAIL: Key (a, b)=(1, 2) is not present in table "referenced_table_xxxxxxx". CONTEXT: while executing command on localhost:xxxxx ROLLBACK; @@ -1040,6 +1397,24 @@ BEGIN; SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'referenced_table'); ERROR: cannot create foreign key constraint since Citus does not support ON DELETE / UPDATE SET DEFAULT actions on the columns that default to sequences ROLLBACK; +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + CREATE TABLE referencing_table(a serial, b int); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'referenced_table'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + ALTER TABLE referencing_table ADD CONSTRAINT fkey_to_dummy_ref_on_update FOREIGN KEY (a) REFERENCES referenced_table(a) ON UPDATE SET DEFAULT; +ERROR: cannot create foreign key constraint since Citus does not support ON DELETE / UPDATE SET DEFAULT actions on the columns that default to sequences +ROLLBACK; -- to a non-colocated null dist key table BEGIN; CREATE TABLE referenced_table(a int UNIQUE, b int); @@ -1088,7 +1463,7 @@ BEGIN; INSERT INTO referencing_table VALUES (1, 2); -- fails INSERT INTO referencing_table VALUES (2, 2); -ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730146" +ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730197" DETAIL: Key (a)=(2) is not present in table "referenced_table_xxxxxxx". CONTEXT: while executing command on localhost:xxxxx ROLLBACK; @@ -1339,10 +1714,6 @@ CREATE TABLE trigger_table_1 (value int); CREATE TRIGGER trigger_1 BEFORE INSERT ON trigger_table_1 FOR EACH ROW EXECUTE FUNCTION increment_value(); -SELECT create_distributed_table('trigger_table_1', NULL, distribution_type=>null); -ERROR: cannot distribute relation "trigger_table_1" because it has triggers -HINT: Consider dropping all the triggers on "trigger_table_1" and retry. -SET citus.enable_unsafe_triggers TO ON; SELECT create_distributed_table('trigger_table_1', NULL, distribution_type=>null); create_distributed_table --------------------------------------------------------------------- @@ -1423,10 +1794,19 @@ TRUNCATE trigger_table_3; NOTICE: notice_truncate() CONTEXT: PL/pgSQL function notice_truncate() line XX at RAISE SET client_min_messages TO WARNING; +-- test rename, disable and drop trigger +ALTER TRIGGER trigger_4 ON trigger_table_3 RENAME TO trigger_new_name; +ALTER TABLE trigger_table_3 DISABLE TRIGGER ALL; +DROP TRIGGER trigger_new_name ON trigger_table_3; +-- enable the remaining triggers +ALTER TABLE trigger_table_3 ENABLE TRIGGER ALL; -- try a few simple queries at least to make sure that we don't crash BEGIN; INSERT INTO nullkey_c1_t1 SELECT * FROM nullkey_c2_t1; ROLLBACK; +DROP TRIGGER IF EXISTS trigger_1 ON trigger_table_1; +DROP TRIGGER trigger_2 ON trigger_table_2 CASCADE; +DROP TRIGGER trigger_3 ON trigger_table_3 RESTRICT; -- cleanup at exit SET client_min_messages TO ERROR; DROP SCHEMA create_null_dist_key, "NULL_!_dist_key" CASCADE; diff --git a/src/test/regress/multi_1_schedule b/src/test/regress/multi_1_schedule index a673a71d0..5d6fbb068 100644 --- a/src/test/regress/multi_1_schedule +++ b/src/test/regress/multi_1_schedule @@ -68,7 +68,7 @@ test: multi_master_protocol multi_load_data multi_load_data_superuser multi_beha test: multi_behavioral_analytics_basics multi_behavioral_analytics_single_shard_queries multi_insert_select_non_pushable_queries multi_insert_select multi_behavioral_analytics_create_table_superuser test: multi_shard_update_delete recursive_dml_with_different_planners_executors test: insert_select_repartition window_functions dml_recursive multi_insert_select_window -test: multi_insert_select_conflict citus_table_triggers +test: multi_insert_select_conflict citus_table_triggers alter_table_null_dist_key test: multi_row_insert insert_select_into_local_table alter_index # following should not run in parallel because it relies on connection counts to workers diff --git a/src/test/regress/sql/alter_table_null_dist_key.sql b/src/test/regress/sql/alter_table_null_dist_key.sql new file mode 100644 index 000000000..bcf0b4f74 --- /dev/null +++ b/src/test/regress/sql/alter_table_null_dist_key.sql @@ -0,0 +1,98 @@ +CREATE SCHEMA alter_null_dist_key; +SET search_path TO alter_null_dist_key; + +SET citus.next_shard_id TO 1720000; +SET citus.shard_count TO 32; +SET citus.shard_replication_factor TO 1; + +CREATE SEQUENCE dist_seq; +CREATE TABLE null_dist_table(a bigint DEFAULT nextval('dist_seq') UNIQUE, "b" text, c bigint GENERATED BY DEFAULT AS IDENTITY); +INSERT INTO null_dist_table("b") VALUES ('test'); +SELECT create_distributed_table('null_dist_table', null, colocate_with=>'none', distribution_type=>null); + +-- add column +ALTER TABLE null_dist_table ADD COLUMN d bigint DEFAULT 2; +SELECT * FROM null_dist_table ORDER BY c; + +-- alter default, set to 3 +ALTER TABLE null_dist_table ALTER COLUMN d SET DEFAULT 3; +INSERT INTO null_dist_table("b") VALUES ('test'); +SELECT * FROM null_dist_table ORDER BY c; + +-- drop default, see null +ALTER TABLE null_dist_table ALTER COLUMN d DROP DEFAULT; +INSERT INTO null_dist_table("b") VALUES ('test'); +SELECT * FROM null_dist_table ORDER BY c; + +-- cleanup the rows that were added to test the default behavior +DELETE FROM null_dist_table WHERE "b" = 'test' AND a > 1; + +-- alter column type +ALTER TABLE null_dist_table ALTER COLUMN d TYPE text; +UPDATE null_dist_table SET d = 'this is a text' WHERE d = '2'; +SELECT * FROM null_dist_table ORDER BY c; + +-- drop seq column +ALTER TABLE null_dist_table DROP COLUMN a; +SELECT * FROM null_dist_table ORDER BY c; + +-- add not null constraint +ALTER TABLE null_dist_table ALTER COLUMN b SET NOT NULL; + +-- not null constraint violation, error out +INSERT INTO null_dist_table VALUES (NULL, 2, 'test'); +-- drop not null constraint and try again +ALTER TABLE null_dist_table ALTER COLUMN b DROP NOT NULL; +INSERT INTO null_dist_table VALUES (NULL, 3, 'test'); +SELECT * FROM null_dist_table ORDER BY c; + +-- add exclusion constraint +ALTER TABLE null_dist_table ADD CONSTRAINT exc_b EXCLUDE USING btree (b with =); +-- rename the exclusion constraint, errors out +ALTER TABLE null_dist_table RENAME CONSTRAINT exc_b TO exc_b_1; +-- create exclusion constraint without a name +ALTER TABLE null_dist_table ADD EXCLUDE USING btree (b with =); + +-- test setting autovacuum option +ALTER TABLE null_dist_table SET (autovacuum_enabled = false); + +-- test multiple subcommands +ALTER TABLE null_dist_table ADD COLUMN int_column1 INTEGER, + DROP COLUMN d; + +SELECT * FROM null_dist_table ORDER BY c; + +-- test policy and row level security +CREATE TABLE null_dist_key_with_policy (table_user text); +INSERT INTO null_dist_key_with_policy VALUES ('user_1'); +SELECT create_distributed_table('null_dist_key_with_policy', null); + +-- enable rls +ALTER TABLE null_dist_key_with_policy ENABLE ROW LEVEL SECURITY; + +-- user_1 will be allowed to see the inserted row +CREATE ROLE user_1 WITH LOGIN; +GRANT ALL ON SCHEMA alter_null_dist_key TO user_1; +GRANT ALL ON TABLE alter_null_dist_key.null_dist_key_with_policy TO user_1; +CREATE POLICY table_policy ON null_dist_key_with_policy TO user_1 + USING (table_user = current_user); + +-- user_2 will not be allowed to see the inserted row +CREATE ROLE user_2 WITH LOGIN; +GRANT ALL ON SCHEMA alter_null_dist_key TO user_2; +GRANT ALL ON TABLE alter_null_dist_key.null_dist_key_with_policy TO user_2; +CREATE POLICY table_policy_1 ON null_dist_key_with_policy TO user_2 + USING (table_user = current_user); + +\c - user_1 - +SELECT * FROM alter_null_dist_key.null_dist_key_with_policy; +\c - user_2 - +SELECT * FROM alter_null_dist_key.null_dist_key_with_policy; +-- postgres will always be allowed to see the row as a superuser +\c - postgres - +SELECT * FROM alter_null_dist_key.null_dist_key_with_policy; + +-- cleanup +SET client_min_messages TO ERROR; +DROP SCHEMA alter_null_dist_key CASCADE; +DROP ROLE user_1, user_2; diff --git a/src/test/regress/sql/create_null_dist_key.sql b/src/test/regress/sql/create_null_dist_key.sql index b03cdde4d..9ca943d75 100644 --- a/src/test/regress/sql/create_null_dist_key.sql +++ b/src/test/regress/sql/create_null_dist_key.sql @@ -317,9 +317,11 @@ DROP TABLE null_dist_key_table_1, null_dist_key_table_2; -- create indexes before creating the null dist key tables -- .. for an initially empty table -CREATE TABLE null_dist_key_table_1(a int); +CREATE TABLE null_dist_key_table_1(a int, b int); +CREATE STATISTICS s1 (dependencies) ON a, b FROM null_dist_key_table_1; CREATE INDEX null_dist_key_table_1_idx ON null_dist_key_table_1(a); SELECT create_distributed_table('null_dist_key_table_1', null, colocate_with=>'none'); +CREATE STATISTICS s2 (dependencies) ON a, b FROM null_dist_key_table_1; -- .. and for another table having data in it before creating null dist key table CREATE TABLE null_dist_key_table_2(a int); @@ -327,6 +329,12 @@ INSERT INTO null_dist_key_table_2 VALUES(1); CREATE INDEX null_dist_key_table_2_idx ON null_dist_key_table_2(a); SELECT create_distributed_table('null_dist_key_table_2', null, colocate_with=>'none'); +-- test create index concurrently, then reindex +CREATE INDEX CONCURRENTLY ind_conc ON null_dist_key_table_2(a); +REINDEX INDEX ind_conc; +REINDEX INDEX CONCURRENTLY ind_conc; +DROP INDEX ind_conc; + SELECT * FROM null_dist_key_table_2 ORDER BY a; -- show that we do not support inheritance relationships @@ -347,11 +355,22 @@ BEGIN; CREATE POLICY table_policy ON null_dist_key_table_3 TO table_users USING (table_user = current_user); + GRANT ALL ON TABLE null_dist_key_table_3 TO table_users; + ALTER TABLE null_dist_key_table_3 OWNER TO table_users; + SELECT create_distributed_table('null_dist_key_table_3', null, colocate_with=>'none'); + + REVOKE ALL ON TABLE null_dist_key_table_3 FROM table_users; + ALTER TABLE null_dist_key_table_3 OWNER TO postgres; + GRANT ALL ON TABLE null_dist_key_table_3 TO table_users; ROLLBACK; +ALTER STATISTICS s2 SET STATISTICS 46; +ALTER TABLE null_dist_key_table_1 SET SCHEMA public; +DROP STATISTICS s1, s2; + -- drop them for next tests -DROP TABLE null_dist_key_table_1, null_dist_key_table_2, distributed_table; +DROP TABLE public.null_dist_key_table_1, null_dist_key_table_2, distributed_table; -- tests for object names that should be escaped properly @@ -438,6 +457,240 @@ SELECT create_distributed_table('sensors_2000', NULL, distribution_type=>null); SELECT create_distributed_table('sensors', NULL, distribution_type=>null); +-- verify we can create new partitions after distributing the parent table +CREATE TABLE sensors_2001 PARTITION OF sensors FOR VALUES FROM ('2001-01-01') TO ('2002-01-01'); + +-- verify we can attach to a null dist key table +CREATE TABLE sensors_2002 (measureid integer, eventdatetime date, measure_data jsonb, PRIMARY KEY (measureid, eventdatetime, measure_data)); +ALTER TABLE sensors ATTACH PARTITION sensors_2002 FOR VALUES FROM ('2002-01-01') TO ('2003-01-01'); + +-- verify we can detach from a null dist key table +ALTER TABLE sensors DETACH PARTITION sensors_2001; + +-- error out when attaching a noncolocated partition +CREATE TABLE sensors_2003 (measureid integer, eventdatetime date, measure_data jsonb, PRIMARY KEY (measureid, eventdatetime, measure_data)); +SELECT create_distributed_table('sensors_2003', NULL, distribution_type=>null, colocate_with=>'none'); +ALTER TABLE sensors ATTACH PARTITION sensors_2003 FOR VALUES FROM ('2003-01-01') TO ('2004-01-01'); +DROP TABLE sensors_2003; + +-- verify we can attach after distributing, if the parent and partition are colocated +CREATE TABLE sensors_2004 (measureid integer, eventdatetime date, measure_data jsonb, PRIMARY KEY (measureid, eventdatetime, measure_data)); +SELECT create_distributed_table('sensors_2004', NULL, distribution_type=>null, colocate_with=>'sensors'); +ALTER TABLE sensors ATTACH PARTITION sensors_2004 FOR VALUES FROM ('2004-01-01') TO ('2005-01-01'); + +-- verify we can attach a citus local table +CREATE TABLE sensors_2005 (measureid integer, eventdatetime date, measure_data jsonb, PRIMARY KEY (measureid, eventdatetime, measure_data)); +SELECT citus_add_local_table_to_metadata('sensors_2005'); +ALTER TABLE sensors ATTACH PARTITION sensors_2005 FOR VALUES FROM ('2005-01-01') TO ('2006-01-01'); + +-- check metadata +-- check all partitions and the parent on pg_dist_partition +SELECT logicalrelid::text FROM pg_dist_partition WHERE logicalrelid::text IN ('sensors', 'sensors_2000', 'sensors_2001', 'sensors_2002', 'sensors_2004', 'sensors_2005') ORDER BY logicalrelid::text; +-- verify they are all colocated +SELECT COUNT(DISTINCT(colocationid)) FROM pg_dist_partition WHERE logicalrelid::text IN ('sensors', 'sensors_2000', 'sensors_2001', 'sensors_2002', 'sensors_2004', 'sensors_2005'); +-- verify all partitions are placed on the same node +SELECT COUNT(DISTINCT(groupid)) FROM pg_dist_placement WHERE shardid IN + (SELECT shardid FROM pg_dist_shard WHERE logicalrelid::text IN ('sensors', 'sensors_2000', 'sensors_2001', 'sensors_2002', 'sensors_2004', 'sensors_2005')); + +-- verify the shard of sensors_2000 is attached to the parent shard, on the worker node +SELECT COUNT(*) FROM run_command_on_workers($$ + SELECT relpartbound FROM pg_class WHERE relname LIKE 'sensors_2000_1______';$$) + WHERE length(result) > 0; + +-- verify the shard of sensors_2001 is detached from the parent shard, on the worker node +SELECT COUNT(*) FROM run_command_on_workers($$ + SELECT relpartbound FROM pg_class WHERE relname LIKE 'sensors_2001_1______';$$) + WHERE length(result) > 0; + +-- verify the shard of sensors_2002 is attached to the parent shard, on the worker node +SELECT COUNT(*) FROM run_command_on_workers($$ + SELECT relpartbound FROM pg_class WHERE relname LIKE 'sensors_2002_1______';$$) + WHERE length(result) > 0; + +-- create a partitioned citus local table and verify we error out when attaching a partition with null dist key +CREATE TABLE partitioned_citus_local_tbl( + measureid integer, + eventdatetime date, + measure_data jsonb, +PRIMARY KEY (measureid, eventdatetime, measure_data)) +PARTITION BY RANGE(eventdatetime); +SELECT citus_add_local_table_to_metadata('partitioned_citus_local_tbl'); +CREATE TABLE partition_with_null_key (measureid integer, eventdatetime date, measure_data jsonb, PRIMARY KEY (measureid, eventdatetime, measure_data)); +SELECT create_distributed_table('partition_with_null_key', NULL, distribution_type=>null); +ALTER TABLE partitioned_citus_local_tbl ATTACH PARTITION partition_with_null_key FOR VALUES FROM ('2004-01-01') TO ('2005-01-01'); + +-- test partitioned tables + indexes with long names +CREATE TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"( + id int PRIMARY KEY, + "TeNANt_Id" int, + "jsondata" jsonb NOT NULL, + name text, + price numeric CHECK (price > 0), + serial_data bigserial, UNIQUE (id, price)) + PARTITION BY LIST(id); + +CREATE TABLE "NULL_!_dist_key"."partition1_nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + PARTITION OF "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + FOR VALUES IN (1); +CREATE TABLE "NULL_!_dist_key"."partition2_nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + PARTITION OF "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + FOR VALUES IN (2); +CREATE TABLE "NULL_!_dist_key"."partition100_nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + PARTITION OF "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + FOR VALUES IN (100); + +-- create some objects before create_distributed_table +CREATE INDEX "my!Index1New" ON "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id) WITH ( fillfactor = 80 ) WHERE id > 10; +CREATE UNIQUE INDEX uniqueIndexNew ON "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" (id); + +-- ingest some data before create_distributed_table +set client_min_messages to ERROR; +INSERT INTO "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" VALUES (1, 1, row_to_json(row(1,1), true)), + (2, 1, row_to_json(row(2,2), 'false')); +reset client_min_messages; +-- create a replica identity before create_distributed_table +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" REPLICA IDENTITY USING INDEX uniqueIndexNew; + +-- test triggers +SET client_min_messages TO ERROR; +CREATE FUNCTION insert_id_100() RETURNS trigger AS $insert_100$ +BEGIN + INSERT INTO "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" VALUES (100, 1, row_to_json(row(1,1), true)); + RETURN NEW; +END; +$insert_100$ LANGUAGE plpgsql; + +CREATE TABLE null_key_table_with_trigger(a INT); +SELECT create_distributed_table('null_key_table_with_trigger', null); +-- try to add a trigger after distributing the table, fails +CREATE TRIGGER insert_100_trigger + AFTER UPDATE ON null_key_table_with_trigger + FOR EACH STATEMENT EXECUTE FUNCTION insert_id_100(); + +-- now try to distribute a table that already has a trigger on it +CREATE TRIGGER insert_100_trigger + AFTER UPDATE ON "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + FOR EACH STATEMENT EXECUTE FUNCTION insert_id_100(); + +-- error out because of the trigger +SELECT create_distributed_table('"NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"', null); + +SET citus.enable_unsafe_triggers TO ON; +RESET client_min_messages; + +-- this shouldn't give any syntax errors +SELECT create_distributed_table('"NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"', null); + +-- now we can add triggers on distributed tables, because we set the GUC to on +CREATE TRIGGER insert_100_trigger_2 + AFTER UPDATE ON null_key_table_with_trigger + FOR EACH STATEMENT EXECUTE FUNCTION insert_id_100(); + +SET client_min_messages TO ERROR; +UPDATE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" SET "TeNANt_Id"="TeNANt_Id"+1; +-- we should see one row with id = 100 +SELECT COUNT(*) FROM "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" WHERE id = 100; + +-- create some objects after create_distributed_table +CREATE INDEX "my!Index2New" ON "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id) WITH ( fillfactor = 90 ) WHERE id < 20; +CREATE UNIQUE INDEX uniqueIndex2New ON "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id); + +-- error out for already existing, because of the unique index +INSERT INTO "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" VALUES (1, 1, row_to_json(row(1,1), true)); + +-- verify all 4 shard indexes are created on the same node +SELECT result FROM run_command_on_workers($$ + SELECT COUNT(*) FROM pg_indexes WHERE indexname LIKE '%my!Index_New_1%' OR indexname LIKE '%uniqueindex%new_1%';$$) + ORDER BY nodeport; + +-- foreign key to a ref table +CREATE TABLE dummy_reference_table (a INT PRIMARY KEY); +SELECT create_reference_table('dummy_reference_table'); +TRUNCATE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"; +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + ADD CONSTRAINT fkey_to_dummy_ref FOREIGN KEY (id) REFERENCES dummy_reference_table(a); +BEGIN; -- try to add the same fkey, reversed + ALTER TABLE dummy_reference_table + ADD CONSTRAINT fkey_to_dummy_ref FOREIGN KEY (a) REFERENCES "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id); +ROLLBACK; + +-- errors out because of foreign key violation +INSERT INTO "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" VALUES (100, 1, row_to_json(row(1,1), true)); + +-- now inserts successfully +INSERT INTO dummy_reference_table VALUES (100); +INSERT INTO "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" VALUES (100, 1, row_to_json(row(1,1), true)); +DELETE FROM "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" WHERE id = 100; + +-- foreign key to a local table, errors out +CREATE TABLE local_table_for_fkey (a INT PRIMARY KEY); +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + ADD CONSTRAINT fkey_to_dummy_local FOREIGN KEY (id) REFERENCES local_table_for_fkey(a); + +-- Normally, we support foreign keys from Postgres tables to distributed +-- tables assuming that the user will soon distribute the local table too +-- anyway. However, this is not the case for null-shard-key tables before +-- we improve SQL support. +ALTER TABLE local_table_for_fkey + ADD CONSTRAINT fkey_from_dummy_local FOREIGN KEY (a) REFERENCES "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id); + +-- foreign key to a citus local table, errors out +CREATE TABLE citus_local_table_for_fkey (a INT PRIMARY KEY); +SELECT citus_add_local_table_to_metadata('citus_local_table_for_fkey'); +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + ADD CONSTRAINT fkey_to_dummy_citus_local FOREIGN KEY (id) REFERENCES citus_local_table_for_fkey(a); +-- reversed, still fails +ALTER TABLE citus_local_table_for_fkey + ADD CONSTRAINT fkey_from_dummy_citus_local FOREIGN KEY (a) REFERENCES "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id); + +-- foreign key to a distributed table, errors out because not colocated +CREATE TABLE dist_table_for_fkey (a INT PRIMARY KEY); +SELECT create_distributed_table('dist_table_for_fkey', 'a'); +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + ADD CONSTRAINT fkey_to_dummy_dist FOREIGN KEY (id) REFERENCES dist_table_for_fkey(a); +-- reversed, still fails +ALTER TABLE dist_table_for_fkey + ADD CONSTRAINT fkey_to_dummy_dist FOREIGN KEY (a) REFERENCES "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id); + +-- create a null key distributed table, not colocated with the partitioned table, and then try to create a fkey +CREATE TABLE null_key_dist_not_colocated (a INT PRIMARY KEY); +SELECT create_distributed_table('null_key_dist_not_colocated', null, colocate_with=>'none'); +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + ADD CONSTRAINT fkey_to_dummy_dist FOREIGN KEY (id) REFERENCES null_key_dist_not_colocated(a); + +-- create a null key distributed table, colocated with the partitioned table, and then create a fkey +CREATE TABLE null_key_dist (a INT PRIMARY KEY); +SELECT create_distributed_table('null_key_dist', null, colocate_with=>'"NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"'); +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + ADD CONSTRAINT fkey_to_dummy_dist FOREIGN KEY (id) REFERENCES null_key_dist(a); + +-- check supported ON DELETE and ON UPDATE commands +ALTER TABLE null_key_dist ADD CONSTRAINT fkey_add_test_1 FOREIGN KEY(a) + REFERENCES "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id) ON DELETE SET DEFAULT; +ALTER TABLE null_key_dist ADD CONSTRAINT fkey_add_test_2 FOREIGN KEY(a) + REFERENCES "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id) ON UPDATE CASCADE; +ALTER TABLE null_key_dist ADD CONSTRAINT fkey_add_test_3 FOREIGN KEY(a) + REFERENCES dummy_reference_table(a) ON DELETE SET DEFAULT; +ALTER TABLE null_key_dist ADD CONSTRAINT fkey_add_test_4 FOREIGN KEY(a) + REFERENCES dummy_reference_table(a) ON UPDATE CASCADE; + +ALTER TABLE null_key_dist DROP CONSTRAINT fkey_add_test_1; +ALTER TABLE null_key_dist DROP CONSTRAINT fkey_add_test_2; +ALTER TABLE null_key_dist DROP CONSTRAINT fkey_add_test_3; +ALTER TABLE null_key_dist DROP CONSTRAINT fkey_add_test_4; +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" DROP CONSTRAINT fkey_to_dummy_dist; + +-- create a view that depends on the null shard key table +CREATE VIEW public.v1 AS SELECT * FROM null_key_dist; +SELECT * FROM public.v1; + +DELETE FROM null_key_dist; +VACUUM null_key_dist; +TRUNCATE null_key_dist; +DROP TABLE null_key_dist CASCADE; + +RESET client_min_messages; + CREATE TABLE multi_level_partitioning_parent( measureid integer, eventdatetime date, @@ -685,6 +938,15 @@ BEGIN; SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'referenced_table'); ROLLBACK; +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); + + CREATE TABLE referencing_table(a serial, b int); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'referenced_table'); + ALTER TABLE referencing_table ADD CONSTRAINT fkey_to_dummy_ref_on_update FOREIGN KEY (a) REFERENCES referenced_table(a) ON UPDATE SET DEFAULT; +ROLLBACK; + -- to a non-colocated null dist key table BEGIN; CREATE TABLE referenced_table(a int UNIQUE, b int); @@ -888,9 +1150,6 @@ FOR EACH ROW EXECUTE FUNCTION increment_value(); SELECT create_distributed_table('trigger_table_1', NULL, distribution_type=>null); -SET citus.enable_unsafe_triggers TO ON; -SELECT create_distributed_table('trigger_table_1', NULL, distribution_type=>null); - INSERT INTO trigger_table_1 VALUES(1), (2); SELECT * FROM trigger_table_1 ORDER BY value; @@ -952,11 +1211,22 @@ SET client_min_messages TO NOTICE; TRUNCATE trigger_table_3; SET client_min_messages TO WARNING; +-- test rename, disable and drop trigger +ALTER TRIGGER trigger_4 ON trigger_table_3 RENAME TO trigger_new_name; +ALTER TABLE trigger_table_3 DISABLE TRIGGER ALL; +DROP TRIGGER trigger_new_name ON trigger_table_3; +-- enable the remaining triggers +ALTER TABLE trigger_table_3 ENABLE TRIGGER ALL; + -- try a few simple queries at least to make sure that we don't crash BEGIN; INSERT INTO nullkey_c1_t1 SELECT * FROM nullkey_c2_t1; ROLLBACK; +DROP TRIGGER IF EXISTS trigger_1 ON trigger_table_1; +DROP TRIGGER trigger_2 ON trigger_table_2 CASCADE; +DROP TRIGGER trigger_3 ON trigger_table_3 RESTRICT; + -- cleanup at exit SET client_min_messages TO ERROR; DROP SCHEMA create_null_dist_key, "NULL_!_dist_key" CASCADE; From ac0ffc9839782478a4400dc77667b5101f9a6f64 Mon Sep 17 00:00:00 2001 From: Onur Tirtir Date: Wed, 22 Mar 2023 11:13:31 +0300 Subject: [PATCH 3/7] Add a config for arbitrary config tests where all the tables are null-shard-key tables (#6783/#6788) --- .../citus_arbitrary_configs.py | 11 +++++ src/test/regress/citus_tests/config.py | 46 +++++++++++++++++++ .../regress/expected/null_dist_key_prep.out | 13 ++++++ src/test/regress/null_dist_key_prep_schedule | 1 + src/test/regress/sql/null_dist_key_prep.sql | 14 ++++++ 5 files changed, 85 insertions(+) create mode 100644 src/test/regress/expected/null_dist_key_prep.out create mode 100644 src/test/regress/null_dist_key_prep_schedule create mode 100644 src/test/regress/sql/null_dist_key_prep.sql diff --git a/src/test/regress/citus_tests/arbitrary_configs/citus_arbitrary_configs.py b/src/test/regress/citus_tests/arbitrary_configs/citus_arbitrary_configs.py index 6c9863434..8785de8f7 100755 --- a/src/test/regress/citus_tests/arbitrary_configs/citus_arbitrary_configs.py +++ b/src/test/regress/citus_tests/arbitrary_configs/citus_arbitrary_configs.py @@ -76,6 +76,17 @@ def run_for_config(config, lock, sql_schedule_name): cfg.SUPER_USER_NAME, ) common.save_regression_diff("postgres", config.output_dir) + elif config.all_null_dist_key: + exitCode |= common.run_pg_regress_without_exit( + config.bindir, + config.pg_srcdir, + config.coordinator_port(), + cfg.NULL_DIST_KEY_PREP_SCHEDULE, + config.output_dir, + config.input_dir, + cfg.SUPER_USER_NAME, + ) + common.save_regression_diff("null_dist_key_prep_regression", config.output_dir) exitCode |= _run_pg_regress_on_port( config, config.coordinator_port(), cfg.CREATE_SCHEDULE diff --git a/src/test/regress/citus_tests/config.py b/src/test/regress/citus_tests/config.py index 3dc47671b..cd1be125b 100644 --- a/src/test/regress/citus_tests/config.py +++ b/src/test/regress/citus_tests/config.py @@ -22,6 +22,7 @@ ARBITRARY_SCHEDULE_NAMES = [ "sql_schedule", "sql_base_schedule", "postgres_schedule", + "null_dist_key_prep_schedule", ] BEFORE_PG_UPGRADE_SCHEDULE = "./before_pg_upgrade_schedule" @@ -29,6 +30,7 @@ AFTER_PG_UPGRADE_SCHEDULE = "./after_pg_upgrade_schedule" CREATE_SCHEDULE = "./create_schedule" POSTGRES_SCHEDULE = "./postgres_schedule" +NULL_DIST_KEY_PREP_SCHEDULE = "./null_dist_key_prep_schedule" SQL_SCHEDULE = "./sql_schedule" SQL_BASE_SCHEDULE = "./sql_base_schedule" @@ -101,6 +103,7 @@ class CitusBaseClusterConfig(object, metaclass=NewInitCaller): self.dbname = DATABASE_NAME self.is_mx = True self.is_citus = True + self.all_null_dist_key = False self.name = type(self).__name__ self.settings = { "shared_preload_libraries": "citus", @@ -203,6 +206,49 @@ class PostgresConfig(CitusDefaultClusterConfig): ] +class AllNullDistKeyDefaultConfig(CitusDefaultClusterConfig): + def __init__(self, arguments): + super().__init__(arguments) + self.all_null_dist_key = True + self.skip_tests += [ + # i) Skip the following tests because they require SQL support beyond + # router planner / supporting more DDL command types. + # + # group 1 + "dropped_columns_create_load", + "dropped_columns_1", + # group 2 + "distributed_planning_create_load", + "distributed_planning", + # group 4 + "views_create", + "views", + # group 5 + "intermediate_result_pruning_create", + "intermediate_result_pruning_queries_1", + "intermediate_result_pruning_queries_2", + # group 6 + "local_dist_join_load", + "local_dist_join", + "arbitrary_configs_recurring_outer_join", + # group 7 + "sequences_create", + "sequences", + # group 8 + "function_create", + "functions", + # group 9 + "merge_arbitrary_create", + "merge_arbitrary", + # group 10 + "arbitrary_configs_router_create", + "arbitrary_configs_router", + # + # ii) Skip the following test as it requires support for create_distributed_function. + "nested_execution", + ] + + class CitusSingleNodeClusterConfig(CitusDefaultClusterConfig): def __init__(self, arguments): super().__init__(arguments) diff --git a/src/test/regress/expected/null_dist_key_prep.out b/src/test/regress/expected/null_dist_key_prep.out new file mode 100644 index 000000000..7a861b06f --- /dev/null +++ b/src/test/regress/expected/null_dist_key_prep.out @@ -0,0 +1,13 @@ +ALTER FUNCTION create_distributed_table RENAME TO create_distributed_table_internal; +CREATE OR REPLACE FUNCTION pg_catalog.create_distributed_table(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 plpgsql +AS $function$ +BEGIN + PERFORM create_distributed_table_internal(table_name, NULL, NULL, colocate_with, NULL); +END; +$function$; diff --git a/src/test/regress/null_dist_key_prep_schedule b/src/test/regress/null_dist_key_prep_schedule new file mode 100644 index 000000000..1a43130ec --- /dev/null +++ b/src/test/regress/null_dist_key_prep_schedule @@ -0,0 +1 @@ +test: null_dist_key_prep diff --git a/src/test/regress/sql/null_dist_key_prep.sql b/src/test/regress/sql/null_dist_key_prep.sql new file mode 100644 index 000000000..5a9a3ac01 --- /dev/null +++ b/src/test/regress/sql/null_dist_key_prep.sql @@ -0,0 +1,14 @@ +ALTER FUNCTION create_distributed_table RENAME TO create_distributed_table_internal; + +CREATE OR REPLACE FUNCTION pg_catalog.create_distributed_table(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 plpgsql +AS $function$ +BEGIN + PERFORM create_distributed_table_internal(table_name, NULL, NULL, colocate_with, NULL); +END; +$function$; From 85745b46d5e98d5b5fa667a9bc30f5bd42ba9ad6 Mon Sep 17 00:00:00 2001 From: Onur Tirtir Date: Tue, 28 Mar 2023 18:47:59 +0300 Subject: [PATCH 4/7] Add initial sql support for distributed tables that don't have a shard key (#6773/#6822) Enable router planner and a limited version of INSERT .. SELECT planner for the queries that reference colocated null shard key tables. * SELECT / UPDATE / DELETE / MERGE is supported as long as it's a router query. * INSERT .. SELECT is supported as long as it only references colocated null shard key tables. Note that this is not only limited to distributed INSERT .. SELECT but also covers a limited set of query types that require pull-to-coordinator, e.g., due to LIMIT clause, generate_series() etc. ... (Ideally distributed INSERT .. SELECT could handle such queries too, e.g., when we're only referencing tables that don't have a shard key, but today this is not the case. See https://github.com/citusdata/citus/pull/6773#discussion_r1140130562. --- .../distributed/planner/distributed_planner.c | 32 + .../planner/fast_path_router_planner.c | 17 +- .../planner/insert_select_planner.c | 92 +- .../distributed/planner/merge_planner.c | 5 + .../planner/multi_logical_planner.c | 16 +- .../planner/multi_router_planner.c | 24 +- src/include/distributed/distributed_planner.h | 13 +- .../distributed/multi_logical_planner.h | 1 + .../distributed/multi_router_planner.h | 1 + src/test/regress/citus_tests/config.py | 6 - .../regress/expected/create_null_dist_key.out | 1 + src/test/regress/expected/merge.out | 147 ++ .../regress/expected/query_null_dist_key.out | 1796 +++++++++++++++++ src/test/regress/multi_1_schedule | 1 + src/test/regress/sql/merge.sql | 112 + src/test/regress/sql/query_null_dist_key.sql | 1132 +++++++++++ 16 files changed, 3375 insertions(+), 21 deletions(-) create mode 100644 src/test/regress/expected/query_null_dist_key.out create mode 100644 src/test/regress/sql/query_null_dist_key.sql diff --git a/src/backend/distributed/planner/distributed_planner.c b/src/backend/distributed/planner/distributed_planner.c index 38962b333..50509baea 100644 --- a/src/backend/distributed/planner/distributed_planner.c +++ b/src/backend/distributed/planner/distributed_planner.c @@ -1025,6 +1025,17 @@ CreateDistributedPlan(uint64 planId, bool allowRecursivePlanning, Query *origina { return distributedPlan; } + else if (ContainsNullDistKeyTable(originalQuery)) + { + /* + * We only support router queries if the query contains reference to + * a null-dist-key table. This temporary restriction will be removed + * once we support recursive planning for the queries that reference + * null-dist-key tables. + */ + WrapRouterErrorForNullDistKeyTable(distributedPlan->planningError); + RaiseDeferredError(distributedPlan->planningError, ERROR); + } else { RaiseDeferredError(distributedPlan->planningError, DEBUG2); @@ -2462,6 +2473,18 @@ HasUnresolvedExternParamsWalker(Node *expression, ParamListInfo boundParams) } +/* + * ContainsNullDistKeyTable returns true if given query contains reference + * to a null-dist-key table. + */ +bool +ContainsNullDistKeyTable(Query *query) +{ + RTEListProperties *rteListProperties = GetRTEListPropertiesForQuery(query); + return rteListProperties->hasDistTableWithoutShardKey; +} + + /* * GetRTEListPropertiesForQuery is a wrapper around GetRTEListProperties that * returns RTEListProperties for the rte list retrieved from query. @@ -2538,6 +2561,15 @@ GetRTEListProperties(List *rangeTableList) else if (IsCitusTableTypeCacheEntry(cacheEntry, DISTRIBUTED_TABLE)) { rteListProperties->hasDistributedTable = true; + + if (!HasDistributionKeyCacheEntry(cacheEntry)) + { + rteListProperties->hasDistTableWithoutShardKey = true; + } + else + { + rteListProperties->hasDistTableWithShardKey = true; + } } else { diff --git a/src/backend/distributed/planner/fast_path_router_planner.c b/src/backend/distributed/planner/fast_path_router_planner.c index ecb62478a..2be4a5626 100644 --- a/src/backend/distributed/planner/fast_path_router_planner.c +++ b/src/backend/distributed/planner/fast_path_router_planner.c @@ -212,6 +212,16 @@ FastPathRouterQuery(Query *query, Node **distributionKeyValue) return false; } + /* + * If the table doesn't have a distribution column, we don't need to + * check anything further. + */ + Var *distributionKey = PartitionColumn(distributedTableId, 1); + if (!distributionKey) + { + return true; + } + /* WHERE clause should not be empty for distributed tables */ if (joinTree == NULL || (IsCitusTableTypeCacheEntry(cacheEntry, DISTRIBUTED_TABLE) && joinTree->quals == @@ -220,13 +230,6 @@ FastPathRouterQuery(Query *query, Node **distributionKeyValue) return false; } - /* if that's a reference table, we don't need to check anything further */ - Var *distributionKey = PartitionColumn(distributedTableId, 1); - if (!distributionKey) - { - return true; - } - /* convert list of expressions into expression tree for further processing */ quals = joinTree->quals; if (quals != NULL && IsA(quals, List)) diff --git a/src/backend/distributed/planner/insert_select_planner.c b/src/backend/distributed/planner/insert_select_planner.c index 53fe58cdb..175f6bc6f 100644 --- a/src/backend/distributed/planner/insert_select_planner.c +++ b/src/backend/distributed/planner/insert_select_planner.c @@ -57,6 +57,7 @@ static DistributedPlan * CreateInsertSelectPlanInternal(uint64 planId, PlannerRestrictionContext * plannerRestrictionContext, ParamListInfo boundParams); +static void ErrorIfInsertSelectWithNullDistKeyNotSupported(Query *originalQuery); static DistributedPlan * CreateDistributedInsertSelectPlan(Query *originalQuery, PlannerRestrictionContext * plannerRestrictionContext); @@ -241,6 +242,12 @@ CreateInsertSelectPlanInternal(uint64 planId, Query *originalQuery, RaiseDeferredError(deferredError, ERROR); } + /* + * We support a limited set of INSERT .. SELECT queries if the query + * references a null-dist-key table. + */ + ErrorIfInsertSelectWithNullDistKeyNotSupported(originalQuery); + DistributedPlan *distributedPlan = CreateDistributedInsertSelectPlan(originalQuery, plannerRestrictionContext); @@ -260,6 +267,74 @@ CreateInsertSelectPlanInternal(uint64 planId, Query *originalQuery, } +/* + * ErrorIfInsertSelectWithNullDistKeyNotSupported throws an error if given INSERT + * .. SELECT query references a null-dist-key table (as the target table or in + * the SELECT clause) and is unsupported. + * + * Such an INSERT .. SELECT query is supported as long as the it only references + * a "colocated" set of null-dist-key tables, no other relation rte types. + */ +static void +ErrorIfInsertSelectWithNullDistKeyNotSupported(Query *originalQuery) +{ + RangeTblEntry *subqueryRte = ExtractSelectRangeTableEntry(originalQuery); + Query *subquery = subqueryRte->subquery; + RTEListProperties *subqueryRteListProperties = GetRTEListPropertiesForQuery(subquery); + + RangeTblEntry *insertRte = ExtractResultRelationRTEOrError(originalQuery); + Oid targetRelationId = insertRte->relid; + if (!IsCitusTableType(targetRelationId, NULL_KEY_DISTRIBUTED_TABLE) && + subqueryRteListProperties->hasDistTableWithoutShardKey) + { + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("cannot select from a distributed table that " + "does not have a shard key when inserting into " + "a different table type"))); + } + else if (IsCitusTableType(targetRelationId, NULL_KEY_DISTRIBUTED_TABLE)) + { + if (subqueryRteListProperties->hasPostgresLocalTable || + subqueryRteListProperties->hasReferenceTable || + subqueryRteListProperties->hasCitusLocalTable || + subqueryRteListProperties->hasDistTableWithShardKey || + subqueryRteListProperties->hasMaterializedView) + { + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("cannot select from different table types " + "when inserting into a distributed table " + "that does not have a shard key"))); + } + + if (!subqueryRteListProperties->hasDistTableWithoutShardKey) + { + /* + * This means that the SELECT doesn't reference any Citus tables, + * Postgres tables or materialized views but references a function + * call, a values claue etc., or a cte from INSERT. + * + * In that case, we rely on the common restrictions enforced by the + * INSERT .. SELECT planners. + */ + Assert(!NeedsDistributedPlanning(subquery)); + return; + } + + List *distributedRelationIdList = DistributedRelationIdList(subquery); + distributedRelationIdList = lappend_oid(distributedRelationIdList, + targetRelationId); + + if (!AllDistributedRelationsInListColocated(distributedRelationIdList)) + { + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("cannot select from a non-colocated distributed " + "table when inserting into a distributed table " + "that does not have a shard key"))); + } + } +} + + /* * CreateDistributedInsertSelectPlan creates a DistributedPlan for distributed * INSERT ... SELECT queries which could consist of multiple tasks. @@ -379,6 +454,16 @@ CreateInsertSelectIntoLocalTablePlan(uint64 planId, Query *insertSelectQuery, { RangeTblEntry *selectRte = ExtractSelectRangeTableEntry(insertSelectQuery); + RTEListProperties *selectRteListProperties = + GetRTEListPropertiesForQuery(selectRte->subquery); + if (selectRteListProperties->hasDistTableWithoutShardKey) + { + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("cannot select from a distributed table that " + "does not have a shard key when inserting into " + "a local table"))); + } + PrepareInsertSelectForCitusPlanner(insertSelectQuery); /* get the SELECT query (may have changed after PrepareInsertSelectForCitusPlanner) */ @@ -717,10 +802,7 @@ DistributedInsertSelectSupported(Query *queryTree, RangeTblEntry *insertRte, } else if (IsCitusTableType(targetRelationId, NULL_KEY_DISTRIBUTED_TABLE)) { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "distributed INSERT ... SELECT cannot target a distributed " - "table with a null shard key", - NULL, NULL); + /* we've already checked the subquery via ErrorIfInsertSelectWithNullDistKeyNotSupported */ } else { @@ -874,7 +956,7 @@ RouterModifyTaskForShardInterval(Query *originalQuery, */ RTEListProperties *subqueryRteListProperties = GetRTEListPropertiesForQuery( copiedSubquery); - if (subqueryRteListProperties->hasDistributedTable) + if (subqueryRteListProperties->hasDistTableWithShardKey) { AddPartitionKeyNotNullFilterToSelect(copiedSubquery); } diff --git a/src/backend/distributed/planner/merge_planner.c b/src/backend/distributed/planner/merge_planner.c index 5b39aeba6..930a44db8 100644 --- a/src/backend/distributed/planner/merge_planner.c +++ b/src/backend/distributed/planner/merge_planner.c @@ -509,6 +509,11 @@ InsertDistributionColumnMatchesSource(Oid targetRelationId, Query *query) return NULL; } + if (!HasDistributionKey(targetRelationId)) + { + return NULL; + } + bool foundDistributionColumn = false; MergeAction *action = NULL; foreach_ptr(action, query->mergeActionList) diff --git a/src/backend/distributed/planner/multi_logical_planner.c b/src/backend/distributed/planner/multi_logical_planner.c index d9322bf5e..7732b6c5e 100644 --- a/src/backend/distributed/planner/multi_logical_planner.c +++ b/src/backend/distributed/planner/multi_logical_planner.c @@ -272,7 +272,7 @@ TargetListOnPartitionColumn(Query *query, List *targetEntryList) if (!targetListOnPartitionColumn) { if (!FindNodeMatchingCheckFunctionInRangeTableList(query->rtable, - IsDistributedTableRTE)) + IsTableWithDistKeyRTE)) { targetListOnPartitionColumn = true; } @@ -379,6 +379,20 @@ IsReferenceTableRTE(Node *node) } +/* + * IsTableWithDistKeyRTE gets a node and returns true if the node + * is a range table relation entry that points to a distributed table + * that has a distribution column. + */ +bool +IsTableWithDistKeyRTE(Node *node) +{ + Oid relationId = NodeTryGetRteRelid(node); + return relationId != InvalidOid && IsCitusTable(relationId) && + HasDistributionKey(relationId); +} + + /* * FullCompositeFieldList gets a composite field list, and checks if all fields * of composite type are used in the list. diff --git a/src/backend/distributed/planner/multi_router_planner.c b/src/backend/distributed/planner/multi_router_planner.c index 97c2cecf6..47d11172f 100644 --- a/src/backend/distributed/planner/multi_router_planner.c +++ b/src/backend/distributed/planner/multi_router_planner.c @@ -258,6 +258,22 @@ CreateModifyPlan(Query *originalQuery, Query *query, } +/* + * WrapRouterErrorForNullDistKeyTable wraps given planning error with a + * generic error message if given query references a distributed table + * that doesn't have a distribution key. + */ +void +WrapRouterErrorForNullDistKeyTable(DeferredErrorMessage *planningError) +{ + planningError->detail = planningError->message; + planningError->message = pstrdup("queries that reference a distributed " + "table without a shard key can only " + "reference colocated distributed " + "tables or reference tables"); +} + + /* * CreateSingleTaskRouterSelectPlan creates a physical plan for given SELECT query. * The returned plan is a router task that returns query results from a single worker. @@ -1870,6 +1886,11 @@ RouterJob(Query *originalQuery, PlannerRestrictionContext *plannerRestrictionCon */ if (IsMergeQuery(originalQuery)) { + if (ContainsNullDistKeyTable(originalQuery)) + { + WrapRouterErrorForNullDistKeyTable(*planningError); + } + RaiseDeferredError(*planningError, ERROR); } else @@ -3855,7 +3876,8 @@ ErrorIfQueryHasUnroutableModifyingCTE(Query *queryTree) CitusTableCacheEntry *modificationTableCacheEntry = GetCitusTableCacheEntry(distributedTableId); - if (!HasDistributionKeyCacheEntry(modificationTableCacheEntry)) + if (!IsCitusTableTypeCacheEntry(modificationTableCacheEntry, + DISTRIBUTED_TABLE)) { return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, "cannot router plan modification of a non-distributed table", diff --git a/src/include/distributed/distributed_planner.h b/src/include/distributed/distributed_planner.h index 412859449..753504131 100644 --- a/src/include/distributed/distributed_planner.h +++ b/src/include/distributed/distributed_planner.h @@ -147,9 +147,19 @@ typedef struct RTEListProperties bool hasReferenceTable; bool hasCitusLocalTable; - /* includes hash, append and range partitioned tables */ + /* includes hash, null dist key, append and range partitioned tables */ bool hasDistributedTable; + /* + * Effectively, hasDistributedTable is equal to + * "hasDistTableWithShardKey || hasDistTableWithoutShardKey". + * + * We provide below two for the callers that want to know what kind of + * distributed tables that given query has references to. + */ + bool hasDistTableWithShardKey; + bool hasDistTableWithoutShardKey; + /* union of hasReferenceTable, hasCitusLocalTable and hasDistributedTable */ bool hasCitusTable; @@ -243,6 +253,7 @@ extern int32 BlessRecordExpression(Expr *expr); extern void DissuadePlannerFromUsingPlan(PlannedStmt *plan); extern PlannedStmt * FinalizePlan(PlannedStmt *localPlan, struct DistributedPlan *distributedPlan); +extern bool ContainsNullDistKeyTable(Query *query); extern RTEListProperties * GetRTEListPropertiesForQuery(Query *query); diff --git a/src/include/distributed/multi_logical_planner.h b/src/include/distributed/multi_logical_planner.h index 189170358..de4901ea2 100644 --- a/src/include/distributed/multi_logical_planner.h +++ b/src/include/distributed/multi_logical_planner.h @@ -200,6 +200,7 @@ extern bool IsCitusTableRTE(Node *node); extern bool IsDistributedOrReferenceTableRTE(Node *node); extern bool IsDistributedTableRTE(Node *node); extern bool IsReferenceTableRTE(Node *node); +extern bool IsTableWithDistKeyRTE(Node *node); extern bool IsCitusExtraDataContainerRelation(RangeTblEntry *rte); extern bool ContainsReadIntermediateResultFunction(Node *node); extern bool ContainsReadIntermediateResultArrayFunction(Node *node); diff --git a/src/include/distributed/multi_router_planner.h b/src/include/distributed/multi_router_planner.h index a255fd520..40d92fead 100644 --- a/src/include/distributed/multi_router_planner.h +++ b/src/include/distributed/multi_router_planner.h @@ -36,6 +36,7 @@ extern DistributedPlan * CreateRouterPlan(Query *originalQuery, Query *query, extern DistributedPlan * CreateModifyPlan(Query *originalQuery, Query *query, PlannerRestrictionContext * plannerRestrictionContext); +extern void WrapRouterErrorForNullDistKeyTable(DeferredErrorMessage *planningError); extern DeferredErrorMessage * PlanRouterQuery(Query *originalQuery, PlannerRestrictionContext * plannerRestrictionContext, diff --git a/src/test/regress/citus_tests/config.py b/src/test/regress/citus_tests/config.py index cd1be125b..69cc5599c 100644 --- a/src/test/regress/citus_tests/config.py +++ b/src/test/regress/citus_tests/config.py @@ -237,12 +237,6 @@ class AllNullDistKeyDefaultConfig(CitusDefaultClusterConfig): # group 8 "function_create", "functions", - # group 9 - "merge_arbitrary_create", - "merge_arbitrary", - # group 10 - "arbitrary_configs_router_create", - "arbitrary_configs_router", # # ii) Skip the following test as it requires support for create_distributed_function. "nested_execution", diff --git a/src/test/regress/expected/create_null_dist_key.out b/src/test/regress/expected/create_null_dist_key.out index af6e66f62..43120a454 100644 --- a/src/test/regress/expected/create_null_dist_key.out +++ b/src/test/regress/expected/create_null_dist_key.out @@ -1803,6 +1803,7 @@ ALTER TABLE trigger_table_3 ENABLE TRIGGER ALL; -- try a few simple queries at least to make sure that we don't crash BEGIN; INSERT INTO nullkey_c1_t1 SELECT * FROM nullkey_c2_t1; +ERROR: cannot select from a non-colocated distributed table when inserting into a distributed table that does not have a shard key ROLLBACK; DROP TRIGGER IF EXISTS trigger_1 ON trigger_table_1; DROP TRIGGER trigger_2 ON trigger_table_2 CASCADE; diff --git a/src/test/regress/expected/merge.out b/src/test/regress/expected/merge.out index 2196d966d..fd82efa8c 100644 --- a/src/test/regress/expected/merge.out +++ b/src/test/regress/expected/merge.out @@ -3228,6 +3228,153 @@ WHEN NOT MATCHED THEN INSERT VALUES(dist_source.id, dist_source.val); ERROR: For MERGE command, all the distributed tables must be colocated, for append/range distribution, colocation is not supported HINT: Consider using hash distribution instead +-- test merge with null shard key tables +CREATE SCHEMA query_null_dist_key; +SET search_path TO query_null_dist_key; +SET client_min_messages TO DEBUG2; +CREATE TABLE nullkey_c1_t1(a int, b int); +CREATE TABLE nullkey_c1_t2(a int, b int); +SELECT create_distributed_table('nullkey_c1_t1', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('nullkey_c1_t2', null, colocate_with=>'nullkey_c1_t1'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE nullkey_c2_t1(a int, b int); +CREATE TABLE nullkey_c2_t2(a int, b int); +SELECT create_distributed_table('nullkey_c2_t1', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('nullkey_c2_t2', null, colocate_with=>'nullkey_c2_t1', distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE reference_table(a int, b int); +SELECT create_reference_table('reference_table'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO reference_table SELECT i, i FROM generate_series(0, 5) i; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +CREATE TABLE distributed_table(a int, b int); +SELECT create_distributed_table('distributed_table', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO distributed_table SELECT i, i FROM generate_series(3, 8) i; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +CREATE TABLE citus_local_table(a int, b int); +SELECT citus_add_local_table_to_metadata('citus_local_table'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO citus_local_table SELECT i, i FROM generate_series(0, 10) i; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +CREATE TABLE postgres_local_table(a int, b int); +INSERT INTO postgres_local_table SELECT i, i FROM generate_series(5, 10) i; +-- with a colocated table +MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a) +WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b; +DEBUG: +DEBUG: Creating MERGE router plan +MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a) +WHEN MATCHED THEN DELETE; +DEBUG: +DEBUG: Creating MERGE router plan +MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a) +WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b +WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b); +DEBUG: +DEBUG: Creating MERGE router plan +MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a) +WHEN MATCHED THEN DELETE +WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b); +DEBUG: +DEBUG: Creating MERGE router plan +-- with non-colocated null-dist-key table +MERGE INTO nullkey_c1_t1 USING nullkey_c2_t1 ON (nullkey_c1_t1.a = nullkey_c2_t1.a) +WHEN MATCHED THEN UPDATE SET b = nullkey_c2_t1.b; +ERROR: For MERGE command, all the distributed tables must be colocated +MERGE INTO nullkey_c1_t1 USING nullkey_c2_t1 ON (nullkey_c1_t1.a = nullkey_c2_t1.a) +WHEN MATCHED THEN UPDATE SET b = nullkey_c2_t1.b +WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c2_t1.a, nullkey_c2_t1.b); +ERROR: For MERGE command, all the distributed tables must be colocated +-- with a distributed table +MERGE INTO nullkey_c1_t1 USING distributed_table ON (nullkey_c1_t1.a = distributed_table.a) +WHEN MATCHED THEN UPDATE SET b = distributed_table.b +WHEN NOT MATCHED THEN INSERT VALUES (distributed_table.a, distributed_table.b); +ERROR: For MERGE command, all the distributed tables must be colocated +MERGE INTO distributed_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = distributed_table.a) +WHEN MATCHED THEN DELETE +WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b); +ERROR: For MERGE command, all the distributed tables must be colocated +-- with a reference table +MERGE INTO nullkey_c1_t1 USING reference_table ON (nullkey_c1_t1.a = reference_table.a) +WHEN MATCHED THEN UPDATE SET b = reference_table.b; +ERROR: MERGE command is not supported on reference tables yet +MERGE INTO reference_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = reference_table.a) +WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t1.b +WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b); +ERROR: MERGE command is not supported on reference tables yet +-- with a citus local table +MERGE INTO nullkey_c1_t1 USING citus_local_table ON (nullkey_c1_t1.a = citus_local_table.a) +WHEN MATCHED THEN UPDATE SET b = citus_local_table.b; +ERROR: MERGE command is not supported with combination of distributed/local tables yet +MERGE INTO citus_local_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = citus_local_table.a) +WHEN MATCHED THEN DELETE; +ERROR: MERGE command is not supported with combination of distributed/local tables yet +-- with a postgres table +MERGE INTO nullkey_c1_t1 USING postgres_local_table ON (nullkey_c1_t1.a = postgres_local_table.a) +WHEN MATCHED THEN UPDATE SET b = postgres_local_table.b; +ERROR: MERGE command is not supported with combination of distributed/local tables yet +MERGE INTO postgres_local_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = postgres_local_table.a) +WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t1.b +WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b); +ERROR: MERGE command is not supported with combination of distributed/local tables yet +-- using ctes +WITH cte AS ( + SELECT * FROM nullkey_c1_t1 +) +MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a) +WHEN MATCHED THEN UPDATE SET b = cte.b; +DEBUG: +DEBUG: Creating MERGE router plan +WITH cte AS ( + SELECT * FROM distributed_table +) +MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a) +WHEN MATCHED THEN UPDATE SET b = cte.b; +ERROR: For MERGE command, all the distributed tables must be colocated +WITH cte AS materialized ( + SELECT * FROM distributed_table +) +MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a) +WHEN MATCHED THEN UPDATE SET b = cte.b; +ERROR: For MERGE command, all the distributed tables must be colocated +SET client_min_messages TO WARNING; +DROP SCHEMA query_null_dist_key CASCADE; +RESET client_min_messages; +SET search_path TO merge_schema; DROP SERVER foreign_server CASCADE; NOTICE: drop cascades to 3 other objects DETAIL: drop cascades to user mapping for postgres on server foreign_server diff --git a/src/test/regress/expected/query_null_dist_key.out b/src/test/regress/expected/query_null_dist_key.out new file mode 100644 index 000000000..09413a3ea --- /dev/null +++ b/src/test/regress/expected/query_null_dist_key.out @@ -0,0 +1,1796 @@ +CREATE SCHEMA query_null_dist_key; +SET search_path TO query_null_dist_key; +SET citus.next_shard_id TO 1620000; +SET citus.shard_count TO 32; +SET client_min_messages TO WARNING; +SELECT 1 FROM citus_add_node('localhost', :master_port, groupid => 0); + ?column? +--------------------------------------------------------------------- + 1 +(1 row) + +SET client_min_messages TO NOTICE; +CREATE TABLE nullkey_c1_t1(a int, b int); +CREATE TABLE nullkey_c1_t2(a int, b int); +SELECT create_distributed_table('nullkey_c1_t1', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('nullkey_c1_t2', null, colocate_with=>'nullkey_c1_t1'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO nullkey_c1_t1 SELECT i, i FROM generate_series(1, 8) i; +INSERT INTO nullkey_c1_t2 SELECT i, i FROM generate_series(2, 7) i; +CREATE TABLE nullkey_c2_t1(a int, b int); +CREATE TABLE nullkey_c2_t2(a int, b int); +SELECT create_distributed_table('nullkey_c2_t1', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('nullkey_c2_t2', null, colocate_with=>'nullkey_c2_t1', distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO nullkey_c2_t1 SELECT i, i FROM generate_series(2, 7) i; +INSERT INTO nullkey_c2_t2 SELECT i, i FROM generate_series(1, 8) i; +CREATE TABLE nullkey_c3_t1(a int, b int); +SELECT create_distributed_table('nullkey_c3_t1', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO nullkey_c3_t1 SELECT i, i FROM generate_series(1, 8) i; +CREATE TABLE reference_table(a int, b int); +SELECT create_reference_table('reference_table'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO reference_table SELECT i, i FROM generate_series(0, 5) i; +CREATE TABLE distributed_table(a int, b int); +SELECT create_distributed_table('distributed_table', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO distributed_table SELECT i, i FROM generate_series(3, 8) i; +CREATE TABLE citus_local_table(a int, b int); +SELECT citus_add_local_table_to_metadata('citus_local_table'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO citus_local_table SELECT i, i FROM generate_series(0, 10) i; +CREATE TABLE postgres_local_table(a int, b int); +INSERT INTO postgres_local_table SELECT i, i FROM generate_series(5, 10) i; +CREATE TABLE articles_hash ( + id bigint NOT NULL, + author_id bigint NOT NULL, + title varchar(20) NOT NULL, + word_count integer +); +INSERT INTO articles_hash VALUES ( 4, 4, 'altdorfer', 14551),( 5, 5, 'aruru', 11389), + (13, 3, 'aseyev', 2255),(15, 5, 'adversa', 3164), + (18, 8, 'assembly', 911),(19, 9, 'aubergiste', 4981), + (28, 8, 'aerophyte', 5454),(29, 9, 'amateur', 9524), + (42, 2, 'ausable', 15885),(43, 3, 'affixal', 12723), + (49, 9, 'anyone', 2681),(50, 10, 'anjanette', 19519); +SELECT create_distributed_table('articles_hash', null, colocate_with=>'none'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$query_null_dist_key.articles_hash$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE raw_events_first (user_id int, time timestamp, value_1 int, value_2 int, value_3 float, value_4 bigint, UNIQUE(user_id, value_1)); +SELECT create_distributed_table('raw_events_first', null, colocate_with=>'none', distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE raw_events_second (user_id int, time timestamp, value_1 int, value_2 int, value_3 float, value_4 bigint, UNIQUE(user_id, value_1)); +SELECT create_distributed_table('raw_events_second', null, colocate_with=>'raw_events_first', distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE agg_events (user_id int, value_1_agg int, value_2_agg int, value_3_agg float, value_4_agg bigint, agg_time timestamp, UNIQUE(user_id, value_1_agg)); +SELECT create_distributed_table('agg_events', null, colocate_with=>'raw_events_first', distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE users_ref_table (user_id int); +SELECT create_reference_table('users_ref_table'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO raw_events_first VALUES (1, '1970-01-01', 10, 100, 1000.1, 10000), (3, '1971-01-01', 30, 300, 3000.1, 30000), + (5, '1972-01-01', 50, 500, 5000.1, 50000), (2, '1973-01-01', 20, 200, 2000.1, 20000), + (4, '1974-01-01', 40, 400, 4000.1, 40000), (6, '1975-01-01', 60, 600, 6000.1, 60000); +CREATE TABLE modify_fast_path(key int, value_1 int, value_2 text); +SELECT create_distributed_table('modify_fast_path', null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE modify_fast_path_reference(key int, value_1 int, value_2 text); +SELECT create_reference_table('modify_fast_path_reference'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE bigserial_test (x int, y int, z bigserial); +SELECT create_distributed_table('bigserial_test', null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE append_table (text_col text, a int); +SELECT create_distributed_table('append_table', 'a', 'append'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT master_create_empty_shard('append_table') AS shardid1 \gset +SELECT master_create_empty_shard('append_table') AS shardid2 \gset +SELECT master_create_empty_shard('append_table') AS shardid3 \gset +COPY append_table (text_col, a) FROM STDIN WITH (format 'csv', append_to_shard :shardid1); +COPY append_table (text_col, a) FROM STDIN WITH (format 'csv', append_to_shard :shardid2); +CREATE TABLE range_table(a int, b int); +SELECT create_distributed_table('range_table', 'a', 'range'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CALL public.create_range_partitioned_shards('range_table', '{"0","25"}','{"24","49"}'); +INSERT INTO range_table VALUES (0, 1), (1, 2), (2, 3), (3, 4), (4, 5), (5, 6), (6, 50); +SET client_min_messages to DEBUG2; +-- simple insert +INSERT INTO nullkey_c1_t1 VALUES (1,2), (2,2), (3,4); +DEBUG: Creating router plan +INSERT INTO nullkey_c1_t2 VALUES (1,3), (3,4), (5,1), (6,2); +DEBUG: Creating router plan +INSERT INTO nullkey_c2_t1 VALUES (1,0), (2,5), (4,3), (5,2); +DEBUG: Creating router plan +INSERT INTO nullkey_c2_t2 VALUES (2,4), (3,2), (5,2), (7,4); +DEBUG: Creating router plan +-- simple select +SELECT * FROM nullkey_c1_t1 ORDER BY 1,2; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan + a | b +--------------------------------------------------------------------- + 1 | 1 + 1 | 2 + 2 | 2 + 2 | 2 + 3 | 3 + 3 | 4 + 4 | 4 + 5 | 5 + 6 | 6 + 7 | 7 + 8 | 8 +(11 rows) + +-- for update / share +SELECT * FROM modify_fast_path WHERE key = 1 FOR UPDATE; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan + key | value_1 | value_2 +--------------------------------------------------------------------- +(0 rows) + +SELECT * FROM modify_fast_path WHERE key = 1 FOR SHARE; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan + key | value_1 | value_2 +--------------------------------------------------------------------- +(0 rows) + +SELECT * FROM modify_fast_path FOR UPDATE; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan + key | value_1 | value_2 +--------------------------------------------------------------------- +(0 rows) + +SELECT * FROM modify_fast_path FOR SHARE; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan + key | value_1 | value_2 +--------------------------------------------------------------------- +(0 rows) + +-- cartesian product with different table types +-- with other table types +SELECT COUNT(*) FROM distributed_table d1, nullkey_c1_t1; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +SELECT COUNT(*) FROM reference_table d1, nullkey_c1_t1; +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 66 +(1 row) + +SELECT COUNT(*) FROM citus_local_table d1, nullkey_c1_t1; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +SELECT COUNT(*) FROM postgres_local_table d1, nullkey_c1_t1; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +-- with a colocated null dist key table +SELECT COUNT(*) FROM nullkey_c1_t1 d1, nullkey_c1_t2; +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 110 +(1 row) + +-- with a non-colocated null dist key table +SELECT COUNT(*) FROM nullkey_c1_t1 d1, nullkey_c2_t1; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: found no worker with all shard placements +-- First, show that nullkey_c1_t1 and nullkey_c3_t1 are not colocated. +SELECT + (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'query_null_dist_key.nullkey_c1_t1'::regclass) != + (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'query_null_dist_key.nullkey_c3_t1'::regclass); + ?column? +--------------------------------------------------------------------- + t +(1 row) + +-- Now verify that we can join them via router planner because it doesn't care +-- about whether two tables are colocated or not but physical location of shards +-- when citus.enable_non_colocated_router_query_pushdown is set to on. +SET citus.enable_non_colocated_router_query_pushdown TO ON; +SELECT COUNT(*) FROM nullkey_c1_t1 JOIN nullkey_c3_t1 USING(a); +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 11 +(1 row) + +SET citus.enable_non_colocated_router_query_pushdown TO OFF; +SELECT COUNT(*) FROM nullkey_c1_t1 JOIN nullkey_c3_t1 USING(a); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: router planner does not support queries that reference non-colocated distributed tables +RESET citus.enable_non_colocated_router_query_pushdown; +-- colocated join between null dist key tables +SELECT COUNT(*) FROM nullkey_c1_t1 JOIN nullkey_c1_t2 USING(a); +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 14 +(1 row) + +SELECT COUNT(*) FROM nullkey_c1_t1 LEFT JOIN nullkey_c1_t2 USING(a); +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 15 +(1 row) + +SELECT COUNT(*) FROM nullkey_c1_t1 FULL JOIN nullkey_c1_t2 USING(a); +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 15 +(1 row) + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +LEFT JOIN LATERAL ( + SELECT * FROM nullkey_c1_t2 t2 WHERE t2.b > t1.a +) q USING(a); +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 11 +(1 row) + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +JOIN LATERAL ( + SELECT * FROM nullkey_c1_t2 t2 WHERE t2.b > t1.a +) q USING(a); +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 4 +(1 row) + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE EXISTS ( + SELECT * FROM nullkey_c1_t2 t2 WHERE t2.b > t1.a +); +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 9 +(1 row) + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE t1.b IN ( + SELECT b+1 FROM nullkey_c1_t2 t2 WHERE t2.b = t1.a +); +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 2 +(1 row) + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE t1.b NOT IN ( + SELECT a FROM nullkey_c1_t2 t2 WHERE t2.b > t1.a +); +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 7 +(1 row) + +-- non-colocated inner joins between null dist key tables +SELECT * FROM nullkey_c1_t1 JOIN nullkey_c2_t1 USING(a) ORDER BY 1,2,3; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: found no worker with all shard placements +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +JOIN LATERAL ( + SELECT * FROM nullkey_c2_t2 t2 WHERE t2.b > t1.a +) q USING(a); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: found no worker with all shard placements +-- non-colocated outer joins between null dist key tables +SELECT * FROM nullkey_c1_t1 LEFT JOIN nullkey_c2_t2 USING(a) ORDER BY 1,2,3 LIMIT 4; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: found no worker with all shard placements +SELECT * FROM nullkey_c1_t1 FULL JOIN nullkey_c2_t2 USING(a) ORDER BY 1,2,3 LIMIT 4; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: found no worker with all shard placements +SELECT * FROM nullkey_c1_t1 t1 +LEFT JOIN LATERAL ( + SELECT * FROM nullkey_c2_t2 t2 WHERE t2.b > t1.a +) q USING(a) ORDER BY 1,2,3 OFFSET 3 LIMIT 4; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: found no worker with all shard placements +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +LEFT JOIN LATERAL ( + SELECT * FROM nullkey_c2_t2 t2 WHERE t2.b > t1.a +) q USING(a); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: found no worker with all shard placements +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE EXISTS ( + SELECT * FROM nullkey_c2_t2 t2 WHERE t2.b > t1.a +); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: found no worker with all shard placements +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE t1.b IN ( + SELECT b+1 FROM nullkey_c2_t2 t2 WHERE t2.b = t1.a +); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: found no worker with all shard placements +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE t1.b NOT IN ( + SELECT a FROM nullkey_c2_t2 t2 WHERE t2.b > t1.a +); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: found no worker with all shard placements +-- join with a reference table +SELECT COUNT(*) FROM nullkey_c1_t1, reference_table WHERE nullkey_c1_t1.a = reference_table.a; +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 8 +(1 row) + +WITH cte_1 AS + (SELECT * FROM nullkey_c1_t1, reference_table WHERE nullkey_c1_t1.a = reference_table.a ORDER BY 1,2,3,4 FOR UPDATE) +SELECT COUNT(*) FROM cte_1; +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 8 +(1 row) + +-- join with postgres / citus local tables +SELECT * FROM nullkey_c1_t1 JOIN postgres_local_table USING(a); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +SELECT * FROM nullkey_c1_t1 JOIN citus_local_table USING(a); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +-- join with a distributed table +SELECT * FROM distributed_table d1 JOIN nullkey_c1_t1 USING(a); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +JOIN LATERAL ( + SELECT * FROM distributed_table t2 WHERE t2.b > t1.a +) q USING(a); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +SELECT COUNT(*) FROM distributed_table t1 +JOIN LATERAL ( + SELECT * FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +) q USING(a); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +-- outer joins with different table types +SELECT COUNT(*) FROM nullkey_c1_t1 LEFT JOIN reference_table USING(a); +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 11 +(1 row) + +SELECT COUNT(*) FROM reference_table LEFT JOIN nullkey_c1_t1 USING(a); +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 9 +(1 row) + +SELECT COUNT(*) FROM nullkey_c1_t1 LEFT JOIN citus_local_table USING(a); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +SELECT COUNT(*) FROM citus_local_table LEFT JOIN nullkey_c1_t1 USING(a); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +SELECT COUNT(*) FROM nullkey_c1_t1 LEFT JOIN postgres_local_table USING(a); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +SELECT COUNT(*) FROM postgres_local_table LEFT JOIN nullkey_c1_t1 USING(a); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +SELECT COUNT(*) FROM nullkey_c1_t1 FULL JOIN citus_local_table USING(a); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +SELECT COUNT(*) FROM nullkey_c1_t1 FULL JOIN postgres_local_table USING(a); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +SELECT COUNT(*) FROM nullkey_c1_t1 FULL JOIN reference_table USING(a); +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 12 +(1 row) + +SELECT COUNT(*) FROM nullkey_c1_t1 JOIN append_table USING(a); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner does not support append-partitioned tables. +SELECT COUNT(*) FROM nullkey_c1_t1 JOIN range_table USING(a); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +SET citus.enable_non_colocated_router_query_pushdown TO ON; +SELECT COUNT(*) FROM nullkey_c1_t1 JOIN range_table USING(a) WHERE range_table.a = 20; +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 0 +(1 row) + +SET citus.enable_non_colocated_router_query_pushdown TO OFF; +SELECT COUNT(*) FROM nullkey_c1_t1 JOIN range_table USING(a) WHERE range_table.a = 20; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: router planner does not support queries that reference non-colocated distributed tables +RESET citus.enable_non_colocated_router_query_pushdown; +-- lateral / semi / anti joins with different table types +-- with a reference table +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +LEFT JOIN LATERAL ( + SELECT * FROM reference_table t2 WHERE t2.b > t1.a +) q USING(a); +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 11 +(1 row) + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE EXISTS ( + SELECT * FROM reference_table t2 WHERE t2.b > t1.a +); +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 7 +(1 row) + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE NOT EXISTS ( + SELECT * FROM reference_table t2 WHERE t2.b > t1.a +); +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 4 +(1 row) + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE t1.b IN ( + SELECT b+1 FROM reference_table t2 WHERE t2.b = t1.a +); +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 2 +(1 row) + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE t1.b NOT IN ( + SELECT a FROM reference_table t2 WHERE t2.b > t1.a +); +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 9 +(1 row) + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +JOIN LATERAL ( + SELECT * FROM reference_table t2 WHERE t2.b > t1.a +) q USING(a); +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 0 +(1 row) + +SELECT COUNT(*) FROM reference_table t1 +LEFT JOIN LATERAL ( + SELECT * FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +) q USING(a); +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 6 +(1 row) + +SELECT COUNT(*) FROM reference_table t1 +WHERE EXISTS ( + SELECT * FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +); +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 6 +(1 row) + +SELECT COUNT(*) FROM reference_table t1 +WHERE t1.b IN ( + SELECT b+1 FROM nullkey_c1_t1 t2 WHERE t2.b = t1.a +); +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 0 +(1 row) + +SELECT COUNT(*) FROM reference_table t1 +WHERE t1.b NOT IN ( + SELECT a FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +); +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 4 +(1 row) + +SELECT COUNT(*) FROM reference_table t1 +JOIN LATERAL ( + SELECT * FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +) q USING(a); +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 2 +(1 row) + +-- with a distributed table +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +LEFT JOIN LATERAL ( + SELECT * FROM distributed_table t2 WHERE t2.b > t1.a +) q USING(a); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE EXISTS ( + SELECT * FROM distributed_table t2 WHERE t2.b > t1.a +); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE NOT EXISTS ( + SELECT * FROM distributed_table t2 WHERE t2.b > t1.a +); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE t1.b IN ( + SELECT b+1 FROM distributed_table t2 WHERE t2.b = t1.a +); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE t1.b NOT IN ( + SELECT a FROM distributed_table t2 WHERE t2.b > t1.a +); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +SELECT COUNT(*) FROM distributed_table t1 +LEFT JOIN LATERAL ( + SELECT * FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +) q USING(a); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +SELECT COUNT(*) FROM distributed_table t1 +WHERE EXISTS ( + SELECT * FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +SELECT COUNT(*) FROM distributed_table t1 +WHERE t1.b IN ( + SELECT b+1 FROM nullkey_c1_t1 t2 WHERE t2.b = t1.a +); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +SELECT COUNT(*) FROM distributed_table t1 +WHERE t1.b NOT IN ( + SELECT a FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +-- with postgres / citus local tables +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +LEFT JOIN LATERAL ( + SELECT * FROM citus_local_table t2 WHERE t2.b > t1.a +) q USING(a); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE EXISTS ( + SELECT * FROM citus_local_table t2 WHERE t2.b > t1.a +); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE NOT EXISTS ( + SELECT * FROM citus_local_table t2 WHERE t2.b > t1.a +); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE t1.b IN ( + SELECT b+1 FROM citus_local_table t2 WHERE t2.b = t1.a +); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: found no worker with all shard placements +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE t1.b NOT IN ( + SELECT a FROM citus_local_table t2 WHERE t2.b > t1.a +); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: found no worker with all shard placements +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +JOIN LATERAL ( + SELECT * FROM citus_local_table t2 WHERE t2.b > t1.a +) q USING(a); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +SELECT COUNT(*) FROM citus_local_table t1 +LEFT JOIN LATERAL ( + SELECT * FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +) q USING(a); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +SELECT COUNT(*) FROM postgres_local_table t1 +LEFT JOIN LATERAL ( + SELECT * FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +) q USING(a); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +SELECT COUNT(*) FROM citus_local_table t1 +WHERE EXISTS ( + SELECT * FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +SELECT COUNT(*) FROM citus_local_table t1 +WHERE t1.b IN ( + SELECT b+1 FROM nullkey_c1_t1 t2 WHERE t2.b = t1.a +); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: found no worker with all shard placements +SELECT COUNT(*) FROM citus_local_table t1 +WHERE t1.b NOT IN ( + SELECT a FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: found no worker with all shard placements +SELECT COUNT(*) FROM citus_local_table t1 +JOIN LATERAL ( + SELECT * FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +) q USING(a); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +LEFT JOIN LATERAL ( + SELECT * FROM postgres_local_table t2 WHERE t2.b > t1.a +) q USING(a); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE EXISTS ( + SELECT * FROM postgres_local_table t2 WHERE t2.b > t1.a +); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE NOT EXISTS ( + SELECT * FROM postgres_local_table t2 WHERE t2.b > t1.a +); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE t1.b IN ( + SELECT b+1 FROM postgres_local_table t2 WHERE t2.b = t1.a +); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: found no worker with all shard placements +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE t1.b NOT IN ( + SELECT a FROM postgres_local_table t2 WHERE t2.b > t1.a +); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: found no worker with all shard placements +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +JOIN LATERAL ( + SELECT * FROM postgres_local_table t2 WHERE t2.b > t1.a +) q USING(a); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +SELECT COUNT(*) FROM postgres_local_table t1 +WHERE EXISTS ( + SELECT * FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +SELECT COUNT(*) FROM postgres_local_table t1 +WHERE t1.b IN ( + SELECT b+1 FROM nullkey_c1_t1 t2 WHERE t2.b = t1.a +); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: found no worker with all shard placements +SELECT COUNT(*) FROM postgres_local_table t1 +WHERE t1.b NOT IN ( + SELECT a FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: found no worker with all shard placements +SELECT COUNT(*) FROM postgres_local_table t1 +JOIN LATERAL ( + SELECT * FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +) q USING(a); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +-- insert .. select +-- between two colocated null dist key tables +-- The target list of "distributed statement"s that we send to workers +-- differ(*) in Postgres versions < 15. For this reason, we temporarily +-- disable debug messages here and run the EXPLAIN'ed version of the +-- command. +-- +-- (*): < SELECT a, b > vs < SELECT table_name.a, table_name.b > +SET client_min_messages TO WARNING; +EXPLAIN (ANALYZE TRUE, TIMING FALSE, COSTS FALSE, SUMMARY FALSE, VERBOSE FALSE) +INSERT INTO nullkey_c1_t1 SELECT * FROM nullkey_c1_t2; + QUERY PLAN +--------------------------------------------------------------------- + Custom Scan (Citus Adaptive) (actual rows=0 loops=1) + Task Count: 1 + Tasks Shown: All + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Insert on nullkey_c1_t1_1620000 citus_table_alias (actual rows=0 loops=1) + -> Seq Scan on nullkey_c1_t2_1620001 nullkey_c1_t2 (actual rows=10 loops=1) +(7 rows) + +SET client_min_messages TO DEBUG2; +-- between two non-colocated null dist key tables +INSERT INTO nullkey_c1_t1 SELECT * FROM nullkey_c2_t1; +ERROR: cannot select from a non-colocated distributed table when inserting into a distributed table that does not have a shard key +-- between a null dist key table and a table of different type +INSERT INTO nullkey_c1_t1 SELECT * FROM reference_table; +ERROR: cannot select from different table types when inserting into a distributed table that does not have a shard key +INSERT INTO nullkey_c1_t1 SELECT * FROM distributed_table; +ERROR: cannot select from different table types when inserting into a distributed table that does not have a shard key +INSERT INTO nullkey_c1_t1 SELECT * FROM citus_local_table; +ERROR: cannot select from different table types when inserting into a distributed table that does not have a shard key +INSERT INTO nullkey_c1_t1 SELECT * FROM postgres_local_table; +ERROR: cannot select from different table types when inserting into a distributed table that does not have a shard key +INSERT INTO reference_table SELECT * FROM nullkey_c1_t1; +ERROR: cannot select from a distributed table that does not have a shard key when inserting into a different table type +INSERT INTO distributed_table SELECT * FROM nullkey_c1_t1; +ERROR: cannot select from a distributed table that does not have a shard key when inserting into a different table type +INSERT INTO citus_local_table SELECT * FROM nullkey_c1_t1; +ERROR: cannot select from a distributed table that does not have a shard key when inserting into a different table type +INSERT INTO postgres_local_table SELECT * FROM nullkey_c1_t1; +ERROR: cannot select from a distributed table that does not have a shard key when inserting into a local table +-- test subquery +SELECT count(*) FROM +( + SELECT * FROM (SELECT * FROM nullkey_c1_t2) as subquery_inner +) AS subquery_top; +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 10 +(1 row) + +-- test cte inlining +WITH cte_nullkey_c1_t1 AS (SELECT * FROM nullkey_c1_t1), + cte_postgres_local_table AS (SELECT * FROM postgres_local_table), + cte_distributed_table AS (SELECT * FROM distributed_table) +SELECT COUNT(*) FROM cte_distributed_table, cte_nullkey_c1_t1, cte_postgres_local_table +WHERE cte_nullkey_c1_t1.a > 3 AND cte_distributed_table.a < 5; +DEBUG: CTE cte_nullkey_c1_t1 is going to be inlined via distributed planning +DEBUG: CTE cte_postgres_local_table is going to be inlined via distributed planning +DEBUG: CTE cte_distributed_table is going to be inlined via distributed planning +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +-- test recursive ctes +WITH level_0 AS ( + WITH level_1 AS ( + WITH RECURSIVE level_2_recursive(x) AS ( + VALUES (1) + UNION ALL + SELECT a + 1 FROM nullkey_c1_t1 JOIN level_2_recursive ON (a = x) WHERE a < 100 + ) + SELECT * FROM level_2_recursive RIGHT JOIN reference_table ON (level_2_recursive.x = reference_table.a) + ) + SELECT * FROM level_1 +) +SELECT COUNT(*) FROM level_0; +DEBUG: CTE level_0 is going to be inlined via distributed planning +DEBUG: CTE level_1 is going to be inlined via distributed planning +DEBUG: Creating router plan + count +--------------------------------------------------------------------- + 122 +(1 row) + +WITH level_0 AS ( + WITH level_1 AS ( + WITH RECURSIVE level_2_recursive(x) AS ( + VALUES (1) + UNION ALL + SELECT a + 1 FROM nullkey_c1_t1 JOIN level_2_recursive ON (a = x) WHERE a < 100 + ) + SELECT * FROM level_2_recursive JOIN distributed_table ON (level_2_recursive.x = distributed_table.a) + ) + SELECT * FROM level_1 +) +SELECT COUNT(*) FROM level_0; +DEBUG: CTE level_0 is going to be inlined via distributed planning +DEBUG: CTE level_1 is going to be inlined via distributed planning +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +-- grouping set +SELECT + id, substring(title, 2, 1) AS subtitle, count(*) + FROM articles_hash + WHERE author_id = 1 or author_id = 2 + GROUP BY GROUPING SETS ((id),(subtitle)) + ORDER BY id, subtitle; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan + id | subtitle | count +--------------------------------------------------------------------- + 42 | | 1 + | u | 1 +(2 rows) + +-- subquery in SELECT clause +SELECT a.title AS name, (SELECT a2.id FROM articles_hash a2 WHERE a.id = a2.id LIMIT 1) + AS special_price FROM articles_hash a +ORDER BY 1,2; +DEBUG: Creating router plan + name | special_price +--------------------------------------------------------------------- + adversa | 15 + aerophyte | 28 + affixal | 43 + altdorfer | 4 + amateur | 29 + anjanette | 50 + anyone | 49 + aruru | 5 + aseyev | 13 + assembly | 18 + aubergiste | 19 + ausable | 42 +(12 rows) + +-- test prepared statements +-- prepare queries can be router plannable +PREPARE author_1_articles as + SELECT * + FROM articles_hash + WHERE author_id = 1; +EXECUTE author_1_articles; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan + id | author_id | title | word_count +--------------------------------------------------------------------- +(0 rows) + +EXECUTE author_1_articles; + id | author_id | title | word_count +--------------------------------------------------------------------- +(0 rows) + +EXECUTE author_1_articles; + id | author_id | title | word_count +--------------------------------------------------------------------- +(0 rows) + +EXECUTE author_1_articles; + id | author_id | title | word_count +--------------------------------------------------------------------- +(0 rows) + +EXECUTE author_1_articles; + id | author_id | title | word_count +--------------------------------------------------------------------- +(0 rows) + +EXECUTE author_1_articles; + id | author_id | title | word_count +--------------------------------------------------------------------- +(0 rows) + +-- parametric prepare queries can be router plannable +PREPARE author_articles(int) as + SELECT * + FROM articles_hash + WHERE author_id = $1; +EXECUTE author_articles(1); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan + id | author_id | title | word_count +--------------------------------------------------------------------- +(0 rows) + +EXECUTE author_articles(1); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan + id | author_id | title | word_count +--------------------------------------------------------------------- +(0 rows) + +EXECUTE author_articles(1); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan + id | author_id | title | word_count +--------------------------------------------------------------------- +(0 rows) + +EXECUTE author_articles(1); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan + id | author_id | title | word_count +--------------------------------------------------------------------- +(0 rows) + +EXECUTE author_articles(1); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan + id | author_id | title | word_count +--------------------------------------------------------------------- +(0 rows) + +EXECUTE author_articles(1); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan + id | author_id | title | word_count +--------------------------------------------------------------------- +(0 rows) + +EXECUTE author_articles(NULL); + id | author_id | title | word_count +--------------------------------------------------------------------- +(0 rows) + +EXECUTE author_articles(NULL); + id | author_id | title | word_count +--------------------------------------------------------------------- +(0 rows) + +EXECUTE author_articles(NULL); + id | author_id | title | word_count +--------------------------------------------------------------------- +(0 rows) + +EXECUTE author_articles(NULL); + id | author_id | title | word_count +--------------------------------------------------------------------- +(0 rows) + +EXECUTE author_articles(NULL); + id | author_id | title | word_count +--------------------------------------------------------------------- +(0 rows) + +EXECUTE author_articles(NULL); + id | author_id | title | word_count +--------------------------------------------------------------------- +(0 rows) + +EXECUTE author_articles(NULL); + id | author_id | title | word_count +--------------------------------------------------------------------- +(0 rows) + +PREPARE author_articles_update(int) AS + UPDATE articles_hash SET title = 'test' WHERE author_id = $1; +EXECUTE author_articles_update(NULL); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE author_articles_update(NULL); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE author_articles_update(NULL); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE author_articles_update(NULL); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE author_articles_update(NULL); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE author_articles_update(NULL); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE author_articles_update(NULL); +-- More tests with insert .. select. +-- +-- The target list of "distributed statement"s that we send to workers +-- might differ(*) in Postgres versions < 15 and they are reported when +-- "log level >= DEBUG2". For this reason, we set log level to DEBUG1 to +-- avoid reporting them. +-- +-- DEBUG1 still allows reporting the reason why given INSERT .. SELECT +-- query is not distributed / requires pull-to-coordinator. +SET client_min_messages TO DEBUG1; +INSERT INTO bigserial_test (x, y) SELECT x, y FROM bigserial_test; +DEBUG: volatile functions are not allowed in distributed INSERT ... SELECT queries +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Sequences cannot be used in router queries +INSERT INTO agg_events + (user_id) +SELECT f2.id FROM +(SELECT + id +FROM (SELECT users_ref_table.user_id AS id + FROM raw_events_first, + users_ref_table + WHERE raw_events_first.user_id = users_ref_table.user_id ) AS foo) as f +INNER JOIN +(SELECT v4, + v1, + id +FROM (SELECT SUM(raw_events_second.value_4) AS v4, + SUM(raw_events_first.value_1) AS v1, + raw_events_second.user_id AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.user_id + GROUP BY raw_events_second.user_id + HAVING SUM(raw_events_second.value_4) > 1000) AS foo2 ) as f2 +ON (f.id = f2.id) +WHERE f.id IN (SELECT user_id + FROM raw_events_second); +ERROR: cannot select from different table types when inserting into a distributed table that does not have a shard key +-- upsert with returning +INSERT INTO agg_events AS ae + ( + user_id, + value_1_agg, + agg_time + ) +SELECT user_id, + value_1, + time +FROM raw_events_first +ON conflict (user_id, value_1_agg) +DO UPDATE + SET agg_time = EXCLUDED.agg_time + WHERE ae.agg_time < EXCLUDED.agg_time +RETURNING user_id, value_1_agg; + user_id | value_1_agg +--------------------------------------------------------------------- + 1 | 10 + 2 | 20 + 3 | 30 + 4 | 40 + 5 | 50 + 6 | 60 +(6 rows) + +-- using a left join +INSERT INTO agg_events (user_id) +SELECT + raw_events_first.user_id +FROM + raw_events_first LEFT JOIN raw_events_second ON raw_events_first.user_id = raw_events_second.user_id + WHERE raw_events_second.user_id = 10 OR raw_events_second.user_id = 11; +-- using a full join +INSERT INTO agg_events (user_id, value_1_agg) +SELECT t1.user_id AS col1, + t2.user_id AS col2 +FROM raw_events_first t1 + FULL JOIN raw_events_second t2 + ON t1.user_id = t2.user_id; +-- using semi join +INSERT INTO raw_events_second + (user_id) +SELECT user_id +FROM raw_events_first +WHERE user_id IN (SELECT raw_events_second.user_id + FROM raw_events_second, raw_events_first + WHERE raw_events_second.user_id = raw_events_first.user_id AND raw_events_first.user_id = 200); +-- using lateral join +INSERT INTO raw_events_second + (user_id) +SELECT user_id +FROM raw_events_first +WHERE NOT EXISTS (SELECT 1 + FROM raw_events_second + WHERE raw_events_second.user_id =raw_events_first.user_id); +-- using inner join +INSERT INTO agg_events (user_id) +SELECT raw_events_first.user_id +FROM raw_events_first INNER JOIN raw_events_second ON raw_events_first.user_id = raw_events_second.value_1 +WHERE raw_events_first.value_1 IN (10, 11,12) OR raw_events_second.user_id IN (1,2,3,4); +-- We could relax distributed insert .. select checks to allow pushing +-- down more clauses down to the worker nodes when inserting into a single +-- shard by selecting from a colocated one. We might want to do something +-- like https://github.com/citusdata/citus/pull/6772. +-- +-- e.g., insert into null_shard_key_1/citus_local/reference +-- select * from null_shard_key_1/citus_local/reference limit 1 +-- +-- Below "limit / offset clause" test and some others are examples of this. +-- limit / offset clause +INSERT INTO agg_events (user_id) SELECT raw_events_first.user_id FROM raw_events_first LIMIT 1; +DEBUG: LIMIT clauses are not allowed in distributed INSERT ... SELECT queries +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO agg_events (user_id) SELECT raw_events_first.user_id FROM raw_events_first OFFSET 1; +DEBUG: OFFSET clauses are not allowed in distributed INSERT ... SELECT queries +DEBUG: Collecting INSERT ... SELECT results on coordinator +-- using a materialized cte +WITH cte AS MATERIALIZED + (SELECT max(value_1)+1 as v1_agg, user_id FROM raw_events_first GROUP BY user_id) +INSERT INTO agg_events (value_1_agg, user_id) +SELECT v1_agg, user_id FROM cte; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO raw_events_second + WITH cte AS MATERIALIZED (SELECT * FROM raw_events_first) + SELECT user_id * 1000, time, value_1, value_2, value_3, value_4 FROM cte; +-- using a regular cte +WITH cte AS (SELECT * FROM raw_events_first) +INSERT INTO raw_events_second + SELECT user_id * 7000, time, value_1, value_2, value_3, value_4 FROM cte; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: CTE cte is going to be inlined via distributed planning +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO raw_events_second + WITH cte AS (SELECT * FROM raw_events_first) + SELECT * FROM cte; +DEBUG: CTE cte is going to be inlined via distributed planning +INSERT INTO agg_events + WITH sub_cte AS (SELECT 1) + SELECT + raw_events_first.user_id, (SELECT * FROM sub_cte) + FROM + raw_events_first; +DEBUG: CTE sub_cte is going to be inlined via distributed planning +DEBUG: Subqueries without relations are not allowed in distributed INSERT ... SELECT queries +DEBUG: Collecting INSERT ... SELECT results on coordinator +-- we still support complex joins via INSERT's cte list .. +WITH cte AS ( + SELECT reference_table.a AS a, 1 AS b + FROM distributed_table RIGHT JOIN reference_table USING (a) +) +INSERT INTO raw_events_second (user_id, value_1) + SELECT (a+5)*-1, b FROM cte; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: CTE cte is going to be inlined via distributed planning +DEBUG: recursively planning left side of the right join since the outer side is a recurring rel +DEBUG: recursively planning distributed relation "distributed_table" since it is part of a distributed join node that is outer joined with a recurring rel +DEBUG: Wrapping relation "distributed_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT a FROM query_null_dist_key.distributed_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT "?column?" AS user_id, b AS value_1 FROM (SELECT ((cte.a OPERATOR(pg_catalog.+) 5) OPERATOR(pg_catalog.*) '-1'::integer), cte.b FROM (SELECT reference_table.a, 1 AS b FROM ((SELECT distributed_table_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) distributed_table_1) distributed_table RIGHT JOIN query_null_dist_key.reference_table USING (a))) cte) citus_insert_select_subquery("?column?", b) +DEBUG: Collecting INSERT ... SELECT results on coordinator +-- .. but can't do so via via SELECT's cte list +INSERT INTO raw_events_second (user_id, value_1) +WITH cte AS ( + SELECT reference_table.a AS a, 1 AS b + FROM distributed_table RIGHT JOIN reference_table USING (a) +) + SELECT (a+5)*-1, b FROM cte; +DEBUG: CTE cte is going to be inlined via distributed planning +ERROR: cannot select from different table types when inserting into a distributed table that does not have a shard key +-- using set operations +INSERT INTO + raw_events_first(user_id) + (SELECT user_id FROM raw_events_first) INTERSECT + (SELECT user_id FROM raw_events_first); +DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries +DEBUG: Collecting INSERT ... SELECT results on coordinator +-- group by clause inside subquery +INSERT INTO agg_events + (user_id) +SELECT f2.id FROM +(SELECT + id +FROM (SELECT raw_events_second.user_id AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.user_id ) AS foo) as f +INNER JOIN +(SELECT v4, + v1, + id +FROM (SELECT SUM(raw_events_second.value_4) AS v4, + SUM(raw_events_first.value_1) AS v1, + raw_events_second.user_id AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.user_id + GROUP BY raw_events_second.user_id + HAVING SUM(raw_events_second.value_4) > 1000) AS foo2 ) as f2 +ON (f.id = f2.id) +WHERE f.id IN (SELECT user_id + FROM raw_events_second); +-- group by clause inside lateral subquery +INSERT INTO agg_events (user_id, value_4_agg) +SELECT + averages.user_id, avg(averages.value_4) +FROM + (SELECT + t1.user_id + FROM + raw_events_second t1 JOIN raw_events_second t2 on (t1.user_id = t2.user_id) + ) reference_ids + JOIN LATERAL + (SELECT + user_id, value_4 + FROM + raw_events_first) as averages ON averages.value_4 = reference_ids.user_id + GROUP BY averages.user_id; +-- using aggregates +INSERT INTO agg_events + (value_3_agg, + value_4_agg, + value_1_agg, + value_2_agg, + user_id) +SELECT SUM(value_3), + Count(value_4), + user_id, + SUM(value_1), + Avg(value_2) +FROM raw_events_first +GROUP BY user_id; +-- using generate_series +INSERT INTO raw_events_first (user_id, value_1, value_2) +SELECT s, s, s FROM generate_series(1, 5) s; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +CREATE SEQUENCE insert_select_test_seq; +-- nextval() expression in select's targetlist +INSERT INTO raw_events_first (user_id, value_1, value_2) +SELECT s, nextval('insert_select_test_seq'), (random()*10)::int +FROM generate_series(100, 105) s; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +-- non-immutable function +INSERT INTO modify_fast_path (key, value_1) VALUES (2,1) RETURNING value_1, random() * key; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: non-IMMUTABLE functions are not allowed in the RETURNING clause +SET client_min_messages TO DEBUG2; +-- update / delete +UPDATE nullkey_c1_t1 SET a = 1 WHERE b = 5; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +UPDATE nullkey_c1_t1 SET a = 1 WHERE a = 5; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +UPDATE nullkey_c1_t1 SET a = random(); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: functions used in UPDATE queries on distributed tables must not be VOLATILE +UPDATE nullkey_c1_t1 SET a = 1 WHERE a = random(); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: functions used in the WHERE/ON/WHEN clause of modification queries on distributed tables must not be VOLATILE +DELETE FROM nullkey_c1_t1 WHERE b = 5; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DELETE FROM nullkey_c1_t1 WHERE a = random(); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: functions used in the WHERE/ON/WHEN clause of modification queries on distributed tables must not be VOLATILE +-- simple update queries between different table types / colocated tables +UPDATE nullkey_c1_t1 SET b = 5 FROM nullkey_c1_t2 WHERE nullkey_c1_t1.b = nullkey_c1_t2.b; +DEBUG: Creating router plan +UPDATE nullkey_c1_t1 SET b = 5 FROM nullkey_c2_t1 WHERE nullkey_c1_t1.b = nullkey_c2_t1.b; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: found no worker with all shard placements +UPDATE nullkey_c1_t1 SET b = 5 FROM reference_table WHERE nullkey_c1_t1.b = reference_table.b; +DEBUG: Creating router plan +UPDATE nullkey_c1_t1 SET b = 5 FROM distributed_table WHERE nullkey_c1_t1.b = distributed_table.b; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns +UPDATE nullkey_c1_t1 SET b = 5 FROM distributed_table WHERE nullkey_c1_t1.b = distributed_table.a; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns +UPDATE nullkey_c1_t1 SET b = 5 FROM citus_local_table WHERE nullkey_c1_t1.b = citus_local_table.b; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: local table citus_local_table cannot be joined with these distributed tables +UPDATE nullkey_c1_t1 SET b = 5 FROM postgres_local_table WHERE nullkey_c1_t1.b = postgres_local_table.b; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: relation postgres_local_table is not distributed +UPDATE reference_table SET b = 5 FROM nullkey_c1_t1 WHERE nullkey_c1_t1.b = reference_table.b; +ERROR: cannot perform select on a distributed table and modify a reference table +UPDATE distributed_table SET b = 5 FROM nullkey_c1_t1 WHERE nullkey_c1_t1.b = distributed_table.b; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns +UPDATE distributed_table SET b = 5 FROM nullkey_c1_t1 WHERE nullkey_c1_t1.b = distributed_table.a; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns +UPDATE citus_local_table SET b = 5 FROM nullkey_c1_t1 WHERE nullkey_c1_t1.b = citus_local_table.b; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: local table citus_local_table cannot be joined with these distributed tables +UPDATE postgres_local_table SET b = 5 FROM nullkey_c1_t1 WHERE nullkey_c1_t1.b = postgres_local_table.b; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: relation postgres_local_table is not distributed +-- simple delete queries between different table types / colocated tables +DELETE FROM nullkey_c1_t1 USING nullkey_c1_t2 WHERE nullkey_c1_t1.b = nullkey_c1_t2.b; +DEBUG: Creating router plan +DELETE FROM nullkey_c1_t1 USING nullkey_c2_t1 WHERE nullkey_c1_t1.b = nullkey_c2_t1.b; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: found no worker with all shard placements +DELETE FROM nullkey_c1_t1 USING reference_table WHERE nullkey_c1_t1.b = reference_table.b; +DEBUG: Creating router plan +DELETE FROM nullkey_c1_t1 USING distributed_table WHERE nullkey_c1_t1.b = distributed_table.b; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns +DELETE FROM nullkey_c1_t1 USING distributed_table WHERE nullkey_c1_t1.b = distributed_table.a; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns +DELETE FROM nullkey_c1_t1 USING citus_local_table WHERE nullkey_c1_t1.b = citus_local_table.b; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: local table citus_local_table cannot be joined with these distributed tables +DELETE FROM nullkey_c1_t1 USING postgres_local_table WHERE nullkey_c1_t1.b = postgres_local_table.b; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: relation postgres_local_table is not distributed +DELETE FROM reference_table USING nullkey_c1_t1 WHERE nullkey_c1_t1.b = reference_table.b; +ERROR: cannot perform select on a distributed table and modify a reference table +DELETE FROM distributed_table USING nullkey_c1_t1 WHERE nullkey_c1_t1.b = distributed_table.b; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns +DELETE FROM distributed_table USING nullkey_c1_t1 WHERE nullkey_c1_t1.b = distributed_table.a; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns +DELETE FROM citus_local_table USING nullkey_c1_t1 WHERE nullkey_c1_t1.b = citus_local_table.b; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: local table citus_local_table cannot be joined with these distributed tables +DELETE FROM postgres_local_table USING nullkey_c1_t1 WHERE nullkey_c1_t1.b = postgres_local_table.b; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: relation postgres_local_table is not distributed +-- slightly more complex update queries +UPDATE nullkey_c1_t1 SET b = 5 WHERE nullkey_c1_t1.b IN (SELECT b FROM distributed_table); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns +WITH cte AS materialized( + SELECT * FROM distributed_table +) +UPDATE nullkey_c1_t1 SET b = 5 FROM cte WHERE nullkey_c1_t1.b = cte.a; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns +WITH cte AS ( + SELECT reference_table.a AS a, 1 AS b + FROM distributed_table RIGHT JOIN reference_table USING (a) +) +UPDATE nullkey_c1_t1 SET b = 5 WHERE nullkey_c1_t1.b IN (SELECT b FROM cte); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns +UPDATE nullkey_c1_t1 SET b = 5 FROM reference_table WHERE EXISTS ( + SELECT 1 FROM reference_table LEFT JOIN nullkey_c1_t1 USING (a) WHERE nullkey_c1_t1.b IS NULL +); +DEBUG: Creating router plan +UPDATE nullkey_c1_t1 tx SET b = ( + SELECT nullkey_c1_t2.b FROM nullkey_c1_t2 JOIN nullkey_c1_t1 ON (nullkey_c1_t1.a != nullkey_c1_t2.a) WHERE nullkey_c1_t1.a = tx.a ORDER BY 1 LIMIT 1 +); +DEBUG: Creating router plan +UPDATE nullkey_c1_t1 tx SET b = t2.b FROM nullkey_c1_t1 t1 JOIN nullkey_c1_t2 t2 ON (t1.a = t2.a); +DEBUG: Creating router plan +WITH cte AS ( + SELECT * FROM nullkey_c1_t2 ORDER BY 1,2 LIMIT 10 +) +UPDATE nullkey_c1_t1 SET b = 5 WHERE nullkey_c1_t1.a IN (SELECT b FROM cte); +DEBUG: Creating router plan +UPDATE modify_fast_path SET value_1 = value_1 + 12 * value_1 WHERE key = 1; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +UPDATE modify_fast_path SET value_1 = NULL WHERE value_1 = 15 AND (key = 1 OR value_2 = 'citus'); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +UPDATE modify_fast_path SET value_1 = 5 WHERE key = 2 RETURNING value_1 * 15, value_1::numeric * 16; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan + ?column? | ?column? +--------------------------------------------------------------------- +(0 rows) + +UPDATE modify_fast_path + SET value_1 = 1 + FROM modify_fast_path_reference + WHERE + modify_fast_path.key = modify_fast_path_reference.key AND + modify_fast_path.key = 1 AND + modify_fast_path_reference.key = 1; +DEBUG: Creating router plan +PREPARE p1 (int, int, int) AS + UPDATE modify_fast_path SET value_1 = value_1 + $1 WHERE key = $2 AND value_1 = $3; +EXECUTE p1(1,1,1); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE p1(2,2,2); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE p1(3,3,3); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE p1(4,4,4); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE p1(5,5,5); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE p1(6,6,6); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE p1(7,7,7); +PREPARE prepared_zero_shard_update(int) AS UPDATE modify_fast_path SET value_1 = 1 WHERE key = $1 AND false; +EXECUTE prepared_zero_shard_update(1); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE prepared_zero_shard_update(2); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE prepared_zero_shard_update(3); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE prepared_zero_shard_update(4); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE prepared_zero_shard_update(5); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE prepared_zero_shard_update(6); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE prepared_zero_shard_update(7); +-- slightly more complex delete queries +DELETE FROM nullkey_c1_t1 WHERE nullkey_c1_t1.b IN (SELECT b FROM distributed_table); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns +WITH cte AS materialized( + SELECT * FROM distributed_table +) +DELETE FROM nullkey_c1_t1 USING cte WHERE nullkey_c1_t1.b = cte.a; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns +WITH cte AS ( + SELECT reference_table.a AS a, 1 AS b + FROM distributed_table RIGHT JOIN reference_table USING (a) +) +DELETE FROM nullkey_c1_t1 WHERE nullkey_c1_t1.b IN (SELECT b FROM cte); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: complex joins are only supported when all distributed tables are co-located and joined on their distribution columns +DELETE FROM nullkey_c1_t1 USING reference_table WHERE EXISTS ( + SELECT 1 FROM reference_table LEFT JOIN nullkey_c1_t1 USING (a) WHERE nullkey_c1_t1.b IS NULL +); +DEBUG: Creating router plan +DELETE FROM nullkey_c1_t1 tx USING nullkey_c1_t1 t1 JOIN nullkey_c1_t2 t2 ON (t1.a = t2.a); +DEBUG: Creating router plan +WITH cte AS ( + SELECT * FROM nullkey_c1_t2 ORDER BY 1,2 LIMIT 10 +) +DELETE FROM nullkey_c1_t1 WHERE nullkey_c1_t1.a IN (SELECT b FROM cte); +DEBUG: Creating router plan +DELETE FROM modify_fast_path WHERE value_1 = 15 AND (key = 1 OR value_2 = 'citus'); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DELETE FROM modify_fast_path WHERE key = 2 RETURNING value_1 * 15, value_1::numeric * 16; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan + ?column? | ?column? +--------------------------------------------------------------------- +(0 rows) + +DELETE FROM modify_fast_path + USING modify_fast_path_reference + WHERE + modify_fast_path.key = modify_fast_path_reference.key AND + modify_fast_path.key = 1 AND + modify_fast_path_reference.key = 1; +DEBUG: Creating router plan +PREPARE p2 (int, int, int) AS + DELETE FROM modify_fast_path WHERE key = ($2)*$1 AND value_1 = $3; +EXECUTE p2(1,1,1); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE p2(2,2,2); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE p2(3,3,3); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE p2(4,4,4); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE p2(5,5,5); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE p2(6,6,6); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE p2(7,7,7); +PREPARE prepared_zero_shard_delete(int) AS DELETE FROM modify_fast_path WHERE key = $1 AND false; +EXECUTE prepared_zero_shard_delete(1); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE prepared_zero_shard_delete(2); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE prepared_zero_shard_delete(3); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE prepared_zero_shard_delete(4); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE prepared_zero_shard_delete(5); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE prepared_zero_shard_delete(6); +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +EXECUTE prepared_zero_shard_delete(7); +-- test modifying ctes +WITH cte AS ( + UPDATE modify_fast_path SET value_1 = value_1 + 1 WHERE key = 1 RETURNING * +) +SELECT * FROM cte; +DEBUG: Creating router plan + key | value_1 | value_2 +--------------------------------------------------------------------- +(0 rows) + +WITH cte AS ( + DELETE FROM modify_fast_path WHERE key = 1 RETURNING * +) +SELECT * FROM modify_fast_path; +DEBUG: Creating router plan + key | value_1 | value_2 +--------------------------------------------------------------------- +(0 rows) + +WITH cte AS ( + DELETE FROM modify_fast_path WHERE key = 1 RETURNING * +) +SELECT * FROM modify_fast_path_reference WHERE key IN (SELECT key FROM cte); +DEBUG: Creating router plan + key | value_1 | value_2 +--------------------------------------------------------------------- +(0 rows) + +WITH cte AS ( + DELETE FROM reference_table WHERE a = 1 RETURNING * +) +SELECT * FROM nullkey_c1_t1 WHERE a IN (SELECT a FROM cte); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: cannot router plan modification of a non-distributed table +WITH cte AS ( + DELETE FROM nullkey_c1_t1 WHERE a = 1 RETURNING * +) +SELECT * FROM nullkey_c1_t2 WHERE a IN (SELECT a FROM cte); +DEBUG: Creating router plan + a | b +--------------------------------------------------------------------- +(0 rows) + +WITH cte AS ( + DELETE FROM nullkey_c1_t1 WHERE a = 1 RETURNING * +) +SELECT * FROM nullkey_c2_t1 WHERE a IN (SELECT a FROM cte); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: found no worker with all shard placements +WITH cte AS ( + DELETE FROM nullkey_c1_t1 WHERE a = 1 RETURNING * +) +SELECT * FROM distributed_table WHERE a IN (SELECT a FROM cte); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +-- Below two queries fail very late when +-- citus.enable_non_colocated_router_query_pushdown is set to on. +SET citus.enable_non_colocated_router_query_pushdown TO ON; +WITH cte AS ( + DELETE FROM distributed_table WHERE a = 1 RETURNING * +) +SELECT * FROM nullkey_c1_t1 WHERE a IN (SELECT a FROM cte); +DEBUG: Creating router plan +DEBUG: query has a single distribution column value: 1 +ERROR: relation "query_null_dist_key.nullkey_c1_t1_1620000" does not exist +CONTEXT: while executing command on localhost:xxxxx +WITH cte AS ( + DELETE FROM distributed_table WHERE a = 1 RETURNING * +) +SELECT * FROM nullkey_c1_t1 WHERE b IN (SELECT b FROM cte); +DEBUG: Creating router plan +DEBUG: query has a single distribution column value: 1 +ERROR: relation "query_null_dist_key.nullkey_c1_t1_1620000" does not exist +CONTEXT: while executing command on localhost:xxxxx +SET citus.enable_non_colocated_router_query_pushdown TO OFF; +WITH cte AS ( + DELETE FROM distributed_table WHERE a = 1 RETURNING * +) +SELECT * FROM nullkey_c1_t1 WHERE a IN (SELECT a FROM cte); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: router planner does not support queries that reference non-colocated distributed tables +WITH cte AS ( + DELETE FROM distributed_table WHERE a = 1 RETURNING * +) +SELECT * FROM nullkey_c1_t1 WHERE b IN (SELECT b FROM cte); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: router planner does not support queries that reference non-colocated distributed tables +RESET citus.enable_non_colocated_router_query_pushdown; +WITH cte AS ( + UPDATE modify_fast_path SET value_1 = value_1 + 1 WHERE key = 1 RETURNING * +) +UPDATE modify_fast_path SET value_1 = value_1 + 1 WHERE key = 1; +DEBUG: Creating router plan +WITH cte AS ( + DELETE FROM modify_fast_path WHERE key = 1 RETURNING * +) +DELETE FROM modify_fast_path WHERE key = 1; +DEBUG: Creating router plan +-- test window functions +SELECT + user_id, avg(avg(value_3)) OVER (PARTITION BY user_id, MIN(value_2)) +FROM + raw_events_first +GROUP BY + 1 +ORDER BY + 2 DESC NULLS LAST, 1 DESC; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan + user_id | avg +--------------------------------------------------------------------- + 6 | 6000.1 + 5 | 5000.1 + 4 | 4000.1 + 3 | 3000.1 + 2 | 2000.1 + 1 | 1000.1 + 105 | + 104 | + 103 | + 102 | + 101 | + 100 | +(12 rows) + +SELECT + user_id, max(value_1) OVER (PARTITION BY user_id, MIN(value_2)) +FROM ( + SELECT + DISTINCT us.user_id, us.value_2, us.value_1, random() as r1 + FROM + raw_events_first as us, raw_events_second + WHERE + us.user_id = raw_events_second.user_id + ORDER BY + user_id, value_2 + ) s +GROUP BY + 1, value_1 +ORDER BY + 2 DESC, 1; +DEBUG: Creating router plan + user_id | max +--------------------------------------------------------------------- + 1 | + 2 | + 3 | + 4 | + 5 | + 6 | + 6 | 60 + 5 | 50 + 4 | 40 + 3 | 30 + 2 | 20 + 1 | 10 + 5 | 5 + 4 | 4 + 3 | 3 + 2 | 2 + 1 | 1 +(17 rows) + +SELECT + DISTINCT ON (raw_events_second.user_id, rnk) raw_events_second.user_id, rank() OVER my_win AS rnk +FROM + raw_events_second, raw_events_first +WHERE + raw_events_first.user_id = raw_events_second.user_id +WINDOW + my_win AS (PARTITION BY raw_events_second.user_id, raw_events_first.value_1 ORDER BY raw_events_second.time DESC) +ORDER BY + rnk DESC, 1 DESC +LIMIT 10; +DEBUG: Creating router plan + user_id | rnk +--------------------------------------------------------------------- + 6 | 2 + 5 | 2 + 4 | 2 + 3 | 2 + 2 | 2 + 1 | 2 + 6 | 1 + 5 | 1 + 4 | 1 + 3 | 1 +(10 rows) + +SET client_min_messages TO ERROR; +DROP SCHEMA query_null_dist_key CASCADE; +SELECT citus_remove_node('localhost', :master_port); + citus_remove_node +--------------------------------------------------------------------- + +(1 row) + diff --git a/src/test/regress/multi_1_schedule b/src/test/regress/multi_1_schedule index 5d6fbb068..45adf469e 100644 --- a/src/test/regress/multi_1_schedule +++ b/src/test/regress/multi_1_schedule @@ -200,6 +200,7 @@ test: local_table_join test: local_dist_join_mixed test: citus_local_dist_joins test: recurring_outer_join +test: query_null_dist_key test: pg_dump # --------- diff --git a/src/test/regress/sql/merge.sql b/src/test/regress/sql/merge.sql index 5b9190516..1fdc3a514 100644 --- a/src/test/regress/sql/merge.sql +++ b/src/test/regress/sql/merge.sql @@ -2051,6 +2051,118 @@ UPDATE SET val = dist_source.val WHEN NOT MATCHED THEN INSERT VALUES(dist_source.id, dist_source.val); +-- test merge with null shard key tables + +CREATE SCHEMA query_null_dist_key; + +SET search_path TO query_null_dist_key; +SET client_min_messages TO DEBUG2; + +CREATE TABLE nullkey_c1_t1(a int, b int); +CREATE TABLE nullkey_c1_t2(a int, b int); +SELECT create_distributed_table('nullkey_c1_t1', null, colocate_with=>'none'); +SELECT create_distributed_table('nullkey_c1_t2', null, colocate_with=>'nullkey_c1_t1'); + +CREATE TABLE nullkey_c2_t1(a int, b int); +CREATE TABLE nullkey_c2_t2(a int, b int); +SELECT create_distributed_table('nullkey_c2_t1', null, colocate_with=>'none'); +SELECT create_distributed_table('nullkey_c2_t2', null, colocate_with=>'nullkey_c2_t1', distribution_type=>null); + +CREATE TABLE reference_table(a int, b int); +SELECT create_reference_table('reference_table'); +INSERT INTO reference_table SELECT i, i FROM generate_series(0, 5) i; + +CREATE TABLE distributed_table(a int, b int); +SELECT create_distributed_table('distributed_table', 'a'); +INSERT INTO distributed_table SELECT i, i FROM generate_series(3, 8) i; + +CREATE TABLE citus_local_table(a int, b int); +SELECT citus_add_local_table_to_metadata('citus_local_table'); +INSERT INTO citus_local_table SELECT i, i FROM generate_series(0, 10) i; + +CREATE TABLE postgres_local_table(a int, b int); +INSERT INTO postgres_local_table SELECT i, i FROM generate_series(5, 10) i; + +-- with a colocated table +MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a) +WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b; + +MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a) +WHEN MATCHED THEN DELETE; + +MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a) +WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b +WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b); + +MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a) +WHEN MATCHED THEN DELETE +WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b); + +-- with non-colocated null-dist-key table +MERGE INTO nullkey_c1_t1 USING nullkey_c2_t1 ON (nullkey_c1_t1.a = nullkey_c2_t1.a) +WHEN MATCHED THEN UPDATE SET b = nullkey_c2_t1.b; + +MERGE INTO nullkey_c1_t1 USING nullkey_c2_t1 ON (nullkey_c1_t1.a = nullkey_c2_t1.a) +WHEN MATCHED THEN UPDATE SET b = nullkey_c2_t1.b +WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c2_t1.a, nullkey_c2_t1.b); + +-- with a distributed table +MERGE INTO nullkey_c1_t1 USING distributed_table ON (nullkey_c1_t1.a = distributed_table.a) +WHEN MATCHED THEN UPDATE SET b = distributed_table.b +WHEN NOT MATCHED THEN INSERT VALUES (distributed_table.a, distributed_table.b); + +MERGE INTO distributed_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = distributed_table.a) +WHEN MATCHED THEN DELETE +WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b); + +-- with a reference table +MERGE INTO nullkey_c1_t1 USING reference_table ON (nullkey_c1_t1.a = reference_table.a) +WHEN MATCHED THEN UPDATE SET b = reference_table.b; + +MERGE INTO reference_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = reference_table.a) +WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t1.b +WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b); + +-- with a citus local table +MERGE INTO nullkey_c1_t1 USING citus_local_table ON (nullkey_c1_t1.a = citus_local_table.a) +WHEN MATCHED THEN UPDATE SET b = citus_local_table.b; + +MERGE INTO citus_local_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = citus_local_table.a) +WHEN MATCHED THEN DELETE; + +-- with a postgres table +MERGE INTO nullkey_c1_t1 USING postgres_local_table ON (nullkey_c1_t1.a = postgres_local_table.a) +WHEN MATCHED THEN UPDATE SET b = postgres_local_table.b; + +MERGE INTO postgres_local_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = postgres_local_table.a) +WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t1.b +WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b); + +-- using ctes +WITH cte AS ( + SELECT * FROM nullkey_c1_t1 +) +MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a) +WHEN MATCHED THEN UPDATE SET b = cte.b; + +WITH cte AS ( + SELECT * FROM distributed_table +) +MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a) +WHEN MATCHED THEN UPDATE SET b = cte.b; + +WITH cte AS materialized ( + SELECT * FROM distributed_table +) +MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a) +WHEN MATCHED THEN UPDATE SET b = cte.b; + +SET client_min_messages TO WARNING; +DROP SCHEMA query_null_dist_key CASCADE; + +RESET client_min_messages; +SET search_path TO merge_schema; + DROP SERVER foreign_server CASCADE; DROP FUNCTION merge_when_and_write(); DROP SCHEMA merge_schema CASCADE; diff --git a/src/test/regress/sql/query_null_dist_key.sql b/src/test/regress/sql/query_null_dist_key.sql new file mode 100644 index 000000000..f5d1fe3fc --- /dev/null +++ b/src/test/regress/sql/query_null_dist_key.sql @@ -0,0 +1,1132 @@ +CREATE SCHEMA query_null_dist_key; +SET search_path TO query_null_dist_key; + +SET citus.next_shard_id TO 1620000; +SET citus.shard_count TO 32; + +SET client_min_messages TO WARNING; +SELECT 1 FROM citus_add_node('localhost', :master_port, groupid => 0); + +SET client_min_messages TO NOTICE; + +CREATE TABLE nullkey_c1_t1(a int, b int); +CREATE TABLE nullkey_c1_t2(a int, b int); +SELECT create_distributed_table('nullkey_c1_t1', null, colocate_with=>'none'); +SELECT create_distributed_table('nullkey_c1_t2', null, colocate_with=>'nullkey_c1_t1'); +INSERT INTO nullkey_c1_t1 SELECT i, i FROM generate_series(1, 8) i; +INSERT INTO nullkey_c1_t2 SELECT i, i FROM generate_series(2, 7) i; + +CREATE TABLE nullkey_c2_t1(a int, b int); +CREATE TABLE nullkey_c2_t2(a int, b int); +SELECT create_distributed_table('nullkey_c2_t1', null, colocate_with=>'none'); +SELECT create_distributed_table('nullkey_c2_t2', null, colocate_with=>'nullkey_c2_t1', distribution_type=>null); +INSERT INTO nullkey_c2_t1 SELECT i, i FROM generate_series(2, 7) i; +INSERT INTO nullkey_c2_t2 SELECT i, i FROM generate_series(1, 8) i; + +CREATE TABLE nullkey_c3_t1(a int, b int); +SELECT create_distributed_table('nullkey_c3_t1', null, colocate_with=>'none'); +INSERT INTO nullkey_c3_t1 SELECT i, i FROM generate_series(1, 8) i; + +CREATE TABLE reference_table(a int, b int); +SELECT create_reference_table('reference_table'); +INSERT INTO reference_table SELECT i, i FROM generate_series(0, 5) i; + +CREATE TABLE distributed_table(a int, b int); +SELECT create_distributed_table('distributed_table', 'a'); +INSERT INTO distributed_table SELECT i, i FROM generate_series(3, 8) i; + +CREATE TABLE citus_local_table(a int, b int); +SELECT citus_add_local_table_to_metadata('citus_local_table'); +INSERT INTO citus_local_table SELECT i, i FROM generate_series(0, 10) i; + +CREATE TABLE postgres_local_table(a int, b int); +INSERT INTO postgres_local_table SELECT i, i FROM generate_series(5, 10) i; + +CREATE TABLE articles_hash ( + id bigint NOT NULL, + author_id bigint NOT NULL, + title varchar(20) NOT NULL, + word_count integer +); + +INSERT INTO articles_hash VALUES ( 4, 4, 'altdorfer', 14551),( 5, 5, 'aruru', 11389), + (13, 3, 'aseyev', 2255),(15, 5, 'adversa', 3164), + (18, 8, 'assembly', 911),(19, 9, 'aubergiste', 4981), + (28, 8, 'aerophyte', 5454),(29, 9, 'amateur', 9524), + (42, 2, 'ausable', 15885),(43, 3, 'affixal', 12723), + (49, 9, 'anyone', 2681),(50, 10, 'anjanette', 19519); + +SELECT create_distributed_table('articles_hash', null, colocate_with=>'none'); + +CREATE TABLE raw_events_first (user_id int, time timestamp, value_1 int, value_2 int, value_3 float, value_4 bigint, UNIQUE(user_id, value_1)); +SELECT create_distributed_table('raw_events_first', null, colocate_with=>'none', distribution_type=>null); + +CREATE TABLE raw_events_second (user_id int, time timestamp, value_1 int, value_2 int, value_3 float, value_4 bigint, UNIQUE(user_id, value_1)); +SELECT create_distributed_table('raw_events_second', null, colocate_with=>'raw_events_first', distribution_type=>null); + +CREATE TABLE agg_events (user_id int, value_1_agg int, value_2_agg int, value_3_agg float, value_4_agg bigint, agg_time timestamp, UNIQUE(user_id, value_1_agg)); +SELECT create_distributed_table('agg_events', null, colocate_with=>'raw_events_first', distribution_type=>null); + +CREATE TABLE users_ref_table (user_id int); +SELECT create_reference_table('users_ref_table'); + +INSERT INTO raw_events_first VALUES (1, '1970-01-01', 10, 100, 1000.1, 10000), (3, '1971-01-01', 30, 300, 3000.1, 30000), + (5, '1972-01-01', 50, 500, 5000.1, 50000), (2, '1973-01-01', 20, 200, 2000.1, 20000), + (4, '1974-01-01', 40, 400, 4000.1, 40000), (6, '1975-01-01', 60, 600, 6000.1, 60000); + +CREATE TABLE modify_fast_path(key int, value_1 int, value_2 text); +SELECT create_distributed_table('modify_fast_path', null); + +CREATE TABLE modify_fast_path_reference(key int, value_1 int, value_2 text); +SELECT create_reference_table('modify_fast_path_reference'); + +CREATE TABLE bigserial_test (x int, y int, z bigserial); +SELECT create_distributed_table('bigserial_test', null); + +CREATE TABLE append_table (text_col text, a int); +SELECT create_distributed_table('append_table', 'a', 'append'); +SELECT master_create_empty_shard('append_table') AS shardid1 \gset +SELECT master_create_empty_shard('append_table') AS shardid2 \gset +SELECT master_create_empty_shard('append_table') AS shardid3 \gset + +COPY append_table (text_col, a) FROM STDIN WITH (format 'csv', append_to_shard :shardid1); +abc,234 +bcd,123 +bcd,234 +cde,345 +def,456 +efg,234 +\. + +COPY append_table (text_col, a) FROM STDIN WITH (format 'csv', append_to_shard :shardid2); +abc,123 +efg,123 +hij,123 +hij,234 +ijk,1 +jkl,0 +\. + +CREATE TABLE range_table(a int, b int); +SELECT create_distributed_table('range_table', 'a', 'range'); +CALL public.create_range_partitioned_shards('range_table', '{"0","25"}','{"24","49"}'); +INSERT INTO range_table VALUES (0, 1), (1, 2), (2, 3), (3, 4), (4, 5), (5, 6), (6, 50); + +SET client_min_messages to DEBUG2; + +-- simple insert +INSERT INTO nullkey_c1_t1 VALUES (1,2), (2,2), (3,4); +INSERT INTO nullkey_c1_t2 VALUES (1,3), (3,4), (5,1), (6,2); + +INSERT INTO nullkey_c2_t1 VALUES (1,0), (2,5), (4,3), (5,2); +INSERT INTO nullkey_c2_t2 VALUES (2,4), (3,2), (5,2), (7,4); + +-- simple select +SELECT * FROM nullkey_c1_t1 ORDER BY 1,2; + +-- for update / share +SELECT * FROM modify_fast_path WHERE key = 1 FOR UPDATE; +SELECT * FROM modify_fast_path WHERE key = 1 FOR SHARE; +SELECT * FROM modify_fast_path FOR UPDATE; +SELECT * FROM modify_fast_path FOR SHARE; + +-- cartesian product with different table types + +-- with other table types +SELECT COUNT(*) FROM distributed_table d1, nullkey_c1_t1; +SELECT COUNT(*) FROM reference_table d1, nullkey_c1_t1; +SELECT COUNT(*) FROM citus_local_table d1, nullkey_c1_t1; +SELECT COUNT(*) FROM postgres_local_table d1, nullkey_c1_t1; + +-- with a colocated null dist key table +SELECT COUNT(*) FROM nullkey_c1_t1 d1, nullkey_c1_t2; + +-- with a non-colocated null dist key table +SELECT COUNT(*) FROM nullkey_c1_t1 d1, nullkey_c2_t1; + +-- First, show that nullkey_c1_t1 and nullkey_c3_t1 are not colocated. +SELECT + (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'query_null_dist_key.nullkey_c1_t1'::regclass) != + (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'query_null_dist_key.nullkey_c3_t1'::regclass); + +-- Now verify that we can join them via router planner because it doesn't care +-- about whether two tables are colocated or not but physical location of shards +-- when citus.enable_non_colocated_router_query_pushdown is set to on. + +SET citus.enable_non_colocated_router_query_pushdown TO ON; + +SELECT COUNT(*) FROM nullkey_c1_t1 JOIN nullkey_c3_t1 USING(a); + +SET citus.enable_non_colocated_router_query_pushdown TO OFF; + +SELECT COUNT(*) FROM nullkey_c1_t1 JOIN nullkey_c3_t1 USING(a); + +RESET citus.enable_non_colocated_router_query_pushdown; + +-- colocated join between null dist key tables +SELECT COUNT(*) FROM nullkey_c1_t1 JOIN nullkey_c1_t2 USING(a); +SELECT COUNT(*) FROM nullkey_c1_t1 LEFT JOIN nullkey_c1_t2 USING(a); +SELECT COUNT(*) FROM nullkey_c1_t1 FULL JOIN nullkey_c1_t2 USING(a); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +LEFT JOIN LATERAL ( + SELECT * FROM nullkey_c1_t2 t2 WHERE t2.b > t1.a +) q USING(a); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +JOIN LATERAL ( + SELECT * FROM nullkey_c1_t2 t2 WHERE t2.b > t1.a +) q USING(a); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE EXISTS ( + SELECT * FROM nullkey_c1_t2 t2 WHERE t2.b > t1.a +); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE t1.b IN ( + SELECT b+1 FROM nullkey_c1_t2 t2 WHERE t2.b = t1.a +); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE t1.b NOT IN ( + SELECT a FROM nullkey_c1_t2 t2 WHERE t2.b > t1.a +); + +-- non-colocated inner joins between null dist key tables +SELECT * FROM nullkey_c1_t1 JOIN nullkey_c2_t1 USING(a) ORDER BY 1,2,3; + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +JOIN LATERAL ( + SELECT * FROM nullkey_c2_t2 t2 WHERE t2.b > t1.a +) q USING(a); + +-- non-colocated outer joins between null dist key tables +SELECT * FROM nullkey_c1_t1 LEFT JOIN nullkey_c2_t2 USING(a) ORDER BY 1,2,3 LIMIT 4; +SELECT * FROM nullkey_c1_t1 FULL JOIN nullkey_c2_t2 USING(a) ORDER BY 1,2,3 LIMIT 4; +SELECT * FROM nullkey_c1_t1 t1 +LEFT JOIN LATERAL ( + SELECT * FROM nullkey_c2_t2 t2 WHERE t2.b > t1.a +) q USING(a) ORDER BY 1,2,3 OFFSET 3 LIMIT 4; + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +LEFT JOIN LATERAL ( + SELECT * FROM nullkey_c2_t2 t2 WHERE t2.b > t1.a +) q USING(a); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE EXISTS ( + SELECT * FROM nullkey_c2_t2 t2 WHERE t2.b > t1.a +); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE t1.b IN ( + SELECT b+1 FROM nullkey_c2_t2 t2 WHERE t2.b = t1.a +); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE t1.b NOT IN ( + SELECT a FROM nullkey_c2_t2 t2 WHERE t2.b > t1.a +); + +-- join with a reference table +SELECT COUNT(*) FROM nullkey_c1_t1, reference_table WHERE nullkey_c1_t1.a = reference_table.a; + +WITH cte_1 AS + (SELECT * FROM nullkey_c1_t1, reference_table WHERE nullkey_c1_t1.a = reference_table.a ORDER BY 1,2,3,4 FOR UPDATE) +SELECT COUNT(*) FROM cte_1; + +-- join with postgres / citus local tables +SELECT * FROM nullkey_c1_t1 JOIN postgres_local_table USING(a); +SELECT * FROM nullkey_c1_t1 JOIN citus_local_table USING(a); + +-- join with a distributed table +SELECT * FROM distributed_table d1 JOIN nullkey_c1_t1 USING(a); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +JOIN LATERAL ( + SELECT * FROM distributed_table t2 WHERE t2.b > t1.a +) q USING(a); + +SELECT COUNT(*) FROM distributed_table t1 +JOIN LATERAL ( + SELECT * FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +) q USING(a); + +-- outer joins with different table types +SELECT COUNT(*) FROM nullkey_c1_t1 LEFT JOIN reference_table USING(a); +SELECT COUNT(*) FROM reference_table LEFT JOIN nullkey_c1_t1 USING(a); + +SELECT COUNT(*) FROM nullkey_c1_t1 LEFT JOIN citus_local_table USING(a); +SELECT COUNT(*) FROM citus_local_table LEFT JOIN nullkey_c1_t1 USING(a); + +SELECT COUNT(*) FROM nullkey_c1_t1 LEFT JOIN postgres_local_table USING(a); +SELECT COUNT(*) FROM postgres_local_table LEFT JOIN nullkey_c1_t1 USING(a); + +SELECT COUNT(*) FROM nullkey_c1_t1 FULL JOIN citus_local_table USING(a); +SELECT COUNT(*) FROM nullkey_c1_t1 FULL JOIN postgres_local_table USING(a); +SELECT COUNT(*) FROM nullkey_c1_t1 FULL JOIN reference_table USING(a); + +SELECT COUNT(*) FROM nullkey_c1_t1 JOIN append_table USING(a); +SELECT COUNT(*) FROM nullkey_c1_t1 JOIN range_table USING(a); + +SET citus.enable_non_colocated_router_query_pushdown TO ON; + +SELECT COUNT(*) FROM nullkey_c1_t1 JOIN range_table USING(a) WHERE range_table.a = 20; + +SET citus.enable_non_colocated_router_query_pushdown TO OFF; + +SELECT COUNT(*) FROM nullkey_c1_t1 JOIN range_table USING(a) WHERE range_table.a = 20; + +RESET citus.enable_non_colocated_router_query_pushdown; + +-- lateral / semi / anti joins with different table types + +-- with a reference table +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +LEFT JOIN LATERAL ( + SELECT * FROM reference_table t2 WHERE t2.b > t1.a +) q USING(a); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE EXISTS ( + SELECT * FROM reference_table t2 WHERE t2.b > t1.a +); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE NOT EXISTS ( + SELECT * FROM reference_table t2 WHERE t2.b > t1.a +); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE t1.b IN ( + SELECT b+1 FROM reference_table t2 WHERE t2.b = t1.a +); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE t1.b NOT IN ( + SELECT a FROM reference_table t2 WHERE t2.b > t1.a +); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +JOIN LATERAL ( + SELECT * FROM reference_table t2 WHERE t2.b > t1.a +) q USING(a); + +SELECT COUNT(*) FROM reference_table t1 +LEFT JOIN LATERAL ( + SELECT * FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +) q USING(a); + +SELECT COUNT(*) FROM reference_table t1 +WHERE EXISTS ( + SELECT * FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +); + +SELECT COUNT(*) FROM reference_table t1 +WHERE t1.b IN ( + SELECT b+1 FROM nullkey_c1_t1 t2 WHERE t2.b = t1.a +); + +SELECT COUNT(*) FROM reference_table t1 +WHERE t1.b NOT IN ( + SELECT a FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +); + +SELECT COUNT(*) FROM reference_table t1 +JOIN LATERAL ( + SELECT * FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +) q USING(a); + +-- with a distributed table +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +LEFT JOIN LATERAL ( + SELECT * FROM distributed_table t2 WHERE t2.b > t1.a +) q USING(a); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE EXISTS ( + SELECT * FROM distributed_table t2 WHERE t2.b > t1.a +); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE NOT EXISTS ( + SELECT * FROM distributed_table t2 WHERE t2.b > t1.a +); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE t1.b IN ( + SELECT b+1 FROM distributed_table t2 WHERE t2.b = t1.a +); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE t1.b NOT IN ( + SELECT a FROM distributed_table t2 WHERE t2.b > t1.a +); + +SELECT COUNT(*) FROM distributed_table t1 +LEFT JOIN LATERAL ( + SELECT * FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +) q USING(a); + +SELECT COUNT(*) FROM distributed_table t1 +WHERE EXISTS ( + SELECT * FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +); + +SELECT COUNT(*) FROM distributed_table t1 +WHERE t1.b IN ( + SELECT b+1 FROM nullkey_c1_t1 t2 WHERE t2.b = t1.a +); + +SELECT COUNT(*) FROM distributed_table t1 +WHERE t1.b NOT IN ( + SELECT a FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +); + +-- with postgres / citus local tables +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +LEFT JOIN LATERAL ( + SELECT * FROM citus_local_table t2 WHERE t2.b > t1.a +) q USING(a); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE EXISTS ( + SELECT * FROM citus_local_table t2 WHERE t2.b > t1.a +); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE NOT EXISTS ( + SELECT * FROM citus_local_table t2 WHERE t2.b > t1.a +); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE t1.b IN ( + SELECT b+1 FROM citus_local_table t2 WHERE t2.b = t1.a +); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE t1.b NOT IN ( + SELECT a FROM citus_local_table t2 WHERE t2.b > t1.a +); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +JOIN LATERAL ( + SELECT * FROM citus_local_table t2 WHERE t2.b > t1.a +) q USING(a); + +SELECT COUNT(*) FROM citus_local_table t1 +LEFT JOIN LATERAL ( + SELECT * FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +) q USING(a); + +SELECT COUNT(*) FROM postgres_local_table t1 +LEFT JOIN LATERAL ( + SELECT * FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +) q USING(a); + +SELECT COUNT(*) FROM citus_local_table t1 +WHERE EXISTS ( + SELECT * FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +); + +SELECT COUNT(*) FROM citus_local_table t1 +WHERE t1.b IN ( + SELECT b+1 FROM nullkey_c1_t1 t2 WHERE t2.b = t1.a +); + +SELECT COUNT(*) FROM citus_local_table t1 +WHERE t1.b NOT IN ( + SELECT a FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +); + +SELECT COUNT(*) FROM citus_local_table t1 +JOIN LATERAL ( + SELECT * FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +) q USING(a); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +LEFT JOIN LATERAL ( + SELECT * FROM postgres_local_table t2 WHERE t2.b > t1.a +) q USING(a); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE EXISTS ( + SELECT * FROM postgres_local_table t2 WHERE t2.b > t1.a +); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE NOT EXISTS ( + SELECT * FROM postgres_local_table t2 WHERE t2.b > t1.a +); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE t1.b IN ( + SELECT b+1 FROM postgres_local_table t2 WHERE t2.b = t1.a +); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +WHERE t1.b NOT IN ( + SELECT a FROM postgres_local_table t2 WHERE t2.b > t1.a +); + +SELECT COUNT(*) FROM nullkey_c1_t1 t1 +JOIN LATERAL ( + SELECT * FROM postgres_local_table t2 WHERE t2.b > t1.a +) q USING(a); + +SELECT COUNT(*) FROM postgres_local_table t1 +WHERE EXISTS ( + SELECT * FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +); + +SELECT COUNT(*) FROM postgres_local_table t1 +WHERE t1.b IN ( + SELECT b+1 FROM nullkey_c1_t1 t2 WHERE t2.b = t1.a +); + +SELECT COUNT(*) FROM postgres_local_table t1 +WHERE t1.b NOT IN ( + SELECT a FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +); + +SELECT COUNT(*) FROM postgres_local_table t1 +JOIN LATERAL ( + SELECT * FROM nullkey_c1_t1 t2 WHERE t2.b > t1.a +) q USING(a); + +-- insert .. select + +-- between two colocated null dist key tables + +-- The target list of "distributed statement"s that we send to workers +-- differ(*) in Postgres versions < 15. For this reason, we temporarily +-- disable debug messages here and run the EXPLAIN'ed version of the +-- command. +-- +-- (*): < SELECT a, b > vs < SELECT table_name.a, table_name.b > +SET client_min_messages TO WARNING; +EXPLAIN (ANALYZE TRUE, TIMING FALSE, COSTS FALSE, SUMMARY FALSE, VERBOSE FALSE) +INSERT INTO nullkey_c1_t1 SELECT * FROM nullkey_c1_t2; +SET client_min_messages TO DEBUG2; + +-- between two non-colocated null dist key tables +INSERT INTO nullkey_c1_t1 SELECT * FROM nullkey_c2_t1; + +-- between a null dist key table and a table of different type +INSERT INTO nullkey_c1_t1 SELECT * FROM reference_table; +INSERT INTO nullkey_c1_t1 SELECT * FROM distributed_table; +INSERT INTO nullkey_c1_t1 SELECT * FROM citus_local_table; +INSERT INTO nullkey_c1_t1 SELECT * FROM postgres_local_table; + +INSERT INTO reference_table SELECT * FROM nullkey_c1_t1; +INSERT INTO distributed_table SELECT * FROM nullkey_c1_t1; +INSERT INTO citus_local_table SELECT * FROM nullkey_c1_t1; +INSERT INTO postgres_local_table SELECT * FROM nullkey_c1_t1; + +-- test subquery +SELECT count(*) FROM +( + SELECT * FROM (SELECT * FROM nullkey_c1_t2) as subquery_inner +) AS subquery_top; + +-- test cte inlining +WITH cte_nullkey_c1_t1 AS (SELECT * FROM nullkey_c1_t1), + cte_postgres_local_table AS (SELECT * FROM postgres_local_table), + cte_distributed_table AS (SELECT * FROM distributed_table) +SELECT COUNT(*) FROM cte_distributed_table, cte_nullkey_c1_t1, cte_postgres_local_table +WHERE cte_nullkey_c1_t1.a > 3 AND cte_distributed_table.a < 5; + +-- test recursive ctes +WITH level_0 AS ( + WITH level_1 AS ( + WITH RECURSIVE level_2_recursive(x) AS ( + VALUES (1) + UNION ALL + SELECT a + 1 FROM nullkey_c1_t1 JOIN level_2_recursive ON (a = x) WHERE a < 100 + ) + SELECT * FROM level_2_recursive RIGHT JOIN reference_table ON (level_2_recursive.x = reference_table.a) + ) + SELECT * FROM level_1 +) +SELECT COUNT(*) FROM level_0; + +WITH level_0 AS ( + WITH level_1 AS ( + WITH RECURSIVE level_2_recursive(x) AS ( + VALUES (1) + UNION ALL + SELECT a + 1 FROM nullkey_c1_t1 JOIN level_2_recursive ON (a = x) WHERE a < 100 + ) + SELECT * FROM level_2_recursive JOIN distributed_table ON (level_2_recursive.x = distributed_table.a) + ) + SELECT * FROM level_1 +) +SELECT COUNT(*) FROM level_0; + +-- grouping set +SELECT + id, substring(title, 2, 1) AS subtitle, count(*) + FROM articles_hash + WHERE author_id = 1 or author_id = 2 + GROUP BY GROUPING SETS ((id),(subtitle)) + ORDER BY id, subtitle; + +-- subquery in SELECT clause +SELECT a.title AS name, (SELECT a2.id FROM articles_hash a2 WHERE a.id = a2.id LIMIT 1) + AS special_price FROM articles_hash a +ORDER BY 1,2; + +-- test prepared statements + +-- prepare queries can be router plannable +PREPARE author_1_articles as + SELECT * + FROM articles_hash + WHERE author_id = 1; + +EXECUTE author_1_articles; +EXECUTE author_1_articles; +EXECUTE author_1_articles; +EXECUTE author_1_articles; +EXECUTE author_1_articles; +EXECUTE author_1_articles; + +-- parametric prepare queries can be router plannable +PREPARE author_articles(int) as + SELECT * + FROM articles_hash + WHERE author_id = $1; + +EXECUTE author_articles(1); +EXECUTE author_articles(1); +EXECUTE author_articles(1); +EXECUTE author_articles(1); +EXECUTE author_articles(1); +EXECUTE author_articles(1); + +EXECUTE author_articles(NULL); +EXECUTE author_articles(NULL); +EXECUTE author_articles(NULL); +EXECUTE author_articles(NULL); +EXECUTE author_articles(NULL); +EXECUTE author_articles(NULL); +EXECUTE author_articles(NULL); + +PREPARE author_articles_update(int) AS + UPDATE articles_hash SET title = 'test' WHERE author_id = $1; + +EXECUTE author_articles_update(NULL); +EXECUTE author_articles_update(NULL); +EXECUTE author_articles_update(NULL); +EXECUTE author_articles_update(NULL); +EXECUTE author_articles_update(NULL); +EXECUTE author_articles_update(NULL); +EXECUTE author_articles_update(NULL); + +-- More tests with insert .. select. +-- +-- The target list of "distributed statement"s that we send to workers +-- might differ(*) in Postgres versions < 15 and they are reported when +-- "log level >= DEBUG2". For this reason, we set log level to DEBUG1 to +-- avoid reporting them. +-- +-- DEBUG1 still allows reporting the reason why given INSERT .. SELECT +-- query is not distributed / requires pull-to-coordinator. + +SET client_min_messages TO DEBUG1; + +INSERT INTO bigserial_test (x, y) SELECT x, y FROM bigserial_test; + +INSERT INTO agg_events + (user_id) +SELECT f2.id FROM + +(SELECT + id +FROM (SELECT users_ref_table.user_id AS id + FROM raw_events_first, + users_ref_table + WHERE raw_events_first.user_id = users_ref_table.user_id ) AS foo) as f +INNER JOIN +(SELECT v4, + v1, + id +FROM (SELECT SUM(raw_events_second.value_4) AS v4, + SUM(raw_events_first.value_1) AS v1, + raw_events_second.user_id AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.user_id + GROUP BY raw_events_second.user_id + HAVING SUM(raw_events_second.value_4) > 1000) AS foo2 ) as f2 +ON (f.id = f2.id) +WHERE f.id IN (SELECT user_id + FROM raw_events_second); + +-- upsert with returning +INSERT INTO agg_events AS ae + ( + user_id, + value_1_agg, + agg_time + ) +SELECT user_id, + value_1, + time +FROM raw_events_first +ON conflict (user_id, value_1_agg) +DO UPDATE + SET agg_time = EXCLUDED.agg_time + WHERE ae.agg_time < EXCLUDED.agg_time +RETURNING user_id, value_1_agg; + +-- using a left join +INSERT INTO agg_events (user_id) +SELECT + raw_events_first.user_id +FROM + raw_events_first LEFT JOIN raw_events_second ON raw_events_first.user_id = raw_events_second.user_id + WHERE raw_events_second.user_id = 10 OR raw_events_second.user_id = 11; + +-- using a full join +INSERT INTO agg_events (user_id, value_1_agg) +SELECT t1.user_id AS col1, + t2.user_id AS col2 +FROM raw_events_first t1 + FULL JOIN raw_events_second t2 + ON t1.user_id = t2.user_id; + +-- using semi join +INSERT INTO raw_events_second + (user_id) +SELECT user_id +FROM raw_events_first +WHERE user_id IN (SELECT raw_events_second.user_id + FROM raw_events_second, raw_events_first + WHERE raw_events_second.user_id = raw_events_first.user_id AND raw_events_first.user_id = 200); + +-- using lateral join +INSERT INTO raw_events_second + (user_id) +SELECT user_id +FROM raw_events_first +WHERE NOT EXISTS (SELECT 1 + FROM raw_events_second + WHERE raw_events_second.user_id =raw_events_first.user_id); + +-- using inner join +INSERT INTO agg_events (user_id) +SELECT raw_events_first.user_id +FROM raw_events_first INNER JOIN raw_events_second ON raw_events_first.user_id = raw_events_second.value_1 +WHERE raw_events_first.value_1 IN (10, 11,12) OR raw_events_second.user_id IN (1,2,3,4); + +-- We could relax distributed insert .. select checks to allow pushing +-- down more clauses down to the worker nodes when inserting into a single +-- shard by selecting from a colocated one. We might want to do something +-- like https://github.com/citusdata/citus/pull/6772. +-- +-- e.g., insert into null_shard_key_1/citus_local/reference +-- select * from null_shard_key_1/citus_local/reference limit 1 +-- +-- Below "limit / offset clause" test and some others are examples of this. + +-- limit / offset clause +INSERT INTO agg_events (user_id) SELECT raw_events_first.user_id FROM raw_events_first LIMIT 1; +INSERT INTO agg_events (user_id) SELECT raw_events_first.user_id FROM raw_events_first OFFSET 1; + +-- using a materialized cte +WITH cte AS MATERIALIZED + (SELECT max(value_1)+1 as v1_agg, user_id FROM raw_events_first GROUP BY user_id) +INSERT INTO agg_events (value_1_agg, user_id) +SELECT v1_agg, user_id FROM cte; + +INSERT INTO raw_events_second + WITH cte AS MATERIALIZED (SELECT * FROM raw_events_first) + SELECT user_id * 1000, time, value_1, value_2, value_3, value_4 FROM cte; + +-- using a regular cte +WITH cte AS (SELECT * FROM raw_events_first) +INSERT INTO raw_events_second + SELECT user_id * 7000, time, value_1, value_2, value_3, value_4 FROM cte; + +INSERT INTO raw_events_second + WITH cte AS (SELECT * FROM raw_events_first) + SELECT * FROM cte; + +INSERT INTO agg_events + WITH sub_cte AS (SELECT 1) + SELECT + raw_events_first.user_id, (SELECT * FROM sub_cte) + FROM + raw_events_first; + +-- we still support complex joins via INSERT's cte list .. +WITH cte AS ( + SELECT reference_table.a AS a, 1 AS b + FROM distributed_table RIGHT JOIN reference_table USING (a) +) +INSERT INTO raw_events_second (user_id, value_1) + SELECT (a+5)*-1, b FROM cte; + +-- .. but can't do so via via SELECT's cte list +INSERT INTO raw_events_second (user_id, value_1) +WITH cte AS ( + SELECT reference_table.a AS a, 1 AS b + FROM distributed_table RIGHT JOIN reference_table USING (a) +) + SELECT (a+5)*-1, b FROM cte; + +-- using set operations +INSERT INTO + raw_events_first(user_id) + (SELECT user_id FROM raw_events_first) INTERSECT + (SELECT user_id FROM raw_events_first); + +-- group by clause inside subquery +INSERT INTO agg_events + (user_id) +SELECT f2.id FROM + +(SELECT + id +FROM (SELECT raw_events_second.user_id AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.user_id ) AS foo) as f +INNER JOIN +(SELECT v4, + v1, + id +FROM (SELECT SUM(raw_events_second.value_4) AS v4, + SUM(raw_events_first.value_1) AS v1, + raw_events_second.user_id AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.user_id + GROUP BY raw_events_second.user_id + HAVING SUM(raw_events_second.value_4) > 1000) AS foo2 ) as f2 +ON (f.id = f2.id) +WHERE f.id IN (SELECT user_id + FROM raw_events_second); + +-- group by clause inside lateral subquery +INSERT INTO agg_events (user_id, value_4_agg) +SELECT + averages.user_id, avg(averages.value_4) +FROM + (SELECT + t1.user_id + FROM + raw_events_second t1 JOIN raw_events_second t2 on (t1.user_id = t2.user_id) + ) reference_ids + JOIN LATERAL + (SELECT + user_id, value_4 + FROM + raw_events_first) as averages ON averages.value_4 = reference_ids.user_id + GROUP BY averages.user_id; + +-- using aggregates +INSERT INTO agg_events + (value_3_agg, + value_4_agg, + value_1_agg, + value_2_agg, + user_id) +SELECT SUM(value_3), + Count(value_4), + user_id, + SUM(value_1), + Avg(value_2) +FROM raw_events_first +GROUP BY user_id; + +-- using generate_series +INSERT INTO raw_events_first (user_id, value_1, value_2) +SELECT s, s, s FROM generate_series(1, 5) s; + +CREATE SEQUENCE insert_select_test_seq; + +-- nextval() expression in select's targetlist +INSERT INTO raw_events_first (user_id, value_1, value_2) +SELECT s, nextval('insert_select_test_seq'), (random()*10)::int +FROM generate_series(100, 105) s; + +-- non-immutable function +INSERT INTO modify_fast_path (key, value_1) VALUES (2,1) RETURNING value_1, random() * key; + +SET client_min_messages TO DEBUG2; + +-- update / delete + +UPDATE nullkey_c1_t1 SET a = 1 WHERE b = 5; +UPDATE nullkey_c1_t1 SET a = 1 WHERE a = 5; +UPDATE nullkey_c1_t1 SET a = random(); +UPDATE nullkey_c1_t1 SET a = 1 WHERE a = random(); + +DELETE FROM nullkey_c1_t1 WHERE b = 5; +DELETE FROM nullkey_c1_t1 WHERE a = random(); + +-- simple update queries between different table types / colocated tables +UPDATE nullkey_c1_t1 SET b = 5 FROM nullkey_c1_t2 WHERE nullkey_c1_t1.b = nullkey_c1_t2.b; +UPDATE nullkey_c1_t1 SET b = 5 FROM nullkey_c2_t1 WHERE nullkey_c1_t1.b = nullkey_c2_t1.b; +UPDATE nullkey_c1_t1 SET b = 5 FROM reference_table WHERE nullkey_c1_t1.b = reference_table.b; +UPDATE nullkey_c1_t1 SET b = 5 FROM distributed_table WHERE nullkey_c1_t1.b = distributed_table.b; +UPDATE nullkey_c1_t1 SET b = 5 FROM distributed_table WHERE nullkey_c1_t1.b = distributed_table.a; +UPDATE nullkey_c1_t1 SET b = 5 FROM citus_local_table WHERE nullkey_c1_t1.b = citus_local_table.b; +UPDATE nullkey_c1_t1 SET b = 5 FROM postgres_local_table WHERE nullkey_c1_t1.b = postgres_local_table.b; + +UPDATE reference_table SET b = 5 FROM nullkey_c1_t1 WHERE nullkey_c1_t1.b = reference_table.b; +UPDATE distributed_table SET b = 5 FROM nullkey_c1_t1 WHERE nullkey_c1_t1.b = distributed_table.b; +UPDATE distributed_table SET b = 5 FROM nullkey_c1_t1 WHERE nullkey_c1_t1.b = distributed_table.a; +UPDATE citus_local_table SET b = 5 FROM nullkey_c1_t1 WHERE nullkey_c1_t1.b = citus_local_table.b; +UPDATE postgres_local_table SET b = 5 FROM nullkey_c1_t1 WHERE nullkey_c1_t1.b = postgres_local_table.b; + +-- simple delete queries between different table types / colocated tables +DELETE FROM nullkey_c1_t1 USING nullkey_c1_t2 WHERE nullkey_c1_t1.b = nullkey_c1_t2.b; +DELETE FROM nullkey_c1_t1 USING nullkey_c2_t1 WHERE nullkey_c1_t1.b = nullkey_c2_t1.b; +DELETE FROM nullkey_c1_t1 USING reference_table WHERE nullkey_c1_t1.b = reference_table.b; +DELETE FROM nullkey_c1_t1 USING distributed_table WHERE nullkey_c1_t1.b = distributed_table.b; +DELETE FROM nullkey_c1_t1 USING distributed_table WHERE nullkey_c1_t1.b = distributed_table.a; +DELETE FROM nullkey_c1_t1 USING citus_local_table WHERE nullkey_c1_t1.b = citus_local_table.b; +DELETE FROM nullkey_c1_t1 USING postgres_local_table WHERE nullkey_c1_t1.b = postgres_local_table.b; + +DELETE FROM reference_table USING nullkey_c1_t1 WHERE nullkey_c1_t1.b = reference_table.b; +DELETE FROM distributed_table USING nullkey_c1_t1 WHERE nullkey_c1_t1.b = distributed_table.b; +DELETE FROM distributed_table USING nullkey_c1_t1 WHERE nullkey_c1_t1.b = distributed_table.a; +DELETE FROM citus_local_table USING nullkey_c1_t1 WHERE nullkey_c1_t1.b = citus_local_table.b; +DELETE FROM postgres_local_table USING nullkey_c1_t1 WHERE nullkey_c1_t1.b = postgres_local_table.b; + +-- slightly more complex update queries +UPDATE nullkey_c1_t1 SET b = 5 WHERE nullkey_c1_t1.b IN (SELECT b FROM distributed_table); + +WITH cte AS materialized( + SELECT * FROM distributed_table +) +UPDATE nullkey_c1_t1 SET b = 5 FROM cte WHERE nullkey_c1_t1.b = cte.a; + +WITH cte AS ( + SELECT reference_table.a AS a, 1 AS b + FROM distributed_table RIGHT JOIN reference_table USING (a) +) +UPDATE nullkey_c1_t1 SET b = 5 WHERE nullkey_c1_t1.b IN (SELECT b FROM cte); + +UPDATE nullkey_c1_t1 SET b = 5 FROM reference_table WHERE EXISTS ( + SELECT 1 FROM reference_table LEFT JOIN nullkey_c1_t1 USING (a) WHERE nullkey_c1_t1.b IS NULL +); + +UPDATE nullkey_c1_t1 tx SET b = ( + SELECT nullkey_c1_t2.b FROM nullkey_c1_t2 JOIN nullkey_c1_t1 ON (nullkey_c1_t1.a != nullkey_c1_t2.a) WHERE nullkey_c1_t1.a = tx.a ORDER BY 1 LIMIT 1 +); + +UPDATE nullkey_c1_t1 tx SET b = t2.b FROM nullkey_c1_t1 t1 JOIN nullkey_c1_t2 t2 ON (t1.a = t2.a); + +WITH cte AS ( + SELECT * FROM nullkey_c1_t2 ORDER BY 1,2 LIMIT 10 +) +UPDATE nullkey_c1_t1 SET b = 5 WHERE nullkey_c1_t1.a IN (SELECT b FROM cte); + +UPDATE modify_fast_path SET value_1 = value_1 + 12 * value_1 WHERE key = 1; +UPDATE modify_fast_path SET value_1 = NULL WHERE value_1 = 15 AND (key = 1 OR value_2 = 'citus'); +UPDATE modify_fast_path SET value_1 = 5 WHERE key = 2 RETURNING value_1 * 15, value_1::numeric * 16; +UPDATE modify_fast_path + SET value_1 = 1 + FROM modify_fast_path_reference + WHERE + modify_fast_path.key = modify_fast_path_reference.key AND + modify_fast_path.key = 1 AND + modify_fast_path_reference.key = 1; + +PREPARE p1 (int, int, int) AS + UPDATE modify_fast_path SET value_1 = value_1 + $1 WHERE key = $2 AND value_1 = $3; +EXECUTE p1(1,1,1); +EXECUTE p1(2,2,2); +EXECUTE p1(3,3,3); +EXECUTE p1(4,4,4); +EXECUTE p1(5,5,5); +EXECUTE p1(6,6,6); +EXECUTE p1(7,7,7); + +PREPARE prepared_zero_shard_update(int) AS UPDATE modify_fast_path SET value_1 = 1 WHERE key = $1 AND false; +EXECUTE prepared_zero_shard_update(1); +EXECUTE prepared_zero_shard_update(2); +EXECUTE prepared_zero_shard_update(3); +EXECUTE prepared_zero_shard_update(4); +EXECUTE prepared_zero_shard_update(5); +EXECUTE prepared_zero_shard_update(6); +EXECUTE prepared_zero_shard_update(7); + +-- slightly more complex delete queries +DELETE FROM nullkey_c1_t1 WHERE nullkey_c1_t1.b IN (SELECT b FROM distributed_table); + +WITH cte AS materialized( + SELECT * FROM distributed_table +) +DELETE FROM nullkey_c1_t1 USING cte WHERE nullkey_c1_t1.b = cte.a; + +WITH cte AS ( + SELECT reference_table.a AS a, 1 AS b + FROM distributed_table RIGHT JOIN reference_table USING (a) +) +DELETE FROM nullkey_c1_t1 WHERE nullkey_c1_t1.b IN (SELECT b FROM cte); + +DELETE FROM nullkey_c1_t1 USING reference_table WHERE EXISTS ( + SELECT 1 FROM reference_table LEFT JOIN nullkey_c1_t1 USING (a) WHERE nullkey_c1_t1.b IS NULL +); + +DELETE FROM nullkey_c1_t1 tx USING nullkey_c1_t1 t1 JOIN nullkey_c1_t2 t2 ON (t1.a = t2.a); + +WITH cte AS ( + SELECT * FROM nullkey_c1_t2 ORDER BY 1,2 LIMIT 10 +) +DELETE FROM nullkey_c1_t1 WHERE nullkey_c1_t1.a IN (SELECT b FROM cte); + +DELETE FROM modify_fast_path WHERE value_1 = 15 AND (key = 1 OR value_2 = 'citus'); +DELETE FROM modify_fast_path WHERE key = 2 RETURNING value_1 * 15, value_1::numeric * 16; +DELETE FROM modify_fast_path + USING modify_fast_path_reference + WHERE + modify_fast_path.key = modify_fast_path_reference.key AND + modify_fast_path.key = 1 AND + modify_fast_path_reference.key = 1; + +PREPARE p2 (int, int, int) AS + DELETE FROM modify_fast_path WHERE key = ($2)*$1 AND value_1 = $3; +EXECUTE p2(1,1,1); +EXECUTE p2(2,2,2); +EXECUTE p2(3,3,3); +EXECUTE p2(4,4,4); +EXECUTE p2(5,5,5); +EXECUTE p2(6,6,6); +EXECUTE p2(7,7,7); + +PREPARE prepared_zero_shard_delete(int) AS DELETE FROM modify_fast_path WHERE key = $1 AND false; +EXECUTE prepared_zero_shard_delete(1); +EXECUTE prepared_zero_shard_delete(2); +EXECUTE prepared_zero_shard_delete(3); +EXECUTE prepared_zero_shard_delete(4); +EXECUTE prepared_zero_shard_delete(5); +EXECUTE prepared_zero_shard_delete(6); +EXECUTE prepared_zero_shard_delete(7); + +-- test modifying ctes + +WITH cte AS ( + UPDATE modify_fast_path SET value_1 = value_1 + 1 WHERE key = 1 RETURNING * +) +SELECT * FROM cte; + +WITH cte AS ( + DELETE FROM modify_fast_path WHERE key = 1 RETURNING * +) +SELECT * FROM modify_fast_path; + +WITH cte AS ( + DELETE FROM modify_fast_path WHERE key = 1 RETURNING * +) +SELECT * FROM modify_fast_path_reference WHERE key IN (SELECT key FROM cte); + +WITH cte AS ( + DELETE FROM reference_table WHERE a = 1 RETURNING * +) +SELECT * FROM nullkey_c1_t1 WHERE a IN (SELECT a FROM cte); + +WITH cte AS ( + DELETE FROM nullkey_c1_t1 WHERE a = 1 RETURNING * +) +SELECT * FROM nullkey_c1_t2 WHERE a IN (SELECT a FROM cte); + +WITH cte AS ( + DELETE FROM nullkey_c1_t1 WHERE a = 1 RETURNING * +) +SELECT * FROM nullkey_c2_t1 WHERE a IN (SELECT a FROM cte); + +WITH cte AS ( + DELETE FROM nullkey_c1_t1 WHERE a = 1 RETURNING * +) +SELECT * FROM distributed_table WHERE a IN (SELECT a FROM cte); + +-- Below two queries fail very late when +-- citus.enable_non_colocated_router_query_pushdown is set to on. + +SET citus.enable_non_colocated_router_query_pushdown TO ON; + +WITH cte AS ( + DELETE FROM distributed_table WHERE a = 1 RETURNING * +) +SELECT * FROM nullkey_c1_t1 WHERE a IN (SELECT a FROM cte); + +WITH cte AS ( + DELETE FROM distributed_table WHERE a = 1 RETURNING * +) +SELECT * FROM nullkey_c1_t1 WHERE b IN (SELECT b FROM cte); + +SET citus.enable_non_colocated_router_query_pushdown TO OFF; + +WITH cte AS ( + DELETE FROM distributed_table WHERE a = 1 RETURNING * +) +SELECT * FROM nullkey_c1_t1 WHERE a IN (SELECT a FROM cte); + +WITH cte AS ( + DELETE FROM distributed_table WHERE a = 1 RETURNING * +) +SELECT * FROM nullkey_c1_t1 WHERE b IN (SELECT b FROM cte); + +RESET citus.enable_non_colocated_router_query_pushdown; + +WITH cte AS ( + UPDATE modify_fast_path SET value_1 = value_1 + 1 WHERE key = 1 RETURNING * +) +UPDATE modify_fast_path SET value_1 = value_1 + 1 WHERE key = 1; + +WITH cte AS ( + DELETE FROM modify_fast_path WHERE key = 1 RETURNING * +) +DELETE FROM modify_fast_path WHERE key = 1; + +-- test window functions + +SELECT + user_id, avg(avg(value_3)) OVER (PARTITION BY user_id, MIN(value_2)) +FROM + raw_events_first +GROUP BY + 1 +ORDER BY + 2 DESC NULLS LAST, 1 DESC; + +SELECT + user_id, max(value_1) OVER (PARTITION BY user_id, MIN(value_2)) +FROM ( + SELECT + DISTINCT us.user_id, us.value_2, us.value_1, random() as r1 + FROM + raw_events_first as us, raw_events_second + WHERE + us.user_id = raw_events_second.user_id + ORDER BY + user_id, value_2 + ) s +GROUP BY + 1, value_1 +ORDER BY + 2 DESC, 1; + +SELECT + DISTINCT ON (raw_events_second.user_id, rnk) raw_events_second.user_id, rank() OVER my_win AS rnk +FROM + raw_events_second, raw_events_first +WHERE + raw_events_first.user_id = raw_events_second.user_id +WINDOW + my_win AS (PARTITION BY raw_events_second.user_id, raw_events_first.value_1 ORDER BY raw_events_second.time DESC) +ORDER BY + rnk DESC, 1 DESC +LIMIT 10; + +SET client_min_messages TO ERROR; +DROP SCHEMA query_null_dist_key CASCADE; + +SELECT citus_remove_node('localhost', :master_port); From 39b7711527aeb48af150a85f29e83a7ab778714e Mon Sep 17 00:00:00 2001 From: Onur Tirtir Date: Mon, 24 Apr 2023 13:19:22 +0300 Subject: [PATCH 5/7] Add support for more pushable / non-pushable insert .. select queries with null-shard-key tables (#6823) * Add support for dist insert select by selecting from a reference table. This was the only pushable insert .. select case that #6773 didn't cover. * For the cases where we insert into a Citus table but the INSERT .. SELECT query cannot be pushed down, allow pull-to-coordinator when possible. Remove the checks that we had at the very beginning of CreateInsertSelectPlanInternal so that we can try insert .. select via pull-to-coordinator for the cases where we cannot push-down the insert .. select query. What we support via pull-to-coordinator is still limited due to lacking of logical planner support for SELECT queries, but this commit at least allows using pull-to-coordinator for the cases where the select query can be planned via router planner, without limiting ourselves to restrictive top-level checks. Also introduce some additional restrictions into CreateDistributedInsertSelectPlan for the cases it was missing to check for null-shard-key tables. Indeed, it would make more sense to have those checks for distributed tables in general, via separate PRs against main branch. See https://github.com/citusdata/citus/pull/6817. * Add support for inserting into a Postgres table. --- .../planner/insert_select_planner.c | 156 ++-- .../regress/expected/create_null_dist_key.out | 1 - .../expected/insert_select_null_dist_key.out | 814 ++++++++++++++++++ .../regress/expected/query_null_dist_key.out | 113 ++- src/test/regress/multi_1_schedule | 1 + .../sql/insert_select_null_dist_key.sql | 470 ++++++++++ src/test/regress/sql/query_null_dist_key.sql | 57 +- 7 files changed, 1484 insertions(+), 128 deletions(-) create mode 100644 src/test/regress/expected/insert_select_null_dist_key.out create mode 100644 src/test/regress/sql/insert_select_null_dist_key.sql diff --git a/src/backend/distributed/planner/insert_select_planner.c b/src/backend/distributed/planner/insert_select_planner.c index 175f6bc6f..62c0e8d68 100644 --- a/src/backend/distributed/planner/insert_select_planner.c +++ b/src/backend/distributed/planner/insert_select_planner.c @@ -57,7 +57,6 @@ static DistributedPlan * CreateInsertSelectPlanInternal(uint64 planId, PlannerRestrictionContext * plannerRestrictionContext, ParamListInfo boundParams); -static void ErrorIfInsertSelectWithNullDistKeyNotSupported(Query *originalQuery); static DistributedPlan * CreateDistributedInsertSelectPlan(Query *originalQuery, PlannerRestrictionContext * plannerRestrictionContext); @@ -242,12 +241,6 @@ CreateInsertSelectPlanInternal(uint64 planId, Query *originalQuery, RaiseDeferredError(deferredError, ERROR); } - /* - * We support a limited set of INSERT .. SELECT queries if the query - * references a null-dist-key table. - */ - ErrorIfInsertSelectWithNullDistKeyNotSupported(originalQuery); - DistributedPlan *distributedPlan = CreateDistributedInsertSelectPlan(originalQuery, plannerRestrictionContext); @@ -267,74 +260,6 @@ CreateInsertSelectPlanInternal(uint64 planId, Query *originalQuery, } -/* - * ErrorIfInsertSelectWithNullDistKeyNotSupported throws an error if given INSERT - * .. SELECT query references a null-dist-key table (as the target table or in - * the SELECT clause) and is unsupported. - * - * Such an INSERT .. SELECT query is supported as long as the it only references - * a "colocated" set of null-dist-key tables, no other relation rte types. - */ -static void -ErrorIfInsertSelectWithNullDistKeyNotSupported(Query *originalQuery) -{ - RangeTblEntry *subqueryRte = ExtractSelectRangeTableEntry(originalQuery); - Query *subquery = subqueryRte->subquery; - RTEListProperties *subqueryRteListProperties = GetRTEListPropertiesForQuery(subquery); - - RangeTblEntry *insertRte = ExtractResultRelationRTEOrError(originalQuery); - Oid targetRelationId = insertRte->relid; - if (!IsCitusTableType(targetRelationId, NULL_KEY_DISTRIBUTED_TABLE) && - subqueryRteListProperties->hasDistTableWithoutShardKey) - { - ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("cannot select from a distributed table that " - "does not have a shard key when inserting into " - "a different table type"))); - } - else if (IsCitusTableType(targetRelationId, NULL_KEY_DISTRIBUTED_TABLE)) - { - if (subqueryRteListProperties->hasPostgresLocalTable || - subqueryRteListProperties->hasReferenceTable || - subqueryRteListProperties->hasCitusLocalTable || - subqueryRteListProperties->hasDistTableWithShardKey || - subqueryRteListProperties->hasMaterializedView) - { - ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("cannot select from different table types " - "when inserting into a distributed table " - "that does not have a shard key"))); - } - - if (!subqueryRteListProperties->hasDistTableWithoutShardKey) - { - /* - * This means that the SELECT doesn't reference any Citus tables, - * Postgres tables or materialized views but references a function - * call, a values claue etc., or a cte from INSERT. - * - * In that case, we rely on the common restrictions enforced by the - * INSERT .. SELECT planners. - */ - Assert(!NeedsDistributedPlanning(subquery)); - return; - } - - List *distributedRelationIdList = DistributedRelationIdList(subquery); - distributedRelationIdList = lappend_oid(distributedRelationIdList, - targetRelationId); - - if (!AllDistributedRelationsInListColocated(distributedRelationIdList)) - { - ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("cannot select from a non-colocated distributed " - "table when inserting into a distributed table " - "that does not have a shard key"))); - } - } -} - - /* * CreateDistributedInsertSelectPlan creates a DistributedPlan for distributed * INSERT ... SELECT queries which could consist of multiple tasks. @@ -454,16 +379,6 @@ CreateInsertSelectIntoLocalTablePlan(uint64 planId, Query *insertSelectQuery, { RangeTblEntry *selectRte = ExtractSelectRangeTableEntry(insertSelectQuery); - RTEListProperties *selectRteListProperties = - GetRTEListPropertiesForQuery(selectRte->subquery); - if (selectRteListProperties->hasDistTableWithoutShardKey) - { - ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), - errmsg("cannot select from a distributed table that " - "does not have a shard key when inserting into " - "a local table"))); - } - PrepareInsertSelectForCitusPlanner(insertSelectQuery); /* get the SELECT query (may have changed after PrepareInsertSelectForCitusPlanner) */ @@ -800,10 +715,6 @@ DistributedInsertSelectSupported(Query *queryTree, RangeTblEntry *insertRte, NULL, NULL); } } - else if (IsCitusTableType(targetRelationId, NULL_KEY_DISTRIBUTED_TABLE)) - { - /* we've already checked the subquery via ErrorIfInsertSelectWithNullDistKeyNotSupported */ - } else { /* @@ -819,25 +730,49 @@ DistributedInsertSelectSupported(Query *queryTree, RangeTblEntry *insertRte, "table", NULL, NULL); } - /* ensure that INSERT's partition column comes from SELECT's partition column */ - error = InsertPartitionColumnMatchesSelect(queryTree, insertRte, subqueryRte, - &selectPartitionColumnTableId); - if (error) + if (!HasDistributionKey(targetRelationId) || + subqueryRteListProperties->hasDistTableWithoutShardKey) { - return error; + /* + * XXX: Better to check this regardless of the fact that the target table + * has a distribution column or not. + */ + List *distributedRelationIdList = DistributedRelationIdList(subquery); + distributedRelationIdList = lappend_oid(distributedRelationIdList, + targetRelationId); + + if (!AllDistributedRelationsInListColocated(distributedRelationIdList)) + { + return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "distributed INSERT ... SELECT cannot reference a " + "distributed table without a shard key together " + "with non-colocated distributed tables", + NULL, NULL); + } } - /* - * We expect partition column values come from colocated tables. Note that we - * skip this check from the reference table case given that all reference tables - * are already (and by default) co-located. - */ - if (!TablesColocated(insertRte->relid, selectPartitionColumnTableId)) + if (HasDistributionKey(targetRelationId)) { - return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, - "INSERT target table and the source relation of the SELECT partition " - "column value must be colocated in distributed INSERT ... SELECT", - NULL, NULL); + /* ensure that INSERT's partition column comes from SELECT's partition column */ + error = InsertPartitionColumnMatchesSelect(queryTree, insertRte, subqueryRte, + &selectPartitionColumnTableId); + if (error) + { + return error; + } + + /* + * We expect partition column values come from colocated tables. Note that we + * skip this check from the reference table case given that all reference tables + * are already (and by default) co-located. + */ + if (!TablesColocated(insertRte->relid, selectPartitionColumnTableId)) + { + return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, + "INSERT target table and the source relation of the SELECT partition " + "column value must be colocated in distributed INSERT ... SELECT", + NULL, NULL); + } } } @@ -1626,6 +1561,19 @@ CreateNonPushableInsertSelectPlan(uint64 planId, Query *parse, ParamListInfo bou bool repartitioned = IsRedistributablePlan(selectPlan->planTree) && IsSupportedRedistributionTarget(targetRelationId); + /* + * Today it's not possible to generate a distributed plan for a SELECT + * having more than one tasks if it references a null-shard-key table. + * This is because, we don't support queries beyond router planner + * if the query references a null-shard-key table. + * + * For this reason, right now we don't expect an INSERT .. SELECT + * query to go through the repartitioned INSERT .. SELECT logic if the + * SELECT query references a null-shard-key table. + */ + Assert(!repartitioned || + !GetRTEListPropertiesForQuery(selectQueryCopy)->hasDistTableWithoutShardKey); + distributedPlan->insertSelectQuery = insertSelectQuery; distributedPlan->selectPlanForInsertSelect = selectPlan; distributedPlan->insertSelectMethod = repartitioned ? diff --git a/src/test/regress/expected/create_null_dist_key.out b/src/test/regress/expected/create_null_dist_key.out index 43120a454..af6e66f62 100644 --- a/src/test/regress/expected/create_null_dist_key.out +++ b/src/test/regress/expected/create_null_dist_key.out @@ -1803,7 +1803,6 @@ ALTER TABLE trigger_table_3 ENABLE TRIGGER ALL; -- try a few simple queries at least to make sure that we don't crash BEGIN; INSERT INTO nullkey_c1_t1 SELECT * FROM nullkey_c2_t1; -ERROR: cannot select from a non-colocated distributed table when inserting into a distributed table that does not have a shard key ROLLBACK; DROP TRIGGER IF EXISTS trigger_1 ON trigger_table_1; DROP TRIGGER trigger_2 ON trigger_table_2 CASCADE; diff --git a/src/test/regress/expected/insert_select_null_dist_key.out b/src/test/regress/expected/insert_select_null_dist_key.out new file mode 100644 index 000000000..b5391063c --- /dev/null +++ b/src/test/regress/expected/insert_select_null_dist_key.out @@ -0,0 +1,814 @@ +CREATE SCHEMA insert_select_null_dist_key; +SET search_path TO insert_select_null_dist_key; +SET citus.next_shard_id TO 1820000; +SET citus.shard_count TO 32; +SET client_min_messages TO WARNING; +SELECT 1 FROM citus_add_node('localhost', :master_port, groupid => 0); + ?column? +--------------------------------------------------------------------- + 1 +(1 row) + +SET client_min_messages TO NOTICE; +CREATE TABLE nullkey_c1_t1(a int, b int); +CREATE TABLE nullkey_c1_t2(a int, b int); +SELECT create_distributed_table('nullkey_c1_t1', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT create_distributed_table('nullkey_c1_t2', null, colocate_with=>'nullkey_c1_t1'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE nullkey_c2_t1(a int, b int); +SELECT create_distributed_table('nullkey_c2_t1', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE reference_table(a int, b int); +SELECT create_reference_table('reference_table'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE distributed_table_c1_t1(a int, b int); +SELECT create_distributed_table('distributed_table_c1_t1', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE distributed_table_c1_t2(a int, b int); +SELECT create_distributed_table('distributed_table_c1_t2', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE distributed_table_c2_t1(a int, b int); +SELECT create_distributed_table('distributed_table_c2_t1', 'a', colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE citus_local_table(a int, b int); +SELECT citus_add_local_table_to_metadata('citus_local_table'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE postgres_local_table(a int, b int); +CREATE FUNCTION reload_tables() RETURNS void AS $$ + BEGIN + SET LOCAL client_min_messages TO WARNING; + + TRUNCATE nullkey_c1_t1, nullkey_c1_t2, nullkey_c2_t1, reference_table, distributed_table_c1_t1, + distributed_table_c1_t2, distributed_table_c2_t1, citus_local_table, postgres_local_table; + + INSERT INTO nullkey_c1_t1 SELECT i, i FROM generate_series(1, 8) i; + INSERT INTO nullkey_c1_t2 SELECT i, i FROM generate_series(2, 7) i; + INSERT INTO nullkey_c2_t1 SELECT i, i FROM generate_series(2, 7) i; + INSERT INTO reference_table SELECT i, i FROM generate_series(0, 5) i; + INSERT INTO distributed_table_c1_t1 SELECT i, i FROM generate_series(3, 8) i; + INSERT INTO distributed_table_c1_t2 SELECT i, i FROM generate_series(2, 9) i; + INSERT INTO distributed_table_c2_t1 SELECT i, i FROM generate_series(5, 10) i; + INSERT INTO citus_local_table SELECT i, i FROM generate_series(0, 10) i; + INSERT INTO postgres_local_table SELECT i, i FROM generate_series(5, 10) i; + END; +$$ LANGUAGE plpgsql; +SELECT reload_tables(); + reload_tables +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE append_table (a int, b int); +SELECT create_distributed_table('append_table', 'a', 'append'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +SELECT master_create_empty_shard('append_table') AS shardid1 \gset +SELECT master_create_empty_shard('append_table') AS shardid2 \gset +SELECT master_create_empty_shard('append_table') AS shardid3 \gset +COPY append_table (a, b) FROM STDIN WITH (format 'csv', append_to_shard :shardid1); +COPY append_table (a, b) FROM STDIN WITH (format 'csv', append_to_shard :shardid2); +CREATE TABLE range_table(a int, b int); +SELECT create_distributed_table('range_table', 'a', 'range'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CALL public.create_range_partitioned_shards('range_table', '{"0","25"}','{"24","49"}'); +INSERT INTO range_table VALUES (0, 1), (1, 2), (2, 3), (3, 4), (4, 5), (5, 6), (6, 50); +CREATE MATERIALIZED VIEW matview AS SELECT b*2+a AS a, a*a AS b FROM nullkey_c1_t1; +SET client_min_messages TO DEBUG2; +-- Test inserting into a distributed table by selecting from a combination of +-- different table types together with null-shard-key tables. +-- use a null-shard-key table +INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1; +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +-- use a reference table +INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN reference_table USING (a); +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 RIGHT JOIN reference_table USING (b) WHERE reference_table.a >= 1 AND reference_table.a <= 5; +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 LEFT JOIN reference_table USING (b); +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 INTERSECT SELECT * FROM reference_table; +DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +-- use a colocated null-shard-key table +INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN nullkey_c1_t2 USING (b); +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 FULL JOIN nullkey_c1_t2 USING (a); +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 FULL JOIN matview USING (a); +DEBUG: distributed INSERT ... SELECT cannot select from a local relation when inserting into a distributed table +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +INSERT INTO distributed_table_c1_t1 SELECT * FROM nullkey_c1_t1 UNION SELECT * FROM nullkey_c1_t2; +DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +-- use a non-colocated null-shard-key table +INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 LEFT JOIN nullkey_c2_t1 USING (a); +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: found no worker with all shard placements +INSERT INTO distributed_table_c1_t1 SELECT * FROM nullkey_c1_t1 UNION SELECT * FROM nullkey_c2_t1; +DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: found no worker with all shard placements +-- use a distributed table that is colocated with the target table +INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a); +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +INSERT INTO distributed_table_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a); +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +INSERT INTO distributed_table_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (b); +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +INSERT INTO distributed_table_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a) WHERE distributed_table_c1_t2.a = 1; +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +DEBUG: Creating router plan +DEBUG: query has a single distribution column value: 1 +DEBUG: Collecting INSERT ... SELECT results on coordinator +-- use a distributed table that is not colocated with the target table +INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 JOIN distributed_table_c2_t1 USING (a); +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +-- use a citus local table +INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN citus_local_table USING (a); +DEBUG: distributed INSERT ... SELECT cannot select from distributed tables and local tables at the same time +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +-- use a postgres local table +INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 JOIN postgres_local_table USING (a); +DEBUG: distributed INSERT ... SELECT cannot select from distributed tables and local tables at the same time +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +-- use append / range distributed tables +INSERT INTO range_table SELECT * FROM nullkey_c1_t1; +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO append_table SELECT * FROM nullkey_c1_t1; +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: INSERT ... SELECT into an append-distributed table is not supported +SELECT avg(a), avg(b) FROM distributed_table_c1_t1 ORDER BY 1, 2; +DEBUG: Router planner cannot handle multi-shard select queries + avg | avg +--------------------------------------------------------------------- + 4.2105263157894737 | 4.2105263157894737 +(1 row) + +TRUNCATE distributed_table_c1_t1; +INSERT INTO distributed_table_c1_t1 SELECT i, i FROM generate_series(3, 8) i; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +-- Test inserting into a reference table by selecting from a combination of +-- different table types together with null-shard-key tables. +-- use a null-shard-key table +INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1; +DEBUG: only reference tables may be queried when targeting a reference table with distributed INSERT ... SELECT +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +-- use a reference table +INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN reference_table USING (a); +DEBUG: only reference tables may be queried when targeting a reference table with distributed INSERT ... SELECT +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO reference_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 LEFT JOIN reference_table USING (b); +DEBUG: only reference tables may be queried when targeting a reference table with distributed INSERT ... SELECT +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 UNION SELECT * FROM reference_table; +DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO reference_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 LEFT JOIN reference_table USING (b) WHERE b IN (SELECT b FROM matview); +DEBUG: only reference tables may be queried when targeting a reference table with distributed INSERT ... SELECT +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +-- use a colocated null-shard-key table +INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN nullkey_c1_t2 USING (b); +DEBUG: only reference tables may be queried when targeting a reference table with distributed INSERT ... SELECT +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 FULL JOIN nullkey_c1_t2 USING (a); +DEBUG: only reference tables may be queried when targeting a reference table with distributed INSERT ... SELECT +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +-- use a non-colocated null-shard-key table +INSERT INTO reference_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 LEFT JOIN nullkey_c2_t1 USING (a); +DEBUG: only reference tables may be queried when targeting a reference table with distributed INSERT ... SELECT +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: found no worker with all shard placements +-- use a distributed table +INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a); +DEBUG: only reference tables may be queried when targeting a reference table with distributed INSERT ... SELECT +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +INSERT INTO reference_table SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a); +DEBUG: only reference tables may be queried when targeting a reference table with distributed INSERT ... SELECT +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +INSERT INTO reference_table SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (b); +DEBUG: only reference tables may be queried when targeting a reference table with distributed INSERT ... SELECT +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +INSERT INTO reference_table SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a) WHERE distributed_table_c1_t2.a = 1; +DEBUG: only reference tables may be queried when targeting a reference table with distributed INSERT ... SELECT +DEBUG: Creating router plan +DEBUG: query has a single distribution column value: 1 +DEBUG: Collecting INSERT ... SELECT results on coordinator +-- use a citus local table +INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN citus_local_table USING (a); +DEBUG: distributed INSERT ... SELECT cannot select from distributed tables and local tables at the same time +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +-- use a postgres local table +INSERT INTO reference_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 JOIN postgres_local_table USING (a); +DEBUG: distributed INSERT ... SELECT cannot select from distributed tables and local tables at the same time +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +SELECT avg(a), avg(b) FROM reference_table ORDER BY 1, 2; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan + avg | avg +--------------------------------------------------------------------- + 4.0428571428571429 | 4.0428571428571429 +(1 row) + +TRUNCATE reference_table; +INSERT INTO reference_table SELECT i, i FROM generate_series(0, 5) i; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +-- Test inserting into a citus local table by selecting from a combination of +-- different table types together with null-shard-key tables. +-- use a null-shard-key table +INSERT INTO citus_local_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1; +DEBUG: distributed INSERT ... SELECT cannot insert into a local table that is added to metadata +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +-- use a reference table +INSERT INTO citus_local_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN reference_table USING (a); +DEBUG: distributed INSERT ... SELECT cannot insert into a local table that is added to metadata +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +-- use a colocated null-shard-key table +INSERT INTO citus_local_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN nullkey_c1_t2 USING (b); +DEBUG: distributed INSERT ... SELECT cannot insert into a local table that is added to metadata +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +-- use a distributed table +INSERT INTO citus_local_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a); +DEBUG: distributed INSERT ... SELECT cannot insert into a local table that is added to metadata +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +-- use a citus local table +INSERT INTO citus_local_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN citus_local_table USING (a); +DEBUG: distributed INSERT ... SELECT cannot select from distributed tables and local tables at the same time +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +-- use a postgres local table +INSERT INTO citus_local_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 JOIN postgres_local_table USING (a); +DEBUG: distributed INSERT ... SELECT cannot select from distributed tables and local tables at the same time +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +SELECT avg(a), avg(b) FROM citus_local_table ORDER BY 1, 2; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan + avg | avg +--------------------------------------------------------------------- + 4.4333333333333333 | 4.4333333333333333 +(1 row) + +TRUNCATE citus_local_table; +INSERT INTO citus_local_table SELECT i, i FROM generate_series(0, 10) i; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +-- Test inserting into a null-shard-key table by selecting from a combination of +-- different table types, together with or without null-shard-key tables. +-- use a postgres local table +INSERT INTO nullkey_c1_t1 SELECT postgres_local_table.a, postgres_local_table.b FROM postgres_local_table; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO nullkey_c1_t1 SELECT postgres_local_table.a, postgres_local_table.b FROM postgres_local_table JOIN reference_table USING (a); +DEBUG: distributed INSERT ... SELECT cannot select from a local table +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO nullkey_c1_t1 SELECT postgres_local_table.a, postgres_local_table.b FROM postgres_local_table LEFT JOIN nullkey_c1_t1 USING (a); +DEBUG: distributed INSERT ... SELECT cannot select from distributed tables and local tables at the same time +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +-- use a citus local table +INSERT INTO nullkey_c1_t1 SELECT citus_local_table.a, citus_local_table.b FROM citus_local_table; +DEBUG: distributed INSERT ... SELECT cannot select from a local relation when inserting into a distributed table +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO nullkey_c1_t1 SELECT citus_local_table.a, citus_local_table.b FROM citus_local_table JOIN reference_table USING (a) JOIN postgres_local_table USING (a) ORDER BY 1,2 OFFSET 7; +DEBUG: distributed INSERT ... SELECT cannot select from a local table +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO nullkey_c1_t1 SELECT citus_local_table.a, citus_local_table.b FROM citus_local_table JOIN nullkey_c1_t1 USING (a); +DEBUG: distributed INSERT ... SELECT cannot select from distributed tables and local tables at the same time +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +-- use a distributed table +INSERT INTO nullkey_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM distributed_table_c1_t2; +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO nullkey_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM distributed_table_c1_t2 JOIN reference_table USING (a); +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO nullkey_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM distributed_table_c1_t2 JOIN nullkey_c1_t1 USING (a); +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +-- use a non-colocated null-shard-key table +INSERT INTO nullkey_c2_t1 SELECT q.* FROM (SELECT reference_table.* FROM reference_table LEFT JOIN nullkey_c1_t1 USING (a)) q JOIN nullkey_c1_t2 USING (a); +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +-- use a materialized view +INSERT INTO nullkey_c1_t1 SELECT * FROM matview; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO nullkey_c1_t1 SELECT reference_table.a, reference_table.b FROM reference_table JOIN matview ON (reference_table.a = matview.a); +DEBUG: distributed INSERT ... SELECT cannot select from a local relation when inserting into a distributed table +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO nullkey_c1_t1 SELECT q.* FROM (SELECT reference_table.* FROM reference_table JOIN nullkey_c1_t1 USING (a)) q JOIN matview USING (a); +DEBUG: distributed INSERT ... SELECT cannot select from a local relation when inserting into a distributed table +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +-- use append / range distributed tables +INSERT INTO nullkey_c1_t1 SELECT * FROM range_table; +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO nullkey_c1_t1 SELECT * FROM append_table; +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +DEBUG: Router planner does not support append-partitioned tables. +DEBUG: Collecting INSERT ... SELECT results on coordinator +SELECT avg(a), avg(b) FROM nullkey_c1_t1 ORDER BY 1, 2; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan + avg | avg +--------------------------------------------------------------------- + 5.8611111111111111 | 13.9305555555555556 +(1 row) + +SELECT avg(a), avg(b) FROM nullkey_c2_t1 ORDER BY 1, 2; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan + avg | avg +--------------------------------------------------------------------- + 3.8750000000000000 | 3.8750000000000000 +(1 row) + +TRUNCATE nullkey_c1_t1, nullkey_c2_t1; +INSERT INTO nullkey_c1_t1 SELECT i, i FROM generate_series(1, 8) i; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO nullkey_c2_t1 SELECT i, i FROM generate_series(2, 7) i; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +-- Test inserting into a local table by selecting from a combination of +-- different table types, together with or without null-shard-key tables. +INSERT INTO postgres_local_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN reference_table USING (a); +DEBUG: Creating router plan +INSERT INTO postgres_local_table SELECT * FROM nullkey_c1_t1 ORDER BY 1,2 OFFSET 3 LIMIT 2; +DEBUG: Creating router plan +WITH cte_1 AS ( + DELETE FROM nullkey_c1_t1 WHERE a >= 1 and a <= 4 RETURNING * +) +INSERT INTO postgres_local_table SELECT cte_1.* FROM cte_1 LEFT JOIN nullkey_c1_t2 USING (a) WHERE nullkey_c1_t2.a IS NULL; +DEBUG: Creating router plan +INSERT INTO postgres_local_table SELECT * FROM nullkey_c1_t1 EXCEPT SELECT * FROM postgres_local_table; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +SELECT avg(a), avg(b) FROM postgres_local_table ORDER BY 1, 2; + avg | avg +--------------------------------------------------------------------- + 5.0000000000000000 | 5.0000000000000000 +(1 row) + +TRUNCATE postgres_local_table; +INSERT INTO postgres_local_table SELECT i, i FROM generate_series(5, 10) i; +-- Try slightly more complex queries. +WITH cte_1 AS ( + SELECT nullkey_c1_t1.a, reference_table.b FROM nullkey_c1_t1 JOIN reference_table USING (a) +), +cte_2 AS ( + SELECT reference_table.a, postgres_local_table.b FROM postgres_local_table LEFT JOIN reference_table USING (b) +) +INSERT INTO distributed_table_c1_t1 +SELECT cte_1.* FROM cte_1 JOIN cte_2 USING (a) JOIN distributed_table_c1_t2 USING (a) ORDER BY 1,2; +DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match +DETAIL: The target table's partition column should correspond to a partition column in the subquery. +DEBUG: CTE cte_1 is going to be inlined via distributed planning +DEBUG: CTE cte_2 is going to be inlined via distributed planning +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +WITH cte_1 AS ( + SELECT nullkey_c1_t1.a, reference_table.b FROM nullkey_c1_t1 JOIN reference_table USING (a) +), +cte_2 AS ( + SELECT * FROM nullkey_c1_t2 WHERE EXISTS ( + SELECT 1 FROM reference_table WHERE reference_table.a = nullkey_c1_t2.a + ) + ORDER BY 1,2 OFFSET 1 LIMIT 4 +) +INSERT INTO distributed_table_c1_t1 +SELECT * FROM cte_1 UNION SELECT * FROM cte_2 EXCEPT SELECT * FROM reference_table; +DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries +DEBUG: CTE cte_1 is going to be inlined via distributed planning +DEBUG: CTE cte_2 is going to be inlined via distributed planning +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO distributed_table_c1_t1 (a, b) +SELECT t1.a, t2.b +FROM nullkey_c1_t1 t1 +JOIN ( + SELECT b FROM nullkey_c1_t2 ORDER BY b DESC LIMIT 1 +) t2 +ON t1.b < t2.b; +DEBUG: LIMIT clauses are not allowed in distributed INSERT ... SELECT queries +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO distributed_table_c1_t1 (a, b) +WITH cte AS ( + SELECT a, b, + (SELECT a FROM nullkey_c1_t2 WHERE b = t.b) AS d1, + (SELECT a FROM reference_table WHERE b = t.b) AS d2 + FROM nullkey_c1_t1 t +) +SELECT d1, COALESCE(d2, a) FROM cte WHERE d1 IS NOT NULL AND d2 IS NOT NULL; +DEBUG: CTE cte is going to be inlined via distributed planning +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO citus_local_table (a, b) +SELECT t1.a, t2.b +FROM nullkey_c1_t1 t1 +CROSS JOIN ( + SELECT b FROM nullkey_c2_t1 ORDER BY b LIMIT 1 +) t2; +DEBUG: distributed INSERT ... SELECT cannot insert into a local table that is added to metadata +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: found no worker with all shard placements +INSERT INTO distributed_table_c1_t1 (a, b) +SELECT t1.a, t2.b +FROM reference_table t1 +LEFT JOIN ( + SELECT b, ROW_NUMBER() OVER (ORDER BY b DESC) AS rn + FROM nullkey_c1_t1 +) t2 ON t1.b = t2.b +WHERE t2.rn > 0; +DEBUG: Window functions without PARTITION BY on distribution column is currently unsupported +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO nullkey_c1_t1 (a, b) +SELECT t1.a, t2.b +FROM nullkey_c1_t1 t1 +JOIN ( + SELECT rn, b + FROM ( + SELECT b, ROW_NUMBER() OVER (ORDER BY b DESC) AS rn + FROM distributed_table_c2_t1 + ) q +) t2 ON t1.b = t2.b +WHERE t2.rn > 2; +DEBUG: Window functions without PARTITION BY on distribution column is currently unsupported +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +INSERT INTO distributed_table_c1_t1 (a, b) +SELECT t1.a, t2.b +FROM nullkey_c1_t1 t1 +JOIN ( + SELECT sum_val, b + FROM ( + SELECT b, SUM(a) OVER (PARTITION BY b) AS sum_val + FROM nullkey_c1_t1 + ) q +) t2 ON t1.b = t2.b +WHERE t2.sum_val > 2; +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +-- MultiTaskRouterSelectQuerySupported() is unnecessarily restrictive +-- about pushing down queries with DISTINCT ON clause even if the table +-- doesn't have a shard key. See https://github.com/citusdata/citus/pull/6752. +INSERT INTO nullkey_c1_t1 SELECT DISTINCT ON (a) a, b FROM nullkey_c1_t2; +DEBUG: DISTINCT ON (non-partition column) clauses are not allowed in distributed INSERT ... SELECT queries +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +-- Similarly, we could push down the following query as well. see +-- https://github.com/citusdata/citus/pull/6831. +INSERT INTO nullkey_c1_t1 SELECT b, SUM(a) OVER (ORDER BY b) AS sum_val FROM nullkey_c1_t1; +DEBUG: Window functions without PARTITION BY on distribution column is currently unsupported +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO nullkey_c2_t1 +SELECT t2.a, t2.b +FROM nullkey_c1_t1 AS t2 +JOIN reference_table AS t3 ON (t2.a = t3.a) +WHERE NOT EXISTS ( + SELECT 1 FROM nullkey_c1_t2 AS t1 WHERE t1.b = t3.b +); +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO distributed_table_c1_t1 +SELECT t1.a, t1.b +FROM nullkey_c1_t1 AS t1 +WHERE t1.a NOT IN ( + SELECT DISTINCT t2.a FROM distributed_table_c1_t2 AS t2 +); +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Router planner cannot handle multi-shard select queries +INSERT INTO distributed_table_c1_t1 +SELECT t1.a, t1.b +FROM reference_table AS t1 +JOIN ( + SELECT t2.a FROM ( + SELECT a FROM nullkey_c1_t1 + UNION + SELECT a FROM nullkey_c1_t2 + ) AS t2 +) AS t3 ON t1.a = t3.a; +DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator +-- Temporaryly reduce the verbosity to avoid noise +-- in the output of the next query. +SET client_min_messages TO DEBUG1; +INSERT INTO nullkey_c1_t1 +SELECT t1.a, t1.b +FROM reference_table AS t1 +WHERE t1.a IN ( + SELECT t2.a FROM ( + SELECT t3.a FROM ( + SELECT a FROM distributed_table_c1_t1 WHERE b > 4 + ) AS t3 + JOIN ( + SELECT a FROM distributed_table_c1_t2 WHERE b < 7 + ) AS t4 ON t3.a = t4.a + ) AS t2 +); +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +DEBUG: generating subplan XXX_1 for subquery SELECT a FROM (SELECT t3.a FROM ((SELECT distributed_table_c1_t1.a FROM insert_select_null_dist_key.distributed_table_c1_t1 WHERE (distributed_table_c1_t1.b OPERATOR(pg_catalog.>) 4)) t3 JOIN (SELECT distributed_table_c1_t2.a FROM insert_select_null_dist_key.distributed_table_c1_t2 WHERE (distributed_table_c1_t2.b OPERATOR(pg_catalog.<) 7)) t4 ON ((t3.a OPERATOR(pg_catalog.=) t4.a)))) t2 +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT a, b FROM insert_select_null_dist_key.reference_table t1 WHERE (a OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer))) +DEBUG: Collecting INSERT ... SELECT results on coordinator +SET client_min_messages TO DEBUG2; +-- test upsert with plain INSERT query +CREATE TABLE upsert_test_1 +( + unique_col int UNIQUE, + other_col int, + third_col int +); +DEBUG: CREATE TABLE / UNIQUE will create implicit index "upsert_test_1_unique_col_key" for table "upsert_test_1" +SELECT create_distributed_table('upsert_test_1', null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE upsert_test_2(key int primary key, value text); +DEBUG: CREATE TABLE / PRIMARY KEY will create implicit index "upsert_test_2_pkey" for table "upsert_test_2" +SELECT create_distributed_table('upsert_test_2', null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO upsert_test_2 AS upsert_test_2_alias (key, value) VALUES (1, '5') ON CONFLICT(key) + DO UPDATE SET value = (upsert_test_2_alias.value::int * 2)::text; +DEBUG: Creating router plan +INSERT INTO upsert_test_2 (key, value) VALUES (1, '5') ON CONFLICT(key) + DO UPDATE SET value = (upsert_test_2.value::int * 3)::text; +DEBUG: Creating router plan +INSERT INTO upsert_test_1 (unique_col, other_col) VALUES (1, 1) ON CONFLICT (unique_col) + DO UPDATE SET other_col = (SELECT count(*) from upsert_test_1); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: subqueries are not supported within INSERT queries +HINT: Try rewriting your queries with 'INSERT INTO ... SELECT' syntax. +INSERT INTO upsert_test_1 (unique_col, other_col) VALUES (1, 1) ON CONFLICT (unique_col) + DO UPDATE SET other_col = random()::int; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: functions used in the DO UPDATE SET clause of INSERTs on distributed tables must be marked IMMUTABLE +INSERT INTO upsert_test_1 (unique_col, other_col) VALUES (1, 1) ON CONFLICT (unique_col) + DO UPDATE SET other_col = 5 WHERE upsert_test_1.other_col = random()::int; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: functions used in the WHERE clause of the ON CONFLICT clause of INSERTs on distributed tables must be marked IMMUTABLE +INSERT INTO upsert_test_1 VALUES (3, 5, 7); +DEBUG: Creating router plan +INSERT INTO upsert_test_1 (unique_col, other_col) VALUES (1, 1) ON CONFLICT (unique_col) WHERE unique_col = random()::int + DO UPDATE SET other_col = 5; +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: functions used in the WHERE clause of the ON CONFLICT clause of INSERTs on distributed tables must be marked IMMUTABLE +CREATE TABLE upsert_test_3 (key_1 int, key_2 bigserial, value text DEFAULT 'default_value', PRIMARY KEY (key_1, key_2)); +DEBUG: CREATE TABLE will create implicit sequence "upsert_test_3_key_2_seq" for serial column "upsert_test_3.key_2" +DEBUG: CREATE TABLE / PRIMARY KEY will create implicit index "upsert_test_3_pkey" for table "upsert_test_3" +SELECT create_distributed_table('upsert_test_3', null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +INSERT INTO upsert_test_3 VALUES (1, DEFAULT, '1') RETURNING *; +DEBUG: Creating router plan + key_1 | key_2 | value +--------------------------------------------------------------------- + 1 | 1 | 1 +(1 row) + +INSERT INTO upsert_test_3 VALUES (5, DEFAULT, DEFAULT) RETURNING *; +DEBUG: Creating router plan + key_1 | key_2 | value +--------------------------------------------------------------------- + 5 | 2 | default_value +(1 row) + +SET client_min_messages TO DEBUG1; +INSERT INTO upsert_test_3 SELECT 7, other_col, 'harcoded_text_value' FROM upsert_test_1 RETURNING *; + key_1 | key_2 | value +--------------------------------------------------------------------- + 7 | 5 | harcoded_text_value +(1 row) + +SET client_min_messages TO DEBUG2; +-- test upsert with INSERT .. SELECT queries +SET client_min_messages TO DEBUG1; +INSERT INTO upsert_test_1 (unique_col, other_col) SELECT unique_col, other_col FROM upsert_test_1 ON CONFLICT (unique_col) + DO UPDATE SET other_col = upsert_test_1.other_col + 1; +-- Fails due to https://github.com/citusdata/citus/issues/6826. +INSERT INTO upsert_test_1 (unique_col, other_col) SELECT unique_col, other_col FROM upsert_test_1 ON CONFLICT (unique_col) + DO UPDATE SET other_col = (SELECT count(*) from upsert_test_1); +ERROR: cannot execute a distributed query from a query on a shard +DETAIL: Executing a distributed query in a function call that may be pushed to a remote node can lead to incorrect results. +HINT: Avoid nesting of distributed queries or use alter user current_user set citus.allow_nested_distributed_execution to on to allow it with possible incorrectness. +CONTEXT: while executing command on localhost:xxxxx +SET client_min_messages TO DEBUG2; +INSERT INTO upsert_test_1 (unique_col, other_col) SELECT unique_col, other_col FROM upsert_test_1 ON CONFLICT (unique_col) + DO UPDATE SET other_col = random()::int; +ERROR: functions used in the DO UPDATE SET clause of INSERTs on distributed tables must be marked IMMUTABLE +INSERT INTO upsert_test_1 (unique_col, other_col) SELECT unique_col, other_col FROM upsert_test_1 ON CONFLICT (unique_col) + DO UPDATE SET other_col = 5 WHERE upsert_test_1.other_col = random()::int; +ERROR: functions used in the WHERE clause of the ON CONFLICT clause of INSERTs on distributed tables must be marked IMMUTABLE +SELECT reload_tables(); +DEBUG: function does not have co-located tables + reload_tables +--------------------------------------------------------------------- + +(1 row) + +ALTER TABLE nullkey_c1_t1 ADD PRIMARY KEY (a); +DEBUG: ALTER TABLE / ADD PRIMARY KEY will create implicit index "nullkey_c1_t1_pkey" for table "nullkey_c1_t1" +DEBUG: verifying table "nullkey_c1_t1" +ALTER TABLE distributed_table_c1_t1 ADD PRIMARY KEY (a,b); +DEBUG: ALTER TABLE / ADD PRIMARY KEY will create implicit index "distributed_table_c1_t1_pkey" for table "distributed_table_c1_t1" +DEBUG: verifying table "distributed_table_c1_t1" +INSERT INTO nullkey_c1_t1 AS t1 (a, b) SELECT t3.a, t3.b FROM nullkey_c1_t2 t2 JOIN reference_table t3 ON (t2.a = t3.a) ON CONFLICT (a) + DO UPDATE SET a = t1.a + 10; +DEBUG: distributed statement: INSERT INTO insert_select_null_dist_key.nullkey_c1_t1_1820000 AS t1 (a, b) SELECT t3.a, t3.b FROM (insert_select_null_dist_key.nullkey_c1_t2_1820001 t2 JOIN insert_select_null_dist_key.reference_table_1820003 t3 ON ((t2.a OPERATOR(pg_catalog.=) t3.a))) ON CONFLICT(a) DO UPDATE SET a = (t1.a OPERATOR(pg_catalog.+) 10) +SET client_min_messages TO DEBUG1; +INSERT INTO distributed_table_c1_t1 AS t1 (a, b) SELECT t3.a, t3.b FROM nullkey_c1_t2 t2 JOIN reference_table t3 ON (t2.a = t3.a) ON CONFLICT (a, b) + DO UPDATE SET b = t1.b + 10; +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO nullkey_c1_t1 AS t1 (a, b) SELECT t3.a, t3.b FROM distributed_table_c1_t1 t2 JOIN reference_table t3 ON (t2.a = t3.a) ON CONFLICT (a) + DO UPDATE SET a = t1.a + 10; +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +-- This also fails due to https://github.com/citusdata/citus/issues/6826. +INSERT INTO nullkey_c1_t1 AS t1 (a, b) SELECT t3.a, t3.b FROM distributed_table_c1_t1 t2 JOIN reference_table t3 ON (t2.a = t3.a) WHERE t2.a = 3 ON CONFLICT (a) + DO UPDATE SET a = (SELECT max(b)+1 FROM distributed_table_c1_t1 WHERE a = 3); +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +ERROR: cannot execute a distributed query from a query on a shard +DETAIL: Executing a distributed query in a function call that may be pushed to a remote node can lead to incorrect results. +HINT: Avoid nesting of distributed queries or use alter user current_user set citus.allow_nested_distributed_execution to on to allow it with possible incorrectness. +CONTEXT: while executing command on localhost:xxxxx +SET client_min_messages TO DEBUG2; +SELECT avg(a), avg(b) FROM distributed_table_c1_t1; +DEBUG: Router planner cannot handle multi-shard select queries + avg | avg +--------------------------------------------------------------------- + 5.0000000000000000 | 9.2857142857142857 +(1 row) + +SELECT avg(a), avg(b) FROM nullkey_c1_t1; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan + avg | avg +--------------------------------------------------------------------- + 7.5000000000000000 | 4.1666666666666667 +(1 row) + +SELECT avg(a), avg(b) FROM nullkey_c1_t2; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan + avg | avg +--------------------------------------------------------------------- + 4.5000000000000000 | 4.5000000000000000 +(1 row) + +SELECT * FROM upsert_test_1 ORDER BY unique_col; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan + unique_col | other_col | third_col +--------------------------------------------------------------------- + 3 | 6 | 7 +(1 row) + +SELECT * FROM upsert_test_2 ORDER BY key; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan + key | value +--------------------------------------------------------------------- + 1 | 15 +(1 row) + +SELECT * FROM upsert_test_3 ORDER BY key_1, key_2; +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan + key_1 | key_2 | value +--------------------------------------------------------------------- + 1 | 1 | 1 + 5 | 2 | default_value + 7 | 5 | harcoded_text_value +(3 rows) + +SET client_min_messages TO WARNING; +DROP SCHEMA insert_select_null_dist_key CASCADE; +SELECT citus_remove_node('localhost', :master_port); + citus_remove_node +--------------------------------------------------------------------- + +(1 row) + diff --git a/src/test/regress/expected/query_null_dist_key.out b/src/test/regress/expected/query_null_dist_key.out index 09413a3ea..09907a99b 100644 --- a/src/test/regress/expected/query_null_dist_key.out +++ b/src/test/regress/expected/query_null_dist_key.out @@ -827,24 +827,55 @@ INSERT INTO nullkey_c1_t1 SELECT * FROM nullkey_c1_t2; SET client_min_messages TO DEBUG2; -- between two non-colocated null dist key tables INSERT INTO nullkey_c1_t1 SELECT * FROM nullkey_c2_t1; -ERROR: cannot select from a non-colocated distributed table when inserting into a distributed table that does not have a shard key +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator -- between a null dist key table and a table of different type +SET client_min_messages TO WARNING; +EXPLAIN (ANALYZE TRUE, TIMING FALSE, COSTS FALSE, SUMMARY FALSE, VERBOSE FALSE) INSERT INTO nullkey_c1_t1 SELECT * FROM reference_table; -ERROR: cannot select from different table types when inserting into a distributed table that does not have a shard key + QUERY PLAN +--------------------------------------------------------------------- + Custom Scan (Citus Adaptive) (actual rows=0 loops=1) + Task Count: 1 + Tasks Shown: All + -> Task + Node: host=localhost port=xxxxx dbname=regression + -> Insert on nullkey_c1_t1_1620000 citus_table_alias (actual rows=0 loops=1) + -> Seq Scan on reference_table_1620005 reference_table (actual rows=6 loops=1) +(7 rows) + +SET client_min_messages TO DEBUG2; INSERT INTO nullkey_c1_t1 SELECT * FROM distributed_table; -ERROR: cannot select from different table types when inserting into a distributed table that does not have a shard key +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: Collecting INSERT ... SELECT results on coordinator INSERT INTO nullkey_c1_t1 SELECT * FROM citus_local_table; -ERROR: cannot select from different table types when inserting into a distributed table that does not have a shard key +DEBUG: distributed INSERT ... SELECT cannot select from a local relation when inserting into a distributed table +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator INSERT INTO nullkey_c1_t1 SELECT * FROM postgres_local_table; -ERROR: cannot select from different table types when inserting into a distributed table that does not have a shard key +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator INSERT INTO reference_table SELECT * FROM nullkey_c1_t1; -ERROR: cannot select from a distributed table that does not have a shard key when inserting into a different table type +DEBUG: only reference tables may be queried when targeting a reference table with distributed INSERT ... SELECT +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator INSERT INTO distributed_table SELECT * FROM nullkey_c1_t1; -ERROR: cannot select from a distributed table that does not have a shard key when inserting into a different table type +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator INSERT INTO citus_local_table SELECT * FROM nullkey_c1_t1; -ERROR: cannot select from a distributed table that does not have a shard key when inserting into a different table type +DEBUG: distributed INSERT ... SELECT cannot insert into a local table that is added to metadata +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Collecting INSERT ... SELECT results on coordinator INSERT INTO postgres_local_table SELECT * FROM nullkey_c1_t1; -ERROR: cannot select from a distributed table that does not have a shard key when inserting into a local table +DEBUG: Creating router plan -- test subquery SELECT count(*) FROM ( @@ -873,7 +904,7 @@ WITH level_0 AS ( WITH RECURSIVE level_2_recursive(x) AS ( VALUES (1) UNION ALL - SELECT a + 1 FROM nullkey_c1_t1 JOIN level_2_recursive ON (a = x) WHERE a < 100 + SELECT a + 1 FROM nullkey_c1_t1 JOIN level_2_recursive ON (a = x) WHERE a < 2 ) SELECT * FROM level_2_recursive RIGHT JOIN reference_table ON (level_2_recursive.x = reference_table.a) ) @@ -885,7 +916,7 @@ DEBUG: CTE level_1 is going to be inlined via distributed planning DEBUG: Creating router plan count --------------------------------------------------------------------- - 122 + 106 (1 row) WITH level_0 AS ( @@ -1095,6 +1126,9 @@ INSERT INTO bigserial_test (x, y) SELECT x, y FROM bigserial_test; DEBUG: volatile functions are not allowed in distributed INSERT ... SELECT queries ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables DETAIL: Sequences cannot be used in router queries +INSERT INTO bigserial_test (x, y) SELECT a, a FROM reference_table; +DEBUG: volatile functions are not allowed in distributed INSERT ... SELECT queries +DEBUG: Collecting INSERT ... SELECT results on coordinator INSERT INTO agg_events (user_id) SELECT f2.id FROM @@ -1119,7 +1153,6 @@ FROM (SELECT SUM(raw_events_second.value_4) AS v4, ON (f.id = f2.id) WHERE f.id IN (SELECT user_id FROM raw_events_second); -ERROR: cannot select from different table types when inserting into a distributed table that does not have a shard key -- upsert with returning INSERT INTO agg_events AS ae ( @@ -1153,6 +1186,17 @@ SELECT FROM raw_events_first LEFT JOIN raw_events_second ON raw_events_first.user_id = raw_events_second.user_id WHERE raw_events_second.user_id = 10 OR raw_events_second.user_id = 11; +INSERT INTO agg_events (user_id) +SELECT + users_ref_table.user_id +FROM + users_ref_table LEFT JOIN raw_events_second ON users_ref_table.user_id = raw_events_second.user_id + WHERE raw_events_second.user_id = 10 OR raw_events_second.user_id = 11; +INSERT INTO agg_events (user_id) +SELECT COALESCE(raw_events_first.user_id, users_ref_table.user_id) +FROM raw_events_first + RIGHT JOIN (users_ref_table LEFT JOIN raw_events_second ON users_ref_table.user_id = raw_events_second.user_id) + ON raw_events_first.user_id = users_ref_table.user_id; -- using a full join INSERT INTO agg_events (user_id, value_1_agg) SELECT t1.user_id AS col1, @@ -1176,11 +1220,22 @@ FROM raw_events_first WHERE NOT EXISTS (SELECT 1 FROM raw_events_second WHERE raw_events_second.user_id =raw_events_first.user_id); +INSERT INTO raw_events_second + (user_id) +SELECT user_id +FROM users_ref_table +WHERE NOT EXISTS (SELECT 1 + FROM raw_events_second + WHERE raw_events_second.user_id = users_ref_table.user_id); -- using inner join INSERT INTO agg_events (user_id) SELECT raw_events_first.user_id FROM raw_events_first INNER JOIN raw_events_second ON raw_events_first.user_id = raw_events_second.value_1 WHERE raw_events_first.value_1 IN (10, 11,12) OR raw_events_second.user_id IN (1,2,3,4); +INSERT INTO agg_events (user_id) +SELECT raw_events_first.user_id +FROM raw_events_first INNER JOIN users_ref_table ON raw_events_first.user_id = users_ref_table.user_id +WHERE raw_events_first.value_1 IN (10, 11,12) OR users_ref_table.user_id IN (1,2,3,4); -- We could relax distributed insert .. select checks to allow pushing -- down more clauses down to the worker nodes when inserting into a single -- shard by selecting from a colocated one. We might want to do something @@ -1197,6 +1252,9 @@ DEBUG: Collecting INSERT ... SELECT results on coordinator INSERT INTO agg_events (user_id) SELECT raw_events_first.user_id FROM raw_events_first OFFSET 1; DEBUG: OFFSET clauses are not allowed in distributed INSERT ... SELECT queries DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO agg_events (user_id) SELECT users_ref_table.user_id FROM users_ref_table LIMIT 1; +DEBUG: LIMIT clauses are not allowed in distributed INSERT ... SELECT queries +DEBUG: Collecting INSERT ... SELECT results on coordinator -- using a materialized cte WITH cte AS MATERIALIZED (SELECT max(value_1)+1 as v1_agg, user_id FROM raw_events_first GROUP BY user_id) @@ -1207,6 +1265,9 @@ DEBUG: Collecting INSERT ... SELECT results on coordinator INSERT INTO raw_events_second WITH cte AS MATERIALIZED (SELECT * FROM raw_events_first) SELECT user_id * 1000, time, value_1, value_2, value_3, value_4 FROM cte; +INSERT INTO raw_events_second (user_id) + WITH cte AS MATERIALIZED (SELECT * FROM users_ref_table) + SELECT user_id FROM cte; -- using a regular cte WITH cte AS (SELECT * FROM raw_events_first) INSERT INTO raw_events_second @@ -1229,7 +1290,7 @@ DEBUG: Subqueries without relations are not allowed in distributed INSERT ... S DEBUG: Collecting INSERT ... SELECT results on coordinator -- we still support complex joins via INSERT's cte list .. WITH cte AS ( - SELECT reference_table.a AS a, 1 AS b + SELECT DISTINCT(reference_table.a) AS a, 1 AS b FROM distributed_table RIGHT JOIN reference_table USING (a) ) INSERT INTO raw_events_second (user_id, value_1) @@ -1240,17 +1301,23 @@ DEBUG: recursively planning left side of the right join since the outer side is DEBUG: recursively planning distributed relation "distributed_table" since it is part of a distributed join node that is outer joined with a recurring rel DEBUG: Wrapping relation "distributed_table" to a subquery DEBUG: generating subplan XXX_1 for subquery SELECT a FROM query_null_dist_key.distributed_table WHERE true -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT "?column?" AS user_id, b AS value_1 FROM (SELECT ((cte.a OPERATOR(pg_catalog.+) 5) OPERATOR(pg_catalog.*) '-1'::integer), cte.b FROM (SELECT reference_table.a, 1 AS b FROM ((SELECT distributed_table_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) distributed_table_1) distributed_table RIGHT JOIN query_null_dist_key.reference_table USING (a))) cte) citus_insert_select_subquery("?column?", b) +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT "?column?" AS user_id, b AS value_1 FROM (SELECT ((cte.a OPERATOR(pg_catalog.+) 5) OPERATOR(pg_catalog.*) '-1'::integer), cte.b FROM (SELECT DISTINCT reference_table.a, 1 AS b FROM ((SELECT distributed_table_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) distributed_table_1) distributed_table RIGHT JOIN query_null_dist_key.reference_table USING (a))) cte) citus_insert_select_subquery("?column?", b) DEBUG: Collecting INSERT ... SELECT results on coordinator --- .. but can't do so via via SELECT's cte list +-- .. and via SELECT's cte list too INSERT INTO raw_events_second (user_id, value_1) WITH cte AS ( - SELECT reference_table.a AS a, 1 AS b + SELECT DISTINCT(reference_table.a) AS a, 1 AS b FROM distributed_table RIGHT JOIN reference_table USING (a) ) - SELECT (a+5)*-1, b FROM cte; + SELECT (a+5)*2, b FROM cte; DEBUG: CTE cte is going to be inlined via distributed planning -ERROR: cannot select from different table types when inserting into a distributed table that does not have a shard key +DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables +DEBUG: recursively planning left side of the right join since the outer side is a recurring rel +DEBUG: recursively planning distributed relation "distributed_table" since it is part of a distributed join node that is outer joined with a recurring rel +DEBUG: Wrapping relation "distributed_table" to a subquery +DEBUG: generating subplan XXX_1 for subquery SELECT a FROM query_null_dist_key.distributed_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT ((a OPERATOR(pg_catalog.+) 5) OPERATOR(pg_catalog.*) 2) AS user_id, b AS value_1 FROM (SELECT DISTINCT reference_table.a, 1 AS b FROM ((SELECT distributed_table_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) distributed_table_1) distributed_table RIGHT JOIN query_null_dist_key.reference_table USING (a))) cte +DEBUG: Collecting INSERT ... SELECT results on coordinator -- using set operations INSERT INTO raw_events_first(user_id) @@ -1258,6 +1325,12 @@ INSERT INTO (SELECT user_id FROM raw_events_first); DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries DEBUG: Collecting INSERT ... SELECT results on coordinator +INSERT INTO + raw_events_first(user_id) + (SELECT user_id FROM users_ref_table) INTERSECT + (SELECT user_id FROM raw_events_first); +DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries +DEBUG: Collecting INSERT ... SELECT results on coordinator -- group by clause inside subquery INSERT INTO agg_events (user_id) @@ -1313,6 +1386,10 @@ SELECT SUM(value_3), Avg(value_2) FROM raw_events_first GROUP BY user_id; +INSERT INTO agg_events (value_3_agg, value_1_agg) +SELECT AVG(user_id), SUM(user_id) +FROM users_ref_table +GROUP BY user_id; -- using generate_series INSERT INTO raw_events_first (user_id, value_1, value_2) SELECT s, s, s FROM generate_series(1, 5) s; diff --git a/src/test/regress/multi_1_schedule b/src/test/regress/multi_1_schedule index 45adf469e..9163a5864 100644 --- a/src/test/regress/multi_1_schedule +++ b/src/test/regress/multi_1_schedule @@ -201,6 +201,7 @@ test: local_dist_join_mixed test: citus_local_dist_joins test: recurring_outer_join test: query_null_dist_key +test: insert_select_null_dist_key test: pg_dump # --------- diff --git a/src/test/regress/sql/insert_select_null_dist_key.sql b/src/test/regress/sql/insert_select_null_dist_key.sql new file mode 100644 index 000000000..29454b0c1 --- /dev/null +++ b/src/test/regress/sql/insert_select_null_dist_key.sql @@ -0,0 +1,470 @@ +CREATE SCHEMA insert_select_null_dist_key; +SET search_path TO insert_select_null_dist_key; + +SET citus.next_shard_id TO 1820000; +SET citus.shard_count TO 32; + +SET client_min_messages TO WARNING; +SELECT 1 FROM citus_add_node('localhost', :master_port, groupid => 0); + +SET client_min_messages TO NOTICE; + +CREATE TABLE nullkey_c1_t1(a int, b int); +CREATE TABLE nullkey_c1_t2(a int, b int); +SELECT create_distributed_table('nullkey_c1_t1', null, colocate_with=>'none'); +SELECT create_distributed_table('nullkey_c1_t2', null, colocate_with=>'nullkey_c1_t1'); + +CREATE TABLE nullkey_c2_t1(a int, b int); +SELECT create_distributed_table('nullkey_c2_t1', null, colocate_with=>'none'); + +CREATE TABLE reference_table(a int, b int); +SELECT create_reference_table('reference_table'); + +CREATE TABLE distributed_table_c1_t1(a int, b int); +SELECT create_distributed_table('distributed_table_c1_t1', 'a'); + +CREATE TABLE distributed_table_c1_t2(a int, b int); +SELECT create_distributed_table('distributed_table_c1_t2', 'a'); + +CREATE TABLE distributed_table_c2_t1(a int, b int); +SELECT create_distributed_table('distributed_table_c2_t1', 'a', colocate_with=>'none'); + +CREATE TABLE citus_local_table(a int, b int); +SELECT citus_add_local_table_to_metadata('citus_local_table'); + +CREATE TABLE postgres_local_table(a int, b int); + +CREATE FUNCTION reload_tables() RETURNS void AS $$ + BEGIN + SET LOCAL client_min_messages TO WARNING; + + TRUNCATE nullkey_c1_t1, nullkey_c1_t2, nullkey_c2_t1, reference_table, distributed_table_c1_t1, + distributed_table_c1_t2, distributed_table_c2_t1, citus_local_table, postgres_local_table; + + INSERT INTO nullkey_c1_t1 SELECT i, i FROM generate_series(1, 8) i; + INSERT INTO nullkey_c1_t2 SELECT i, i FROM generate_series(2, 7) i; + INSERT INTO nullkey_c2_t1 SELECT i, i FROM generate_series(2, 7) i; + INSERT INTO reference_table SELECT i, i FROM generate_series(0, 5) i; + INSERT INTO distributed_table_c1_t1 SELECT i, i FROM generate_series(3, 8) i; + INSERT INTO distributed_table_c1_t2 SELECT i, i FROM generate_series(2, 9) i; + INSERT INTO distributed_table_c2_t1 SELECT i, i FROM generate_series(5, 10) i; + INSERT INTO citus_local_table SELECT i, i FROM generate_series(0, 10) i; + INSERT INTO postgres_local_table SELECT i, i FROM generate_series(5, 10) i; + END; +$$ LANGUAGE plpgsql; + +SELECT reload_tables(); + +CREATE TABLE append_table (a int, b int); +SELECT create_distributed_table('append_table', 'a', 'append'); +SELECT master_create_empty_shard('append_table') AS shardid1 \gset +SELECT master_create_empty_shard('append_table') AS shardid2 \gset +SELECT master_create_empty_shard('append_table') AS shardid3 \gset + +COPY append_table (a, b) FROM STDIN WITH (format 'csv', append_to_shard :shardid1); +1, 40 +2, 42 +3, 44 +4, 46 +5, 48 +\. + +COPY append_table (a, b) FROM STDIN WITH (format 'csv', append_to_shard :shardid2); +6, 50 +7, 52 +8, 54 +9, 56 +10, 58 +\. + +CREATE TABLE range_table(a int, b int); +SELECT create_distributed_table('range_table', 'a', 'range'); +CALL public.create_range_partitioned_shards('range_table', '{"0","25"}','{"24","49"}'); +INSERT INTO range_table VALUES (0, 1), (1, 2), (2, 3), (3, 4), (4, 5), (5, 6), (6, 50); + +CREATE MATERIALIZED VIEW matview AS SELECT b*2+a AS a, a*a AS b FROM nullkey_c1_t1; + +SET client_min_messages TO DEBUG2; + +-- Test inserting into a distributed table by selecting from a combination of +-- different table types together with null-shard-key tables. + +-- use a null-shard-key table +INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1; + +-- use a reference table +INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN reference_table USING (a); +INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 RIGHT JOIN reference_table USING (b) WHERE reference_table.a >= 1 AND reference_table.a <= 5; +INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 LEFT JOIN reference_table USING (b); +INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 INTERSECT SELECT * FROM reference_table; + +-- use a colocated null-shard-key table +INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN nullkey_c1_t2 USING (b); +INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 FULL JOIN nullkey_c1_t2 USING (a); +INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 FULL JOIN matview USING (a); +INSERT INTO distributed_table_c1_t1 SELECT * FROM nullkey_c1_t1 UNION SELECT * FROM nullkey_c1_t2; + +-- use a non-colocated null-shard-key table +INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 LEFT JOIN nullkey_c2_t1 USING (a); +INSERT INTO distributed_table_c1_t1 SELECT * FROM nullkey_c1_t1 UNION SELECT * FROM nullkey_c2_t1; + +-- use a distributed table that is colocated with the target table +INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a); +INSERT INTO distributed_table_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a); +INSERT INTO distributed_table_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (b); +INSERT INTO distributed_table_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a) WHERE distributed_table_c1_t2.a = 1; + +-- use a distributed table that is not colocated with the target table +INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 JOIN distributed_table_c2_t1 USING (a); + +-- use a citus local table +INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN citus_local_table USING (a); + +-- use a postgres local table +INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 JOIN postgres_local_table USING (a); + +-- use append / range distributed tables +INSERT INTO range_table SELECT * FROM nullkey_c1_t1; +INSERT INTO append_table SELECT * FROM nullkey_c1_t1; + +SELECT avg(a), avg(b) FROM distributed_table_c1_t1 ORDER BY 1, 2; +TRUNCATE distributed_table_c1_t1; +INSERT INTO distributed_table_c1_t1 SELECT i, i FROM generate_series(3, 8) i; + +-- Test inserting into a reference table by selecting from a combination of +-- different table types together with null-shard-key tables. + +-- use a null-shard-key table +INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1; + +-- use a reference table +INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN reference_table USING (a); +INSERT INTO reference_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 LEFT JOIN reference_table USING (b); +INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 UNION SELECT * FROM reference_table; +INSERT INTO reference_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 LEFT JOIN reference_table USING (b) WHERE b IN (SELECT b FROM matview); + +-- use a colocated null-shard-key table +INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN nullkey_c1_t2 USING (b); +INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 FULL JOIN nullkey_c1_t2 USING (a); + +-- use a non-colocated null-shard-key table +INSERT INTO reference_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 LEFT JOIN nullkey_c2_t1 USING (a); + +-- use a distributed table +INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a); +INSERT INTO reference_table SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a); +INSERT INTO reference_table SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (b); +INSERT INTO reference_table SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a) WHERE distributed_table_c1_t2.a = 1; + +-- use a citus local table +INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN citus_local_table USING (a); + +-- use a postgres local table +INSERT INTO reference_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 JOIN postgres_local_table USING (a); + +SELECT avg(a), avg(b) FROM reference_table ORDER BY 1, 2; +TRUNCATE reference_table; +INSERT INTO reference_table SELECT i, i FROM generate_series(0, 5) i; + +-- Test inserting into a citus local table by selecting from a combination of +-- different table types together with null-shard-key tables. + +-- use a null-shard-key table +INSERT INTO citus_local_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1; + +-- use a reference table +INSERT INTO citus_local_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN reference_table USING (a); + +-- use a colocated null-shard-key table +INSERT INTO citus_local_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN nullkey_c1_t2 USING (b); + +-- use a distributed table +INSERT INTO citus_local_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a); + +-- use a citus local table +INSERT INTO citus_local_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN citus_local_table USING (a); + +-- use a postgres local table +INSERT INTO citus_local_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 JOIN postgres_local_table USING (a); + +SELECT avg(a), avg(b) FROM citus_local_table ORDER BY 1, 2; +TRUNCATE citus_local_table; +INSERT INTO citus_local_table SELECT i, i FROM generate_series(0, 10) i; + +-- Test inserting into a null-shard-key table by selecting from a combination of +-- different table types, together with or without null-shard-key tables. + +-- use a postgres local table +INSERT INTO nullkey_c1_t1 SELECT postgres_local_table.a, postgres_local_table.b FROM postgres_local_table; +INSERT INTO nullkey_c1_t1 SELECT postgres_local_table.a, postgres_local_table.b FROM postgres_local_table JOIN reference_table USING (a); +INSERT INTO nullkey_c1_t1 SELECT postgres_local_table.a, postgres_local_table.b FROM postgres_local_table LEFT JOIN nullkey_c1_t1 USING (a); + +-- use a citus local table +INSERT INTO nullkey_c1_t1 SELECT citus_local_table.a, citus_local_table.b FROM citus_local_table; +INSERT INTO nullkey_c1_t1 SELECT citus_local_table.a, citus_local_table.b FROM citus_local_table JOIN reference_table USING (a) JOIN postgres_local_table USING (a) ORDER BY 1,2 OFFSET 7; +INSERT INTO nullkey_c1_t1 SELECT citus_local_table.a, citus_local_table.b FROM citus_local_table JOIN nullkey_c1_t1 USING (a); + +-- use a distributed table +INSERT INTO nullkey_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM distributed_table_c1_t2; +INSERT INTO nullkey_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM distributed_table_c1_t2 JOIN reference_table USING (a); +INSERT INTO nullkey_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM distributed_table_c1_t2 JOIN nullkey_c1_t1 USING (a); + +-- use a non-colocated null-shard-key table +INSERT INTO nullkey_c2_t1 SELECT q.* FROM (SELECT reference_table.* FROM reference_table LEFT JOIN nullkey_c1_t1 USING (a)) q JOIN nullkey_c1_t2 USING (a); + +-- use a materialized view +INSERT INTO nullkey_c1_t1 SELECT * FROM matview; +INSERT INTO nullkey_c1_t1 SELECT reference_table.a, reference_table.b FROM reference_table JOIN matview ON (reference_table.a = matview.a); +INSERT INTO nullkey_c1_t1 SELECT q.* FROM (SELECT reference_table.* FROM reference_table JOIN nullkey_c1_t1 USING (a)) q JOIN matview USING (a); + +-- use append / range distributed tables +INSERT INTO nullkey_c1_t1 SELECT * FROM range_table; +INSERT INTO nullkey_c1_t1 SELECT * FROM append_table; + +SELECT avg(a), avg(b) FROM nullkey_c1_t1 ORDER BY 1, 2; +SELECT avg(a), avg(b) FROM nullkey_c2_t1 ORDER BY 1, 2; +TRUNCATE nullkey_c1_t1, nullkey_c2_t1; +INSERT INTO nullkey_c1_t1 SELECT i, i FROM generate_series(1, 8) i; +INSERT INTO nullkey_c2_t1 SELECT i, i FROM generate_series(2, 7) i; + +-- Test inserting into a local table by selecting from a combination of +-- different table types, together with or without null-shard-key tables. + +INSERT INTO postgres_local_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN reference_table USING (a); + +INSERT INTO postgres_local_table SELECT * FROM nullkey_c1_t1 ORDER BY 1,2 OFFSET 3 LIMIT 2; + +WITH cte_1 AS ( + DELETE FROM nullkey_c1_t1 WHERE a >= 1 and a <= 4 RETURNING * +) +INSERT INTO postgres_local_table SELECT cte_1.* FROM cte_1 LEFT JOIN nullkey_c1_t2 USING (a) WHERE nullkey_c1_t2.a IS NULL; + +INSERT INTO postgres_local_table SELECT * FROM nullkey_c1_t1 EXCEPT SELECT * FROM postgres_local_table; + +SELECT avg(a), avg(b) FROM postgres_local_table ORDER BY 1, 2; +TRUNCATE postgres_local_table; +INSERT INTO postgres_local_table SELECT i, i FROM generate_series(5, 10) i; + +-- Try slightly more complex queries. + +WITH cte_1 AS ( + SELECT nullkey_c1_t1.a, reference_table.b FROM nullkey_c1_t1 JOIN reference_table USING (a) +), +cte_2 AS ( + SELECT reference_table.a, postgres_local_table.b FROM postgres_local_table LEFT JOIN reference_table USING (b) +) +INSERT INTO distributed_table_c1_t1 +SELECT cte_1.* FROM cte_1 JOIN cte_2 USING (a) JOIN distributed_table_c1_t2 USING (a) ORDER BY 1,2; + +WITH cte_1 AS ( + SELECT nullkey_c1_t1.a, reference_table.b FROM nullkey_c1_t1 JOIN reference_table USING (a) +), +cte_2 AS ( + SELECT * FROM nullkey_c1_t2 WHERE EXISTS ( + SELECT 1 FROM reference_table WHERE reference_table.a = nullkey_c1_t2.a + ) + ORDER BY 1,2 OFFSET 1 LIMIT 4 +) +INSERT INTO distributed_table_c1_t1 +SELECT * FROM cte_1 UNION SELECT * FROM cte_2 EXCEPT SELECT * FROM reference_table; + +INSERT INTO distributed_table_c1_t1 (a, b) +SELECT t1.a, t2.b +FROM nullkey_c1_t1 t1 +JOIN ( + SELECT b FROM nullkey_c1_t2 ORDER BY b DESC LIMIT 1 +) t2 +ON t1.b < t2.b; + +INSERT INTO distributed_table_c1_t1 (a, b) +WITH cte AS ( + SELECT a, b, + (SELECT a FROM nullkey_c1_t2 WHERE b = t.b) AS d1, + (SELECT a FROM reference_table WHERE b = t.b) AS d2 + FROM nullkey_c1_t1 t +) +SELECT d1, COALESCE(d2, a) FROM cte WHERE d1 IS NOT NULL AND d2 IS NOT NULL; + +INSERT INTO citus_local_table (a, b) +SELECT t1.a, t2.b +FROM nullkey_c1_t1 t1 +CROSS JOIN ( + SELECT b FROM nullkey_c2_t1 ORDER BY b LIMIT 1 +) t2; + +INSERT INTO distributed_table_c1_t1 (a, b) +SELECT t1.a, t2.b +FROM reference_table t1 +LEFT JOIN ( + SELECT b, ROW_NUMBER() OVER (ORDER BY b DESC) AS rn + FROM nullkey_c1_t1 +) t2 ON t1.b = t2.b +WHERE t2.rn > 0; + +INSERT INTO nullkey_c1_t1 (a, b) +SELECT t1.a, t2.b +FROM nullkey_c1_t1 t1 +JOIN ( + SELECT rn, b + FROM ( + SELECT b, ROW_NUMBER() OVER (ORDER BY b DESC) AS rn + FROM distributed_table_c2_t1 + ) q +) t2 ON t1.b = t2.b +WHERE t2.rn > 2; + +INSERT INTO distributed_table_c1_t1 (a, b) +SELECT t1.a, t2.b +FROM nullkey_c1_t1 t1 +JOIN ( + SELECT sum_val, b + FROM ( + SELECT b, SUM(a) OVER (PARTITION BY b) AS sum_val + FROM nullkey_c1_t1 + ) q +) t2 ON t1.b = t2.b +WHERE t2.sum_val > 2; + +-- MultiTaskRouterSelectQuerySupported() is unnecessarily restrictive +-- about pushing down queries with DISTINCT ON clause even if the table +-- doesn't have a shard key. See https://github.com/citusdata/citus/pull/6752. +INSERT INTO nullkey_c1_t1 SELECT DISTINCT ON (a) a, b FROM nullkey_c1_t2; + +-- Similarly, we could push down the following query as well. see +-- https://github.com/citusdata/citus/pull/6831. +INSERT INTO nullkey_c1_t1 SELECT b, SUM(a) OVER (ORDER BY b) AS sum_val FROM nullkey_c1_t1; + +INSERT INTO nullkey_c2_t1 +SELECT t2.a, t2.b +FROM nullkey_c1_t1 AS t2 +JOIN reference_table AS t3 ON (t2.a = t3.a) +WHERE NOT EXISTS ( + SELECT 1 FROM nullkey_c1_t2 AS t1 WHERE t1.b = t3.b +); + +INSERT INTO distributed_table_c1_t1 +SELECT t1.a, t1.b +FROM nullkey_c1_t1 AS t1 +WHERE t1.a NOT IN ( + SELECT DISTINCT t2.a FROM distributed_table_c1_t2 AS t2 +); + +INSERT INTO distributed_table_c1_t1 +SELECT t1.a, t1.b +FROM reference_table AS t1 +JOIN ( + SELECT t2.a FROM ( + SELECT a FROM nullkey_c1_t1 + UNION + SELECT a FROM nullkey_c1_t2 + ) AS t2 +) AS t3 ON t1.a = t3.a; + +-- Temporaryly reduce the verbosity to avoid noise +-- in the output of the next query. +SET client_min_messages TO DEBUG1; + +INSERT INTO nullkey_c1_t1 +SELECT t1.a, t1.b +FROM reference_table AS t1 +WHERE t1.a IN ( + SELECT t2.a FROM ( + SELECT t3.a FROM ( + SELECT a FROM distributed_table_c1_t1 WHERE b > 4 + ) AS t3 + JOIN ( + SELECT a FROM distributed_table_c1_t2 WHERE b < 7 + ) AS t4 ON t3.a = t4.a + ) AS t2 +); + +SET client_min_messages TO DEBUG2; + +-- test upsert with plain INSERT query + +CREATE TABLE upsert_test_1 +( + unique_col int UNIQUE, + other_col int, + third_col int +); +SELECT create_distributed_table('upsert_test_1', null); + +CREATE TABLE upsert_test_2(key int primary key, value text); +SELECT create_distributed_table('upsert_test_2', null); + +INSERT INTO upsert_test_2 AS upsert_test_2_alias (key, value) VALUES (1, '5') ON CONFLICT(key) + DO UPDATE SET value = (upsert_test_2_alias.value::int * 2)::text; + +INSERT INTO upsert_test_2 (key, value) VALUES (1, '5') ON CONFLICT(key) + DO UPDATE SET value = (upsert_test_2.value::int * 3)::text; + +INSERT INTO upsert_test_1 (unique_col, other_col) VALUES (1, 1) ON CONFLICT (unique_col) + DO UPDATE SET other_col = (SELECT count(*) from upsert_test_1); + +INSERT INTO upsert_test_1 (unique_col, other_col) VALUES (1, 1) ON CONFLICT (unique_col) + DO UPDATE SET other_col = random()::int; + +INSERT INTO upsert_test_1 (unique_col, other_col) VALUES (1, 1) ON CONFLICT (unique_col) + DO UPDATE SET other_col = 5 WHERE upsert_test_1.other_col = random()::int; + +INSERT INTO upsert_test_1 VALUES (3, 5, 7); + +INSERT INTO upsert_test_1 (unique_col, other_col) VALUES (1, 1) ON CONFLICT (unique_col) WHERE unique_col = random()::int + DO UPDATE SET other_col = 5; + +CREATE TABLE upsert_test_3 (key_1 int, key_2 bigserial, value text DEFAULT 'default_value', PRIMARY KEY (key_1, key_2)); +SELECT create_distributed_table('upsert_test_3', null); + +INSERT INTO upsert_test_3 VALUES (1, DEFAULT, '1') RETURNING *; +INSERT INTO upsert_test_3 VALUES (5, DEFAULT, DEFAULT) RETURNING *; + +SET client_min_messages TO DEBUG1; +INSERT INTO upsert_test_3 SELECT 7, other_col, 'harcoded_text_value' FROM upsert_test_1 RETURNING *; +SET client_min_messages TO DEBUG2; + +-- test upsert with INSERT .. SELECT queries + +SET client_min_messages TO DEBUG1; +INSERT INTO upsert_test_1 (unique_col, other_col) SELECT unique_col, other_col FROM upsert_test_1 ON CONFLICT (unique_col) + DO UPDATE SET other_col = upsert_test_1.other_col + 1; +-- Fails due to https://github.com/citusdata/citus/issues/6826. +INSERT INTO upsert_test_1 (unique_col, other_col) SELECT unique_col, other_col FROM upsert_test_1 ON CONFLICT (unique_col) + DO UPDATE SET other_col = (SELECT count(*) from upsert_test_1); +SET client_min_messages TO DEBUG2; + +INSERT INTO upsert_test_1 (unique_col, other_col) SELECT unique_col, other_col FROM upsert_test_1 ON CONFLICT (unique_col) + DO UPDATE SET other_col = random()::int; + +INSERT INTO upsert_test_1 (unique_col, other_col) SELECT unique_col, other_col FROM upsert_test_1 ON CONFLICT (unique_col) + DO UPDATE SET other_col = 5 WHERE upsert_test_1.other_col = random()::int; + +SELECT reload_tables(); + +ALTER TABLE nullkey_c1_t1 ADD PRIMARY KEY (a); +ALTER TABLE distributed_table_c1_t1 ADD PRIMARY KEY (a,b); + +INSERT INTO nullkey_c1_t1 AS t1 (a, b) SELECT t3.a, t3.b FROM nullkey_c1_t2 t2 JOIN reference_table t3 ON (t2.a = t3.a) ON CONFLICT (a) + DO UPDATE SET a = t1.a + 10; + +SET client_min_messages TO DEBUG1; +INSERT INTO distributed_table_c1_t1 AS t1 (a, b) SELECT t3.a, t3.b FROM nullkey_c1_t2 t2 JOIN reference_table t3 ON (t2.a = t3.a) ON CONFLICT (a, b) + DO UPDATE SET b = t1.b + 10; +INSERT INTO nullkey_c1_t1 AS t1 (a, b) SELECT t3.a, t3.b FROM distributed_table_c1_t1 t2 JOIN reference_table t3 ON (t2.a = t3.a) ON CONFLICT (a) + DO UPDATE SET a = t1.a + 10; +-- This also fails due to https://github.com/citusdata/citus/issues/6826. +INSERT INTO nullkey_c1_t1 AS t1 (a, b) SELECT t3.a, t3.b FROM distributed_table_c1_t1 t2 JOIN reference_table t3 ON (t2.a = t3.a) WHERE t2.a = 3 ON CONFLICT (a) + DO UPDATE SET a = (SELECT max(b)+1 FROM distributed_table_c1_t1 WHERE a = 3); +SET client_min_messages TO DEBUG2; + +SELECT avg(a), avg(b) FROM distributed_table_c1_t1; +SELECT avg(a), avg(b) FROM nullkey_c1_t1; +SELECT avg(a), avg(b) FROM nullkey_c1_t2; +SELECT * FROM upsert_test_1 ORDER BY unique_col; +SELECT * FROM upsert_test_2 ORDER BY key; +SELECT * FROM upsert_test_3 ORDER BY key_1, key_2; + +SET client_min_messages TO WARNING; +DROP SCHEMA insert_select_null_dist_key CASCADE; + +SELECT citus_remove_node('localhost', :master_port); diff --git a/src/test/regress/sql/query_null_dist_key.sql b/src/test/regress/sql/query_null_dist_key.sql index f5d1fe3fc..02eac5c80 100644 --- a/src/test/regress/sql/query_null_dist_key.sql +++ b/src/test/regress/sql/query_null_dist_key.sql @@ -514,7 +514,11 @@ SET client_min_messages TO DEBUG2; INSERT INTO nullkey_c1_t1 SELECT * FROM nullkey_c2_t1; -- between a null dist key table and a table of different type +SET client_min_messages TO WARNING; +EXPLAIN (ANALYZE TRUE, TIMING FALSE, COSTS FALSE, SUMMARY FALSE, VERBOSE FALSE) INSERT INTO nullkey_c1_t1 SELECT * FROM reference_table; +SET client_min_messages TO DEBUG2; + INSERT INTO nullkey_c1_t1 SELECT * FROM distributed_table; INSERT INTO nullkey_c1_t1 SELECT * FROM citus_local_table; INSERT INTO nullkey_c1_t1 SELECT * FROM postgres_local_table; @@ -543,7 +547,7 @@ WITH level_0 AS ( WITH RECURSIVE level_2_recursive(x) AS ( VALUES (1) UNION ALL - SELECT a + 1 FROM nullkey_c1_t1 JOIN level_2_recursive ON (a = x) WHERE a < 100 + SELECT a + 1 FROM nullkey_c1_t1 JOIN level_2_recursive ON (a = x) WHERE a < 2 ) SELECT * FROM level_2_recursive RIGHT JOIN reference_table ON (level_2_recursive.x = reference_table.a) ) @@ -638,6 +642,8 @@ SET client_min_messages TO DEBUG1; INSERT INTO bigserial_test (x, y) SELECT x, y FROM bigserial_test; +INSERT INTO bigserial_test (x, y) SELECT a, a FROM reference_table; + INSERT INTO agg_events (user_id) SELECT f2.id FROM @@ -689,6 +695,19 @@ FROM raw_events_first LEFT JOIN raw_events_second ON raw_events_first.user_id = raw_events_second.user_id WHERE raw_events_second.user_id = 10 OR raw_events_second.user_id = 11; +INSERT INTO agg_events (user_id) +SELECT + users_ref_table.user_id +FROM + users_ref_table LEFT JOIN raw_events_second ON users_ref_table.user_id = raw_events_second.user_id + WHERE raw_events_second.user_id = 10 OR raw_events_second.user_id = 11; + +INSERT INTO agg_events (user_id) +SELECT COALESCE(raw_events_first.user_id, users_ref_table.user_id) +FROM raw_events_first + RIGHT JOIN (users_ref_table LEFT JOIN raw_events_second ON users_ref_table.user_id = raw_events_second.user_id) + ON raw_events_first.user_id = users_ref_table.user_id; + -- using a full join INSERT INTO agg_events (user_id, value_1_agg) SELECT t1.user_id AS col1, @@ -715,12 +734,25 @@ WHERE NOT EXISTS (SELECT 1 FROM raw_events_second WHERE raw_events_second.user_id =raw_events_first.user_id); +INSERT INTO raw_events_second + (user_id) +SELECT user_id +FROM users_ref_table +WHERE NOT EXISTS (SELECT 1 + FROM raw_events_second + WHERE raw_events_second.user_id = users_ref_table.user_id); + -- using inner join INSERT INTO agg_events (user_id) SELECT raw_events_first.user_id FROM raw_events_first INNER JOIN raw_events_second ON raw_events_first.user_id = raw_events_second.value_1 WHERE raw_events_first.value_1 IN (10, 11,12) OR raw_events_second.user_id IN (1,2,3,4); +INSERT INTO agg_events (user_id) +SELECT raw_events_first.user_id +FROM raw_events_first INNER JOIN users_ref_table ON raw_events_first.user_id = users_ref_table.user_id +WHERE raw_events_first.value_1 IN (10, 11,12) OR users_ref_table.user_id IN (1,2,3,4); + -- We could relax distributed insert .. select checks to allow pushing -- down more clauses down to the worker nodes when inserting into a single -- shard by selecting from a colocated one. We might want to do something @@ -734,6 +766,7 @@ WHERE raw_events_first.value_1 IN (10, 11,12) OR raw_events_second.user_id IN (1 -- limit / offset clause INSERT INTO agg_events (user_id) SELECT raw_events_first.user_id FROM raw_events_first LIMIT 1; INSERT INTO agg_events (user_id) SELECT raw_events_first.user_id FROM raw_events_first OFFSET 1; +INSERT INTO agg_events (user_id) SELECT users_ref_table.user_id FROM users_ref_table LIMIT 1; -- using a materialized cte WITH cte AS MATERIALIZED @@ -745,6 +778,10 @@ INSERT INTO raw_events_second WITH cte AS MATERIALIZED (SELECT * FROM raw_events_first) SELECT user_id * 1000, time, value_1, value_2, value_3, value_4 FROM cte; +INSERT INTO raw_events_second (user_id) + WITH cte AS MATERIALIZED (SELECT * FROM users_ref_table) + SELECT user_id FROM cte; + -- using a regular cte WITH cte AS (SELECT * FROM raw_events_first) INSERT INTO raw_events_second @@ -763,19 +800,19 @@ INSERT INTO agg_events -- we still support complex joins via INSERT's cte list .. WITH cte AS ( - SELECT reference_table.a AS a, 1 AS b + SELECT DISTINCT(reference_table.a) AS a, 1 AS b FROM distributed_table RIGHT JOIN reference_table USING (a) ) INSERT INTO raw_events_second (user_id, value_1) SELECT (a+5)*-1, b FROM cte; --- .. but can't do so via via SELECT's cte list +-- .. and via SELECT's cte list too INSERT INTO raw_events_second (user_id, value_1) WITH cte AS ( - SELECT reference_table.a AS a, 1 AS b + SELECT DISTINCT(reference_table.a) AS a, 1 AS b FROM distributed_table RIGHT JOIN reference_table USING (a) ) - SELECT (a+5)*-1, b FROM cte; + SELECT (a+5)*2, b FROM cte; -- using set operations INSERT INTO @@ -783,6 +820,11 @@ INSERT INTO (SELECT user_id FROM raw_events_first) INTERSECT (SELECT user_id FROM raw_events_first); +INSERT INTO + raw_events_first(user_id) + (SELECT user_id FROM users_ref_table) INTERSECT + (SELECT user_id FROM raw_events_first); + -- group by clause inside subquery INSERT INTO agg_events (user_id) @@ -842,6 +884,11 @@ SELECT SUM(value_3), FROM raw_events_first GROUP BY user_id; +INSERT INTO agg_events (value_3_agg, value_1_agg) +SELECT AVG(user_id), SUM(user_id) +FROM users_ref_table +GROUP BY user_id; + -- using generate_series INSERT INTO raw_events_first (user_id, value_1, value_2) SELECT s, s, s FROM generate_series(1, 5) s; From db2514ef7864ac8de04c64741a1c2087716d2d7d Mon Sep 17 00:00:00 2001 From: Onur Tirtir Date: Wed, 3 May 2023 14:32:43 +0300 Subject: [PATCH 6/7] Call null-shard-key tables as single-shard distributed tables in code --- .../commands/create_distributed_table.c | 40 +++--- .../distributed/commands/foreign_constraint.c | 14 +- src/backend/distributed/commands/multi_copy.c | 2 +- src/backend/distributed/commands/table.c | 8 +- .../distributed/metadata/metadata_cache.c | 12 +- .../distributed/operations/create_shards.c | 12 +- .../distributed/operations/stage_protocol.c | 2 +- .../distributed/planner/distributed_planner.c | 18 +-- .../planner/insert_select_planner.c | 10 +- .../planner/multi_router_planner.c | 10 +- .../distributed/utils/distribution_column.c | 2 +- .../distributed/coordinator_protocol.h | 3 +- src/include/distributed/distributed_planner.h | 8 +- src/include/distributed/metadata_cache.h | 6 +- src/include/distributed/metadata_utility.h | 2 +- .../distributed/multi_router_planner.h | 2 +- .../citus_arbitrary_configs.py | 6 +- src/test/regress/citus_tests/config.py | 6 +- src/test/regress/citus_tests/run_test.py | 2 +- ...out => alter_table_single_shard_table.out} | 0 ..._key.out => create_single_shard_table.out} | 136 +++++++++--------- ...t => insert_select_single_shard_table.out} | 42 +++--- src/test/regress/expected/merge.out | 25 ++-- src/test/regress/expected/multi_extension.out | 2 +- ...t_key.out => query_single_shard_table.out} | 40 +++--- src/test/regress/expected/single_node.out | 2 +- src/test/regress/expected/single_node_0.out | 2 +- ...y_prep.out => single_shard_table_prep.out} | 0 src/test/regress/multi_1_schedule | 8 +- src/test/regress/null_dist_key_prep_schedule | 1 - .../regress/single_shard_table_prep_schedule | 1 + ...sql => alter_table_single_shard_table.sql} | 0 ..._key.sql => create_single_shard_table.sql} | 130 ++++++++--------- ...l => insert_select_single_shard_table.sql} | 36 ++--- src/test/regress/sql/merge.sql | 10 +- src/test/regress/sql/multi_extension.sql | 2 +- ...t_key.sql => query_single_shard_table.sql} | 26 ++-- src/test/regress/sql/single_node.sql | 2 +- ...y_prep.sql => single_shard_table_prep.sql} | 0 39 files changed, 318 insertions(+), 312 deletions(-) rename src/test/regress/expected/{alter_table_null_dist_key.out => alter_table_single_shard_table.out} (100%) rename src/test/regress/expected/{create_null_dist_key.out => create_single_shard_table.out} (94%) rename src/test/regress/expected/{insert_select_null_dist_key.out => insert_select_single_shard_table.out} (96%) rename src/test/regress/expected/{query_null_dist_key.out => query_single_shard_table.out} (98%) rename src/test/regress/expected/{null_dist_key_prep.out => single_shard_table_prep.out} (100%) delete mode 100644 src/test/regress/null_dist_key_prep_schedule create mode 100644 src/test/regress/single_shard_table_prep_schedule rename src/test/regress/sql/{alter_table_null_dist_key.sql => alter_table_single_shard_table.sql} (100%) rename src/test/regress/sql/{create_null_dist_key.sql => create_single_shard_table.sql} (91%) rename src/test/regress/sql/{insert_select_null_dist_key.sql => insert_select_single_shard_table.sql} (95%) rename src/test/regress/sql/{query_null_dist_key.sql => query_single_shard_table.sql} (98%) rename src/test/regress/sql/{null_dist_key_prep.sql => single_shard_table_prep.sql} (100%) diff --git a/src/backend/distributed/commands/create_distributed_table.c b/src/backend/distributed/commands/create_distributed_table.c index 544d8f04e..e55196100 100644 --- a/src/backend/distributed/commands/create_distributed_table.c +++ b/src/backend/distributed/commands/create_distributed_table.c @@ -141,8 +141,8 @@ static void CreateCitusTable(Oid relationId, CitusTableType tableType, DistributedTableParams *distributedTableParams); static void CreateHashDistributedTableShards(Oid relationId, int shardCount, Oid colocatedTableId, bool localTableEmpty); -static void CreateNullShardKeyDistTableShard(Oid relationId, Oid colocatedTableId, - uint32 colocationId); +static void CreateSingleShardTableShard(Oid relationId, Oid colocatedTableId, + uint32 colocationId); static uint32 ColocationIdForNewTable(Oid relationId, CitusTableType tableType, DistributedTableParams *distributedTableParams, Var *distributionColumn); @@ -285,7 +285,7 @@ create_distributed_table(PG_FUNCTION_ARGS) { /* * As we do for shard_count parameter, we could throw an error if - * distribution_type is not NULL when creating a null-shard-key table. + * distribution_type is not NULL when creating a single-shard table. * However, this requires changing the default value of distribution_type * parameter to NULL and this would mean a breaking change for most * users because they're mostly using this API to create sharded @@ -296,7 +296,7 @@ create_distributed_table(PG_FUNCTION_ARGS) "when the distribution column is null "))); } - CreateNullShardKeyDistTable(relationId, colocateWithTableName); + CreateSingleShardTable(relationId, colocateWithTableName); } PG_RETURN_VOID(); @@ -1027,11 +1027,11 @@ CreateReferenceTable(Oid relationId) /* - * CreateNullShardKeyDistTable is a wrapper around CreateCitusTable that creates a + * CreateSingleShardTable is a wrapper around CreateCitusTable that creates a * single shard distributed table that doesn't have a shard key. */ void -CreateNullShardKeyDistTable(Oid relationId, char *colocateWithTableName) +CreateSingleShardTable(Oid relationId, char *colocateWithTableName) { DistributedTableParams distributedTableParams = { .colocateWithTableName = colocateWithTableName, @@ -1039,7 +1039,7 @@ CreateNullShardKeyDistTable(Oid relationId, char *colocateWithTableName) .shardCountIsStrict = true, .distributionColumnName = NULL }; - CreateCitusTable(relationId, NULL_KEY_DISTRIBUTED_TABLE, &distributedTableParams); + CreateCitusTable(relationId, SINGLE_SHARD_DISTRIBUTED, &distributedTableParams); } @@ -1061,7 +1061,7 @@ CreateCitusTable(Oid relationId, CitusTableType tableType, DistributedTableParams *distributedTableParams) { if ((tableType == HASH_DISTRIBUTED || tableType == APPEND_DISTRIBUTED || - tableType == RANGE_DISTRIBUTED || tableType == NULL_KEY_DISTRIBUTED_TABLE) != + tableType == RANGE_DISTRIBUTED || tableType == SINGLE_SHARD_DISTRIBUTED) != (distributedTableParams != NULL)) { ereport(ERROR, (errmsg("distributed table params must be provided " @@ -1212,10 +1212,10 @@ CreateCitusTable(Oid relationId, CitusTableType tableType, { CreateReferenceTableShard(relationId); } - else if (tableType == NULL_KEY_DISTRIBUTED_TABLE) + else if (tableType == SINGLE_SHARD_DISTRIBUTED) { - CreateNullShardKeyDistTableShard(relationId, colocatedTableId, - colocationId); + CreateSingleShardTableShard(relationId, colocatedTableId, + colocationId); } if (ShouldSyncTableMetadata(relationId)) @@ -1271,7 +1271,7 @@ CreateCitusTable(Oid relationId, CitusTableType tableType, } /* copy over data for hash distributed and reference tables */ - if (tableType == HASH_DISTRIBUTED || tableType == NULL_KEY_DISTRIBUTED_TABLE || + if (tableType == HASH_DISTRIBUTED || tableType == SINGLE_SHARD_DISTRIBUTED || tableType == REFERENCE_TABLE) { if (RegularTable(relationId)) @@ -1336,7 +1336,7 @@ DecideCitusTableParams(CitusTableType tableType, break; } - case NULL_KEY_DISTRIBUTED_TABLE: + case SINGLE_SHARD_DISTRIBUTED: { citusTableParams.distributionMethod = DISTRIBUTE_BY_NONE; citusTableParams.replicationModel = REPLICATION_MODEL_STREAMING; @@ -1706,12 +1706,12 @@ CreateHashDistributedTableShards(Oid relationId, int shardCount, /* - * CreateHashDistributedTableShards creates the shard of given null-shard-key + * CreateHashDistributedTableShards creates the shard of given single-shard * distributed table. */ static void -CreateNullShardKeyDistTableShard(Oid relationId, Oid colocatedTableId, - uint32 colocationId) +CreateSingleShardTableShard(Oid relationId, Oid colocatedTableId, + uint32 colocationId) { if (colocatedTableId != InvalidOid) { @@ -1735,7 +1735,7 @@ CreateNullShardKeyDistTableShard(Oid relationId, Oid colocatedTableId, } else { - CreateNullKeyShardWithRoundRobinPolicy(relationId, colocationId); + CreateSingleShardTableShardWithRoundRobinPolicy(relationId, colocationId); } } @@ -1984,13 +1984,13 @@ EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn, { /* * Distributing partitioned tables is only supported for hash-distribution - * or null-shard-key tables. + * or single-shard tables. */ - bool isNullShardKeyTable = + bool isSingleShardTable = distributionMethod == DISTRIBUTE_BY_NONE && replicationModel == REPLICATION_MODEL_STREAMING && colocationId != INVALID_COLOCATION_ID; - if (distributionMethod != DISTRIBUTE_BY_HASH && !isNullShardKeyTable) + if (distributionMethod != DISTRIBUTE_BY_HASH && !isSingleShardTable) { ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), errmsg("distributing partitioned tables in only supported " diff --git a/src/backend/distributed/commands/foreign_constraint.c b/src/backend/distributed/commands/foreign_constraint.c index 0b2c5573e..09133f5a3 100644 --- a/src/backend/distributed/commands/foreign_constraint.c +++ b/src/backend/distributed/commands/foreign_constraint.c @@ -304,9 +304,9 @@ ErrorIfUnsupportedForeignConstraintExists(Relation relation, char referencingDis * Foreign keys from citus local tables or reference tables to distributed * tables are not supported. * - * We could support foreign keys from references tables to null-shard-key + * We could support foreign keys from references tables to single-shard * tables but this doesn't seem useful a lot. However, if we decide supporting - * this, then we need to expand relation access tracking check for the null-shard-key + * this, then we need to expand relation access tracking check for the single-shard * tables too. */ if (referencingIsCitusLocalOrRefTable && !referencedIsCitusLocalOrRefTable) @@ -366,11 +366,11 @@ ErrorIfUnsupportedForeignConstraintExists(Relation relation, char referencingDis * if tables are hash-distributed and colocated, we need to make sure that * the distribution key is included in foreign constraint. */ - bool referencedIsNullShardKeyTable = - IsNullShardKeyTableByDistParams(referencedDistMethod, - referencedReplicationModel, - referencedColocationId); - if (!referencedIsCitusLocalOrRefTable && !referencedIsNullShardKeyTable && + bool referencedIsSingleShardTable = + IsSingleShardTableByDistParams(referencedDistMethod, + referencedReplicationModel, + referencedColocationId); + if (!referencedIsCitusLocalOrRefTable && !referencedIsSingleShardTable && !foreignConstraintOnDistKey) { ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), diff --git a/src/backend/distributed/commands/multi_copy.c b/src/backend/distributed/commands/multi_copy.c index fdb9552ef..8e92fd7a8 100644 --- a/src/backend/distributed/commands/multi_copy.c +++ b/src/backend/distributed/commands/multi_copy.c @@ -2146,7 +2146,7 @@ CitusCopyDestReceiverStartup(DestReceiver *dest, int operation, } if (IsCitusTableTypeCacheEntry(cacheEntry, DISTRIBUTED_TABLE) && - !IsCitusTableTypeCacheEntry(cacheEntry, NULL_KEY_DISTRIBUTED_TABLE) && + !IsCitusTableTypeCacheEntry(cacheEntry, SINGLE_SHARD_DISTRIBUTED) && copyDest->partitionColumnIndex == INVALID_PARTITION_COLUMN_INDEX) { ereport(ERROR, (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED), diff --git a/src/backend/distributed/commands/table.c b/src/backend/distributed/commands/table.c index af3439ab5..87a6a11b8 100644 --- a/src/backend/distributed/commands/table.c +++ b/src/backend/distributed/commands/table.c @@ -385,7 +385,7 @@ PostprocessCreateTableStmtPartitionOf(CreateStmt *createStatement, const if (IsCitusTable(parentRelationId)) { /* - * We can create Citus local tables and distributed tables with null shard keys + * We can create Citus local tables and single-shard distributed tables * right away, without switching to sequential mode, because they are going to * have only one shard. */ @@ -398,9 +398,9 @@ PostprocessCreateTableStmtPartitionOf(CreateStmt *createStatement, const char *parentRelationName = generate_qualified_relation_name(parentRelationId); - if (IsCitusTableType(parentRelationId, NULL_KEY_DISTRIBUTED_TABLE)) + if (IsCitusTableType(parentRelationId, SINGLE_SHARD_DISTRIBUTED)) { - CreateNullShardKeyDistTable(relationId, parentRelationName); + CreateSingleShardTable(relationId, parentRelationName); return; } @@ -618,7 +618,7 @@ DistributePartitionUsingParent(Oid parentCitusRelationId, Oid partitionRelationI * If the parent is null key distributed, we should distribute the partition * with null distribution key as well. */ - CreateNullShardKeyDistTable(partitionRelationId, parentRelationName); + CreateSingleShardTable(partitionRelationId, parentRelationName); return; } diff --git a/src/backend/distributed/metadata/metadata_cache.c b/src/backend/distributed/metadata/metadata_cache.c index 4eab2aeed..cfcaa4e65 100644 --- a/src/backend/distributed/metadata/metadata_cache.c +++ b/src/backend/distributed/metadata/metadata_cache.c @@ -508,7 +508,7 @@ IsCitusTableTypeInternal(char partitionMethod, char replicationModel, return partitionMethod == DISTRIBUTE_BY_RANGE; } - case NULL_KEY_DISTRIBUTED_TABLE: + case SINGLE_SHARD_DISTRIBUTED: { return partitionMethod == DISTRIBUTE_BY_NONE && replicationModel != REPLICATION_MODEL_2PC && @@ -826,13 +826,13 @@ IsCitusLocalTableByDistParams(char partitionMethod, char replicationModel, /* - * IsNullShardKeyTableByDistParams returns true if given partitionMethod, - * replicationModel and colocationId would identify a distributed table that - * has a null shard key. + * IsSingleShardTableByDistParams returns true if given partitionMethod, + * replicationModel and colocationId would identify a single-shard distributed + * table that has a null shard key. */ bool -IsNullShardKeyTableByDistParams(char partitionMethod, char replicationModel, - uint32 colocationId) +IsSingleShardTableByDistParams(char partitionMethod, char replicationModel, + uint32 colocationId) { return partitionMethod == DISTRIBUTE_BY_NONE && replicationModel != REPLICATION_MODEL_2PC && diff --git a/src/backend/distributed/operations/create_shards.c b/src/backend/distributed/operations/create_shards.c index 1a2ce5f1c..358927a09 100644 --- a/src/backend/distributed/operations/create_shards.c +++ b/src/backend/distributed/operations/create_shards.c @@ -373,16 +373,18 @@ CreateReferenceTableShard(Oid distributedTableId) /* - * CreateNullKeyShardWithRoundRobinPolicy creates a single shard for the given - * distributedTableId. The created shard does not have min/max values. - * Unlike CreateReferenceTableShard, the shard is _not_ replicated to - * all nodes but would have a single placement like Citus local tables. + * CreateSingleShardTableShardWithRoundRobinPolicy creates a single + * shard for the given distributedTableId. The created shard does not + * have min/max values. Unlike CreateReferenceTableShard, the shard is + * _not_ replicated to all nodes but would have a single placement like + * Citus local tables. + * * However, this placement doesn't necessarily need to be placed on * coordinator. This is determined based on modulo of the colocation * id that given table has been associated to. */ void -CreateNullKeyShardWithRoundRobinPolicy(Oid relationId, uint32 colocationId) +CreateSingleShardTableShardWithRoundRobinPolicy(Oid relationId, uint32 colocationId) { EnsureTableOwner(relationId); diff --git a/src/backend/distributed/operations/stage_protocol.c b/src/backend/distributed/operations/stage_protocol.c index c1031ffdf..7ef988e5f 100644 --- a/src/backend/distributed/operations/stage_protocol.c +++ b/src/backend/distributed/operations/stage_protocol.c @@ -522,7 +522,7 @@ RelationShardListForShardCreate(ShardInterval *shardInterval) List *relationShardList = list_make1(relationShard); if ((IsCitusTableTypeCacheEntry(cacheEntry, HASH_DISTRIBUTED) || - IsCitusTableTypeCacheEntry(cacheEntry, NULL_KEY_DISTRIBUTED_TABLE)) && + IsCitusTableTypeCacheEntry(cacheEntry, SINGLE_SHARD_DISTRIBUTED)) && cacheEntry->colocationId != INVALID_COLOCATION_ID) { shardIndex = ShardIndex(shardInterval); diff --git a/src/backend/distributed/planner/distributed_planner.c b/src/backend/distributed/planner/distributed_planner.c index 50509baea..6c5d0f32a 100644 --- a/src/backend/distributed/planner/distributed_planner.c +++ b/src/backend/distributed/planner/distributed_planner.c @@ -1025,15 +1025,15 @@ CreateDistributedPlan(uint64 planId, bool allowRecursivePlanning, Query *origina { return distributedPlan; } - else if (ContainsNullDistKeyTable(originalQuery)) + else if (ContainsSingleShardTable(originalQuery)) { /* * We only support router queries if the query contains reference to - * a null-dist-key table. This temporary restriction will be removed + * a single-shard table. This temporary restriction will be removed * once we support recursive planning for the queries that reference - * null-dist-key tables. + * single-shard tables. */ - WrapRouterErrorForNullDistKeyTable(distributedPlan->planningError); + WrapRouterErrorForSingleShardTable(distributedPlan->planningError); RaiseDeferredError(distributedPlan->planningError, ERROR); } else @@ -2474,14 +2474,14 @@ HasUnresolvedExternParamsWalker(Node *expression, ParamListInfo boundParams) /* - * ContainsNullDistKeyTable returns true if given query contains reference - * to a null-dist-key table. + * ContainsSingleShardTable returns true if given query contains reference + * to a single-shard table. */ bool -ContainsNullDistKeyTable(Query *query) +ContainsSingleShardTable(Query *query) { RTEListProperties *rteListProperties = GetRTEListPropertiesForQuery(query); - return rteListProperties->hasDistTableWithoutShardKey; + return rteListProperties->hasSingleShardDistTable; } @@ -2564,7 +2564,7 @@ GetRTEListProperties(List *rangeTableList) if (!HasDistributionKeyCacheEntry(cacheEntry)) { - rteListProperties->hasDistTableWithoutShardKey = true; + rteListProperties->hasSingleShardDistTable = true; } else { diff --git a/src/backend/distributed/planner/insert_select_planner.c b/src/backend/distributed/planner/insert_select_planner.c index 62c0e8d68..cae71845b 100644 --- a/src/backend/distributed/planner/insert_select_planner.c +++ b/src/backend/distributed/planner/insert_select_planner.c @@ -731,7 +731,7 @@ DistributedInsertSelectSupported(Query *queryTree, RangeTblEntry *insertRte, } if (!HasDistributionKey(targetRelationId) || - subqueryRteListProperties->hasDistTableWithoutShardKey) + subqueryRteListProperties->hasSingleShardDistTable) { /* * XXX: Better to check this regardless of the fact that the target table @@ -1563,16 +1563,16 @@ CreateNonPushableInsertSelectPlan(uint64 planId, Query *parse, ParamListInfo bou /* * Today it's not possible to generate a distributed plan for a SELECT - * having more than one tasks if it references a null-shard-key table. + * having more than one tasks if it references a single-shard table. * This is because, we don't support queries beyond router planner - * if the query references a null-shard-key table. + * if the query references a single-shard table. * * For this reason, right now we don't expect an INSERT .. SELECT * query to go through the repartitioned INSERT .. SELECT logic if the - * SELECT query references a null-shard-key table. + * SELECT query references a single-shard table. */ Assert(!repartitioned || - !GetRTEListPropertiesForQuery(selectQueryCopy)->hasDistTableWithoutShardKey); + !GetRTEListPropertiesForQuery(selectQueryCopy)->hasSingleShardDistTable); distributedPlan->insertSelectQuery = insertSelectQuery; distributedPlan->selectPlanForInsertSelect = selectPlan; diff --git a/src/backend/distributed/planner/multi_router_planner.c b/src/backend/distributed/planner/multi_router_planner.c index 47d11172f..0c6ec9dca 100644 --- a/src/backend/distributed/planner/multi_router_planner.c +++ b/src/backend/distributed/planner/multi_router_planner.c @@ -259,12 +259,12 @@ CreateModifyPlan(Query *originalQuery, Query *query, /* - * WrapRouterErrorForNullDistKeyTable wraps given planning error with a + * WrapRouterErrorForSingleShardTable wraps given planning error with a * generic error message if given query references a distributed table * that doesn't have a distribution key. */ void -WrapRouterErrorForNullDistKeyTable(DeferredErrorMessage *planningError) +WrapRouterErrorForSingleShardTable(DeferredErrorMessage *planningError) { planningError->detail = planningError->message; planningError->message = pstrdup("queries that reference a distributed " @@ -1886,9 +1886,9 @@ RouterJob(Query *originalQuery, PlannerRestrictionContext *plannerRestrictionCon */ if (IsMergeQuery(originalQuery)) { - if (ContainsNullDistKeyTable(originalQuery)) + if (ContainsSingleShardTable(originalQuery)) { - WrapRouterErrorForNullDistKeyTable(*planningError); + WrapRouterErrorForSingleShardTable(*planningError); } RaiseDeferredError(*planningError, ERROR); @@ -3013,7 +3013,7 @@ BuildRoutesForInsert(Query *query, DeferredErrorMessage **planningError) ereport(ERROR, (errmsg("local table cannot have %d shards", shardCount))); } - else if (IsCitusTableTypeCacheEntry(cacheEntry, NULL_KEY_DISTRIBUTED_TABLE)) + else if (IsCitusTableTypeCacheEntry(cacheEntry, SINGLE_SHARD_DISTRIBUTED)) { ereport(ERROR, (errmsg("distributed tables having a null shard key " "cannot have %d shards", diff --git a/src/backend/distributed/utils/distribution_column.c b/src/backend/distributed/utils/distribution_column.c index 1f5ac9ec5..474133f73 100644 --- a/src/backend/distributed/utils/distribution_column.c +++ b/src/backend/distributed/utils/distribution_column.c @@ -135,7 +135,7 @@ BuildDistributionKeyFromColumnName(Oid relationId, char *columnName, LOCKMODE lo char *tableName = get_rel_name(relationId); - /* short circuit for reference tables and null-shard key tables */ + /* short circuit for reference tables and single-shard tables */ if (columnName == NULL) { return NULL; diff --git a/src/include/distributed/coordinator_protocol.h b/src/include/distributed/coordinator_protocol.h index 351efb790..7f90eadda 100644 --- a/src/include/distributed/coordinator_protocol.h +++ b/src/include/distributed/coordinator_protocol.h @@ -262,7 +262,8 @@ extern void CreateShardsWithRoundRobinPolicy(Oid distributedTableId, int32 shard extern void CreateColocatedShards(Oid targetRelationId, Oid sourceRelationId, bool useExclusiveConnections); extern void CreateReferenceTableShard(Oid distributedTableId); -extern void CreateNullKeyShardWithRoundRobinPolicy(Oid relationId, uint32 colocationId); +extern void CreateSingleShardTableShardWithRoundRobinPolicy(Oid relationId, + uint32 colocationId); extern List * WorkerCreateShardCommandList(Oid relationId, int shardIndex, uint64 shardId, List *ddlCommandList, List *foreignConstraintCommandList); diff --git a/src/include/distributed/distributed_planner.h b/src/include/distributed/distributed_planner.h index 753504131..aac936a98 100644 --- a/src/include/distributed/distributed_planner.h +++ b/src/include/distributed/distributed_planner.h @@ -147,18 +147,18 @@ typedef struct RTEListProperties bool hasReferenceTable; bool hasCitusLocalTable; - /* includes hash, null dist key, append and range partitioned tables */ + /* includes hash, single-shard, append and range partitioned tables */ bool hasDistributedTable; /* * Effectively, hasDistributedTable is equal to - * "hasDistTableWithShardKey || hasDistTableWithoutShardKey". + * "hasDistTableWithShardKey || hasSingleShardDistTable". * * We provide below two for the callers that want to know what kind of * distributed tables that given query has references to. */ bool hasDistTableWithShardKey; - bool hasDistTableWithoutShardKey; + bool hasSingleShardDistTable; /* union of hasReferenceTable, hasCitusLocalTable and hasDistributedTable */ bool hasCitusTable; @@ -253,7 +253,7 @@ extern int32 BlessRecordExpression(Expr *expr); extern void DissuadePlannerFromUsingPlan(PlannedStmt *plan); extern PlannedStmt * FinalizePlan(PlannedStmt *localPlan, struct DistributedPlan *distributedPlan); -extern bool ContainsNullDistKeyTable(Query *query); +extern bool ContainsSingleShardTable(Query *query); extern RTEListProperties * GetRTEListPropertiesForQuery(Query *query); diff --git a/src/include/distributed/metadata_cache.h b/src/include/distributed/metadata_cache.h index 6f29f9d63..5dfb80519 100644 --- a/src/include/distributed/metadata_cache.h +++ b/src/include/distributed/metadata_cache.h @@ -123,7 +123,7 @@ typedef enum HASH_DISTRIBUTED, APPEND_DISTRIBUTED, RANGE_DISTRIBUTED, - NULL_KEY_DISTRIBUTED_TABLE, + SINGLE_SHARD_DISTRIBUTED, /* hash, range or append distributed table */ DISTRIBUTED_TABLE, @@ -158,8 +158,8 @@ extern uint32 ColocationIdViaCatalog(Oid relationId); bool IsReferenceTableByDistParams(char partitionMethod, char replicationModel); extern bool IsCitusLocalTableByDistParams(char partitionMethod, char replicationModel, uint32 colocationId); -extern bool IsNullShardKeyTableByDistParams(char partitionMethod, char replicationModel, - uint32 colocationId); +extern bool IsSingleShardTableByDistParams(char partitionMethod, char replicationModel, + uint32 colocationId); extern List * CitusTableList(void); extern ShardInterval * LoadShardInterval(uint64 shardId); extern bool ShardExists(uint64 shardId); diff --git a/src/include/distributed/metadata_utility.h b/src/include/distributed/metadata_utility.h index fe404acf8..08d4896c1 100644 --- a/src/include/distributed/metadata_utility.h +++ b/src/include/distributed/metadata_utility.h @@ -326,7 +326,7 @@ extern void DeletePartitionRow(Oid distributedRelationId); extern void DeleteShardRow(uint64 shardId); extern void UpdatePlacementGroupId(uint64 placementId, int groupId); extern void DeleteShardPlacementRow(uint64 placementId); -extern void CreateNullShardKeyDistTable(Oid relationId, char *colocateWithTableName); +extern void CreateSingleShardTable(Oid relationId, char *colocateWithTableName); extern void CreateDistributedTable(Oid relationId, char *distributionColumnName, char distributionMethod, int shardCount, bool shardCountIsStrict, char *colocateWithTableName); diff --git a/src/include/distributed/multi_router_planner.h b/src/include/distributed/multi_router_planner.h index 40d92fead..506e50135 100644 --- a/src/include/distributed/multi_router_planner.h +++ b/src/include/distributed/multi_router_planner.h @@ -36,7 +36,7 @@ extern DistributedPlan * CreateRouterPlan(Query *originalQuery, Query *query, extern DistributedPlan * CreateModifyPlan(Query *originalQuery, Query *query, PlannerRestrictionContext * plannerRestrictionContext); -extern void WrapRouterErrorForNullDistKeyTable(DeferredErrorMessage *planningError); +extern void WrapRouterErrorForSingleShardTable(DeferredErrorMessage *planningError); extern DeferredErrorMessage * PlanRouterQuery(Query *originalQuery, PlannerRestrictionContext * plannerRestrictionContext, diff --git a/src/test/regress/citus_tests/arbitrary_configs/citus_arbitrary_configs.py b/src/test/regress/citus_tests/arbitrary_configs/citus_arbitrary_configs.py index 8785de8f7..52924aa11 100755 --- a/src/test/regress/citus_tests/arbitrary_configs/citus_arbitrary_configs.py +++ b/src/test/regress/citus_tests/arbitrary_configs/citus_arbitrary_configs.py @@ -81,12 +81,14 @@ def run_for_config(config, lock, sql_schedule_name): config.bindir, config.pg_srcdir, config.coordinator_port(), - cfg.NULL_DIST_KEY_PREP_SCHEDULE, + cfg.SINGLE_SHARD_PREP_SCHEDULE, config.output_dir, config.input_dir, cfg.SUPER_USER_NAME, ) - common.save_regression_diff("null_dist_key_prep_regression", config.output_dir) + common.save_regression_diff( + "single_shard_table_prep_regression", config.output_dir + ) exitCode |= _run_pg_regress_on_port( config, config.coordinator_port(), cfg.CREATE_SCHEDULE diff --git a/src/test/regress/citus_tests/config.py b/src/test/regress/citus_tests/config.py index 69cc5599c..0973ac58a 100644 --- a/src/test/regress/citus_tests/config.py +++ b/src/test/regress/citus_tests/config.py @@ -22,7 +22,7 @@ ARBITRARY_SCHEDULE_NAMES = [ "sql_schedule", "sql_base_schedule", "postgres_schedule", - "null_dist_key_prep_schedule", + "single_shard_table_prep_schedule", ] BEFORE_PG_UPGRADE_SCHEDULE = "./before_pg_upgrade_schedule" @@ -30,7 +30,7 @@ AFTER_PG_UPGRADE_SCHEDULE = "./after_pg_upgrade_schedule" CREATE_SCHEDULE = "./create_schedule" POSTGRES_SCHEDULE = "./postgres_schedule" -NULL_DIST_KEY_PREP_SCHEDULE = "./null_dist_key_prep_schedule" +SINGLE_SHARD_PREP_SCHEDULE = "./single_shard_table_prep_schedule" SQL_SCHEDULE = "./sql_schedule" SQL_BASE_SCHEDULE = "./sql_base_schedule" @@ -206,7 +206,7 @@ class PostgresConfig(CitusDefaultClusterConfig): ] -class AllNullDistKeyDefaultConfig(CitusDefaultClusterConfig): +class AllSingleShardTableDefaultConfig(CitusDefaultClusterConfig): def __init__(self, arguments): super().__init__(arguments) self.all_null_dist_key = True diff --git a/src/test/regress/citus_tests/run_test.py b/src/test/regress/citus_tests/run_test.py index cda9fc0ae..c9f86a3c4 100755 --- a/src/test/regress/citus_tests/run_test.py +++ b/src/test/regress/citus_tests/run_test.py @@ -128,7 +128,7 @@ DEPS = { "multi_mx_copy_data": TestDeps(None, ["multi_mx_create_table"]), "multi_mx_schema_support": TestDeps(None, ["multi_mx_copy_data"]), "multi_simple_queries": TestDeps("base_schedule"), - "create_null_dist_key": TestDeps("minimal_schedule"), + "create_single_shard_table": TestDeps("minimal_schedule"), } diff --git a/src/test/regress/expected/alter_table_null_dist_key.out b/src/test/regress/expected/alter_table_single_shard_table.out similarity index 100% rename from src/test/regress/expected/alter_table_null_dist_key.out rename to src/test/regress/expected/alter_table_single_shard_table.out diff --git a/src/test/regress/expected/create_null_dist_key.out b/src/test/regress/expected/create_single_shard_table.out similarity index 94% rename from src/test/regress/expected/create_null_dist_key.out rename to src/test/regress/expected/create_single_shard_table.out index af6e66f62..8478dc293 100644 --- a/src/test/regress/expected/create_null_dist_key.out +++ b/src/test/regress/expected/create_single_shard_table.out @@ -1,5 +1,5 @@ -CREATE SCHEMA create_null_dist_key; -SET search_path TO create_null_dist_key; +CREATE SCHEMA create_single_shard_table; +SET search_path TO create_single_shard_table; SET citus.next_shard_id TO 1720000; SET citus.shard_count TO 32; SET citus.shard_replication_factor TO 1; @@ -16,7 +16,7 @@ SELECT 1 FROM citus_remove_node('localhost', :worker_2_port); (1 row) CREATE TABLE add_node_test(a int, "b" text); --- add a node before creating the null-shard-key table +-- add a node before creating the single-shard table SELECT 1 FROM citus_add_node('localhost', :worker_1_port); ?column? --------------------------------------------------------------------- @@ -29,7 +29,7 @@ SELECT create_distributed_table('add_node_test', null, colocate_with=>'none', di (1 row) --- add a node after creating the null-shard-key table +-- add a node after creating the single-shard table SELECT 1 FROM citus_add_node('localhost', :worker_2_port); ?column? --------------------------------------------------------------------- @@ -38,7 +38,7 @@ SELECT 1 FROM citus_add_node('localhost', :worker_2_port); -- make sure that table is created on the worker nodes added before/after create_distributed_table SELECT result FROM run_command_on_workers($$ - SELECT COUNT(*)=1 FROM pg_class WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + SELECT COUNT(*)=1 FROM pg_class WHERE relnamespace = 'create_single_shard_table'::regnamespace AND relname='add_node_test' $$); result @@ -50,7 +50,7 @@ $$); -- and check the metadata tables SELECT result FROM run_command_on_workers($$ SELECT (partmethod, partkey, repmodel, autoconverted) FROM pg_dist_partition - WHERE logicalrelid = 'create_null_dist_key.add_node_test'::regclass + WHERE logicalrelid = 'create_single_shard_table.add_node_test'::regclass $$); result --------------------------------------------------------------------- @@ -60,7 +60,7 @@ $$); SELECT result FROM run_command_on_workers($$ SELECT (shardstorage, shardminvalue, shardmaxvalue) FROM pg_dist_shard - WHERE logicalrelid = 'create_null_dist_key.add_node_test'::regclass + WHERE logicalrelid = 'create_single_shard_table.add_node_test'::regclass $$); result --------------------------------------------------------------------- @@ -72,7 +72,7 @@ SELECT result FROM run_command_on_workers($$ SELECT COUNT(*)=1 FROM pg_dist_placement WHERE shardid = ( SELECT shardid FROM pg_dist_shard - WHERE logicalrelid = 'create_null_dist_key.add_node_test'::regclass + WHERE logicalrelid = 'create_single_shard_table.add_node_test'::regclass ); $$); result @@ -85,7 +85,7 @@ SELECT result FROM run_command_on_workers($$ SELECT (shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation) FROM pg_dist_colocation WHERE colocationid = ( SELECT colocationid FROM pg_dist_partition - WHERE logicalrelid = 'create_null_dist_key.add_node_test'::regclass + WHERE logicalrelid = 'create_single_shard_table.add_node_test'::regclass ); $$); result @@ -116,7 +116,7 @@ SELECT create_distributed_table('nullkey_c1_t1', null, colocate_with=>'none'); (1 row) -SELECT colocationid AS nullkey_c1_t1_colocation_id FROM pg_dist_partition WHERE logicalrelid = 'create_null_dist_key.nullkey_c1_t1'::regclass \gset +SELECT colocationid AS nullkey_c1_t1_colocation_id FROM pg_dist_partition WHERE logicalrelid = 'create_single_shard_table.nullkey_c1_t1'::regclass \gset BEGIN; DROP TABLE nullkey_c1_t1; -- make sure that we delete the colocation group after dropping the last table that belongs to it @@ -171,7 +171,7 @@ SELECT create_distributed_table('nullkey_c2_t3', null, colocate_with=>'nullkey_c SELECT logicalrelid, partmethod, partkey, repmodel, autoconverted FROM pg_dist_partition WHERE logicalrelid IN ( SELECT oid FROM pg_class - WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + WHERE relnamespace = 'create_single_shard_table'::regnamespace AND relname LIKE 'nullkey_c1_%' ) ORDER BY 1; @@ -186,7 +186,7 @@ ORDER BY 1; SELECT COUNT(*) FROM pg_dist_partition WHERE logicalrelid IN ( SELECT oid FROM pg_class - WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + WHERE relnamespace = 'create_single_shard_table'::regnamespace AND relname LIKE 'nullkey_c1_%' ) GROUP BY colocationid; @@ -198,7 +198,7 @@ GROUP BY colocationid; SELECT logicalrelid, shardstorage, shardminvalue, shardmaxvalue FROM pg_dist_shard WHERE logicalrelid IN ( SELECT oid FROM pg_class - WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + WHERE relnamespace = 'create_single_shard_table'::regnamespace AND relname LIKE 'nullkey_c1_%' ) ORDER BY 1; @@ -215,7 +215,7 @@ WHERE shardid IN ( SELECT shardid FROM pg_dist_shard WHERE logicalrelid IN ( SELECT oid FROM pg_class - WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + WHERE relnamespace = 'create_single_shard_table'::regnamespace AND relname LIKE 'nullkey_c1_%' ) ) @@ -229,7 +229,7 @@ GROUP BY groupid; SELECT logicalrelid, partmethod, partkey, repmodel, autoconverted FROM pg_dist_partition WHERE logicalrelid IN ( SELECT oid FROM pg_class - WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + WHERE relnamespace = 'create_single_shard_table'::regnamespace AND relname LIKE 'nullkey_c2_%' ) ORDER BY 1; @@ -244,7 +244,7 @@ ORDER BY 1; SELECT COUNT(*) FROM pg_dist_partition WHERE logicalrelid IN ( SELECT oid FROM pg_class - WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + WHERE relnamespace = 'create_single_shard_table'::regnamespace AND relname LIKE 'nullkey_c2_%' ) GROUP BY colocationid; @@ -256,7 +256,7 @@ GROUP BY colocationid; SELECT logicalrelid, shardstorage, shardminvalue, shardmaxvalue FROM pg_dist_shard WHERE logicalrelid IN ( SELECT oid FROM pg_class - WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + WHERE relnamespace = 'create_single_shard_table'::regnamespace AND relname LIKE 'nullkey_c2_%' ) ORDER BY 1; @@ -273,7 +273,7 @@ WHERE shardid IN ( SELECT shardid FROM pg_dist_shard WHERE logicalrelid IN ( SELECT oid FROM pg_class - WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + WHERE relnamespace = 'create_single_shard_table'::regnamespace AND relname LIKE 'nullkey_c2_%' ) ) @@ -293,12 +293,12 @@ GROUP BY groupid; SELECT ( SELECT colocationid FROM pg_dist_partition - WHERE logicalrelid = 'create_null_dist_key.nullkey_c1_t1'::regclass + WHERE logicalrelid = 'create_single_shard_table.nullkey_c1_t1'::regclass ) != ( SELECT colocationid FROM pg_dist_partition - WHERE logicalrelid = 'create_null_dist_key.nullkey_c2_t1'::regclass + WHERE logicalrelid = 'create_single_shard_table.nullkey_c2_t1'::regclass ); ?column? --------------------------------------------------------------------- @@ -312,7 +312,7 @@ SELECT SELECT groupid FROM pg_dist_placement WHERE shardid = ( SELECT shardid FROM pg_dist_shard - WHERE logicalrelid = 'create_null_dist_key.nullkey_c1_t1'::regclass + WHERE logicalrelid = 'create_single_shard_table.nullkey_c1_t1'::regclass ) ) != @@ -320,7 +320,7 @@ SELECT SELECT groupid FROM pg_dist_placement WHERE shardid = ( SELECT shardid FROM pg_dist_shard - WHERE logicalrelid = 'create_null_dist_key.nullkey_c2_t1'::regclass + WHERE logicalrelid = 'create_single_shard_table.nullkey_c2_t1'::regclass ) ); ?column? @@ -334,7 +334,7 @@ SELECT SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation WHERE colocationid = ( SELECT colocationid FROM pg_dist_partition - WHERE logicalrelid = 'create_null_dist_key.nullkey_c1_t1'::regclass + WHERE logicalrelid = 'create_single_shard_table.nullkey_c1_t1'::regclass ); shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation --------------------------------------------------------------------- @@ -344,7 +344,7 @@ WHERE colocationid = ( SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation WHERE colocationid = ( SELECT colocationid FROM pg_dist_partition - WHERE logicalrelid = 'create_null_dist_key.nullkey_c2_t1'::regclass + WHERE logicalrelid = 'create_single_shard_table.nullkey_c2_t1'::regclass ); shardcount | replicationfactor | distributioncolumntype | distributioncolumncollation --------------------------------------------------------------------- @@ -359,7 +359,7 @@ SELECT create_distributed_table('round_robin_test_c1', null, colocate_with=>'non (1 row) \c - - - :master_port -SET search_path TO create_null_dist_key; +SET search_path TO create_single_shard_table; SET citus.next_shard_id TO 1730000; SET citus.shard_count TO 32; SET citus.shard_replication_factor TO 1; @@ -379,7 +379,7 @@ SELECT SELECT groupid FROM pg_dist_placement WHERE shardid = ( SELECT shardid FROM pg_dist_shard - WHERE logicalrelid = 'create_null_dist_key.round_robin_test_c1'::regclass + WHERE logicalrelid = 'create_single_shard_table.round_robin_test_c1'::regclass ) ) != @@ -387,7 +387,7 @@ SELECT SELECT groupid FROM pg_dist_placement WHERE shardid = ( SELECT shardid FROM pg_dist_shard - WHERE logicalrelid = 'create_null_dist_key.round_robin_test_c2'::regclass + WHERE logicalrelid = 'create_single_shard_table.round_robin_test_c2'::regclass ) ); ?column? @@ -396,7 +396,7 @@ SELECT (1 row) CREATE TABLE distributed_table(a int, b int); --- cannot colocate a sharded table with null shard key table +-- cannot colocate a sharded table with single-shard table SELECT create_distributed_table('distributed_table', 'a', colocate_with=>'nullkey_c1_t1'); ERROR: cannot colocate tables nullkey_c1_t1 and distributed_table DETAIL: Distribution column types don't match for nullkey_c1_t1 and distributed_table. @@ -414,7 +414,7 @@ SELECT create_distributed_table('distributed_table', 'a'); (1 row) --- cannot colocate null shard key tables with other table types +-- cannot colocate single-shard tables with other table types CREATE TABLE cannot_colocate_with_other_types (a int, b int); SELECT create_distributed_table('cannot_colocate_with_other_types', null, colocate_with=>'reference_table'); ERROR: cannot colocate tables reference_table and cannot_colocate_with_other_types @@ -430,7 +430,7 @@ SELECT citus_add_local_table_to_metadata('local'); (1 row) --- cannot colocate null shard key tables with citus local tables +-- cannot colocate single-shard tables with citus local tables SELECT create_distributed_table('cannot_colocate_with_other_types', null, colocate_with=>'local'); -- citus local ERROR: cannot distribute relation DETAIL: Currently, colocate_with option is not supported with append / range distributed tables and local tables added to metadata. @@ -447,15 +447,15 @@ SELECT create_distributed_table('local', null, colocate_with=>'none'); (1 row) BEGIN; - -- creating a null-shard-key table from a temporary table is not supported + -- creating a single-shard table from a temporary table is not supported CREATE TEMPORARY TABLE temp_table (a int); SELECT create_distributed_table('temp_table', null, colocate_with=>'none', distribution_type=>null); ERROR: cannot distribute a temporary table ROLLBACK; --- creating a null-shard-key table from a catalog table is not supported +-- creating a single-shard table from a catalog table is not supported SELECT create_distributed_table('pg_catalog.pg_index', NULL, distribution_type=>null); ERROR: cannot create a citus table from a catalog table --- creating a null-shard-key table from an unlogged table is supported +-- creating a single-shard table from an unlogged table is supported CREATE UNLOGGED TABLE unlogged_table (a int); SELECT create_distributed_table('unlogged_table', null, colocate_with=>'none', distribution_type=>null); create_distributed_table @@ -463,15 +463,15 @@ SELECT create_distributed_table('unlogged_table', null, colocate_with=>'none', d (1 row) --- creating a null-shard-key table from a foreign table is not supported +-- creating a single-shard table from a foreign table is not supported CREATE FOREIGN TABLE foreign_table ( id bigint not null, full_name text not null default '' ) SERVER fake_fdw_server OPTIONS (encoding 'utf-8', compression 'true', table_name 'foreign_table'); SELECT create_distributed_table('foreign_table', null, colocate_with=>'none', distribution_type=>null); ERROR: foreign tables cannot be distributed -HINT: Can add foreign table "foreign_table" to metadata by running: SELECT citus_add_local_table_to_metadata($$create_null_dist_key.foreign_table$$); --- create a null dist key table that has no tuples +HINT: Can add foreign table "foreign_table" to metadata by running: SELECT citus_add_local_table_to_metadata($$create_single_shard_table.foreign_table$$); +-- create a single-shard table that has no tuples CREATE TABLE null_dist_key_table_1 (a int primary key); SELECT create_distributed_table('null_dist_key_table_1', null, colocate_with=>'none'); create_distributed_table @@ -479,7 +479,7 @@ SELECT create_distributed_table('null_dist_key_table_1', null, colocate_with=>'n (1 row) --- create a null dist key table that has some tuples +-- create a single-shard table that has some tuples CREATE TABLE null_dist_key_table_2(a int primary key); INSERT INTO null_dist_key_table_2 VALUES(1); SELECT create_distributed_table('null_dist_key_table_2', null, colocate_with=>'none'); @@ -495,7 +495,7 @@ SELECT * FROM null_dist_key_table_2 ORDER BY a; (1 row) DROP TABLE null_dist_key_table_1, null_dist_key_table_2; --- create indexes before creating the null dist key tables +-- create indexes before creating the single-shard tables -- .. for an initially empty table CREATE TABLE null_dist_key_table_1(a int, b int); CREATE STATISTICS s1 (dependencies) ON a, b FROM null_dist_key_table_1; @@ -507,7 +507,7 @@ SELECT create_distributed_table('null_dist_key_table_1', null, colocate_with=>'n (1 row) CREATE STATISTICS s2 (dependencies) ON a, b FROM null_dist_key_table_1; --- .. and for another table having data in it before creating null dist key table +-- .. and for another table having data in it before creating single-shard table CREATE TABLE null_dist_key_table_2(a int); INSERT INTO null_dist_key_table_2 VALUES(1); CREATE INDEX null_dist_key_table_2_idx ON null_dist_key_table_2(a); @@ -634,7 +634,7 @@ SELECT * FROM "Table?!.1Table" ORDER BY id; 10 | 15 | (150,"{""f1"": 4, ""f2"": 8}") | {} | text_1 | 10 | 27 | yes | 60 | 70 | 4204 (4 rows) -SET search_path TO create_null_dist_key; +SET search_path TO create_single_shard_table; -- create a partitioned table with some columns that -- are going to be dropped within the tests CREATE TABLE sensors( @@ -660,10 +660,10 @@ SELECT create_distributed_table('sensors', NULL, distribution_type=>null); -- verify we can create new partitions after distributing the parent table CREATE TABLE sensors_2001 PARTITION OF sensors FOR VALUES FROM ('2001-01-01') TO ('2002-01-01'); --- verify we can attach to a null dist key table +-- verify we can attach to a single-shard table CREATE TABLE sensors_2002 (measureid integer, eventdatetime date, measure_data jsonb, PRIMARY KEY (measureid, eventdatetime, measure_data)); ALTER TABLE sensors ATTACH PARTITION sensors_2002 FOR VALUES FROM ('2002-01-01') TO ('2003-01-01'); --- verify we can detach from a null dist key table +-- verify we can detach from a single-shard table ALTER TABLE sensors DETACH PARTITION sensors_2001; -- error out when attaching a noncolocated partition CREATE TABLE sensors_2003 (measureid integer, eventdatetime date, measure_data jsonb, PRIMARY KEY (measureid, eventdatetime, measure_data)); @@ -749,7 +749,7 @@ SELECT COUNT(*) FROM run_command_on_workers($$ 1 (1 row) --- create a partitioned citus local table and verify we error out when attaching a partition with null dist key +-- create a partitioned citus local table and verify we error out when attaching a partition with single-shard CREATE TABLE partitioned_citus_local_tbl( measureid integer, eventdatetime date, @@ -911,13 +911,13 @@ DETAIL: To enforce foreign keys, the referencing and referenced rows need to be HINT: You could use SELECT create_reference_table('local_table_for_fkey') to replicate the referenced table to all nodes or consider dropping the foreign key -- Normally, we support foreign keys from Postgres tables to distributed -- tables assuming that the user will soon distribute the local table too --- anyway. However, this is not the case for null-shard-key tables before +-- anyway. However, this is not the case for single-shard tables before -- we improve SQL support. ALTER TABLE local_table_for_fkey ADD CONSTRAINT fkey_from_dummy_local FOREIGN KEY (a) REFERENCES "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id); ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables DETAIL: Local tables cannot be used in distributed queries. -CONTEXT: SQL statement "SELECT fk."a" FROM ONLY "create_null_dist_key"."local_table_for_fkey" fk LEFT OUTER JOIN "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234" pk ON ( pk."id" OPERATOR(pg_catalog.=) fk."a") WHERE pk."id" IS NULL AND (fk."a" IS NOT NULL)" +CONTEXT: SQL statement "SELECT fk."a" FROM ONLY "create_single_shard_table"."local_table_for_fkey" fk LEFT OUTER JOIN "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234" pk ON ( pk."id" OPERATOR(pg_catalog.=) fk."a") WHERE pk."id" IS NULL AND (fk."a" IS NOT NULL)" -- foreign key to a citus local table, errors out CREATE TABLE citus_local_table_for_fkey (a INT PRIMARY KEY); SELECT citus_add_local_table_to_metadata('citus_local_table_for_fkey'); @@ -988,7 +988,7 @@ ALTER TABLE null_key_dist DROP CONSTRAINT fkey_add_test_2; ALTER TABLE null_key_dist DROP CONSTRAINT fkey_add_test_3; ALTER TABLE null_key_dist DROP CONSTRAINT fkey_add_test_4; ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" DROP CONSTRAINT fkey_to_dummy_dist; --- create a view that depends on the null shard key table +-- create a view that depends on the single-shard table CREATE VIEW public.v1 AS SELECT * FROM null_key_dist; SELECT * FROM public.v1; a @@ -1037,7 +1037,7 @@ CREATE TABLE identity_test ( c bigint GENERATED BY DEFAULT AS IDENTITY (START WITH 1000 INCREMENT BY 1000) ); SELECT create_distributed_table('identity_test', NULL, distribution_type=>null); -ERROR: cannot complete operation on create_null_dist_key.identity_test with smallint/int identity column +ERROR: cannot complete operation on create_single_shard_table.identity_test with smallint/int identity column HINT: Use bigint identity column instead. DROP TABLE identity_test; -- Above failed because we don't support using a data type other than BIGINT @@ -1061,7 +1061,7 @@ CONTEXT: PL/pgSQL function normalize_generate_always_as_error(text) line XX at INSERT INTO identity_test (b) OVERRIDING SYSTEM VALUE VALUES (5); INSERT INTO identity_test (c) VALUES (5); SELECT result, success FROM run_command_on_workers($$ - INSERT INTO create_null_dist_key.identity_test (a) VALUES (6) + INSERT INTO create_single_shard_table.identity_test (a) VALUES (6) $$); result | success --------------------------------------------------------------------- @@ -1070,7 +1070,7 @@ $$); (2 rows) SELECT result, success FROM run_command_on_workers($$ - SELECT create_null_dist_key.normalize_generate_always_as_error('INSERT INTO create_null_dist_key.identity_test (b) VALUES (1)') + SELECT create_single_shard_table.normalize_generate_always_as_error('INSERT INTO create_single_shard_table.identity_test (b) VALUES (1)') $$); result | success --------------------------------------------------------------------- @@ -1080,7 +1080,7 @@ $$); -- This should fail due to missing OVERRIDING SYSTEM VALUE. SELECT result, success FROM run_command_on_workers($$ - SELECT create_null_dist_key.normalize_generate_always_as_error('INSERT INTO create_null_dist_key.identity_test (a, b) VALUES (1, 1)') + SELECT create_single_shard_table.normalize_generate_always_as_error('INSERT INTO create_single_shard_table.identity_test (a, b) VALUES (1, 1)') $$); result | success --------------------------------------------------------------------- @@ -1089,7 +1089,7 @@ $$); (2 rows) SELECT result, success FROM run_command_on_workers($$ - INSERT INTO create_null_dist_key.identity_test (a, b) OVERRIDING SYSTEM VALUE VALUES (7, 7) + INSERT INTO create_single_shard_table.identity_test (a, b) OVERRIDING SYSTEM VALUE VALUES (7, 7) $$); result | success --------------------------------------------------------------------- @@ -1098,7 +1098,7 @@ $$); (2 rows) SELECT result, success FROM run_command_on_workers($$ - INSERT INTO create_null_dist_key.identity_test (c, a) OVERRIDING SYSTEM VALUE VALUES (8, 8) + INSERT INTO create_single_shard_table.identity_test (c, a) OVERRIDING SYSTEM VALUE VALUES (8, 8) $$); result | success --------------------------------------------------------------------- @@ -1110,7 +1110,7 @@ $$); CREATE TABLE referenced_table(a int UNIQUE, b int); CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a) REFERENCES referenced_table(a)); --- to a colocated null dist key table +-- to a colocated single-shard table BEGIN; SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); create_distributed_table @@ -1132,7 +1132,7 @@ ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign k DETAIL: Key (a)=(2) is not present in table "referenced_table_xxxxxxx". CONTEXT: while executing command on localhost:xxxxx ROLLBACK; --- to a non-colocated null dist key table +-- to a non-colocated single-shard table BEGIN; SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); create_distributed_table @@ -1278,7 +1278,7 @@ SELECT create_distributed_table('referencing_table', NULL, distribution_type=>nu INSERT INTO referenced_table VALUES (1, 1); -- ok SELECT result, success FROM run_command_on_workers($$ - INSERT INTO create_null_dist_key.referencing_table VALUES (1, 2) + INSERT INTO create_single_shard_table.referencing_table VALUES (1, 2) $$); result | success --------------------------------------------------------------------- @@ -1288,7 +1288,7 @@ $$); -- fails SELECT result, success FROM run_command_on_workers($$ - INSERT INTO create_null_dist_key.referencing_table VALUES (2, 2) + INSERT INTO create_single_shard_table.referencing_table VALUES (2, 2) $$); result | success --------------------------------------------------------------------- @@ -1312,8 +1312,8 @@ ERROR: insert or update on table "self_fkey_test_1730152" violates foreign key DETAIL: Key (b)=(3) is not present in table "self_fkey_test_1730152". CONTEXT: while executing command on localhost:xxxxx -- similar foreign key tests but this time create the referencing table later on --- referencing table is a null shard key table --- to a colocated null dist key table +-- referencing table is a single-shard table +-- to a colocated single-shard table BEGIN; CREATE TABLE referenced_table(a int UNIQUE, b int); SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); @@ -1415,7 +1415,7 @@ BEGIN; ALTER TABLE referencing_table ADD CONSTRAINT fkey_to_dummy_ref_on_update FOREIGN KEY (a) REFERENCES referenced_table(a) ON UPDATE SET DEFAULT; ERROR: cannot create foreign key constraint since Citus does not support ON DELETE / UPDATE SET DEFAULT actions on the columns that default to sequences ROLLBACK; --- to a non-colocated null dist key table +-- to a non-colocated single-shard table BEGIN; CREATE TABLE referenced_table(a int UNIQUE, b int); SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); @@ -1514,7 +1514,7 @@ ERROR: referenced table "referenced_table" must be a distributed table or a ref DETAIL: To enforce foreign keys, the referencing and referenced rows need to be stored on the same node. HINT: You could use SELECT create_reference_table('referenced_table') to replicate the referenced table to all nodes or consider dropping the foreign key ROLLBACK; --- referenced table is a null shard key table +-- referenced table is a single-shard table -- from a sharded table BEGIN; CREATE TABLE referenced_table(a int UNIQUE, b int); @@ -1587,9 +1587,9 @@ SELECT create_distributed_table('referencing_table', null, colocate_with=>'none' SET client_min_messages TO DEBUG1; BEGIN; -- Switches to sequential execution because referenced_table is a reference table - -- and referenced by a null-shard-key distributed table. + -- and referenced by a single-shard table. -- - -- Given that we cannot do parallel access on null-shard-key, this is not useful. + -- Given that we cannot do parallel access on a single-shard table, this is not useful. -- However, this is already what we're doing for, e.g., a foreign key from a -- reference table to another reference table. TRUNCATE referenced_table CASCADE; @@ -1642,9 +1642,9 @@ BEGIN; ROLLBACK; BEGIN; -- Switches to sequential execution because referenced_table is a reference table - -- and referenced by a null-shard-key distributed table. + -- and referenced by a single-shard table. -- - -- Given that we cannot do parallel access on null-shard-key, this is not useful. + -- Given that we cannot do parallel access on a single-shard table, this is not useful. -- However, this is already what we're doing for, e.g., a foreign key from a -- reference table to another reference table. UPDATE referenced_table SET id = 101 WHERE id = 99; @@ -1662,7 +1662,7 @@ ROLLBACK; SET client_min_messages TO WARNING; DROP TABLE referenced_table, referencing_table; -- Test whether we unnecessarily switch to sequential execution --- when the referenced relation is a null-shard-key table. +-- when the referenced relation is a single-shard table. CREATE TABLE referenced_table(id int PRIMARY KEY, value_1 int); SELECT create_distributed_table('referenced_table', null, colocate_with=>'none', distribution_type=>null); create_distributed_table @@ -1686,12 +1686,12 @@ BEGIN; (1 row) -- Doesn't switch to sequential execution because the referenced_table is - -- a null-shard-key distributed table. + -- a single-shard table. ALTER TABLE referencing_table ADD COLUMN X INT; ROLLBACK; BEGIN; -- Doesn't switch to sequential execution because the referenced_table is - -- a null-shard-key distributed table. + -- a single-shard table. TRUNCATE referenced_table CASCADE; NOTICE: truncate cascades to table "referencing_table" DEBUG: truncate cascades to table "referencing_table_xxxxxxx" @@ -1809,4 +1809,4 @@ DROP TRIGGER trigger_2 ON trigger_table_2 CASCADE; DROP TRIGGER trigger_3 ON trigger_table_3 RESTRICT; -- cleanup at exit SET client_min_messages TO ERROR; -DROP SCHEMA create_null_dist_key, "NULL_!_dist_key" CASCADE; +DROP SCHEMA create_single_shard_table, "NULL_!_dist_key" CASCADE; diff --git a/src/test/regress/expected/insert_select_null_dist_key.out b/src/test/regress/expected/insert_select_single_shard_table.out similarity index 96% rename from src/test/regress/expected/insert_select_null_dist_key.out rename to src/test/regress/expected/insert_select_single_shard_table.out index b5391063c..b80d62ad4 100644 --- a/src/test/regress/expected/insert_select_null_dist_key.out +++ b/src/test/regress/expected/insert_select_single_shard_table.out @@ -1,5 +1,5 @@ -CREATE SCHEMA insert_select_null_dist_key; -SET search_path TO insert_select_null_dist_key; +CREATE SCHEMA insert_select_single_shard_table; +SET search_path TO insert_select_single_shard_table; SET citus.next_shard_id TO 1820000; SET citus.shard_count TO 32; SET client_min_messages TO WARNING; @@ -115,8 +115,8 @@ INSERT INTO range_table VALUES (0, 1), (1, 2), (2, 3), (3, 4), (4, 5), (5, 6), ( CREATE MATERIALIZED VIEW matview AS SELECT b*2+a AS a, a*a AS b FROM nullkey_c1_t1; SET client_min_messages TO DEBUG2; -- Test inserting into a distributed table by selecting from a combination of --- different table types together with null-shard-key tables. --- use a null-shard-key table +-- different table types together with single-shard tables. +-- use a single-shard table INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1; DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables DEBUG: Distributed planning for a fast-path router query @@ -139,7 +139,7 @@ INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries DEBUG: Creating router plan DEBUG: Collecting INSERT ... SELECT results on coordinator --- use a colocated null-shard-key table +-- use a colocated single-shard table INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN nullkey_c1_t2 USING (b); DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables DEBUG: Creating router plan @@ -156,7 +156,7 @@ INSERT INTO distributed_table_c1_t1 SELECT * FROM nullkey_c1_t1 UNION SELECT * F DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries DEBUG: Creating router plan DEBUG: Collecting INSERT ... SELECT results on coordinator --- use a non-colocated null-shard-key table +-- use a non-colocated single-shard table INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 LEFT JOIN nullkey_c2_t1 USING (a); DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables @@ -220,8 +220,8 @@ INSERT INTO distributed_table_c1_t1 SELECT i, i FROM generate_series(3, 8) i; DEBUG: distributed INSERT ... SELECT can only select from distributed tables DEBUG: Collecting INSERT ... SELECT results on coordinator -- Test inserting into a reference table by selecting from a combination of --- different table types together with null-shard-key tables. --- use a null-shard-key table +-- different table types together with single-shard tables. +-- use a single-shard table INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1; DEBUG: only reference tables may be queried when targeting a reference table with distributed INSERT ... SELECT DEBUG: Distributed planning for a fast-path router query @@ -244,7 +244,7 @@ INSERT INTO reference_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey DEBUG: only reference tables may be queried when targeting a reference table with distributed INSERT ... SELECT ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables DETAIL: Local tables cannot be used in distributed queries. --- use a colocated null-shard-key table +-- use a colocated single-shard table INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN nullkey_c1_t2 USING (b); DEBUG: only reference tables may be queried when targeting a reference table with distributed INSERT ... SELECT DEBUG: Creating router plan @@ -253,7 +253,7 @@ INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey DEBUG: only reference tables may be queried when targeting a reference table with distributed INSERT ... SELECT DEBUG: Creating router plan DEBUG: Collecting INSERT ... SELECT results on coordinator --- use a non-colocated null-shard-key table +-- use a non-colocated single-shard table INSERT INTO reference_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 LEFT JOIN nullkey_c2_t1 USING (a); DEBUG: only reference tables may be queried when targeting a reference table with distributed INSERT ... SELECT ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables @@ -299,8 +299,8 @@ INSERT INTO reference_table SELECT i, i FROM generate_series(0, 5) i; DEBUG: distributed INSERT ... SELECT can only select from distributed tables DEBUG: Collecting INSERT ... SELECT results on coordinator -- Test inserting into a citus local table by selecting from a combination of --- different table types together with null-shard-key tables. --- use a null-shard-key table +-- different table types together with single-shard tables. +-- use a single-shard table INSERT INTO citus_local_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1; DEBUG: distributed INSERT ... SELECT cannot insert into a local table that is added to metadata DEBUG: Distributed planning for a fast-path router query @@ -311,7 +311,7 @@ INSERT INTO citus_local_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullk DEBUG: distributed INSERT ... SELECT cannot insert into a local table that is added to metadata DEBUG: Creating router plan DEBUG: Collecting INSERT ... SELECT results on coordinator --- use a colocated null-shard-key table +-- use a colocated single-shard table INSERT INTO citus_local_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN nullkey_c1_t2 USING (b); DEBUG: distributed INSERT ... SELECT cannot insert into a local table that is added to metadata DEBUG: Creating router plan @@ -343,8 +343,8 @@ TRUNCATE citus_local_table; INSERT INTO citus_local_table SELECT i, i FROM generate_series(0, 10) i; DEBUG: distributed INSERT ... SELECT can only select from distributed tables DEBUG: Collecting INSERT ... SELECT results on coordinator --- Test inserting into a null-shard-key table by selecting from a combination of --- different table types, together with or without null-shard-key tables. +-- Test inserting into a single-shard table by selecting from a combination of +-- different table types, together with or without single-shard tables. -- use a postgres local table INSERT INTO nullkey_c1_t1 SELECT postgres_local_table.a, postgres_local_table.b FROM postgres_local_table; DEBUG: distributed INSERT ... SELECT can only select from distributed tables @@ -384,7 +384,7 @@ INSERT INTO nullkey_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1 DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables DETAIL: Router planner cannot handle multi-shard select queries --- use a non-colocated null-shard-key table +-- use a non-colocated single-shard table INSERT INTO nullkey_c2_t1 SELECT q.* FROM (SELECT reference_table.* FROM reference_table LEFT JOIN nullkey_c1_t1 USING (a)) q JOIN nullkey_c1_t2 USING (a); DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables DEBUG: Creating router plan @@ -434,7 +434,7 @@ INSERT INTO nullkey_c2_t1 SELECT i, i FROM generate_series(2, 7) i; DEBUG: distributed INSERT ... SELECT can only select from distributed tables DEBUG: Collecting INSERT ... SELECT results on coordinator -- Test inserting into a local table by selecting from a combination of --- different table types, together with or without null-shard-key tables. +-- different table types, together with or without single-shard tables. INSERT INTO postgres_local_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN reference_table USING (a); DEBUG: Creating router plan INSERT INTO postgres_local_table SELECT * FROM nullkey_c1_t1 ORDER BY 1,2 OFFSET 3 LIMIT 2; @@ -620,8 +620,8 @@ WHERE t1.a IN ( ) AS t2 ); DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables -DEBUG: generating subplan XXX_1 for subquery SELECT a FROM (SELECT t3.a FROM ((SELECT distributed_table_c1_t1.a FROM insert_select_null_dist_key.distributed_table_c1_t1 WHERE (distributed_table_c1_t1.b OPERATOR(pg_catalog.>) 4)) t3 JOIN (SELECT distributed_table_c1_t2.a FROM insert_select_null_dist_key.distributed_table_c1_t2 WHERE (distributed_table_c1_t2.b OPERATOR(pg_catalog.<) 7)) t4 ON ((t3.a OPERATOR(pg_catalog.=) t4.a)))) t2 -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT a, b FROM insert_select_null_dist_key.reference_table t1 WHERE (a OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer))) +DEBUG: generating subplan XXX_1 for subquery SELECT a FROM (SELECT t3.a FROM ((SELECT distributed_table_c1_t1.a FROM insert_select_single_shard_table.distributed_table_c1_t1 WHERE (distributed_table_c1_t1.b OPERATOR(pg_catalog.>) 4)) t3 JOIN (SELECT distributed_table_c1_t2.a FROM insert_select_single_shard_table.distributed_table_c1_t2 WHERE (distributed_table_c1_t2.b OPERATOR(pg_catalog.<) 7)) t4 ON ((t3.a OPERATOR(pg_catalog.=) t4.a)))) t2 +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT a, b FROM insert_select_single_shard_table.reference_table t1 WHERE (a OPERATOR(pg_catalog.=) ANY (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer))) DEBUG: Collecting INSERT ... SELECT results on coordinator SET client_min_messages TO DEBUG2; -- test upsert with plain INSERT query @@ -735,7 +735,7 @@ DEBUG: ALTER TABLE / ADD PRIMARY KEY will create implicit index "distributed_ta DEBUG: verifying table "distributed_table_c1_t1" INSERT INTO nullkey_c1_t1 AS t1 (a, b) SELECT t3.a, t3.b FROM nullkey_c1_t2 t2 JOIN reference_table t3 ON (t2.a = t3.a) ON CONFLICT (a) DO UPDATE SET a = t1.a + 10; -DEBUG: distributed statement: INSERT INTO insert_select_null_dist_key.nullkey_c1_t1_1820000 AS t1 (a, b) SELECT t3.a, t3.b FROM (insert_select_null_dist_key.nullkey_c1_t2_1820001 t2 JOIN insert_select_null_dist_key.reference_table_1820003 t3 ON ((t2.a OPERATOR(pg_catalog.=) t3.a))) ON CONFLICT(a) DO UPDATE SET a = (t1.a OPERATOR(pg_catalog.+) 10) +DEBUG: distributed statement: INSERT INTO insert_select_single_shard_table.nullkey_c1_t1_1820000 AS t1 (a, b) SELECT t3.a, t3.b FROM (insert_select_single_shard_table.nullkey_c1_t2_1820001 t2 JOIN insert_select_single_shard_table.reference_table_1820003 t3 ON ((t2.a OPERATOR(pg_catalog.=) t3.a))) ON CONFLICT(a) DO UPDATE SET a = (t1.a OPERATOR(pg_catalog.+) 10) SET client_min_messages TO DEBUG1; INSERT INTO distributed_table_c1_t1 AS t1 (a, b) SELECT t3.a, t3.b FROM nullkey_c1_t2 t2 JOIN reference_table t3 ON (t2.a = t3.a) ON CONFLICT (a, b) DO UPDATE SET b = t1.b + 10; @@ -805,7 +805,7 @@ DEBUG: Creating router plan (3 rows) SET client_min_messages TO WARNING; -DROP SCHEMA insert_select_null_dist_key CASCADE; +DROP SCHEMA insert_select_single_shard_table CASCADE; SELECT citus_remove_node('localhost', :master_port); citus_remove_node --------------------------------------------------------------------- diff --git a/src/test/regress/expected/merge.out b/src/test/regress/expected/merge.out index fd82efa8c..190e6b2b3 100644 --- a/src/test/regress/expected/merge.out +++ b/src/test/regress/expected/merge.out @@ -3228,9 +3228,9 @@ WHEN NOT MATCHED THEN INSERT VALUES(dist_source.id, dist_source.val); ERROR: For MERGE command, all the distributed tables must be colocated, for append/range distribution, colocation is not supported HINT: Consider using hash distribution instead --- test merge with null shard key tables -CREATE SCHEMA query_null_dist_key; -SET search_path TO query_null_dist_key; +-- test merge with single-shard tables +CREATE SCHEMA query_single_shard_table; +SET search_path TO query_single_shard_table; SET client_min_messages TO DEBUG2; CREATE TABLE nullkey_c1_t1(a int, b int); CREATE TABLE nullkey_c1_t2(a int, b int); @@ -3295,23 +3295,23 @@ INSERT INTO postgres_local_table SELECT i, i FROM generate_series(5, 10) i; -- with a colocated table MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b; -DEBUG: +DEBUG: DEBUG: Creating MERGE router plan MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a) WHEN MATCHED THEN DELETE; -DEBUG: +DEBUG: DEBUG: Creating MERGE router plan MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b); -DEBUG: +DEBUG: DEBUG: Creating MERGE router plan MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a) WHEN MATCHED THEN DELETE WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b); -DEBUG: +DEBUG: DEBUG: Creating MERGE router plan --- with non-colocated null-dist-key table +-- with non-colocated single-shard table MERGE INTO nullkey_c1_t1 USING nullkey_c2_t1 ON (nullkey_c1_t1.a = nullkey_c2_t1.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c2_t1.b; ERROR: For MERGE command, all the distributed tables must be colocated @@ -3331,11 +3331,12 @@ ERROR: For MERGE command, all the distributed tables must be colocated -- with a reference table MERGE INTO nullkey_c1_t1 USING reference_table ON (nullkey_c1_t1.a = reference_table.a) WHEN MATCHED THEN UPDATE SET b = reference_table.b; -ERROR: MERGE command is not supported on reference tables yet +ERROR: MERGE command is not supported with combination of distributed/reference yet +HINT: If target is distributed, source must be distributed and co-located MERGE INTO reference_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = reference_table.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t1.b WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b); -ERROR: MERGE command is not supported on reference tables yet +ERROR: Reference table as target is not allowed in MERGE command -- with a citus local table MERGE INTO nullkey_c1_t1 USING citus_local_table ON (nullkey_c1_t1.a = citus_local_table.a) WHEN MATCHED THEN UPDATE SET b = citus_local_table.b; @@ -3357,7 +3358,7 @@ WITH cte AS ( ) MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a) WHEN MATCHED THEN UPDATE SET b = cte.b; -DEBUG: +DEBUG: DEBUG: Creating MERGE router plan WITH cte AS ( SELECT * FROM distributed_table @@ -3372,7 +3373,7 @@ MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a) WHEN MATCHED THEN UPDATE SET b = cte.b; ERROR: For MERGE command, all the distributed tables must be colocated SET client_min_messages TO WARNING; -DROP SCHEMA query_null_dist_key CASCADE; +DROP SCHEMA query_single_shard_table CASCADE; RESET client_min_messages; SET search_path TO merge_schema; DROP SERVER foreign_server CASCADE; diff --git a/src/test/regress/expected/multi_extension.out b/src/test/regress/expected/multi_extension.out index 56de93802..dad925072 100644 --- a/src/test/regress/expected/multi_extension.out +++ b/src/test/regress/expected/multi_extension.out @@ -1735,7 +1735,7 @@ HINT: To remove the local data, run: SELECT truncate_local_data_after_distribut DROP TABLE test; TRUNCATE pg_dist_node; --- confirm that we can create a null shard key table on an empty node +-- confirm that we can create a single-shard table on an empty node CREATE TABLE test (x int, y int); INSERT INTO test VALUES (1,2); SET citus.shard_replication_factor TO 1; diff --git a/src/test/regress/expected/query_null_dist_key.out b/src/test/regress/expected/query_single_shard_table.out similarity index 98% rename from src/test/regress/expected/query_null_dist_key.out rename to src/test/regress/expected/query_single_shard_table.out index 09907a99b..a7efbf7be 100644 --- a/src/test/regress/expected/query_null_dist_key.out +++ b/src/test/regress/expected/query_single_shard_table.out @@ -1,5 +1,5 @@ -CREATE SCHEMA query_null_dist_key; -SET search_path TO query_null_dist_key; +CREATE SCHEMA query_single_shard_table; +SET search_path TO query_single_shard_table; SET citus.next_shard_id TO 1620000; SET citus.shard_count TO 32; SET client_min_messages TO WARNING; @@ -92,7 +92,7 @@ SELECT create_distributed_table('articles_hash', null, colocate_with=>'none'); NOTICE: Copying data from local table... NOTICE: copying the data has completed DETAIL: The local data in the table is no longer visible, but is still on disk. -HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$query_null_dist_key.articles_hash$$) +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$query_single_shard_table.articles_hash$$) create_distributed_table --------------------------------------------------------------------- @@ -247,7 +247,7 @@ DETAIL: Local tables cannot be used in distributed queries. SELECT COUNT(*) FROM postgres_local_table d1, nullkey_c1_t1; ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables DETAIL: Local tables cannot be used in distributed queries. --- with a colocated null dist key table +-- with a colocated single-shard table SELECT COUNT(*) FROM nullkey_c1_t1 d1, nullkey_c1_t2; DEBUG: Creating router plan count @@ -255,14 +255,14 @@ DEBUG: Creating router plan 110 (1 row) --- with a non-colocated null dist key table +-- with a non-colocated single-shard table SELECT COUNT(*) FROM nullkey_c1_t1 d1, nullkey_c2_t1; ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables DETAIL: found no worker with all shard placements -- First, show that nullkey_c1_t1 and nullkey_c3_t1 are not colocated. SELECT - (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'query_null_dist_key.nullkey_c1_t1'::regclass) != - (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'query_null_dist_key.nullkey_c3_t1'::regclass); + (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'query_single_shard_table.nullkey_c1_t1'::regclass) != + (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'query_single_shard_table.nullkey_c3_t1'::regclass); ?column? --------------------------------------------------------------------- t @@ -284,7 +284,7 @@ SELECT COUNT(*) FROM nullkey_c1_t1 JOIN nullkey_c3_t1 USING(a); ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables DETAIL: router planner does not support queries that reference non-colocated distributed tables RESET citus.enable_non_colocated_router_query_pushdown; --- colocated join between null dist key tables +-- colocated join between single-shard tables SELECT COUNT(*) FROM nullkey_c1_t1 JOIN nullkey_c1_t2 USING(a); DEBUG: Creating router plan count @@ -356,7 +356,7 @@ DEBUG: Creating router plan 7 (1 row) --- non-colocated inner joins between null dist key tables +-- non-colocated inner joins between single-shard tables SELECT * FROM nullkey_c1_t1 JOIN nullkey_c2_t1 USING(a) ORDER BY 1,2,3; ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables DETAIL: found no worker with all shard placements @@ -366,7 +366,7 @@ JOIN LATERAL ( ) q USING(a); ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables DETAIL: found no worker with all shard placements --- non-colocated outer joins between null dist key tables +-- non-colocated outer joins between single-shard tables SELECT * FROM nullkey_c1_t1 LEFT JOIN nullkey_c2_t2 USING(a) ORDER BY 1,2,3 LIMIT 4; ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables DETAIL: found no worker with all shard placements @@ -803,7 +803,7 @@ JOIN LATERAL ( ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables DETAIL: Local tables cannot be used in distributed queries. -- insert .. select --- between two colocated null dist key tables +-- between two colocated single-shard tables -- The target list of "distributed statement"s that we send to workers -- differ(*) in Postgres versions < 15. For this reason, we temporarily -- disable debug messages here and run the EXPLAIN'ed version of the @@ -825,13 +825,13 @@ INSERT INTO nullkey_c1_t1 SELECT * FROM nullkey_c1_t2; (7 rows) SET client_min_messages TO DEBUG2; --- between two non-colocated null dist key tables +-- between two non-colocated single-shard tables INSERT INTO nullkey_c1_t1 SELECT * FROM nullkey_c2_t1; DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables DEBUG: Distributed planning for a fast-path router query DEBUG: Creating router plan DEBUG: Collecting INSERT ... SELECT results on coordinator --- between a null dist key table and a table of different type +-- between a single-shard table and a table of different type SET client_min_messages TO WARNING; EXPLAIN (ANALYZE TRUE, TIMING FALSE, COSTS FALSE, SUMMARY FALSE, VERBOSE FALSE) INSERT INTO nullkey_c1_t1 SELECT * FROM reference_table; @@ -1300,8 +1300,8 @@ DEBUG: CTE cte is going to be inlined via distributed planning DEBUG: recursively planning left side of the right join since the outer side is a recurring rel DEBUG: recursively planning distributed relation "distributed_table" since it is part of a distributed join node that is outer joined with a recurring rel DEBUG: Wrapping relation "distributed_table" to a subquery -DEBUG: generating subplan XXX_1 for subquery SELECT a FROM query_null_dist_key.distributed_table WHERE true -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT "?column?" AS user_id, b AS value_1 FROM (SELECT ((cte.a OPERATOR(pg_catalog.+) 5) OPERATOR(pg_catalog.*) '-1'::integer), cte.b FROM (SELECT DISTINCT reference_table.a, 1 AS b FROM ((SELECT distributed_table_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) distributed_table_1) distributed_table RIGHT JOIN query_null_dist_key.reference_table USING (a))) cte) citus_insert_select_subquery("?column?", b) +DEBUG: generating subplan XXX_1 for subquery SELECT a FROM query_single_shard_table.distributed_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT "?column?" AS user_id, b AS value_1 FROM (SELECT ((cte.a OPERATOR(pg_catalog.+) 5) OPERATOR(pg_catalog.*) '-1'::integer), cte.b FROM (SELECT DISTINCT reference_table.a, 1 AS b FROM ((SELECT distributed_table_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) distributed_table_1) distributed_table RIGHT JOIN query_single_shard_table.reference_table USING (a))) cte) citus_insert_select_subquery("?column?", b) DEBUG: Collecting INSERT ... SELECT results on coordinator -- .. and via SELECT's cte list too INSERT INTO raw_events_second (user_id, value_1) @@ -1315,8 +1315,8 @@ DEBUG: distributed INSERT ... SELECT cannot reference a distributed table witho DEBUG: recursively planning left side of the right join since the outer side is a recurring rel DEBUG: recursively planning distributed relation "distributed_table" since it is part of a distributed join node that is outer joined with a recurring rel DEBUG: Wrapping relation "distributed_table" to a subquery -DEBUG: generating subplan XXX_1 for subquery SELECT a FROM query_null_dist_key.distributed_table WHERE true -DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT ((a OPERATOR(pg_catalog.+) 5) OPERATOR(pg_catalog.*) 2) AS user_id, b AS value_1 FROM (SELECT DISTINCT reference_table.a, 1 AS b FROM ((SELECT distributed_table_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) distributed_table_1) distributed_table RIGHT JOIN query_null_dist_key.reference_table USING (a))) cte +DEBUG: generating subplan XXX_1 for subquery SELECT a FROM query_single_shard_table.distributed_table WHERE true +DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT ((a OPERATOR(pg_catalog.+) 5) OPERATOR(pg_catalog.*) 2) AS user_id, b AS value_1 FROM (SELECT DISTINCT reference_table.a, 1 AS b FROM ((SELECT distributed_table_1.a, NULL::integer AS b FROM (SELECT intermediate_result.a FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer)) distributed_table_1) distributed_table RIGHT JOIN query_single_shard_table.reference_table USING (a))) cte DEBUG: Collecting INSERT ... SELECT results on coordinator -- using set operations INSERT INTO @@ -1738,7 +1738,7 @@ WITH cte AS ( SELECT * FROM nullkey_c1_t1 WHERE a IN (SELECT a FROM cte); DEBUG: Creating router plan DEBUG: query has a single distribution column value: 1 -ERROR: relation "query_null_dist_key.nullkey_c1_t1_1620000" does not exist +ERROR: relation "query_single_shard_table.nullkey_c1_t1_1620000" does not exist CONTEXT: while executing command on localhost:xxxxx WITH cte AS ( DELETE FROM distributed_table WHERE a = 1 RETURNING * @@ -1746,7 +1746,7 @@ WITH cte AS ( SELECT * FROM nullkey_c1_t1 WHERE b IN (SELECT b FROM cte); DEBUG: Creating router plan DEBUG: query has a single distribution column value: 1 -ERROR: relation "query_null_dist_key.nullkey_c1_t1_1620000" does not exist +ERROR: relation "query_single_shard_table.nullkey_c1_t1_1620000" does not exist CONTEXT: while executing command on localhost:xxxxx SET citus.enable_non_colocated_router_query_pushdown TO OFF; WITH cte AS ( @@ -1864,7 +1864,7 @@ DEBUG: Creating router plan (10 rows) SET client_min_messages TO ERROR; -DROP SCHEMA query_null_dist_key CASCADE; +DROP SCHEMA query_single_shard_table CASCADE; SELECT citus_remove_node('localhost', :master_port); citus_remove_node --------------------------------------------------------------------- diff --git a/src/test/regress/expected/single_node.out b/src/test/regress/expected/single_node.out index ecb652931..5efad578e 100644 --- a/src/test/regress/expected/single_node.out +++ b/src/test/regress/expected/single_node.out @@ -152,7 +152,7 @@ WHERE shardid = ( t (1 row) --- try creating a null-shard-key distributed table from a shard relation +-- try creating a single-shard table from a shard relation SELECT shardid AS round_robin_test_c1_shard_id FROM pg_dist_shard WHERE logicalrelid = 'single_node.single_node_nullkey_c1'::regclass \gset SELECT create_distributed_table('single_node_nullkey_c1_' || :round_robin_test_c1_shard_id , null, colocate_with=>'none', distribution_type=>null); ERROR: relation "single_node_nullkey_c1_90630532" is a shard relation diff --git a/src/test/regress/expected/single_node_0.out b/src/test/regress/expected/single_node_0.out index 86b4982e6..446db9aed 100644 --- a/src/test/regress/expected/single_node_0.out +++ b/src/test/regress/expected/single_node_0.out @@ -152,7 +152,7 @@ WHERE shardid = ( t (1 row) --- try creating a null-shard-key distributed table from a shard relation +-- try creating a single-shard table from a shard relation SELECT shardid AS round_robin_test_c1_shard_id FROM pg_dist_shard WHERE logicalrelid = 'single_node.single_node_nullkey_c1'::regclass \gset SELECT create_distributed_table('single_node_nullkey_c1_' || :round_robin_test_c1_shard_id , null, colocate_with=>'none', distribution_type=>null); ERROR: relation "single_node_nullkey_c1_90630532" is a shard relation diff --git a/src/test/regress/expected/null_dist_key_prep.out b/src/test/regress/expected/single_shard_table_prep.out similarity index 100% rename from src/test/regress/expected/null_dist_key_prep.out rename to src/test/regress/expected/single_shard_table_prep.out diff --git a/src/test/regress/multi_1_schedule b/src/test/regress/multi_1_schedule index 9163a5864..b5943f899 100644 --- a/src/test/regress/multi_1_schedule +++ b/src/test/regress/multi_1_schedule @@ -32,7 +32,7 @@ test: escape_extension_name test: ref_citus_local_fkeys test: alter_database_owner test: distributed_triggers -test: create_null_dist_key +test: create_single_shard_table test: multi_test_catalog_views test: multi_table_ddl @@ -68,7 +68,7 @@ test: multi_master_protocol multi_load_data multi_load_data_superuser multi_beha test: multi_behavioral_analytics_basics multi_behavioral_analytics_single_shard_queries multi_insert_select_non_pushable_queries multi_insert_select multi_behavioral_analytics_create_table_superuser test: multi_shard_update_delete recursive_dml_with_different_planners_executors test: insert_select_repartition window_functions dml_recursive multi_insert_select_window -test: multi_insert_select_conflict citus_table_triggers alter_table_null_dist_key +test: multi_insert_select_conflict citus_table_triggers alter_table_single_shard_table test: multi_row_insert insert_select_into_local_table alter_index # following should not run in parallel because it relies on connection counts to workers @@ -200,8 +200,8 @@ test: local_table_join test: local_dist_join_mixed test: citus_local_dist_joins test: recurring_outer_join -test: query_null_dist_key -test: insert_select_null_dist_key +test: query_single_shard_table +test: insert_select_single_shard_table test: pg_dump # --------- diff --git a/src/test/regress/null_dist_key_prep_schedule b/src/test/regress/null_dist_key_prep_schedule deleted file mode 100644 index 1a43130ec..000000000 --- a/src/test/regress/null_dist_key_prep_schedule +++ /dev/null @@ -1 +0,0 @@ -test: null_dist_key_prep diff --git a/src/test/regress/single_shard_table_prep_schedule b/src/test/regress/single_shard_table_prep_schedule new file mode 100644 index 000000000..7410cf05f --- /dev/null +++ b/src/test/regress/single_shard_table_prep_schedule @@ -0,0 +1 @@ +test: single_shard_table_prep diff --git a/src/test/regress/sql/alter_table_null_dist_key.sql b/src/test/regress/sql/alter_table_single_shard_table.sql similarity index 100% rename from src/test/regress/sql/alter_table_null_dist_key.sql rename to src/test/regress/sql/alter_table_single_shard_table.sql diff --git a/src/test/regress/sql/create_null_dist_key.sql b/src/test/regress/sql/create_single_shard_table.sql similarity index 91% rename from src/test/regress/sql/create_null_dist_key.sql rename to src/test/regress/sql/create_single_shard_table.sql index 9ca943d75..c7a2d49c2 100644 --- a/src/test/regress/sql/create_null_dist_key.sql +++ b/src/test/regress/sql/create_single_shard_table.sql @@ -1,5 +1,5 @@ -CREATE SCHEMA create_null_dist_key; -SET search_path TO create_null_dist_key; +CREATE SCHEMA create_single_shard_table; +SET search_path TO create_single_shard_table; SET citus.next_shard_id TO 1720000; SET citus.shard_count TO 32; @@ -10,17 +10,17 @@ SELECT 1 FROM citus_remove_node('localhost', :worker_2_port); CREATE TABLE add_node_test(a int, "b" text); --- add a node before creating the null-shard-key table +-- add a node before creating the single-shard table SELECT 1 FROM citus_add_node('localhost', :worker_1_port); SELECT create_distributed_table('add_node_test', null, colocate_with=>'none', distribution_type=>null); --- add a node after creating the null-shard-key table +-- add a node after creating the single-shard table SELECT 1 FROM citus_add_node('localhost', :worker_2_port); -- make sure that table is created on the worker nodes added before/after create_distributed_table SELECT result FROM run_command_on_workers($$ - SELECT COUNT(*)=1 FROM pg_class WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + SELECT COUNT(*)=1 FROM pg_class WHERE relnamespace = 'create_single_shard_table'::regnamespace AND relname='add_node_test' $$); @@ -28,19 +28,19 @@ $$); SELECT result FROM run_command_on_workers($$ SELECT (partmethod, partkey, repmodel, autoconverted) FROM pg_dist_partition - WHERE logicalrelid = 'create_null_dist_key.add_node_test'::regclass + WHERE logicalrelid = 'create_single_shard_table.add_node_test'::regclass $$); SELECT result FROM run_command_on_workers($$ SELECT (shardstorage, shardminvalue, shardmaxvalue) FROM pg_dist_shard - WHERE logicalrelid = 'create_null_dist_key.add_node_test'::regclass + WHERE logicalrelid = 'create_single_shard_table.add_node_test'::regclass $$); SELECT result FROM run_command_on_workers($$ SELECT COUNT(*)=1 FROM pg_dist_placement WHERE shardid = ( SELECT shardid FROM pg_dist_shard - WHERE logicalrelid = 'create_null_dist_key.add_node_test'::regclass + WHERE logicalrelid = 'create_single_shard_table.add_node_test'::regclass ); $$); @@ -48,7 +48,7 @@ SELECT result FROM run_command_on_workers($$ SELECT (shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation) FROM pg_dist_colocation WHERE colocationid = ( SELECT colocationid FROM pg_dist_partition - WHERE logicalrelid = 'create_null_dist_key.add_node_test'::regclass + WHERE logicalrelid = 'create_single_shard_table.add_node_test'::regclass ); $$); @@ -66,7 +66,7 @@ CREATE TABLE nullkey_c1_t2(a int, b int); CREATE TABLE nullkey_c1_t3(a int, b int); SELECT create_distributed_table('nullkey_c1_t1', null, colocate_with=>'none'); -SELECT colocationid AS nullkey_c1_t1_colocation_id FROM pg_dist_partition WHERE logicalrelid = 'create_null_dist_key.nullkey_c1_t1'::regclass \gset +SELECT colocationid AS nullkey_c1_t1_colocation_id FROM pg_dist_partition WHERE logicalrelid = 'create_single_shard_table.nullkey_c1_t1'::regclass \gset BEGIN; DROP TABLE nullkey_c1_t1; @@ -95,7 +95,7 @@ SELECT create_distributed_table('nullkey_c2_t3', null, colocate_with=>'nullkey_c SELECT logicalrelid, partmethod, partkey, repmodel, autoconverted FROM pg_dist_partition WHERE logicalrelid IN ( SELECT oid FROM pg_class - WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + WHERE relnamespace = 'create_single_shard_table'::regnamespace AND relname LIKE 'nullkey_c1_%' ) ORDER BY 1; @@ -104,7 +104,7 @@ ORDER BY 1; SELECT COUNT(*) FROM pg_dist_partition WHERE logicalrelid IN ( SELECT oid FROM pg_class - WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + WHERE relnamespace = 'create_single_shard_table'::regnamespace AND relname LIKE 'nullkey_c1_%' ) GROUP BY colocationid; @@ -112,7 +112,7 @@ GROUP BY colocationid; SELECT logicalrelid, shardstorage, shardminvalue, shardmaxvalue FROM pg_dist_shard WHERE logicalrelid IN ( SELECT oid FROM pg_class - WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + WHERE relnamespace = 'create_single_shard_table'::regnamespace AND relname LIKE 'nullkey_c1_%' ) ORDER BY 1; @@ -123,7 +123,7 @@ WHERE shardid IN ( SELECT shardid FROM pg_dist_shard WHERE logicalrelid IN ( SELECT oid FROM pg_class - WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + WHERE relnamespace = 'create_single_shard_table'::regnamespace AND relname LIKE 'nullkey_c1_%' ) ) @@ -134,7 +134,7 @@ GROUP BY groupid; SELECT logicalrelid, partmethod, partkey, repmodel, autoconverted FROM pg_dist_partition WHERE logicalrelid IN ( SELECT oid FROM pg_class - WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + WHERE relnamespace = 'create_single_shard_table'::regnamespace AND relname LIKE 'nullkey_c2_%' ) ORDER BY 1; @@ -143,7 +143,7 @@ ORDER BY 1; SELECT COUNT(*) FROM pg_dist_partition WHERE logicalrelid IN ( SELECT oid FROM pg_class - WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + WHERE relnamespace = 'create_single_shard_table'::regnamespace AND relname LIKE 'nullkey_c2_%' ) GROUP BY colocationid; @@ -151,7 +151,7 @@ GROUP BY colocationid; SELECT logicalrelid, shardstorage, shardminvalue, shardmaxvalue FROM pg_dist_shard WHERE logicalrelid IN ( SELECT oid FROM pg_class - WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + WHERE relnamespace = 'create_single_shard_table'::regnamespace AND relname LIKE 'nullkey_c2_%' ) ORDER BY 1; @@ -162,7 +162,7 @@ WHERE shardid IN ( SELECT shardid FROM pg_dist_shard WHERE logicalrelid IN ( SELECT oid FROM pg_class - WHERE relnamespace = 'create_null_dist_key'::regnamespace AND + WHERE relnamespace = 'create_single_shard_table'::regnamespace AND relname LIKE 'nullkey_c2_%' ) ) @@ -178,12 +178,12 @@ GROUP BY groupid; SELECT ( SELECT colocationid FROM pg_dist_partition - WHERE logicalrelid = 'create_null_dist_key.nullkey_c1_t1'::regclass + WHERE logicalrelid = 'create_single_shard_table.nullkey_c1_t1'::regclass ) != ( SELECT colocationid FROM pg_dist_partition - WHERE logicalrelid = 'create_null_dist_key.nullkey_c2_t1'::regclass + WHERE logicalrelid = 'create_single_shard_table.nullkey_c2_t1'::regclass ); -- Since we determine node for the placement based on the module of colocation id, @@ -193,7 +193,7 @@ SELECT SELECT groupid FROM pg_dist_placement WHERE shardid = ( SELECT shardid FROM pg_dist_shard - WHERE logicalrelid = 'create_null_dist_key.nullkey_c1_t1'::regclass + WHERE logicalrelid = 'create_single_shard_table.nullkey_c1_t1'::regclass ) ) != @@ -201,7 +201,7 @@ SELECT SELECT groupid FROM pg_dist_placement WHERE shardid = ( SELECT shardid FROM pg_dist_shard - WHERE logicalrelid = 'create_null_dist_key.nullkey_c2_t1'::regclass + WHERE logicalrelid = 'create_single_shard_table.nullkey_c2_t1'::regclass ) ); @@ -211,20 +211,20 @@ SELECT SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation WHERE colocationid = ( SELECT colocationid FROM pg_dist_partition - WHERE logicalrelid = 'create_null_dist_key.nullkey_c1_t1'::regclass + WHERE logicalrelid = 'create_single_shard_table.nullkey_c1_t1'::regclass ); SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation WHERE colocationid = ( SELECT colocationid FROM pg_dist_partition - WHERE logicalrelid = 'create_null_dist_key.nullkey_c2_t1'::regclass + WHERE logicalrelid = 'create_single_shard_table.nullkey_c2_t1'::regclass ); CREATE TABLE round_robin_test_c1(a int, b int); SELECT create_distributed_table('round_robin_test_c1', null, colocate_with=>'none', distribution_type=>null); \c - - - :master_port -SET search_path TO create_null_dist_key; +SET search_path TO create_single_shard_table; SET citus.next_shard_id TO 1730000; SET citus.shard_count TO 32; SET citus.shard_replication_factor TO 1; @@ -241,7 +241,7 @@ SELECT SELECT groupid FROM pg_dist_placement WHERE shardid = ( SELECT shardid FROM pg_dist_shard - WHERE logicalrelid = 'create_null_dist_key.round_robin_test_c1'::regclass + WHERE logicalrelid = 'create_single_shard_table.round_robin_test_c1'::regclass ) ) != @@ -249,13 +249,13 @@ SELECT SELECT groupid FROM pg_dist_placement WHERE shardid = ( SELECT shardid FROM pg_dist_shard - WHERE logicalrelid = 'create_null_dist_key.round_robin_test_c2'::regclass + WHERE logicalrelid = 'create_single_shard_table.round_robin_test_c2'::regclass ) ); CREATE TABLE distributed_table(a int, b int); --- cannot colocate a sharded table with null shard key table +-- cannot colocate a sharded table with single-shard table SELECT create_distributed_table('distributed_table', 'a', colocate_with=>'nullkey_c1_t1'); CREATE TABLE reference_table(a int, b int); @@ -263,7 +263,7 @@ CREATE TABLE local(a int, b int); SELECT create_reference_table('reference_table'); SELECT create_distributed_table('distributed_table', 'a'); --- cannot colocate null shard key tables with other table types +-- cannot colocate single-shard tables with other table types CREATE TABLE cannot_colocate_with_other_types (a int, b int); SELECT create_distributed_table('cannot_colocate_with_other_types', null, colocate_with=>'reference_table'); SELECT create_distributed_table('cannot_colocate_with_other_types', null, colocate_with=>'distributed_table'); @@ -271,7 +271,7 @@ SELECT create_distributed_table('cannot_colocate_with_other_types', null, coloca SELECT citus_add_local_table_to_metadata('local'); --- cannot colocate null shard key tables with citus local tables +-- cannot colocate single-shard tables with citus local tables SELECT create_distributed_table('cannot_colocate_with_other_types', null, colocate_with=>'local'); -- citus local SET client_min_messages TO WARNING; @@ -282,30 +282,30 @@ SELECT create_distributed_table('distributed_table', null, colocate_with=>'none' SELECT create_distributed_table('local', null, colocate_with=>'none'); BEGIN; - -- creating a null-shard-key table from a temporary table is not supported + -- creating a single-shard table from a temporary table is not supported CREATE TEMPORARY TABLE temp_table (a int); SELECT create_distributed_table('temp_table', null, colocate_with=>'none', distribution_type=>null); ROLLBACK; --- creating a null-shard-key table from a catalog table is not supported +-- creating a single-shard table from a catalog table is not supported SELECT create_distributed_table('pg_catalog.pg_index', NULL, distribution_type=>null); --- creating a null-shard-key table from an unlogged table is supported +-- creating a single-shard table from an unlogged table is supported CREATE UNLOGGED TABLE unlogged_table (a int); SELECT create_distributed_table('unlogged_table', null, colocate_with=>'none', distribution_type=>null); --- creating a null-shard-key table from a foreign table is not supported +-- creating a single-shard table from a foreign table is not supported CREATE FOREIGN TABLE foreign_table ( id bigint not null, full_name text not null default '' ) SERVER fake_fdw_server OPTIONS (encoding 'utf-8', compression 'true', table_name 'foreign_table'); SELECT create_distributed_table('foreign_table', null, colocate_with=>'none', distribution_type=>null); --- create a null dist key table that has no tuples +-- create a single-shard table that has no tuples CREATE TABLE null_dist_key_table_1 (a int primary key); SELECT create_distributed_table('null_dist_key_table_1', null, colocate_with=>'none'); --- create a null dist key table that has some tuples +-- create a single-shard table that has some tuples CREATE TABLE null_dist_key_table_2(a int primary key); INSERT INTO null_dist_key_table_2 VALUES(1); SELECT create_distributed_table('null_dist_key_table_2', null, colocate_with=>'none'); @@ -314,7 +314,7 @@ SELECT * FROM null_dist_key_table_2 ORDER BY a; DROP TABLE null_dist_key_table_1, null_dist_key_table_2; --- create indexes before creating the null dist key tables +-- create indexes before creating the single-shard tables -- .. for an initially empty table CREATE TABLE null_dist_key_table_1(a int, b int); @@ -323,7 +323,7 @@ CREATE INDEX null_dist_key_table_1_idx ON null_dist_key_table_1(a); SELECT create_distributed_table('null_dist_key_table_1', null, colocate_with=>'none'); CREATE STATISTICS s2 (dependencies) ON a, b FROM null_dist_key_table_1; --- .. and for another table having data in it before creating null dist key table +-- .. and for another table having data in it before creating single-shard table CREATE TABLE null_dist_key_table_2(a int); INSERT INTO null_dist_key_table_2 VALUES(1); CREATE INDEX null_dist_key_table_2_idx ON null_dist_key_table_2(a); @@ -435,7 +435,7 @@ INSERT INTO "Table?!.1Table"(id, jsondata, yes_no_enum_col) VALUES (101, '{"a": SELECT * FROM "Table?!.1Table" ORDER BY id; -SET search_path TO create_null_dist_key; +SET search_path TO create_single_shard_table; -- create a partitioned table with some columns that -- are going to be dropped within the tests @@ -460,11 +460,11 @@ SELECT create_distributed_table('sensors', NULL, distribution_type=>null); -- verify we can create new partitions after distributing the parent table CREATE TABLE sensors_2001 PARTITION OF sensors FOR VALUES FROM ('2001-01-01') TO ('2002-01-01'); --- verify we can attach to a null dist key table +-- verify we can attach to a single-shard table CREATE TABLE sensors_2002 (measureid integer, eventdatetime date, measure_data jsonb, PRIMARY KEY (measureid, eventdatetime, measure_data)); ALTER TABLE sensors ATTACH PARTITION sensors_2002 FOR VALUES FROM ('2002-01-01') TO ('2003-01-01'); --- verify we can detach from a null dist key table +-- verify we can detach from a single-shard table ALTER TABLE sensors DETACH PARTITION sensors_2001; -- error out when attaching a noncolocated partition @@ -507,7 +507,7 @@ SELECT COUNT(*) FROM run_command_on_workers($$ SELECT relpartbound FROM pg_class WHERE relname LIKE 'sensors_2002_1______';$$) WHERE length(result) > 0; --- create a partitioned citus local table and verify we error out when attaching a partition with null dist key +-- create a partitioned citus local table and verify we error out when attaching a partition with single-shard CREATE TABLE partitioned_citus_local_tbl( measureid integer, eventdatetime date, @@ -629,7 +629,7 @@ ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!90123456789012345678901234567890 -- Normally, we support foreign keys from Postgres tables to distributed -- tables assuming that the user will soon distribute the local table too --- anyway. However, this is not the case for null-shard-key tables before +-- anyway. However, this is not the case for single-shard tables before -- we improve SQL support. ALTER TABLE local_table_for_fkey ADD CONSTRAINT fkey_from_dummy_local FOREIGN KEY (a) REFERENCES "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id); @@ -680,7 +680,7 @@ ALTER TABLE null_key_dist DROP CONSTRAINT fkey_add_test_3; ALTER TABLE null_key_dist DROP CONSTRAINT fkey_add_test_4; ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" DROP CONSTRAINT fkey_to_dummy_dist; --- create a view that depends on the null shard key table +-- create a view that depends on the single-shard table CREATE VIEW public.v1 AS SELECT * FROM null_key_dist; SELECT * FROM public.v1; @@ -755,24 +755,24 @@ INSERT INTO identity_test (b) OVERRIDING SYSTEM VALUE VALUES (5); INSERT INTO identity_test (c) VALUES (5); SELECT result, success FROM run_command_on_workers($$ - INSERT INTO create_null_dist_key.identity_test (a) VALUES (6) + INSERT INTO create_single_shard_table.identity_test (a) VALUES (6) $$); SELECT result, success FROM run_command_on_workers($$ - SELECT create_null_dist_key.normalize_generate_always_as_error('INSERT INTO create_null_dist_key.identity_test (b) VALUES (1)') + SELECT create_single_shard_table.normalize_generate_always_as_error('INSERT INTO create_single_shard_table.identity_test (b) VALUES (1)') $$); -- This should fail due to missing OVERRIDING SYSTEM VALUE. SELECT result, success FROM run_command_on_workers($$ - SELECT create_null_dist_key.normalize_generate_always_as_error('INSERT INTO create_null_dist_key.identity_test (a, b) VALUES (1, 1)') + SELECT create_single_shard_table.normalize_generate_always_as_error('INSERT INTO create_single_shard_table.identity_test (a, b) VALUES (1, 1)') $$); SELECT result, success FROM run_command_on_workers($$ - INSERT INTO create_null_dist_key.identity_test (a, b) OVERRIDING SYSTEM VALUE VALUES (7, 7) + INSERT INTO create_single_shard_table.identity_test (a, b) OVERRIDING SYSTEM VALUE VALUES (7, 7) $$); SELECT result, success FROM run_command_on_workers($$ - INSERT INTO create_null_dist_key.identity_test (c, a) OVERRIDING SYSTEM VALUE VALUES (8, 8) + INSERT INTO create_single_shard_table.identity_test (c, a) OVERRIDING SYSTEM VALUE VALUES (8, 8) $$); -- test foreign keys @@ -781,7 +781,7 @@ CREATE TABLE referenced_table(a int UNIQUE, b int); CREATE TABLE referencing_table(a int, b int, FOREIGN KEY (a) REFERENCES referenced_table(a)); --- to a colocated null dist key table +-- to a colocated single-shard table BEGIN; SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'referenced_table'); @@ -793,7 +793,7 @@ BEGIN; INSERT INTO referencing_table VALUES (2, 2); ROLLBACK; --- to a non-colocated null dist key table +-- to a non-colocated single-shard table BEGIN; SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'none'); @@ -866,11 +866,11 @@ SELECT create_distributed_table('referencing_table', NULL, distribution_type=>nu INSERT INTO referenced_table VALUES (1, 1); -- ok SELECT result, success FROM run_command_on_workers($$ - INSERT INTO create_null_dist_key.referencing_table VALUES (1, 2) + INSERT INTO create_single_shard_table.referencing_table VALUES (1, 2) $$); -- fails SELECT result, success FROM run_command_on_workers($$ - INSERT INTO create_null_dist_key.referencing_table VALUES (2, 2) + INSERT INTO create_single_shard_table.referencing_table VALUES (2, 2) $$); DROP TABLE referencing_table, referenced_table; @@ -885,9 +885,9 @@ INSERT INTO self_fkey_test VALUES (2, 3); -- fails -- similar foreign key tests but this time create the referencing table later on --- referencing table is a null shard key table +-- referencing table is a single-shard table --- to a colocated null dist key table +-- to a colocated single-shard table BEGIN; CREATE TABLE referenced_table(a int UNIQUE, b int); SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); @@ -947,7 +947,7 @@ BEGIN; ALTER TABLE referencing_table ADD CONSTRAINT fkey_to_dummy_ref_on_update FOREIGN KEY (a) REFERENCES referenced_table(a) ON UPDATE SET DEFAULT; ROLLBACK; --- to a non-colocated null dist key table +-- to a non-colocated single-shard table BEGIN; CREATE TABLE referenced_table(a int UNIQUE, b int); SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); @@ -1010,7 +1010,7 @@ BEGIN; SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'none'); ROLLBACK; --- referenced table is a null shard key table +-- referenced table is a single-shard table -- from a sharded table BEGIN; @@ -1060,9 +1060,9 @@ SET client_min_messages TO DEBUG1; BEGIN; -- Switches to sequential execution because referenced_table is a reference table - -- and referenced by a null-shard-key distributed table. + -- and referenced by a single-shard table. -- - -- Given that we cannot do parallel access on null-shard-key, this is not useful. + -- Given that we cannot do parallel access on a single-shard table, this is not useful. -- However, this is already what we're doing for, e.g., a foreign key from a -- reference table to another reference table. TRUNCATE referenced_table CASCADE; @@ -1089,9 +1089,9 @@ ROLLBACK; BEGIN; -- Switches to sequential execution because referenced_table is a reference table - -- and referenced by a null-shard-key distributed table. + -- and referenced by a single-shard table. -- - -- Given that we cannot do parallel access on null-shard-key, this is not useful. + -- Given that we cannot do parallel access on a single-shard table, this is not useful. -- However, this is already what we're doing for, e.g., a foreign key from a -- reference table to another reference table. UPDATE referenced_table SET id = 101 WHERE id = 99; @@ -1109,7 +1109,7 @@ SET client_min_messages TO WARNING; DROP TABLE referenced_table, referencing_table; -- Test whether we unnecessarily switch to sequential execution --- when the referenced relation is a null-shard-key table. +-- when the referenced relation is a single-shard table. CREATE TABLE referenced_table(id int PRIMARY KEY, value_1 int); SELECT create_distributed_table('referenced_table', null, colocate_with=>'none', distribution_type=>null); @@ -1122,13 +1122,13 @@ SET client_min_messages TO DEBUG1; BEGIN; SELECT COUNT(*) FROM referenced_table; -- Doesn't switch to sequential execution because the referenced_table is - -- a null-shard-key distributed table. + -- a single-shard table. ALTER TABLE referencing_table ADD COLUMN X INT; ROLLBACK; BEGIN; -- Doesn't switch to sequential execution because the referenced_table is - -- a null-shard-key distributed table. + -- a single-shard table. TRUNCATE referenced_table CASCADE; SELECT COUNT(*) FROM referencing_table; COMMIT; @@ -1229,4 +1229,4 @@ DROP TRIGGER trigger_3 ON trigger_table_3 RESTRICT; -- cleanup at exit SET client_min_messages TO ERROR; -DROP SCHEMA create_null_dist_key, "NULL_!_dist_key" CASCADE; +DROP SCHEMA create_single_shard_table, "NULL_!_dist_key" CASCADE; diff --git a/src/test/regress/sql/insert_select_null_dist_key.sql b/src/test/regress/sql/insert_select_single_shard_table.sql similarity index 95% rename from src/test/regress/sql/insert_select_null_dist_key.sql rename to src/test/regress/sql/insert_select_single_shard_table.sql index 29454b0c1..f428752ec 100644 --- a/src/test/regress/sql/insert_select_null_dist_key.sql +++ b/src/test/regress/sql/insert_select_single_shard_table.sql @@ -1,5 +1,5 @@ -CREATE SCHEMA insert_select_null_dist_key; -SET search_path TO insert_select_null_dist_key; +CREATE SCHEMA insert_select_single_shard_table; +SET search_path TO insert_select_single_shard_table; SET citus.next_shard_id TO 1820000; SET citus.shard_count TO 32; @@ -87,9 +87,9 @@ CREATE MATERIALIZED VIEW matview AS SELECT b*2+a AS a, a*a AS b FROM nullkey_c1_ SET client_min_messages TO DEBUG2; -- Test inserting into a distributed table by selecting from a combination of --- different table types together with null-shard-key tables. +-- different table types together with single-shard tables. --- use a null-shard-key table +-- use a single-shard table INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1; -- use a reference table @@ -98,13 +98,13 @@ INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 LEFT JOIN reference_table USING (b); INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 INTERSECT SELECT * FROM reference_table; --- use a colocated null-shard-key table +-- use a colocated single-shard table INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN nullkey_c1_t2 USING (b); INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 FULL JOIN nullkey_c1_t2 USING (a); INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 FULL JOIN matview USING (a); INSERT INTO distributed_table_c1_t1 SELECT * FROM nullkey_c1_t1 UNION SELECT * FROM nullkey_c1_t2; --- use a non-colocated null-shard-key table +-- use a non-colocated single-shard table INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 LEFT JOIN nullkey_c2_t1 USING (a); INSERT INTO distributed_table_c1_t1 SELECT * FROM nullkey_c1_t1 UNION SELECT * FROM nullkey_c2_t1; @@ -132,9 +132,9 @@ TRUNCATE distributed_table_c1_t1; INSERT INTO distributed_table_c1_t1 SELECT i, i FROM generate_series(3, 8) i; -- Test inserting into a reference table by selecting from a combination of --- different table types together with null-shard-key tables. +-- different table types together with single-shard tables. --- use a null-shard-key table +-- use a single-shard table INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1; -- use a reference table @@ -143,11 +143,11 @@ INSERT INTO reference_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 UNION SELECT * FROM reference_table; INSERT INTO reference_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 LEFT JOIN reference_table USING (b) WHERE b IN (SELECT b FROM matview); --- use a colocated null-shard-key table +-- use a colocated single-shard table INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN nullkey_c1_t2 USING (b); INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 FULL JOIN nullkey_c1_t2 USING (a); --- use a non-colocated null-shard-key table +-- use a non-colocated single-shard table INSERT INTO reference_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 LEFT JOIN nullkey_c2_t1 USING (a); -- use a distributed table @@ -167,15 +167,15 @@ TRUNCATE reference_table; INSERT INTO reference_table SELECT i, i FROM generate_series(0, 5) i; -- Test inserting into a citus local table by selecting from a combination of --- different table types together with null-shard-key tables. +-- different table types together with single-shard tables. --- use a null-shard-key table +-- use a single-shard table INSERT INTO citus_local_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1; -- use a reference table INSERT INTO citus_local_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN reference_table USING (a); --- use a colocated null-shard-key table +-- use a colocated single-shard table INSERT INTO citus_local_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN nullkey_c1_t2 USING (b); -- use a distributed table @@ -191,8 +191,8 @@ SELECT avg(a), avg(b) FROM citus_local_table ORDER BY 1, 2; TRUNCATE citus_local_table; INSERT INTO citus_local_table SELECT i, i FROM generate_series(0, 10) i; --- Test inserting into a null-shard-key table by selecting from a combination of --- different table types, together with or without null-shard-key tables. +-- Test inserting into a single-shard table by selecting from a combination of +-- different table types, together with or without single-shard tables. -- use a postgres local table INSERT INTO nullkey_c1_t1 SELECT postgres_local_table.a, postgres_local_table.b FROM postgres_local_table; @@ -209,7 +209,7 @@ INSERT INTO nullkey_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1 INSERT INTO nullkey_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM distributed_table_c1_t2 JOIN reference_table USING (a); INSERT INTO nullkey_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM distributed_table_c1_t2 JOIN nullkey_c1_t1 USING (a); --- use a non-colocated null-shard-key table +-- use a non-colocated single-shard table INSERT INTO nullkey_c2_t1 SELECT q.* FROM (SELECT reference_table.* FROM reference_table LEFT JOIN nullkey_c1_t1 USING (a)) q JOIN nullkey_c1_t2 USING (a); -- use a materialized view @@ -228,7 +228,7 @@ INSERT INTO nullkey_c1_t1 SELECT i, i FROM generate_series(1, 8) i; INSERT INTO nullkey_c2_t1 SELECT i, i FROM generate_series(2, 7) i; -- Test inserting into a local table by selecting from a combination of --- different table types, together with or without null-shard-key tables. +-- different table types, together with or without single-shard tables. INSERT INTO postgres_local_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN reference_table USING (a); @@ -465,6 +465,6 @@ SELECT * FROM upsert_test_2 ORDER BY key; SELECT * FROM upsert_test_3 ORDER BY key_1, key_2; SET client_min_messages TO WARNING; -DROP SCHEMA insert_select_null_dist_key CASCADE; +DROP SCHEMA insert_select_single_shard_table CASCADE; SELECT citus_remove_node('localhost', :master_port); diff --git a/src/test/regress/sql/merge.sql b/src/test/regress/sql/merge.sql index 1fdc3a514..f10ab6c99 100644 --- a/src/test/regress/sql/merge.sql +++ b/src/test/regress/sql/merge.sql @@ -2051,11 +2051,11 @@ UPDATE SET val = dist_source.val WHEN NOT MATCHED THEN INSERT VALUES(dist_source.id, dist_source.val); --- test merge with null shard key tables +-- test merge with single-shard tables -CREATE SCHEMA query_null_dist_key; +CREATE SCHEMA query_single_shard_table; -SET search_path TO query_null_dist_key; +SET search_path TO query_single_shard_table; SET client_min_messages TO DEBUG2; CREATE TABLE nullkey_c1_t1(a int, b int); @@ -2098,7 +2098,7 @@ MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2 WHEN MATCHED THEN DELETE WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b); --- with non-colocated null-dist-key table +-- with non-colocated single-shard table MERGE INTO nullkey_c1_t1 USING nullkey_c2_t1 ON (nullkey_c1_t1.a = nullkey_c2_t1.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c2_t1.b; @@ -2158,7 +2158,7 @@ MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a) WHEN MATCHED THEN UPDATE SET b = cte.b; SET client_min_messages TO WARNING; -DROP SCHEMA query_null_dist_key CASCADE; +DROP SCHEMA query_single_shard_table CASCADE; RESET client_min_messages; SET search_path TO merge_schema; diff --git a/src/test/regress/sql/multi_extension.sql b/src/test/regress/sql/multi_extension.sql index 50b821b0c..03cf4c7fb 100644 --- a/src/test/regress/sql/multi_extension.sql +++ b/src/test/regress/sql/multi_extension.sql @@ -904,7 +904,7 @@ SELECT create_distributed_table('test','x'); DROP TABLE test; TRUNCATE pg_dist_node; --- confirm that we can create a null shard key table on an empty node +-- confirm that we can create a single-shard table on an empty node CREATE TABLE test (x int, y int); INSERT INTO test VALUES (1,2); SET citus.shard_replication_factor TO 1; diff --git a/src/test/regress/sql/query_null_dist_key.sql b/src/test/regress/sql/query_single_shard_table.sql similarity index 98% rename from src/test/regress/sql/query_null_dist_key.sql rename to src/test/regress/sql/query_single_shard_table.sql index 02eac5c80..0a05558af 100644 --- a/src/test/regress/sql/query_null_dist_key.sql +++ b/src/test/regress/sql/query_single_shard_table.sql @@ -1,5 +1,5 @@ -CREATE SCHEMA query_null_dist_key; -SET search_path TO query_null_dist_key; +CREATE SCHEMA query_single_shard_table; +SET search_path TO query_single_shard_table; SET citus.next_shard_id TO 1620000; SET citus.shard_count TO 32; @@ -138,16 +138,16 @@ SELECT COUNT(*) FROM reference_table d1, nullkey_c1_t1; SELECT COUNT(*) FROM citus_local_table d1, nullkey_c1_t1; SELECT COUNT(*) FROM postgres_local_table d1, nullkey_c1_t1; --- with a colocated null dist key table +-- with a colocated single-shard table SELECT COUNT(*) FROM nullkey_c1_t1 d1, nullkey_c1_t2; --- with a non-colocated null dist key table +-- with a non-colocated single-shard table SELECT COUNT(*) FROM nullkey_c1_t1 d1, nullkey_c2_t1; -- First, show that nullkey_c1_t1 and nullkey_c3_t1 are not colocated. SELECT - (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'query_null_dist_key.nullkey_c1_t1'::regclass) != - (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'query_null_dist_key.nullkey_c3_t1'::regclass); + (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'query_single_shard_table.nullkey_c1_t1'::regclass) != + (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'query_single_shard_table.nullkey_c3_t1'::regclass); -- Now verify that we can join them via router planner because it doesn't care -- about whether two tables are colocated or not but physical location of shards @@ -163,7 +163,7 @@ SELECT COUNT(*) FROM nullkey_c1_t1 JOIN nullkey_c3_t1 USING(a); RESET citus.enable_non_colocated_router_query_pushdown; --- colocated join between null dist key tables +-- colocated join between single-shard tables SELECT COUNT(*) FROM nullkey_c1_t1 JOIN nullkey_c1_t2 USING(a); SELECT COUNT(*) FROM nullkey_c1_t1 LEFT JOIN nullkey_c1_t2 USING(a); SELECT COUNT(*) FROM nullkey_c1_t1 FULL JOIN nullkey_c1_t2 USING(a); @@ -193,7 +193,7 @@ WHERE t1.b NOT IN ( SELECT a FROM nullkey_c1_t2 t2 WHERE t2.b > t1.a ); --- non-colocated inner joins between null dist key tables +-- non-colocated inner joins between single-shard tables SELECT * FROM nullkey_c1_t1 JOIN nullkey_c2_t1 USING(a) ORDER BY 1,2,3; SELECT COUNT(*) FROM nullkey_c1_t1 t1 @@ -201,7 +201,7 @@ JOIN LATERAL ( SELECT * FROM nullkey_c2_t2 t2 WHERE t2.b > t1.a ) q USING(a); --- non-colocated outer joins between null dist key tables +-- non-colocated outer joins between single-shard tables SELECT * FROM nullkey_c1_t1 LEFT JOIN nullkey_c2_t2 USING(a) ORDER BY 1,2,3 LIMIT 4; SELECT * FROM nullkey_c1_t1 FULL JOIN nullkey_c2_t2 USING(a) ORDER BY 1,2,3 LIMIT 4; SELECT * FROM nullkey_c1_t1 t1 @@ -497,7 +497,7 @@ JOIN LATERAL ( -- insert .. select --- between two colocated null dist key tables +-- between two colocated single-shard tables -- The target list of "distributed statement"s that we send to workers -- differ(*) in Postgres versions < 15. For this reason, we temporarily @@ -510,10 +510,10 @@ EXPLAIN (ANALYZE TRUE, TIMING FALSE, COSTS FALSE, SUMMARY FALSE, VERBOSE FALSE) INSERT INTO nullkey_c1_t1 SELECT * FROM nullkey_c1_t2; SET client_min_messages TO DEBUG2; --- between two non-colocated null dist key tables +-- between two non-colocated single-shard tables INSERT INTO nullkey_c1_t1 SELECT * FROM nullkey_c2_t1; --- between a null dist key table and a table of different type +-- between a single-shard table and a table of different type SET client_min_messages TO WARNING; EXPLAIN (ANALYZE TRUE, TIMING FALSE, COSTS FALSE, SUMMARY FALSE, VERBOSE FALSE) INSERT INTO nullkey_c1_t1 SELECT * FROM reference_table; @@ -1174,6 +1174,6 @@ ORDER BY LIMIT 10; SET client_min_messages TO ERROR; -DROP SCHEMA query_null_dist_key CASCADE; +DROP SCHEMA query_single_shard_table CASCADE; SELECT citus_remove_node('localhost', :master_port); diff --git a/src/test/regress/sql/single_node.sql b/src/test/regress/sql/single_node.sql index 8c612c1bb..bdfb3e260 100644 --- a/src/test/regress/sql/single_node.sql +++ b/src/test/regress/sql/single_node.sql @@ -94,7 +94,7 @@ WHERE shardid = ( WHERE logicalrelid = 'single_node.single_node_nullkey_c2'::regclass ); --- try creating a null-shard-key distributed table from a shard relation +-- try creating a single-shard table from a shard relation SELECT shardid AS round_robin_test_c1_shard_id FROM pg_dist_shard WHERE logicalrelid = 'single_node.single_node_nullkey_c1'::regclass \gset SELECT create_distributed_table('single_node_nullkey_c1_' || :round_robin_test_c1_shard_id , null, colocate_with=>'none', distribution_type=>null); diff --git a/src/test/regress/sql/null_dist_key_prep.sql b/src/test/regress/sql/single_shard_table_prep.sql similarity index 100% rename from src/test/regress/sql/null_dist_key_prep.sql rename to src/test/regress/sql/single_shard_table_prep.sql From 4321286005f7f4768ffb02200d30ad2b29f8cbba Mon Sep 17 00:00:00 2001 From: Ahmet Gedemenli Date: Wed, 3 May 2023 16:04:56 +0300 Subject: [PATCH 7/7] Disable master_create_empty_shard udf for single shard tables (#6902) --- src/backend/distributed/operations/stage_protocol.c | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/backend/distributed/operations/stage_protocol.c b/src/backend/distributed/operations/stage_protocol.c index 7ef988e5f..db7ebefca 100644 --- a/src/backend/distributed/operations/stage_protocol.c +++ b/src/backend/distributed/operations/stage_protocol.c @@ -138,6 +138,13 @@ master_create_empty_shard(PG_FUNCTION_ARGS) errdetail("We currently don't support creating shards " "on hash-partitioned tables"))); } + else if (IsCitusTableType(relationId, SINGLE_SHARD_DISTRIBUTED)) + { + ereport(ERROR, (errmsg("relation \"%s\" is a single shard table", + relationName), + errdetail("We currently don't support creating shards " + "on single shard tables"))); + } else if (IsCitusTableType(relationId, REFERENCE_TABLE)) { ereport(ERROR, (errmsg("relation \"%s\" is a reference table",