Call null-shard-key tables as single-shard distributed tables in code

pull/6867/head
Onur Tirtir 2023-05-03 14:32:43 +03:00
parent 39b7711527
commit db2514ef78
39 changed files with 318 additions and 312 deletions

View File

@ -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 "

View File

@ -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),

View File

@ -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),

View File

@ -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;
}

View File

@ -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 &&

View File

@ -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);

View File

@ -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);

View File

@ -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
{

View File

@ -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;

View File

@ -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",

View File

@ -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;

View File

@ -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);

View File

@ -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);

View File

@ -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);

View File

@ -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);

View File

@ -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,

View File

@ -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

View File

@ -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

View File

@ -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"),
}

View File

@ -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;

View File

@ -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
---------------------------------------------------------------------

View File

@ -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: <Deparsed MERGE query: MERGE INTO query_null_dist_key.nullkey_c1_t1_4000145 nullkey_c1_t1 USING query_null_dist_key.nullkey_c1_t2_4000146 nullkey_c1_t2 ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) nullkey_c1_t2.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b>
DEBUG: <Deparsed MERGE query: MERGE INTO query_single_shard_table.nullkey_c1_t1_4000147 nullkey_c1_t1 USING query_single_shard_table.nullkey_c1_t2_4000148 nullkey_c1_t2 ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) nullkey_c1_t2.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b>
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: <Deparsed MERGE query: MERGE INTO query_null_dist_key.nullkey_c1_t1_4000145 nullkey_c1_t1 USING query_null_dist_key.nullkey_c1_t2_4000146 nullkey_c1_t2 ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) nullkey_c1_t2.a) WHEN MATCHED THEN DELETE>
DEBUG: <Deparsed MERGE query: MERGE INTO query_single_shard_table.nullkey_c1_t1_4000147 nullkey_c1_t1 USING query_single_shard_table.nullkey_c1_t2_4000148 nullkey_c1_t2 ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) nullkey_c1_t2.a) WHEN MATCHED THEN DELETE>
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: <Deparsed MERGE query: MERGE INTO query_null_dist_key.nullkey_c1_t1_4000145 nullkey_c1_t1 USING query_null_dist_key.nullkey_c1_t2_4000146 nullkey_c1_t2 ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) nullkey_c1_t2.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b WHEN NOT MATCHED THEN INSERT (a, b) VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b)>
DEBUG: <Deparsed MERGE query: MERGE INTO query_single_shard_table.nullkey_c1_t1_4000147 nullkey_c1_t1 USING query_single_shard_table.nullkey_c1_t2_4000148 nullkey_c1_t2 ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) nullkey_c1_t2.a) WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b WHEN NOT MATCHED THEN INSERT (a, b) VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b)>
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: <Deparsed MERGE query: MERGE INTO query_null_dist_key.nullkey_c1_t1_4000145 nullkey_c1_t1 USING query_null_dist_key.nullkey_c1_t2_4000146 nullkey_c1_t2 ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) nullkey_c1_t2.a) WHEN MATCHED THEN DELETE WHEN NOT MATCHED THEN INSERT (a, b) VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b)>
DEBUG: <Deparsed MERGE query: MERGE INTO query_single_shard_table.nullkey_c1_t1_4000147 nullkey_c1_t1 USING query_single_shard_table.nullkey_c1_t2_4000148 nullkey_c1_t2 ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) nullkey_c1_t2.a) WHEN MATCHED THEN DELETE WHEN NOT MATCHED THEN INSERT (a, b) VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b)>
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: <Deparsed MERGE query: WITH cte AS (SELECT nullkey_c1_t1_1.a, nullkey_c1_t1_1.b FROM query_null_dist_key.nullkey_c1_t1_4000145 nullkey_c1_t1_1) MERGE INTO query_null_dist_key.nullkey_c1_t1_4000145 nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) cte.a) WHEN MATCHED THEN UPDATE SET b = cte.b>
DEBUG: <Deparsed MERGE query: WITH cte AS (SELECT nullkey_c1_t1_1.a, nullkey_c1_t1_1.b FROM query_single_shard_table.nullkey_c1_t1_4000147 nullkey_c1_t1_1) MERGE INTO query_single_shard_table.nullkey_c1_t1_4000147 nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a OPERATOR(pg_catalog.=) cte.a) WHEN MATCHED THEN UPDATE SET b = cte.b>
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;

View File

@ -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;

View File

@ -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
---------------------------------------------------------------------

View File

@ -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

View File

@ -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

View File

@ -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
# ---------

View File

@ -1 +0,0 @@
test: null_dist_key_prep

View File

@ -0,0 +1 @@
test: single_shard_table_prep

View File

@ -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;

View File

@ -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);

View File

@ -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;

View File

@ -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;

View File

@ -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);

View File

@ -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);