Merge branch 'main' into gokhangulbiz/tenant-stats-perf-improvements

pull/6868/head
Gokhan Gulbiz 2023-05-04 08:17:44 +03:00 committed by GitHub
commit f5befff77c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
79 changed files with 9929 additions and 324 deletions

View File

@ -474,6 +474,9 @@ jobs:
- store_artifacts: - store_artifacts:
name: 'Save dmls' name: 'Save dmls'
path: src/test/regress/citus_tests/query_generator/out/queries.sql path: src/test/regress/citus_tests/query_generator/out/queries.sql
- store_artifacts:
name: 'Save diffs'
path: src/test/regress/citus_tests/query_generator/out/local_dist.diffs
- stack_trace - stack_trace
- coverage: - coverage:
flags: 'test_<< parameters.pg_major >>,querygen' flags: 'test_<< parameters.pg_major >>,querygen'

View File

@ -1,3 +1,71 @@
### citus v11.3.0 (May 2, 2023) ###
* Introduces CDC implementation for Citus using logical replication
(#6623, #6810, #6827)
* Adds support for `MERGE` command on co-located distributed tables joined on
distribution column (#6696, #6733)
* Adds the view `citus_stats_tenants` that monitor statistics on tenant usages
(#6725)
* Adds the GUC `citus.max_background_task_executors_per_node` to control number
of background task executors involving a node (#6771)
* Allows parallel shard moves in background rebalancer (#6756)
* Introduces the GUC `citus.metadata_sync_mode` that introduces nontransactional
mode for metadata sync (#6728, #6889)
* Propagates CREATE/ALTER/DROP PUBLICATION statements for distributed tables
(#6776)
* Adds the GUC `citus.enable_non_colocated_router_query_pushdown` to ensure
generating a consistent distributed plan for the queries that reference
non-colocated distributed tables when set to "false" (#6793)
* Checks if all moves are able to be done via logical replication for rebalancer
(#6754)
* Correctly reports shard size in `citus_shards` view (#6748)
* Fixes a bug in shard copy operations (#6721)
* Fixes a bug that prevents enforcing identity column restrictions on worker
nodes (#6738)
* Fixes a bug with `INSERT .. SELECT` queries with identity columns (#6802)
* Fixes an issue that caused some queries with custom aggregates to fail (#6805)
* Fixes an issue when `citus_set_coordinator_host` is called more than once
(#6837)
* Fixes an uninitialized memory access in shard split API (#6845)
* Fixes memory leak and max allocation block errors during metadata syncing
(#6728)
* Fixes memory leak in `undistribute_table` (#6693)
* Fixes memory leak in `alter_distributed_table` (#6726)
* Fixes memory leak in `create_distributed_table` (#6722)
* Fixes memory leak issue with query results that returns single row (#6724)
* Improves rebalancer when shard groups have placement count less than worker
count (#6739)
* Makes sure to stop maintenance daemon when dropping a database even without
Citus extension (#6688)
* Prevents using `alter_distributed_table` and `undistribute_table` UDFs when a
table has identity columns (#6738)
* Prevents using identity columns on data types other than `bigint` on
distributed tables (#6738)
### citus v11.2.1 (April 20, 2023) ### ### citus v11.2.1 (April 20, 2023) ###
* Correctly reports shard size in `citus_shards` view (#6748) * Correctly reports shard size in `citus_shards` view (#6748)
@ -106,6 +174,9 @@
### citus v10.0.8 (April 20, 2023) ### ### citus v10.0.8 (April 20, 2023) ###
* Fixes a bug that could break `DROP SCHEMA/EXTENSON` commands when there is a
columnar table (#5458)
* Fixes a crash that occurs when the aggregate that cannot be pushed-down * Fixes a crash that occurs when the aggregate that cannot be pushed-down
returns empty result from a worker (#5679) returns empty result from a worker (#5679)

View File

@ -1,6 +1,6 @@
# Columnar extension # Columnar extension
comment = 'Citus Columnar extension' comment = 'Citus Columnar extension'
default_version = '11.2-1' default_version = '11.3-1'
module_pathname = '$libdir/citus_columnar' module_pathname = '$libdir/citus_columnar'
relocatable = false relocatable = false
schema = pg_catalog schema = pg_catalog

View File

@ -0,0 +1 @@
-- citus_columnar--11.2-1--11.3-1

View File

@ -0,0 +1 @@
-- citus_columnar--11.3-1--11.2-1

View File

@ -1710,20 +1710,13 @@ ReplaceTable(Oid sourceId, Oid targetId, List *justBeforeDropCommands,
} }
else if (ShouldSyncTableMetadata(sourceId)) else if (ShouldSyncTableMetadata(sourceId))
{ {
char *qualifiedTableName = quote_qualified_identifier(schemaName, sourceName);
/* /*
* We are converting a citus local table to a distributed/reference table, * We are converting a citus local table to a distributed/reference table,
* so we should prevent dropping the sequence on the table. Otherwise, we'd * so we should prevent dropping the sequence on the table. Otherwise, we'd
* lose track of the previous changes in the sequence. * lose track of the previous changes in the sequence.
*/ */
StringInfo command = makeStringInfo(); char *command = WorkerDropSequenceDependencyCommand(sourceId);
SendCommandToWorkersWithMetadata(command);
appendStringInfo(command,
"SELECT pg_catalog.worker_drop_sequence_dependency(%s);",
quote_literal_cstr(qualifiedTableName));
SendCommandToWorkersWithMetadata(command->data);
} }
} }

View File

@ -141,6 +141,8 @@ static void CreateCitusTable(Oid relationId, CitusTableType tableType,
DistributedTableParams *distributedTableParams); DistributedTableParams *distributedTableParams);
static void CreateHashDistributedTableShards(Oid relationId, int shardCount, static void CreateHashDistributedTableShards(Oid relationId, int shardCount,
Oid colocatedTableId, bool localTableEmpty); Oid colocatedTableId, bool localTableEmpty);
static void CreateSingleShardTableShard(Oid relationId, Oid colocatedTableId,
uint32 colocationId);
static uint32 ColocationIdForNewTable(Oid relationId, CitusTableType tableType, static uint32 ColocationIdForNewTable(Oid relationId, CitusTableType tableType,
DistributedTableParams *distributedTableParams, DistributedTableParams *distributedTableParams,
Var *distributionColumn); Var *distributionColumn);
@ -216,51 +218,86 @@ create_distributed_table(PG_FUNCTION_ARGS)
{ {
CheckCitusVersion(ERROR); CheckCitusVersion(ERROR);
if (PG_ARGISNULL(0) || PG_ARGISNULL(1) || PG_ARGISNULL(2) || PG_ARGISNULL(3)) if (PG_ARGISNULL(0) || PG_ARGISNULL(3))
{ {
PG_RETURN_VOID(); PG_RETURN_VOID();
} }
Oid relationId = PG_GETARG_OID(0); Oid relationId = PG_GETARG_OID(0);
text *distributionColumnText = PG_GETARG_TEXT_P(1); text *distributionColumnText = PG_ARGISNULL(1) ? NULL : PG_GETARG_TEXT_P(1);
Oid distributionMethodOid = PG_GETARG_OID(2); Oid distributionMethodOid = PG_GETARG_OID(2);
text *colocateWithTableNameText = PG_GETARG_TEXT_P(3); text *colocateWithTableNameText = PG_GETARG_TEXT_P(3);
char *colocateWithTableName = text_to_cstring(colocateWithTableNameText); char *colocateWithTableName = text_to_cstring(colocateWithTableNameText);
bool shardCountIsStrict = false; bool shardCountIsStrict = false;
int shardCount = ShardCount; if (distributionColumnText)
if (!PG_ARGISNULL(4))
{ {
if (pg_strncasecmp(colocateWithTableName, "default", NAMEDATALEN) != 0 && if (PG_ARGISNULL(2))
pg_strncasecmp(colocateWithTableName, "none", NAMEDATALEN) != 0)
{ {
ereport(ERROR, (errmsg("Cannot use colocate_with with a table " PG_RETURN_VOID();
"and shard_count at the same time")));
} }
shardCount = PG_GETARG_INT32(4); int shardCount = ShardCount;
if (!PG_ARGISNULL(4))
{
if (!IsColocateWithDefault(colocateWithTableName) &&
!IsColocateWithNone(colocateWithTableName))
{
ereport(ERROR, (errmsg("Cannot use colocate_with with a table "
"and shard_count at the same time")));
}
/* shardCount = PG_GETARG_INT32(4);
* if shard_count parameter is given than we have to
* make sure table has that many shards /*
*/ * If shard_count parameter is given, then we have to
shardCountIsStrict = true; * make sure table has that many shards.
*/
shardCountIsStrict = true;
}
char *distributionColumnName = text_to_cstring(distributionColumnText);
Assert(distributionColumnName != NULL);
char distributionMethod = LookupDistributionMethod(distributionMethodOid);
if (shardCount < 1 || shardCount > MAX_SHARD_COUNT)
{
ereport(ERROR, (errmsg("%d is outside the valid range for "
"parameter \"shard_count\" (1 .. %d)",
shardCount, MAX_SHARD_COUNT)));
}
CreateDistributedTable(relationId, distributionColumnName, distributionMethod,
shardCount, shardCountIsStrict, colocateWithTableName);
} }
else
char *distributionColumnName = text_to_cstring(distributionColumnText);
Assert(distributionColumnName != NULL);
char distributionMethod = LookupDistributionMethod(distributionMethodOid);
if (shardCount < 1 || shardCount > MAX_SHARD_COUNT)
{ {
ereport(ERROR, (errmsg("%d is outside the valid range for " if (!PG_ARGISNULL(4))
"parameter \"shard_count\" (1 .. %d)", {
shardCount, MAX_SHARD_COUNT))); ereport(ERROR, (errmsg("shard_count can't be specified when the "
} "distribution column is null because in "
"that case it's automatically set to 1")));
}
CreateDistributedTable(relationId, distributionColumnName, distributionMethod, if (!PG_ARGISNULL(2) &&
shardCount, shardCountIsStrict, colocateWithTableName); LookupDistributionMethod(PG_GETARG_OID(2)) != DISTRIBUTE_BY_HASH)
{
/*
* As we do for shard_count parameter, we could throw an error if
* distribution_type is not NULL when creating a 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
* tables. For this reason, here we instead do nothing if the distribution
* method is DISTRIBUTE_BY_HASH.
*/
ereport(ERROR, (errmsg("distribution_type can't be specified "
"when the distribution column is null ")));
}
CreateSingleShardTable(relationId, colocateWithTableName);
}
PG_RETURN_VOID(); PG_RETURN_VOID();
} }
@ -276,11 +313,18 @@ create_distributed_table_concurrently(PG_FUNCTION_ARGS)
{ {
CheckCitusVersion(ERROR); CheckCitusVersion(ERROR);
if (PG_ARGISNULL(0) || PG_ARGISNULL(1) || PG_ARGISNULL(2) || PG_ARGISNULL(3)) if (PG_ARGISNULL(0) || PG_ARGISNULL(2) || PG_ARGISNULL(3))
{ {
PG_RETURN_VOID(); PG_RETURN_VOID();
} }
if (PG_ARGISNULL(1))
{
ereport(ERROR, (errmsg("cannot use create_distributed_table_concurrently "
"to create a distributed table with a null shard "
"key, consider using create_distributed_table()")));
}
Oid relationId = PG_GETARG_OID(0); Oid relationId = PG_GETARG_OID(0);
text *distributionColumnText = PG_GETARG_TEXT_P(1); text *distributionColumnText = PG_GETARG_TEXT_P(1);
char *distributionColumnName = text_to_cstring(distributionColumnText); char *distributionColumnName = text_to_cstring(distributionColumnText);
@ -982,6 +1026,23 @@ CreateReferenceTable(Oid relationId)
} }
/*
* CreateSingleShardTable is a wrapper around CreateCitusTable that creates a
* single shard distributed table that doesn't have a shard key.
*/
void
CreateSingleShardTable(Oid relationId, char *colocateWithTableName)
{
DistributedTableParams distributedTableParams = {
.colocateWithTableName = colocateWithTableName,
.shardCount = 1,
.shardCountIsStrict = true,
.distributionColumnName = NULL
};
CreateCitusTable(relationId, SINGLE_SHARD_DISTRIBUTED, &distributedTableParams);
}
/* /*
* CreateCitusTable is the internal method that creates a Citus table in * CreateCitusTable is the internal method that creates a Citus table in
* given configuration. * given configuration.
@ -1000,7 +1061,8 @@ CreateCitusTable(Oid relationId, CitusTableType tableType,
DistributedTableParams *distributedTableParams) DistributedTableParams *distributedTableParams)
{ {
if ((tableType == HASH_DISTRIBUTED || tableType == APPEND_DISTRIBUTED || if ((tableType == HASH_DISTRIBUTED || tableType == APPEND_DISTRIBUTED ||
tableType == RANGE_DISTRIBUTED) != (distributedTableParams != NULL)) tableType == RANGE_DISTRIBUTED || tableType == SINGLE_SHARD_DISTRIBUTED) !=
(distributedTableParams != NULL))
{ {
ereport(ERROR, (errmsg("distributed table params must be provided " ereport(ERROR, (errmsg("distributed table params must be provided "
"when creating a distributed table and must " "when creating a distributed table and must "
@ -1078,7 +1140,7 @@ CreateCitusTable(Oid relationId, CitusTableType tableType,
PropagatePrerequisiteObjectsForDistributedTable(relationId); PropagatePrerequisiteObjectsForDistributedTable(relationId);
Var *distributionColumn = NULL; Var *distributionColumn = NULL;
if (distributedTableParams) if (distributedTableParams && distributedTableParams->distributionColumnName)
{ {
distributionColumn = BuildDistributionKeyFromColumnName(relationId, distributionColumn = BuildDistributionKeyFromColumnName(relationId,
distributedTableParams-> distributedTableParams->
@ -1150,6 +1212,11 @@ CreateCitusTable(Oid relationId, CitusTableType tableType,
{ {
CreateReferenceTableShard(relationId); CreateReferenceTableShard(relationId);
} }
else if (tableType == SINGLE_SHARD_DISTRIBUTED)
{
CreateSingleShardTableShard(relationId, colocatedTableId,
colocationId);
}
if (ShouldSyncTableMetadata(relationId)) if (ShouldSyncTableMetadata(relationId))
{ {
@ -1204,7 +1271,8 @@ CreateCitusTable(Oid relationId, CitusTableType tableType,
} }
/* copy over data for hash distributed and reference tables */ /* copy over data for hash distributed and reference tables */
if (tableType == HASH_DISTRIBUTED || tableType == REFERENCE_TABLE) if (tableType == HASH_DISTRIBUTED || tableType == SINGLE_SHARD_DISTRIBUTED ||
tableType == REFERENCE_TABLE)
{ {
if (RegularTable(relationId)) if (RegularTable(relationId))
{ {
@ -1268,6 +1336,13 @@ DecideCitusTableParams(CitusTableType tableType,
break; break;
} }
case SINGLE_SHARD_DISTRIBUTED:
{
citusTableParams.distributionMethod = DISTRIBUTE_BY_NONE;
citusTableParams.replicationModel = REPLICATION_MODEL_STREAMING;
break;
}
case REFERENCE_TABLE: case REFERENCE_TABLE:
{ {
citusTableParams.distributionMethod = DISTRIBUTE_BY_NONE; citusTableParams.distributionMethod = DISTRIBUTE_BY_NONE;
@ -1630,6 +1705,41 @@ CreateHashDistributedTableShards(Oid relationId, int shardCount,
} }
/*
* CreateHashDistributedTableShards creates the shard of given single-shard
* distributed table.
*/
static void
CreateSingleShardTableShard(Oid relationId, Oid colocatedTableId,
uint32 colocationId)
{
if (colocatedTableId != InvalidOid)
{
/*
* We currently allow concurrent distribution of colocated tables (which
* we probably should not be allowing because of foreign keys /
* partitioning etc).
*
* We also prevent concurrent shard moves / copy / splits) while creating
* a colocated table.
*/
AcquirePlacementColocationLock(colocatedTableId, ShareLock,
"colocate distributed table");
/*
* We don't need to force using exclusive connections because we're anyway
* creating a single shard.
*/
bool useExclusiveConnection = false;
CreateColocatedShards(relationId, colocatedTableId, useExclusiveConnection);
}
else
{
CreateSingleShardTableShardWithRoundRobinPolicy(relationId, colocationId);
}
}
/* /*
* ColocationIdForNewTable returns a colocation id for given table * ColocationIdForNewTable returns a colocation id for given table
* according to given configuration. If there is no such configuration, it * according to given configuration. If there is no such configuration, it
@ -1662,8 +1772,8 @@ ColocationIdForNewTable(Oid relationId, CitusTableType tableType,
{ {
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot distribute relation"), errmsg("cannot distribute relation"),
errdetail("Currently, colocate_with option is only supported " errdetail("Currently, colocate_with option is not supported "
"for hash distributed tables."))); "for append / range distributed tables.")));
} }
return colocationId; return colocationId;
@ -1679,10 +1789,11 @@ ColocationIdForNewTable(Oid relationId, CitusTableType tableType,
* can be sure that there will no modifications on the colocation table * can be sure that there will no modifications on the colocation table
* until this transaction is committed. * until this transaction is committed.
*/ */
Assert(citusTableParams.distributionMethod == DISTRIBUTE_BY_HASH);
Oid distributionColumnType = distributionColumn->vartype; Oid distributionColumnType =
Oid distributionColumnCollation = get_typcollation(distributionColumnType); distributionColumn ? distributionColumn->vartype : InvalidOid;
Oid distributionColumnCollation =
distributionColumn ? get_typcollation(distributionColumnType) : InvalidOid;
/* get an advisory lock to serialize concurrent default group creations */ /* get an advisory lock to serialize concurrent default group creations */
if (IsColocateWithDefault(distributedTableParams->colocateWithTableName)) if (IsColocateWithDefault(distributedTableParams->colocateWithTableName))
@ -1871,8 +1982,15 @@ EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn,
*/ */
if (PartitionedTableNoLock(relationId)) if (PartitionedTableNoLock(relationId))
{ {
/* distributing partitioned tables in only supported for hash-distribution */ /*
if (distributionMethod != DISTRIBUTE_BY_HASH) * Distributing partitioned tables is only supported for hash-distribution
* or single-shard tables.
*/
bool isSingleShardTable =
distributionMethod == DISTRIBUTE_BY_NONE &&
replicationModel == REPLICATION_MODEL_STREAMING &&
colocationId != INVALID_COLOCATION_ID;
if (distributionMethod != DISTRIBUTE_BY_HASH && !isSingleShardTable)
{ {
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("distributing partitioned tables in only supported " errmsg("distributing partitioned tables in only supported "

View File

@ -393,9 +393,17 @@ GetDependencyCreateDDLCommands(const ObjectAddress *dependency)
tableDDLCommand)); tableDDLCommand));
} }
/* we need to drop table, if exists, first to make table creation idempotent */ /*
* We need to drop table, if exists, first to make table creation
* idempotent. Before dropping the table, we should also break
* dependencies with sequences since `drop cascade table` would also
* drop depended sequences. This is safe as we still record dependency
* with the sequence during table creation.
*/
commandList = lcons(DropTableIfExistsCommand(relationId), commandList = lcons(DropTableIfExistsCommand(relationId),
commandList); commandList);
commandList = lcons(WorkerDropSequenceDependencyCommand(relationId),
commandList);
} }
return commandList; return commandList;

View File

@ -303,6 +303,11 @@ ErrorIfUnsupportedForeignConstraintExists(Relation relation, char referencingDis
/* /*
* Foreign keys from citus local tables or reference tables to distributed * Foreign keys from citus local tables or reference tables to distributed
* tables are not supported. * tables are not supported.
*
* 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 single-shard
* tables too.
*/ */
if (referencingIsCitusLocalOrRefTable && !referencedIsCitusLocalOrRefTable) if (referencingIsCitusLocalOrRefTable && !referencedIsCitusLocalOrRefTable)
{ {
@ -361,7 +366,12 @@ ErrorIfUnsupportedForeignConstraintExists(Relation relation, char referencingDis
* if tables are hash-distributed and colocated, we need to make sure that * if tables are hash-distributed and colocated, we need to make sure that
* the distribution key is included in foreign constraint. * the distribution key is included in foreign constraint.
*/ */
if (!referencedIsCitusLocalOrRefTable && !foreignConstraintOnDistKey) bool referencedIsSingleShardTable =
IsSingleShardTableByDistParams(referencedDistMethod,
referencedReplicationModel,
referencedColocationId);
if (!referencedIsCitusLocalOrRefTable && !referencedIsSingleShardTable &&
!foreignConstraintOnDistKey)
{ {
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create foreign key constraint"), errmsg("cannot create foreign key constraint"),

View File

@ -2146,6 +2146,7 @@ CitusCopyDestReceiverStartup(DestReceiver *dest, int operation,
} }
if (IsCitusTableTypeCacheEntry(cacheEntry, DISTRIBUTED_TABLE) && if (IsCitusTableTypeCacheEntry(cacheEntry, DISTRIBUTED_TABLE) &&
!IsCitusTableTypeCacheEntry(cacheEntry, SINGLE_SHARD_DISTRIBUTED) &&
copyDest->partitionColumnIndex == INVALID_PARTITION_COLUMN_INDEX) copyDest->partitionColumnIndex == INVALID_PARTITION_COLUMN_INDEX)
{ {
ereport(ERROR, (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED), ereport(ERROR, (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),

View File

@ -384,6 +384,11 @@ PostprocessCreateTableStmtPartitionOf(CreateStmt *createStatement, const
*/ */
if (IsCitusTable(parentRelationId)) if (IsCitusTable(parentRelationId))
{ {
/*
* 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.
*/
if (IsCitusTableType(parentRelationId, CITUS_LOCAL_TABLE)) if (IsCitusTableType(parentRelationId, CITUS_LOCAL_TABLE))
{ {
CreateCitusLocalTablePartitionOf(createStatement, relationId, CreateCitusLocalTablePartitionOf(createStatement, relationId,
@ -391,11 +396,18 @@ PostprocessCreateTableStmtPartitionOf(CreateStmt *createStatement, const
return; return;
} }
char *parentRelationName = generate_qualified_relation_name(parentRelationId);
if (IsCitusTableType(parentRelationId, SINGLE_SHARD_DISTRIBUTED))
{
CreateSingleShardTable(relationId, parentRelationName);
return;
}
Var *parentDistributionColumn = DistPartitionKeyOrError(parentRelationId); Var *parentDistributionColumn = DistPartitionKeyOrError(parentRelationId);
char *distributionColumnName = char *distributionColumnName =
ColumnToColumnName(parentRelationId, (Node *) parentDistributionColumn); ColumnToColumnName(parentRelationId, (Node *) parentDistributionColumn);
char parentDistributionMethod = DISTRIBUTE_BY_HASH; char parentDistributionMethod = DISTRIBUTE_BY_HASH;
char *parentRelationName = generate_qualified_relation_name(parentRelationId);
SwitchToSequentialAndLocalExecutionIfPartitionNameTooLong(parentRelationId, SwitchToSequentialAndLocalExecutionIfPartitionNameTooLong(parentRelationId,
relationId); relationId);
@ -589,19 +601,32 @@ PreprocessAttachCitusPartitionToCitusTable(Oid parentCitusRelationId, Oid
/* /*
* DistributePartitionUsingParent takes a parent and a partition relation and * DistributePartitionUsingParent takes a parent and a partition relation and
* distributes the partition, using the same distribution column as the parent. * distributes the partition, using the same distribution column as the parent, if the
* It creates a *hash* distributed table by default, as partitioned tables can only be * parent has a distribution column. It creates a *hash* distributed table by default, as
* distributed by hash. * partitioned tables can only be distributed by hash, unless it's null key distributed.
*
* If the parent has no distribution key, we distribute the partition with null key too.
*/ */
static void static void
DistributePartitionUsingParent(Oid parentCitusRelationId, Oid partitionRelationId) DistributePartitionUsingParent(Oid parentCitusRelationId, Oid partitionRelationId)
{ {
char *parentRelationName = generate_qualified_relation_name(parentCitusRelationId);
if (!HasDistributionKey(parentCitusRelationId))
{
/*
* If the parent is null key distributed, we should distribute the partition
* with null distribution key as well.
*/
CreateSingleShardTable(partitionRelationId, parentRelationName);
return;
}
Var *distributionColumn = DistPartitionKeyOrError(parentCitusRelationId); Var *distributionColumn = DistPartitionKeyOrError(parentCitusRelationId);
char *distributionColumnName = ColumnToColumnName(parentCitusRelationId, char *distributionColumnName = ColumnToColumnName(parentCitusRelationId,
(Node *) distributionColumn); (Node *) distributionColumn);
char distributionMethod = DISTRIBUTE_BY_HASH; char distributionMethod = DISTRIBUTE_BY_HASH;
char *parentRelationName = generate_qualified_relation_name(parentCitusRelationId);
SwitchToSequentialAndLocalExecutionIfPartitionNameTooLong( SwitchToSequentialAndLocalExecutionIfPartitionNameTooLong(
parentCitusRelationId, partitionRelationId); parentCitusRelationId, partitionRelationId);

View File

@ -324,7 +324,8 @@ ExecuteTruncateStmtSequentialIfNecessary(TruncateStmt *command)
{ {
Oid relationId = RangeVarGetRelid(rangeVar, NoLock, failOK); Oid relationId = RangeVarGetRelid(rangeVar, NoLock, failOK);
if (IsCitusTable(relationId) && !HasDistributionKey(relationId) && if ((IsCitusTableType(relationId, REFERENCE_TABLE) ||
IsCitusTableType(relationId, CITUS_LOCAL_TABLE)) &&
TableReferenced(relationId)) TableReferenced(relationId))
{ {
char *relationName = get_rel_name(relationId); char *relationName = get_rel_name(relationId);

View File

@ -508,11 +508,21 @@ IsCitusTableTypeInternal(char partitionMethod, char replicationModel,
return partitionMethod == DISTRIBUTE_BY_RANGE; return partitionMethod == DISTRIBUTE_BY_RANGE;
} }
case SINGLE_SHARD_DISTRIBUTED:
{
return partitionMethod == DISTRIBUTE_BY_NONE &&
replicationModel != REPLICATION_MODEL_2PC &&
colocationId != INVALID_COLOCATION_ID;
}
case DISTRIBUTED_TABLE: case DISTRIBUTED_TABLE:
{ {
return partitionMethod == DISTRIBUTE_BY_HASH || return partitionMethod == DISTRIBUTE_BY_HASH ||
partitionMethod == DISTRIBUTE_BY_RANGE || partitionMethod == DISTRIBUTE_BY_RANGE ||
partitionMethod == DISTRIBUTE_BY_APPEND; partitionMethod == DISTRIBUTE_BY_APPEND ||
(partitionMethod == DISTRIBUTE_BY_NONE &&
replicationModel != REPLICATION_MODEL_2PC &&
colocationId != INVALID_COLOCATION_ID);
} }
case STRICTLY_PARTITIONED_DISTRIBUTED_TABLE: case STRICTLY_PARTITIONED_DISTRIBUTED_TABLE:
@ -815,6 +825,21 @@ IsCitusLocalTableByDistParams(char partitionMethod, char replicationModel,
} }
/*
* IsSingleShardTableByDistParams returns true if given partitionMethod,
* replicationModel and colocationId would identify a single-shard distributed
* table that has a null shard key.
*/
bool
IsSingleShardTableByDistParams(char partitionMethod, char replicationModel,
uint32 colocationId)
{
return partitionMethod == DISTRIBUTE_BY_NONE &&
replicationModel != REPLICATION_MODEL_2PC &&
colocationId != INVALID_COLOCATION_ID;
}
/* /*
* CitusTableList returns a list that includes all the valid distributed table * CitusTableList returns a list that includes all the valid distributed table
* cache entries. * cache entries.

View File

@ -515,7 +515,7 @@ ShouldSyncUserCommandForObject(ObjectAddress objectAddress)
/* /*
* ShouldSyncTableMetadata checks if the metadata of a distributed table should be * ShouldSyncTableMetadata checks if the metadata of a distributed table should be
* propagated to metadata workers, i.e. the table is a hash distributed table or * propagated to metadata workers, i.e. the table is a hash distributed table or
* reference/citus local table. * a Citus table that doesn't have shard key.
*/ */
bool bool
ShouldSyncTableMetadata(Oid relationId) ShouldSyncTableMetadata(Oid relationId)
@ -537,10 +537,11 @@ ShouldSyncTableMetadata(Oid relationId)
/* /*
* ShouldSyncTableMetadataViaCatalog checks if the metadata of a distributed table should * ShouldSyncTableMetadataViaCatalog checks if the metadata of a Citus table should
* be propagated to metadata workers, i.e. the table is an MX table or reference table. * be propagated to metadata workers, i.e. the table is an MX table or Citus table
* that doesn't have shard key.
* Tables with streaming replication model (which means RF=1) and hash distribution are * Tables with streaming replication model (which means RF=1) and hash distribution are
* considered as MX tables while tables with none distribution are reference tables. * considered as MX tables.
* *
* ShouldSyncTableMetadataViaCatalog does not use the CitusTableCache and instead reads * ShouldSyncTableMetadataViaCatalog does not use the CitusTableCache and instead reads
* from catalog tables directly. * from catalog tables directly.
@ -686,7 +687,7 @@ DropMetadataSnapshotOnNode(WorkerNode *workerNode)
bool singleTransaction = true; bool singleTransaction = true;
List *dropMetadataCommandList = DetachPartitionCommandList(); List *dropMetadataCommandList = DetachPartitionCommandList();
dropMetadataCommandList = lappend(dropMetadataCommandList, dropMetadataCommandList = lappend(dropMetadataCommandList,
BREAK_CITUS_TABLE_SEQUENCE_DEPENDENCY_COMMAND); BREAK_ALL_CITUS_TABLE_SEQUENCE_DEPENDENCY_COMMAND);
dropMetadataCommandList = lappend(dropMetadataCommandList, dropMetadataCommandList = lappend(dropMetadataCommandList,
WorkerDropAllShellTablesCommand(singleTransaction)); WorkerDropAllShellTablesCommand(singleTransaction));
dropMetadataCommandList = list_concat(dropMetadataCommandList, dropMetadataCommandList = list_concat(dropMetadataCommandList,
@ -1080,7 +1081,7 @@ EnsureObjectMetadataIsSane(int distributionArgumentIndex, int colocationId)
/* /*
* DistributionCreateCommands generates a commands that can be * DistributionCreateCommands generates a commands that can be
* executed to replicate the metadata for a distributed table. * executed to replicate the metadata for a Citus table.
*/ */
char * char *
DistributionCreateCommand(CitusTableCacheEntry *cacheEntry) DistributionCreateCommand(CitusTableCacheEntry *cacheEntry)
@ -4235,6 +4236,22 @@ WorkerDropAllShellTablesCommand(bool singleTransaction)
} }
/*
* WorkerDropSequenceDependencyCommand returns command to drop sequence dependencies for
* given table.
*/
char *
WorkerDropSequenceDependencyCommand(Oid relationId)
{
char *qualifiedTableName = generate_qualified_relation_name(relationId);
StringInfo breakSequenceDepCommand = makeStringInfo();
appendStringInfo(breakSequenceDepCommand,
BREAK_CITUS_TABLE_SEQUENCE_DEPENDENCY_COMMAND,
quote_literal_cstr(qualifiedTableName));
return breakSequenceDepCommand->data;
}
/* /*
* PropagateNodeWideObjectsCommandList is called during node activation to * PropagateNodeWideObjectsCommandList is called during node activation to
* propagate any object that should be propagated for every node. These are * propagate any object that should be propagated for every node. These are
@ -4352,8 +4369,8 @@ SendNodeWideObjectsSyncCommands(MetadataSyncContext *context)
void void
SendShellTableDeletionCommands(MetadataSyncContext *context) SendShellTableDeletionCommands(MetadataSyncContext *context)
{ {
/* break all sequence deps for citus tables and remove all shell tables */ /* break all sequence deps for citus tables */
char *breakSeqDepsCommand = BREAK_CITUS_TABLE_SEQUENCE_DEPENDENCY_COMMAND; char *breakSeqDepsCommand = BREAK_ALL_CITUS_TABLE_SEQUENCE_DEPENDENCY_COMMAND;
SendOrCollectCommandListToActivatedNodes(context, list_make1(breakSeqDepsCommand)); SendOrCollectCommandListToActivatedNodes(context, list_make1(breakSeqDepsCommand));
/* remove shell tables */ /* remove shell tables */

View File

@ -36,6 +36,7 @@
#include "distributed/multi_join_order.h" #include "distributed/multi_join_order.h"
#include "distributed/multi_router_planner.h" #include "distributed/multi_router_planner.h"
#include "distributed/pg_dist_node.h" #include "distributed/pg_dist_node.h"
#include "distributed/pg_dist_node_metadata.h"
#include "distributed/reference_table_utils.h" #include "distributed/reference_table_utils.h"
#include "distributed/remote_commands.h" #include "distributed/remote_commands.h"
#include "distributed/resource_lock.h" #include "distributed/resource_lock.h"
@ -119,7 +120,6 @@ static char * NodeMetadataSyncedUpdateCommand(uint32 nodeId, bool metadataSynced
static void ErrorIfCoordinatorMetadataSetFalse(WorkerNode *workerNode, Datum value, static void ErrorIfCoordinatorMetadataSetFalse(WorkerNode *workerNode, Datum value,
char *field); char *field);
static WorkerNode * SetShouldHaveShards(WorkerNode *workerNode, bool shouldHaveShards); static WorkerNode * SetShouldHaveShards(WorkerNode *workerNode, bool shouldHaveShards);
static int FindCoordinatorNodeId(void);
static WorkerNode * FindNodeAnyClusterByNodeId(uint32 nodeId); static WorkerNode * FindNodeAnyClusterByNodeId(uint32 nodeId);
static void ErrorIfAnyNodeNotExist(List *nodeList); static void ErrorIfAnyNodeNotExist(List *nodeList);
static void UpdateLocalGroupIdsViaMetadataContext(MetadataSyncContext *context); static void UpdateLocalGroupIdsViaMetadataContext(MetadataSyncContext *context);
@ -1800,7 +1800,7 @@ FindNodeWithNodeId(int nodeId, bool missingOk)
/* /*
* FindCoordinatorNodeId returns the node id of the coordinator node * FindCoordinatorNodeId returns the node id of the coordinator node
*/ */
static int int
FindCoordinatorNodeId() FindCoordinatorNodeId()
{ {
bool includeNodesFromOtherClusters = false; bool includeNodesFromOtherClusters = false;

View File

@ -217,9 +217,9 @@ CreateColocatedShards(Oid targetRelationId, Oid sourceRelationId, bool
List *insertedShardPlacements = NIL; List *insertedShardPlacements = NIL;
List *insertedShardIds = NIL; List *insertedShardIds = NIL;
/* make sure that tables are hash partitioned */ CitusTableCacheEntry *targetCacheEntry = GetCitusTableCacheEntry(targetRelationId);
CheckHashPartitionedTable(targetRelationId); Assert(targetCacheEntry->partitionMethod == DISTRIBUTE_BY_HASH ||
CheckHashPartitionedTable(sourceRelationId); targetCacheEntry->partitionMethod == DISTRIBUTE_BY_NONE);
/* /*
* In contrast to append/range partitioned tables it makes more sense to * In contrast to append/range partitioned tables it makes more sense to
@ -259,10 +259,20 @@ CreateColocatedShards(Oid targetRelationId, Oid sourceRelationId, bool
*newShardIdPtr = GetNextShardId(); *newShardIdPtr = GetNextShardId();
insertedShardIds = lappend(insertedShardIds, newShardIdPtr); insertedShardIds = lappend(insertedShardIds, newShardIdPtr);
int32 shardMinValue = DatumGetInt32(sourceShardInterval->minValue); text *shardMinValueText = NULL;
int32 shardMaxValue = DatumGetInt32(sourceShardInterval->maxValue); text *shardMaxValueText = NULL;
text *shardMinValueText = IntegerToText(shardMinValue); if (targetCacheEntry->partitionMethod == DISTRIBUTE_BY_NONE)
text *shardMaxValueText = IntegerToText(shardMaxValue); {
Assert(list_length(sourceShardIntervalList) == 1);
}
else
{
int32 shardMinValue = DatumGetInt32(sourceShardInterval->minValue);
int32 shardMaxValue = DatumGetInt32(sourceShardInterval->maxValue);
shardMinValueText = IntegerToText(shardMinValue);
shardMaxValueText = IntegerToText(shardMaxValue);
}
List *sourceShardPlacementList = ShardPlacementListSortedByWorker( List *sourceShardPlacementList = ShardPlacementListSortedByWorker(
sourceShardId); sourceShardId);
@ -362,6 +372,72 @@ CreateReferenceTableShard(Oid distributedTableId)
} }
/*
* 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
CreateSingleShardTableShardWithRoundRobinPolicy(Oid relationId, uint32 colocationId)
{
EnsureTableOwner(relationId);
/* we plan to add shards: get an exclusive lock on relation oid */
LockRelationOid(relationId, ExclusiveLock);
/*
* Load and sort the worker node list for deterministic placement.
*
* Also take a RowShareLock on pg_dist_node to disallow concurrent
* node list changes that require an exclusive lock.
*/
List *workerNodeList = DistributedTablePlacementNodeList(RowShareLock);
workerNodeList = SortList(workerNodeList, CompareWorkerNodes);
int32 workerNodeCount = list_length(workerNodeList);
if (workerNodeCount == 0)
{
ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("couldn't find any worker nodes"),
errhint("Add more worker nodes")));
}
char shardStorageType = ShardStorageType(relationId);
text *minHashTokenText = NULL;
text *maxHashTokenText = NULL;
uint64 shardId = GetNextShardId();
InsertShardRow(relationId, shardId, shardStorageType,
minHashTokenText, maxHashTokenText);
/* determine the node index based on colocation id */
int roundRobinNodeIdx = colocationId % workerNodeCount;
int replicationFactor = 1;
List *insertedShardPlacements = InsertShardPlacementRows(
relationId,
shardId,
workerNodeList,
roundRobinNodeIdx,
replicationFactor);
/*
* We don't need to force using exclusive connections because we're anyway
* creating a single shard.
*/
bool useExclusiveConnection = false;
bool colocatedShard = false;
CreateShardsOnWorkers(relationId, insertedShardPlacements,
useExclusiveConnection, colocatedShard);
}
/* /*
* CheckHashPartitionedTable looks up the partition information for the given * CheckHashPartitionedTable looks up the partition information for the given
* tableId and checks if the table is hash partitioned. If not, the function * tableId and checks if the table is hash partitioned. If not, the function

View File

@ -515,6 +515,16 @@ GetRebalanceSteps(RebalanceOptions *options)
/* sort the lists to make the function more deterministic */ /* sort the lists to make the function more deterministic */
List *activeWorkerList = SortedActiveWorkers(); List *activeWorkerList = SortedActiveWorkers();
int shardAllowedNodeCount = 0;
WorkerNode *workerNode = NULL;
foreach_ptr(workerNode, activeWorkerList)
{
if (workerNode->shouldHaveShards)
{
shardAllowedNodeCount++;
}
}
List *activeShardPlacementListList = NIL; List *activeShardPlacementListList = NIL;
List *unbalancedShards = NIL; List *unbalancedShards = NIL;
@ -532,8 +542,7 @@ GetRebalanceSteps(RebalanceOptions *options)
shardPlacementList, options->workerNode); shardPlacementList, options->workerNode);
} }
if (list_length(activeShardPlacementListForRelation) >= list_length( if (list_length(activeShardPlacementListForRelation) >= shardAllowedNodeCount)
activeWorkerList))
{ {
activeShardPlacementListList = lappend(activeShardPlacementListList, activeShardPlacementListList = lappend(activeShardPlacementListList,
activeShardPlacementListForRelation); activeShardPlacementListForRelation);

View File

@ -138,6 +138,13 @@ master_create_empty_shard(PG_FUNCTION_ARGS)
errdetail("We currently don't support creating shards " errdetail("We currently don't support creating shards "
"on hash-partitioned tables"))); "on hash-partitioned tables")));
} }
else if (IsCitusTableType(relationId, SINGLE_SHARD_DISTRIBUTED))
{
ereport(ERROR, (errmsg("relation \"%s\" is a single shard table",
relationName),
errdetail("We currently don't support creating shards "
"on single shard tables")));
}
else if (IsCitusTableType(relationId, REFERENCE_TABLE)) else if (IsCitusTableType(relationId, REFERENCE_TABLE))
{ {
ereport(ERROR, (errmsg("relation \"%s\" is a reference table", ereport(ERROR, (errmsg("relation \"%s\" is a reference table",
@ -521,7 +528,8 @@ RelationShardListForShardCreate(ShardInterval *shardInterval)
relationShard->shardId = shardInterval->shardId; relationShard->shardId = shardInterval->shardId;
List *relationShardList = list_make1(relationShard); List *relationShardList = list_make1(relationShard);
if (IsCitusTableTypeCacheEntry(cacheEntry, HASH_DISTRIBUTED) && if ((IsCitusTableTypeCacheEntry(cacheEntry, HASH_DISTRIBUTED) ||
IsCitusTableTypeCacheEntry(cacheEntry, SINGLE_SHARD_DISTRIBUTED)) &&
cacheEntry->colocationId != INVALID_COLOCATION_ID) cacheEntry->colocationId != INVALID_COLOCATION_ID)
{ {
shardIndex = ShardIndex(shardInterval); shardIndex = ShardIndex(shardInterval);

View File

@ -1025,6 +1025,17 @@ CreateDistributedPlan(uint64 planId, bool allowRecursivePlanning, Query *origina
{ {
return distributedPlan; return distributedPlan;
} }
else if (ContainsSingleShardTable(originalQuery))
{
/*
* We only support router queries if the query contains reference to
* a single-shard table. This temporary restriction will be removed
* once we support recursive planning for the queries that reference
* single-shard tables.
*/
WrapRouterErrorForSingleShardTable(distributedPlan->planningError);
RaiseDeferredError(distributedPlan->planningError, ERROR);
}
else else
{ {
RaiseDeferredError(distributedPlan->planningError, DEBUG2); RaiseDeferredError(distributedPlan->planningError, DEBUG2);
@ -2462,6 +2473,18 @@ HasUnresolvedExternParamsWalker(Node *expression, ParamListInfo boundParams)
} }
/*
* ContainsSingleShardTable returns true if given query contains reference
* to a single-shard table.
*/
bool
ContainsSingleShardTable(Query *query)
{
RTEListProperties *rteListProperties = GetRTEListPropertiesForQuery(query);
return rteListProperties->hasSingleShardDistTable;
}
/* /*
* GetRTEListPropertiesForQuery is a wrapper around GetRTEListProperties that * GetRTEListPropertiesForQuery is a wrapper around GetRTEListProperties that
* returns RTEListProperties for the rte list retrieved from query. * returns RTEListProperties for the rte list retrieved from query.
@ -2538,6 +2561,15 @@ GetRTEListProperties(List *rangeTableList)
else if (IsCitusTableTypeCacheEntry(cacheEntry, DISTRIBUTED_TABLE)) else if (IsCitusTableTypeCacheEntry(cacheEntry, DISTRIBUTED_TABLE))
{ {
rteListProperties->hasDistributedTable = true; rteListProperties->hasDistributedTable = true;
if (!HasDistributionKeyCacheEntry(cacheEntry))
{
rteListProperties->hasSingleShardDistTable = true;
}
else
{
rteListProperties->hasDistTableWithShardKey = true;
}
} }
else else
{ {

View File

@ -212,6 +212,16 @@ FastPathRouterQuery(Query *query, Node **distributionKeyValue)
return false; return false;
} }
/*
* If the table doesn't have a distribution column, we don't need to
* check anything further.
*/
Var *distributionKey = PartitionColumn(distributedTableId, 1);
if (!distributionKey)
{
return true;
}
/* WHERE clause should not be empty for distributed tables */ /* WHERE clause should not be empty for distributed tables */
if (joinTree == NULL || if (joinTree == NULL ||
(IsCitusTableTypeCacheEntry(cacheEntry, DISTRIBUTED_TABLE) && joinTree->quals == (IsCitusTableTypeCacheEntry(cacheEntry, DISTRIBUTED_TABLE) && joinTree->quals ==
@ -220,13 +230,6 @@ FastPathRouterQuery(Query *query, Node **distributionKeyValue)
return false; return false;
} }
/* if that's a reference table, we don't need to check anything further */
Var *distributionKey = PartitionColumn(distributedTableId, 1);
if (!distributionKey)
{
return true;
}
/* convert list of expressions into expression tree for further processing */ /* convert list of expressions into expression tree for further processing */
quals = joinTree->quals; quals = joinTree->quals;
if (quals != NULL && IsA(quals, List)) if (quals != NULL && IsA(quals, List))

View File

@ -730,25 +730,49 @@ DistributedInsertSelectSupported(Query *queryTree, RangeTblEntry *insertRte,
"table", NULL, NULL); "table", NULL, NULL);
} }
/* ensure that INSERT's partition column comes from SELECT's partition column */ if (!HasDistributionKey(targetRelationId) ||
error = InsertPartitionColumnMatchesSelect(queryTree, insertRte, subqueryRte, subqueryRteListProperties->hasSingleShardDistTable)
&selectPartitionColumnTableId);
if (error)
{ {
return error; /*
* XXX: Better to check this regardless of the fact that the target table
* has a distribution column or not.
*/
List *distributedRelationIdList = DistributedRelationIdList(subquery);
distributedRelationIdList = lappend_oid(distributedRelationIdList,
targetRelationId);
if (!AllDistributedRelationsInListColocated(distributedRelationIdList))
{
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
"distributed INSERT ... SELECT cannot reference a "
"distributed table without a shard key together "
"with non-colocated distributed tables",
NULL, NULL);
}
} }
/* if (HasDistributionKey(targetRelationId))
* We expect partition column values come from colocated tables. Note that we
* skip this check from the reference table case given that all reference tables
* are already (and by default) co-located.
*/
if (!TablesColocated(insertRte->relid, selectPartitionColumnTableId))
{ {
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, /* ensure that INSERT's partition column comes from SELECT's partition column */
"INSERT target table and the source relation of the SELECT partition " error = InsertPartitionColumnMatchesSelect(queryTree, insertRte, subqueryRte,
"column value must be colocated in distributed INSERT ... SELECT", &selectPartitionColumnTableId);
NULL, NULL); if (error)
{
return error;
}
/*
* We expect partition column values come from colocated tables. Note that we
* skip this check from the reference table case given that all reference tables
* are already (and by default) co-located.
*/
if (!TablesColocated(insertRte->relid, selectPartitionColumnTableId))
{
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
"INSERT target table and the source relation of the SELECT partition "
"column value must be colocated in distributed INSERT ... SELECT",
NULL, NULL);
}
} }
} }
@ -867,7 +891,7 @@ RouterModifyTaskForShardInterval(Query *originalQuery,
*/ */
RTEListProperties *subqueryRteListProperties = GetRTEListPropertiesForQuery( RTEListProperties *subqueryRteListProperties = GetRTEListPropertiesForQuery(
copiedSubquery); copiedSubquery);
if (subqueryRteListProperties->hasDistributedTable) if (subqueryRteListProperties->hasDistTableWithShardKey)
{ {
AddPartitionKeyNotNullFilterToSelect(copiedSubquery); AddPartitionKeyNotNullFilterToSelect(copiedSubquery);
} }
@ -1537,6 +1561,19 @@ CreateNonPushableInsertSelectPlan(uint64 planId, Query *parse, ParamListInfo bou
bool repartitioned = IsRedistributablePlan(selectPlan->planTree) && bool repartitioned = IsRedistributablePlan(selectPlan->planTree) &&
IsSupportedRedistributionTarget(targetRelationId); IsSupportedRedistributionTarget(targetRelationId);
/*
* Today it's not possible to generate a distributed plan for a SELECT
* 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 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 single-shard table.
*/
Assert(!repartitioned ||
!GetRTEListPropertiesForQuery(selectQueryCopy)->hasSingleShardDistTable);
distributedPlan->insertSelectQuery = insertSelectQuery; distributedPlan->insertSelectQuery = insertSelectQuery;
distributedPlan->selectPlanForInsertSelect = selectPlan; distributedPlan->selectPlanForInsertSelect = selectPlan;
distributedPlan->insertSelectMethod = repartitioned ? distributedPlan->insertSelectMethod = repartitioned ?

View File

@ -22,29 +22,27 @@
#include "distributed/merge_planner.h" #include "distributed/merge_planner.h"
#include "distributed/multi_logical_optimizer.h" #include "distributed/multi_logical_optimizer.h"
#include "distributed/multi_router_planner.h" #include "distributed/multi_router_planner.h"
#include "distributed/pg_dist_node_metadata.h"
#include "distributed/pg_version_constants.h" #include "distributed/pg_version_constants.h"
#include "distributed/query_pushdown_planning.h" #include "distributed/query_pushdown_planning.h"
#if PG_VERSION_NUM >= PG_VERSION_15 #if PG_VERSION_NUM >= PG_VERSION_15
static DeferredErrorMessage * CheckIfRTETypeIsUnsupported(Query *parse,
RangeTblEntry *rangeTableEntry);
static DeferredErrorMessage * ErrorIfDistTablesNotColocated(Query *parse, static DeferredErrorMessage * ErrorIfDistTablesNotColocated(Query *parse,
List * List *
distTablesList, distTablesList,
PlannerRestrictionContext PlannerRestrictionContext
* *
plannerRestrictionContext); plannerRestrictionContext);
static DeferredErrorMessage * ErrorIfMergeHasUnsupportedTables(Query *parse, static DeferredErrorMessage * ErrorIfMergeHasUnsupportedTables(Oid targetRelationId,
Query *parse,
List *rangeTableList, List *rangeTableList,
PlannerRestrictionContext * PlannerRestrictionContext *
restrictionContext); restrictionContext);
static bool IsDistributionColumnInMergeSource(Expr *columnExpression, Query *query, bool static bool IsDistributionColumnInMergeSource(Expr *columnExpression, Query *query, bool
skipOuterVars); skipOuterVars);
static DeferredErrorMessage * InsertDistributionColumnMatchesSource(Query *query, static DeferredErrorMessage * InsertDistributionColumnMatchesSource(Oid targetRelationId,
RangeTblEntry * Query *query);
resultRte);
static DeferredErrorMessage * MergeQualAndTargetListFunctionsSupported(Oid static DeferredErrorMessage * MergeQualAndTargetListFunctionsSupported(Oid
resultRelationId, resultRelationId,
FromExpr *joinTree, FromExpr *joinTree,
@ -65,12 +63,15 @@ CreateMergePlan(Query *originalQuery, Query *query,
{ {
DistributedPlan *distributedPlan = CitusMakeNode(DistributedPlan); DistributedPlan *distributedPlan = CitusMakeNode(DistributedPlan);
bool multiShardQuery = false; bool multiShardQuery = false;
Oid targetRelationId = ModifyQueryResultRelationId(originalQuery);
Assert(originalQuery->commandType == CMD_MERGE); Assert(originalQuery->commandType == CMD_MERGE);
Assert(OidIsValid(targetRelationId));
distributedPlan->targetRelationId = targetRelationId;
distributedPlan->modLevel = RowModifyLevelForQuery(query); distributedPlan->modLevel = RowModifyLevelForQuery(query);
distributedPlan->planningError = MergeQuerySupported(targetRelationId,
distributedPlan->planningError = MergeQuerySupported(originalQuery, originalQuery,
multiShardQuery, multiShardQuery,
plannerRestrictionContext); plannerRestrictionContext);
@ -94,8 +95,6 @@ CreateMergePlan(Query *originalQuery, Query *query,
/* MERGE doesn't support RETURNING clause */ /* MERGE doesn't support RETURNING clause */
distributedPlan->expectResults = false; distributedPlan->expectResults = false;
distributedPlan->targetRelationId = ResultRelationOidForQuery(query);
distributedPlan->fastPathRouterPlan = distributedPlan->fastPathRouterPlan =
plannerRestrictionContext->fastPathRestrictionContext->fastPathRouterQuery; plannerRestrictionContext->fastPathRestrictionContext->fastPathRouterQuery;
@ -111,7 +110,7 @@ CreateMergePlan(Query *originalQuery, Query *query,
* - Checks target-lists and functions-in-quals in TargetlistAndFunctionsSupported * - Checks target-lists and functions-in-quals in TargetlistAndFunctionsSupported
*/ */
DeferredErrorMessage * DeferredErrorMessage *
MergeQuerySupported(Query *originalQuery, bool multiShardQuery, MergeQuerySupported(Oid resultRelationId, Query *originalQuery, bool multiShardQuery,
PlannerRestrictionContext *plannerRestrictionContext) PlannerRestrictionContext *plannerRestrictionContext)
{ {
/* function is void for pre-15 versions of Postgres */ /* function is void for pre-15 versions of Postgres */
@ -138,7 +137,6 @@ MergeQuerySupported(Query *originalQuery, bool multiShardQuery,
} }
List *rangeTableList = ExtractRangeTableEntryList(originalQuery); List *rangeTableList = ExtractRangeTableEntryList(originalQuery);
RangeTblEntry *resultRte = ExtractResultRelationRTE(originalQuery);
/* /*
* Fast path queries cannot have merge command, and we prevent the remaining here. * Fast path queries cannot have merge command, and we prevent the remaining here.
@ -149,7 +147,8 @@ MergeQuerySupported(Query *originalQuery, bool multiShardQuery,
* ErrorIfDistTablesNotColocated for details. * ErrorIfDistTablesNotColocated for details.
*/ */
DeferredErrorMessage *deferredError = DeferredErrorMessage *deferredError =
ErrorIfMergeHasUnsupportedTables(originalQuery, ErrorIfMergeHasUnsupportedTables(resultRelationId,
originalQuery,
rangeTableList, rangeTableList,
plannerRestrictionContext); plannerRestrictionContext);
if (deferredError) if (deferredError)
@ -158,7 +157,6 @@ MergeQuerySupported(Query *originalQuery, bool multiShardQuery,
RaiseDeferredError(deferredError, ERROR); RaiseDeferredError(deferredError, ERROR);
} }
Oid resultRelationId = resultRte->relid;
deferredError = MergeQualAndTargetListFunctionsSupported(resultRelationId, deferredError = MergeQualAndTargetListFunctionsSupported(resultRelationId,
originalQuery->jointree, originalQuery->jointree,
originalQuery->jointree-> originalQuery->jointree->
@ -191,7 +189,7 @@ MergeQuerySupported(Query *originalQuery, bool multiShardQuery,
} }
deferredError = deferredError =
InsertDistributionColumnMatchesSource(originalQuery, resultRte); InsertDistributionColumnMatchesSource(resultRelationId, originalQuery);
if (deferredError) if (deferredError)
{ {
/* MERGE's unsupported scenario, raise the exception */ /* MERGE's unsupported scenario, raise the exception */
@ -222,32 +220,6 @@ MergeQuerySupported(Query *originalQuery, bool multiShardQuery,
} }
/*
* IsMergeAllowedOnRelation takes a relation entry and checks if MERGE command is
* permitted on special relations, such as materialized view, returns true only if
* it's a "source" relation.
*/
bool
IsMergeAllowedOnRelation(Query *parse, RangeTblEntry *rte)
{
if (!IsMergeQuery(parse))
{
return false;
}
/* Fetch the MERGE target relation */
RangeTblEntry *targetRte = rt_fetch(parse->resultRelation, parse->rtable);
/* Is it a target relation? */
if (targetRte->relid == rte->relid)
{
return false;
}
return true;
}
#if PG_VERSION_NUM >= PG_VERSION_15 #if PG_VERSION_NUM >= PG_VERSION_15
/* /*
@ -283,70 +255,6 @@ ErrorIfDistTablesNotColocated(Query *parse, List *distTablesList,
} }
/*
* ErrorIfRTETypeIsUnsupported Checks for types of tables that are not supported, such
* as, reference tables, append-distributed tables and materialized view as target relation.
* Routine returns NULL for the supported types, error message for everything else.
*/
static DeferredErrorMessage *
CheckIfRTETypeIsUnsupported(Query *parse, RangeTblEntry *rangeTableEntry)
{
if (rangeTableEntry->relkind == RELKIND_MATVIEW ||
rangeTableEntry->relkind == RELKIND_FOREIGN_TABLE)
{
/* Materialized view or Foreign table as target is not allowed */
if (IsMergeAllowedOnRelation(parse, rangeTableEntry))
{
/* Non target relation is ok */
return NULL;
}
else
{
/* Usually we don't reach this exception as the Postgres parser catches it */
StringInfo errorMessage = makeStringInfo();
appendStringInfo(errorMessage, "MERGE command is not allowed on "
"relation type(relkind:%c)",
rangeTableEntry->relkind);
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
errorMessage->data, NULL, NULL);
}
}
if (rangeTableEntry->relkind != RELKIND_RELATION &&
rangeTableEntry->relkind != RELKIND_PARTITIONED_TABLE)
{
StringInfo errorMessage = makeStringInfo();
appendStringInfo(errorMessage, "Unexpected table type(relkind:%c) "
"in MERGE command", rangeTableEntry->relkind);
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
errorMessage->data, NULL, NULL);
}
Assert(rangeTableEntry->relid != 0);
/* Reference tables are not supported yet */
if (IsCitusTableType(rangeTableEntry->relid, REFERENCE_TABLE))
{
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
"MERGE command is not supported on reference "
"tables yet", NULL, NULL);
}
/* Append/Range tables are not supported */
if (IsCitusTableType(rangeTableEntry->relid, APPEND_DISTRIBUTED) ||
IsCitusTableType(rangeTableEntry->relid, RANGE_DISTRIBUTED))
{
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
"For MERGE command, all the distributed tables "
"must be colocated, for append/range distribution, "
"colocation is not supported", NULL,
"Consider using hash distribution instead");
}
return NULL;
}
/* /*
* ErrorIfMergeHasUnsupportedTables checks if all the tables(target, source or any CTE * ErrorIfMergeHasUnsupportedTables checks if all the tables(target, source or any CTE
* present) in the MERGE command are local i.e. a combination of Citus local and Non-Citus * present) in the MERGE command are local i.e. a combination of Citus local and Non-Citus
@ -355,11 +263,12 @@ CheckIfRTETypeIsUnsupported(Query *parse, RangeTblEntry *rangeTableEntry)
* for all other combinations. * for all other combinations.
*/ */
static DeferredErrorMessage * static DeferredErrorMessage *
ErrorIfMergeHasUnsupportedTables(Query *parse, List *rangeTableList, ErrorIfMergeHasUnsupportedTables(Oid targetRelationId, Query *parse, List *rangeTableList,
PlannerRestrictionContext *restrictionContext) PlannerRestrictionContext *restrictionContext)
{ {
List *distTablesList = NIL; List *distTablesList = NIL;
bool foundLocalTables = false; bool foundLocalTables = false;
bool foundReferenceTables = false;
RangeTblEntry *rangeTableEntry = NULL; RangeTblEntry *rangeTableEntry = NULL;
foreach_ptr(rangeTableEntry, rangeTableList) foreach_ptr(rangeTableEntry, rangeTableList)
@ -410,18 +319,48 @@ ErrorIfMergeHasUnsupportedTables(Query *parse, List *rangeTableList,
} }
/* RTE Relation can be of various types, check them now */ /* RTE Relation can be of various types, check them now */
switch (rangeTableEntry->relkind)
/* skip the regular views as they are replaced with subqueries */
if (rangeTableEntry->relkind == RELKIND_VIEW)
{ {
continue; /* skip the regular views as they are replaced with subqueries */
} case RELKIND_VIEW:
{
continue;
}
DeferredErrorMessage *errorMessage = case RELKIND_MATVIEW:
CheckIfRTETypeIsUnsupported(parse, rangeTableEntry); case RELKIND_FOREIGN_TABLE:
if (errorMessage) {
{ /* These two cases as a target is not allowed */
return errorMessage; if (relationId == targetRelationId)
{
/* Usually we don't reach this exception as the Postgres parser catches it */
StringInfo errorMessage = makeStringInfo();
appendStringInfo(errorMessage, "MERGE command is not allowed on "
"relation type(relkind:%c)",
rangeTableEntry->relkind);
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
errorMessage->data, NULL, NULL);
}
break;
}
case RELKIND_RELATION:
case RELKIND_PARTITIONED_TABLE:
{
/* Check for citus/postgres table types */
Assert(OidIsValid(relationId));
break;
}
default:
{
StringInfo errorMessage = makeStringInfo();
appendStringInfo(errorMessage, "Unexpected table type(relkind:%c) "
"in MERGE command",
rangeTableEntry->relkind);
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
errorMessage->data, NULL, NULL);
}
} }
/* /*
@ -430,28 +369,63 @@ ErrorIfMergeHasUnsupportedTables(Query *parse, List *rangeTableList,
*/ */
if (IsCitusTableType(relationId, DISTRIBUTED_TABLE)) if (IsCitusTableType(relationId, DISTRIBUTED_TABLE))
{ {
distTablesList = lappend(distTablesList, rangeTableEntry); /* Append/Range distributed tables are not supported */
continue; if (IsCitusTableType(relationId, APPEND_DISTRIBUTED) ||
} IsCitusTableType(relationId, RANGE_DISTRIBUTED))
{
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
"For MERGE command, all the distributed tables "
"must be colocated, for append/range distribution, "
"colocation is not supported", NULL,
"Consider using hash distribution instead");
}
/* Regular Postgres tables and Citus local tables are allowed */ distTablesList = lappend(distTablesList, rangeTableEntry);
if (!IsCitusTable(relationId) || }
IsCitusTableType(relationId, CITUS_LOCAL_TABLE)) else if (IsCitusTableType(relationId, REFERENCE_TABLE))
{ {
/* Reference table as a target is not allowed */
if (relationId == targetRelationId)
{
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
"Reference table as target "
"is not allowed in "
"MERGE command", NULL, NULL);
}
foundReferenceTables = true;
}
else if (IsCitusTableType(relationId, CITUS_LOCAL_TABLE))
{
/* Citus local tables */
foundLocalTables = true;
}
else if (!IsCitusTable(relationId))
{
/* Regular Postgres table */
foundLocalTables = true; foundLocalTables = true;
continue;
} }
/* Any other Citus table type missing ? */ /* Any other Citus table type missing ? */
} }
/* Ensure all tables are indeed local */ /* Ensure all tables are indeed local (or a combination of reference and local) */
if (foundLocalTables && list_length(distTablesList) == 0) if (list_length(distTablesList) == 0)
{ {
/* All the tables are local, supported */ /*
* All the tables are local/reference, supported as long as
* coordinator is in the metadata.
*/
if (FindCoordinatorNodeId() == -1)
{
elog(ERROR, "Coordinator node is not in the metadata. TODO better meesage");
}
/* All the tables are local/reference, supported */
return NULL; return NULL;
} }
else if (foundLocalTables && list_length(distTablesList) > 0)
if (foundLocalTables)
{ {
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
"MERGE command is not supported with " "MERGE command is not supported with "
@ -459,6 +433,17 @@ ErrorIfMergeHasUnsupportedTables(Query *parse, List *rangeTableList,
NULL, NULL); NULL, NULL);
} }
if (foundReferenceTables)
{
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
"MERGE command is not supported with "
"combination of distributed/reference yet",
NULL,
"If target is distributed, source "
"must be distributed and co-located");
}
/* Ensure all distributed tables are indeed co-located */ /* Ensure all distributed tables are indeed co-located */
return ErrorIfDistTablesNotColocated(parse, return ErrorIfDistTablesNotColocated(parse,
distTablesList, distTablesList,
@ -515,11 +500,16 @@ IsDistributionColumnInMergeSource(Expr *columnExpression, Query *query, bool
* prevent such mishaps, we disallow such inserts here. * prevent such mishaps, we disallow such inserts here.
*/ */
static DeferredErrorMessage * static DeferredErrorMessage *
InsertDistributionColumnMatchesSource(Query *query, RangeTblEntry *resultRte) InsertDistributionColumnMatchesSource(Oid targetRelationId, Query *query)
{ {
Assert(IsMergeQuery(query)); Assert(IsMergeQuery(query));
if (!IsCitusTableType(resultRte->relid, DISTRIBUTED_TABLE)) if (!IsCitusTableType(targetRelationId, DISTRIBUTED_TABLE))
{
return NULL;
}
if (!HasDistributionKey(targetRelationId))
{ {
return NULL; return NULL;
} }
@ -549,7 +539,7 @@ InsertDistributionColumnMatchesSource(Query *query, RangeTblEntry *resultRte)
} }
Assert(action->commandType == CMD_INSERT); Assert(action->commandType == CMD_INSERT);
Var *targetKey = PartitionColumn(resultRte->relid, 1); Var *targetKey = PartitionColumn(targetRelationId, 1);
TargetEntry *targetEntry = NULL; TargetEntry *targetEntry = NULL;
foreach_ptr(targetEntry, action->targetList) foreach_ptr(targetEntry, action->targetList)
@ -736,3 +726,34 @@ MergeQualAndTargetListFunctionsSupported(Oid resultRelationId, FromExpr *joinTre
#endif #endif
/*
* IsLocalTableModification returns true if the table modified is a Postgres table.
* We do not support recursive planning for MERGE yet, so we could have a join
* between local and Citus tables. Only allow local tables when it is the target table.
*/
bool
IsLocalTableModification(Oid targetRelationId, Query *query, uint64 shardId,
RTEListProperties *rteProperties)
{
/* No-op for SELECT command */
if (!IsModifyCommand(query))
{
return false;
}
/* For MERGE, we have to check only the target relation */
if (IsMergeQuery(query) && !IsCitusTable(targetRelationId))
{
/* Postgres table */
return true;
}
if (shardId == INVALID_SHARD_ID && ContainsOnlyLocalTables(rteProperties))
{
return true;
}
return false;
}

View File

@ -1404,7 +1404,7 @@ DistPartitionKeyOrError(Oid relationId)
if (partitionKey == NULL) if (partitionKey == NULL)
{ {
ereport(ERROR, (errmsg( ereport(ERROR, (errmsg(
"no distribution column found for relation %d, because it is a reference table", "no distribution column found for relation %d",
relationId))); relationId)));
} }

View File

@ -272,7 +272,7 @@ TargetListOnPartitionColumn(Query *query, List *targetEntryList)
if (!targetListOnPartitionColumn) if (!targetListOnPartitionColumn)
{ {
if (!FindNodeMatchingCheckFunctionInRangeTableList(query->rtable, if (!FindNodeMatchingCheckFunctionInRangeTableList(query->rtable,
IsDistributedTableRTE)) IsTableWithDistKeyRTE))
{ {
targetListOnPartitionColumn = true; targetListOnPartitionColumn = true;
} }
@ -379,6 +379,20 @@ IsReferenceTableRTE(Node *node)
} }
/*
* IsTableWithDistKeyRTE gets a node and returns true if the node
* is a range table relation entry that points to a distributed table
* that has a distribution column.
*/
bool
IsTableWithDistKeyRTE(Node *node)
{
Oid relationId = NodeTryGetRteRelid(node);
return relationId != InvalidOid && IsCitusTable(relationId) &&
HasDistributionKey(relationId);
}
/* /*
* FullCompositeFieldList gets a composite field list, and checks if all fields * FullCompositeFieldList gets a composite field list, and checks if all fields
* of composite type are used in the list. * of composite type are used in the list.

View File

@ -2487,7 +2487,7 @@ QueryPushdownTaskCreate(Query *originalQuery, int shardIndex,
/* non-distributed tables have only one shard */ /* non-distributed tables have only one shard */
shardInterval = cacheEntry->sortedShardIntervalArray[0]; shardInterval = cacheEntry->sortedShardIntervalArray[0];
/* only use reference table as anchor shard if none exists yet */ /* use as anchor shard only if we couldn't find any yet */
if (anchorShardId == INVALID_SHARD_ID) if (anchorShardId == INVALID_SHARD_ID)
{ {
anchorShardId = shardInterval->shardId; anchorShardId = shardInterval->shardId;

View File

@ -258,6 +258,22 @@ CreateModifyPlan(Query *originalQuery, Query *query,
} }
/*
* 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
WrapRouterErrorForSingleShardTable(DeferredErrorMessage *planningError)
{
planningError->detail = planningError->message;
planningError->message = pstrdup("queries that reference a distributed "
"table without a shard key can only "
"reference colocated distributed "
"tables or reference tables");
}
/* /*
* CreateSingleTaskRouterSelectPlan creates a physical plan for given SELECT query. * CreateSingleTaskRouterSelectPlan creates a physical plan for given SELECT query.
* The returned plan is a router task that returns query results from a single worker. * The returned plan is a router task that returns query results from a single worker.
@ -1870,6 +1886,11 @@ RouterJob(Query *originalQuery, PlannerRestrictionContext *plannerRestrictionCon
*/ */
if (IsMergeQuery(originalQuery)) if (IsMergeQuery(originalQuery))
{ {
if (ContainsSingleShardTable(originalQuery))
{
WrapRouterErrorForSingleShardTable(*planningError);
}
RaiseDeferredError(*planningError, ERROR); RaiseDeferredError(*planningError, ERROR);
} }
else else
@ -2246,10 +2267,8 @@ SelectsFromDistributedTable(List *rangeTableList, Query *query)
} }
static bool ContainsOnlyLocalTables(RTEListProperties *rteProperties);
/* /*
* RouterQuery runs router pruning logic for SELECT, UPDATE and DELETE queries. * RouterQuery runs router pruning logic for SELECT, UPDATE, DELETE, and MERGE queries.
* If there are shards present and query is routable, all RTEs have been updated * If there are shards present and query is routable, all RTEs have been updated
* to point to the relevant shards in the originalQuery. Also, placementList is * to point to the relevant shards in the originalQuery. Also, placementList is
* filled with the list of worker nodes that has all the required shard placements * filled with the list of worker nodes that has all the required shard placements
@ -2282,6 +2301,7 @@ PlanRouterQuery(Query *originalQuery,
DeferredErrorMessage *planningError = NULL; DeferredErrorMessage *planningError = NULL;
bool shardsPresent = false; bool shardsPresent = false;
CmdType commandType = originalQuery->commandType; CmdType commandType = originalQuery->commandType;
Oid targetRelationId = InvalidOid;
bool fastPathRouterQuery = bool fastPathRouterQuery =
plannerRestrictionContext->fastPathRestrictionContext->fastPathRouterQuery; plannerRestrictionContext->fastPathRestrictionContext->fastPathRouterQuery;
@ -2350,7 +2370,8 @@ PlanRouterQuery(Query *originalQuery,
if (IsMergeQuery(originalQuery)) if (IsMergeQuery(originalQuery))
{ {
planningError = MergeQuerySupported(originalQuery, targetRelationId = ModifyQueryResultRelationId(originalQuery);
planningError = MergeQuerySupported(targetRelationId, originalQuery,
isMultiShardQuery, isMultiShardQuery,
plannerRestrictionContext); plannerRestrictionContext);
} }
@ -2403,13 +2424,14 @@ PlanRouterQuery(Query *originalQuery,
/* both Postgres tables and materialized tables are locally avaliable */ /* both Postgres tables and materialized tables are locally avaliable */
RTEListProperties *rteProperties = GetRTEListPropertiesForQuery(originalQuery); RTEListProperties *rteProperties = GetRTEListPropertiesForQuery(originalQuery);
if (shardId == INVALID_SHARD_ID && ContainsOnlyLocalTables(rteProperties))
if (isLocalTableModification)
{ {
if (commandType != CMD_SELECT) *isLocalTableModification =
{ IsLocalTableModification(targetRelationId, originalQuery, shardId,
*isLocalTableModification = true; rteProperties);
}
} }
bool hasPostgresLocalRelation = bool hasPostgresLocalRelation =
rteProperties->hasPostgresLocalTable || rteProperties->hasMaterializedView; rteProperties->hasPostgresLocalTable || rteProperties->hasMaterializedView;
List *taskPlacementList = List *taskPlacementList =
@ -2447,7 +2469,7 @@ PlanRouterQuery(Query *originalQuery,
* ContainsOnlyLocalTables returns true if there is only * ContainsOnlyLocalTables returns true if there is only
* local tables and not any distributed or reference table. * local tables and not any distributed or reference table.
*/ */
static bool bool
ContainsOnlyLocalTables(RTEListProperties *rteProperties) ContainsOnlyLocalTables(RTEListProperties *rteProperties)
{ {
return !rteProperties->hasDistributedTable && !rteProperties->hasReferenceTable; return !rteProperties->hasDistributedTable && !rteProperties->hasReferenceTable;
@ -2683,7 +2705,7 @@ TargetShardIntervalForFastPathQuery(Query *query, bool *isMultiShardQuery,
if (!HasDistributionKey(relationId)) if (!HasDistributionKey(relationId))
{ {
/* we don't need to do shard pruning for non-distributed tables */ /* we don't need to do shard pruning for single shard tables */
return list_make1(LoadShardIntervalList(relationId)); return list_make1(LoadShardIntervalList(relationId));
} }
@ -2973,7 +2995,7 @@ BuildRoutesForInsert(Query *query, DeferredErrorMessage **planningError)
Assert(query->commandType == CMD_INSERT); Assert(query->commandType == CMD_INSERT);
/* reference tables and citus local tables can only have one shard */ /* tables that don't have distribution column can only have one shard */
if (!HasDistributionKeyCacheEntry(cacheEntry)) if (!HasDistributionKeyCacheEntry(cacheEntry))
{ {
List *shardIntervalList = LoadShardIntervalList(distributedTableId); List *shardIntervalList = LoadShardIntervalList(distributedTableId);
@ -2991,6 +3013,12 @@ BuildRoutesForInsert(Query *query, DeferredErrorMessage **planningError)
ereport(ERROR, (errmsg("local table cannot have %d shards", ereport(ERROR, (errmsg("local table cannot have %d shards",
shardCount))); shardCount)));
} }
else if (IsCitusTableTypeCacheEntry(cacheEntry, SINGLE_SHARD_DISTRIBUTED))
{
ereport(ERROR, (errmsg("distributed tables having a null shard key "
"cannot have %d shards",
shardCount)));
}
} }
ShardInterval *shardInterval = linitial(shardIntervalList); ShardInterval *shardInterval = linitial(shardIntervalList);
@ -3848,7 +3876,8 @@ ErrorIfQueryHasUnroutableModifyingCTE(Query *queryTree)
CitusTableCacheEntry *modificationTableCacheEntry = CitusTableCacheEntry *modificationTableCacheEntry =
GetCitusTableCacheEntry(distributedTableId); GetCitusTableCacheEntry(distributedTableId);
if (!HasDistributionKeyCacheEntry(modificationTableCacheEntry)) if (!IsCitusTableTypeCacheEntry(modificationTableCacheEntry,
DISTRIBUTED_TABLE))
{ {
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED, return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
"cannot router plan modification of a non-distributed table", "cannot router plan modification of a non-distributed table",

View File

@ -195,7 +195,7 @@ RecordRelationAccessIfNonDistTable(Oid relationId, ShardPlacementAccessType acce
* recursively calling RecordRelationAccessBase(), so becareful about * recursively calling RecordRelationAccessBase(), so becareful about
* removing this check. * removing this check.
*/ */
if (IsCitusTable(relationId) && HasDistributionKey(relationId)) if (IsCitusTableType(relationId, DISTRIBUTED_TABLE))
{ {
return; return;
} }
@ -732,7 +732,7 @@ CheckConflictingRelationAccesses(Oid relationId, ShardPlacementAccessType access
CitusTableCacheEntry *cacheEntry = GetCitusTableCacheEntry(relationId); CitusTableCacheEntry *cacheEntry = GetCitusTableCacheEntry(relationId);
if (HasDistributionKeyCacheEntry(cacheEntry) || if (IsCitusTableTypeCacheEntry(cacheEntry, DISTRIBUTED_TABLE) ||
cacheEntry->referencingRelationsViaForeignKey == NIL) cacheEntry->referencingRelationsViaForeignKey == NIL)
{ {
return; return;
@ -931,7 +931,7 @@ HoldsConflictingLockWithReferencedRelations(Oid relationId, ShardPlacementAccess
* We're only interested in foreign keys to reference tables and citus * We're only interested in foreign keys to reference tables and citus
* local tables. * local tables.
*/ */
if (IsCitusTable(referencedRelation) && HasDistributionKey(referencedRelation)) if (IsCitusTableType(referencedRelation, DISTRIBUTED_TABLE))
{ {
continue; continue;
} }
@ -993,7 +993,7 @@ HoldsConflictingLockWithReferencingRelations(Oid relationId, ShardPlacementAcces
CitusTableCacheEntry *cacheEntry = GetCitusTableCacheEntry(relationId); CitusTableCacheEntry *cacheEntry = GetCitusTableCacheEntry(relationId);
bool holdsConflictingLocks = false; bool holdsConflictingLocks = false;
Assert(!HasDistributionKeyCacheEntry(cacheEntry)); Assert(!IsCitusTableTypeCacheEntry(cacheEntry, DISTRIBUTED_TABLE));
Oid referencingRelation = InvalidOid; Oid referencingRelation = InvalidOid;
foreach_oid(referencingRelation, cacheEntry->referencingRelationsViaForeignKey) foreach_oid(referencingRelation, cacheEntry->referencingRelationsViaForeignKey)

View File

@ -1384,17 +1384,19 @@ EnsureTableCanBeColocatedWith(Oid relationId, char replicationModel,
Oid distributionColumnType, Oid sourceRelationId) Oid distributionColumnType, Oid sourceRelationId)
{ {
CitusTableCacheEntry *sourceTableEntry = GetCitusTableCacheEntry(sourceRelationId); CitusTableCacheEntry *sourceTableEntry = GetCitusTableCacheEntry(sourceRelationId);
char sourceReplicationModel = sourceTableEntry->replicationModel;
Var *sourceDistributionColumn = DistPartitionKeyOrError(sourceRelationId);
if (!IsCitusTableTypeCacheEntry(sourceTableEntry, HASH_DISTRIBUTED)) if (IsCitusTableTypeCacheEntry(sourceTableEntry, APPEND_DISTRIBUTED) ||
IsCitusTableTypeCacheEntry(sourceTableEntry, RANGE_DISTRIBUTED) ||
IsCitusTableTypeCacheEntry(sourceTableEntry, CITUS_LOCAL_TABLE))
{ {
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot distribute relation"), errmsg("cannot distribute relation"),
errdetail("Currently, colocate_with option is only supported " errdetail("Currently, colocate_with option is not supported "
"for hash distributed tables."))); "with append / range distributed tables and local "
"tables added to metadata.")));
} }
char sourceReplicationModel = sourceTableEntry->replicationModel;
if (sourceReplicationModel != replicationModel) if (sourceReplicationModel != replicationModel)
{ {
char *relationName = get_rel_name(relationId); char *relationName = get_rel_name(relationId);
@ -1406,7 +1408,9 @@ EnsureTableCanBeColocatedWith(Oid relationId, char replicationModel,
sourceRelationName, relationName))); sourceRelationName, relationName)));
} }
Oid sourceDistributionColumnType = sourceDistributionColumn->vartype; Var *sourceDistributionColumn = DistPartitionKey(sourceRelationId);
Oid sourceDistributionColumnType = !sourceDistributionColumn ? InvalidOid :
sourceDistributionColumn->vartype;
if (sourceDistributionColumnType != distributionColumnType) if (sourceDistributionColumnType != distributionColumnType)
{ {
char *relationName = get_rel_name(relationId); char *relationName = get_rel_name(relationId);

View File

@ -135,7 +135,7 @@ BuildDistributionKeyFromColumnName(Oid relationId, char *columnName, LOCKMODE lo
char *tableName = get_rel_name(relationId); char *tableName = get_rel_name(relationId);
/* short circuit for reference tables */ /* short circuit for reference tables and single-shard tables */
if (columnName == NULL) if (columnName == NULL)
{ {
return NULL; return NULL;

View File

@ -206,7 +206,7 @@ CompareRelationShards(const void *leftElement, const void *rightElement)
* *
* For hash partitioned tables, it calculates hash value of a number in its * For hash partitioned tables, it calculates hash value of a number in its
* range (e.g. min value) and finds which shard should contain the hashed * range (e.g. min value) and finds which shard should contain the hashed
* value. For reference tables and citus local tables, it simply returns 0. * value. For the tables that don't have a shard key, it simply returns 0.
* For the other table types, the function errors out. * For the other table types, the function errors out.
*/ */
int int
@ -231,12 +231,11 @@ ShardIndex(ShardInterval *shardInterval)
"tables that are added to citus metadata"))); "tables that are added to citus metadata")));
} }
/* short-circuit for reference tables */ /* short-circuit for the tables that don't have a distribution key */
if (!HasDistributionKeyCacheEntry(cacheEntry)) if (!HasDistributionKeyCacheEntry(cacheEntry))
{ {
/* /*
* Reference tables and citus local tables have only a single shard, * Such tables have only a single shard, so the index is fixed to 0.
* so the index is fixed to 0.
*/ */
shardIndex = 0; shardIndex = 0;

View File

@ -262,6 +262,8 @@ extern void CreateShardsWithRoundRobinPolicy(Oid distributedTableId, int32 shard
extern void CreateColocatedShards(Oid targetRelationId, Oid sourceRelationId, extern void CreateColocatedShards(Oid targetRelationId, Oid sourceRelationId,
bool useExclusiveConnections); bool useExclusiveConnections);
extern void CreateReferenceTableShard(Oid distributedTableId); extern void CreateReferenceTableShard(Oid distributedTableId);
extern void CreateSingleShardTableShardWithRoundRobinPolicy(Oid relationId,
uint32 colocationId);
extern List * WorkerCreateShardCommandList(Oid relationId, int shardIndex, uint64 shardId, extern List * WorkerCreateShardCommandList(Oid relationId, int shardIndex, uint64 shardId,
List *ddlCommandList, List *ddlCommandList,
List *foreignConstraintCommandList); List *foreignConstraintCommandList);

View File

@ -147,9 +147,19 @@ typedef struct RTEListProperties
bool hasReferenceTable; bool hasReferenceTable;
bool hasCitusLocalTable; bool hasCitusLocalTable;
/* includes hash, append and range partitioned tables */ /* includes hash, single-shard, append and range partitioned tables */
bool hasDistributedTable; bool hasDistributedTable;
/*
* Effectively, hasDistributedTable is equal to
* "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 hasSingleShardDistTable;
/* union of hasReferenceTable, hasCitusLocalTable and hasDistributedTable */ /* union of hasReferenceTable, hasCitusLocalTable and hasDistributedTable */
bool hasCitusTable; bool hasCitusTable;
@ -243,6 +253,7 @@ extern int32 BlessRecordExpression(Expr *expr);
extern void DissuadePlannerFromUsingPlan(PlannedStmt *plan); extern void DissuadePlannerFromUsingPlan(PlannedStmt *plan);
extern PlannedStmt * FinalizePlan(PlannedStmt *localPlan, extern PlannedStmt * FinalizePlan(PlannedStmt *localPlan,
struct DistributedPlan *distributedPlan); struct DistributedPlan *distributedPlan);
extern bool ContainsSingleShardTable(Query *query);
extern RTEListProperties * GetRTEListPropertiesForQuery(Query *query); extern RTEListProperties * GetRTEListPropertiesForQuery(Query *query);

View File

@ -19,13 +19,16 @@
#include "distributed/errormessage.h" #include "distributed/errormessage.h"
#include "distributed/multi_physical_planner.h" #include "distributed/multi_physical_planner.h"
extern bool IsMergeAllowedOnRelation(Query *parse, RangeTblEntry *rte); extern DeferredErrorMessage * MergeQuerySupported(Oid resultRelationId,
extern DeferredErrorMessage * MergeQuerySupported(Query *originalQuery, Query *originalQuery,
bool multiShardQuery, bool multiShardQuery,
PlannerRestrictionContext * PlannerRestrictionContext *
plannerRestrictionContext); plannerRestrictionContext);
extern DistributedPlan * CreateMergePlan(Query *originalQuery, Query *query, extern DistributedPlan * CreateMergePlan(Query *originalQuery, Query *query,
PlannerRestrictionContext * PlannerRestrictionContext *
plannerRestrictionContext); plannerRestrictionContext);
extern bool IsLocalTableModification(Oid targetRelationId, Query *query,
uint64 shardId,
RTEListProperties *rteProperties);
#endif /* MERGE_PLANNER_H */ #endif /* MERGE_PLANNER_H */

View File

@ -123,6 +123,7 @@ typedef enum
HASH_DISTRIBUTED, HASH_DISTRIBUTED,
APPEND_DISTRIBUTED, APPEND_DISTRIBUTED,
RANGE_DISTRIBUTED, RANGE_DISTRIBUTED,
SINGLE_SHARD_DISTRIBUTED,
/* hash, range or append distributed table */ /* hash, range or append distributed table */
DISTRIBUTED_TABLE, DISTRIBUTED_TABLE,
@ -157,6 +158,8 @@ extern uint32 ColocationIdViaCatalog(Oid relationId);
bool IsReferenceTableByDistParams(char partitionMethod, char replicationModel); bool IsReferenceTableByDistParams(char partitionMethod, char replicationModel);
extern bool IsCitusLocalTableByDistParams(char partitionMethod, char replicationModel, extern bool IsCitusLocalTableByDistParams(char partitionMethod, char replicationModel,
uint32 colocationId); uint32 colocationId);
extern bool IsSingleShardTableByDistParams(char partitionMethod, char replicationModel,
uint32 colocationId);
extern List * CitusTableList(void); extern List * CitusTableList(void);
extern ShardInterval * LoadShardInterval(uint64 shardId); extern ShardInterval * LoadShardInterval(uint64 shardId);
extern bool ShardExists(uint64 shardId); extern bool ShardExists(uint64 shardId);

View File

@ -156,6 +156,7 @@ extern void SendOrCollectCommandListToSingleNode(MetadataSyncContext *context,
extern void ActivateNodeList(MetadataSyncContext *context); extern void ActivateNodeList(MetadataSyncContext *context);
extern char * WorkerDropAllShellTablesCommand(bool singleTransaction); extern char * WorkerDropAllShellTablesCommand(bool singleTransaction);
extern char * WorkerDropSequenceDependencyCommand(Oid relationId);
extern void SyncDistributedObjects(MetadataSyncContext *context); extern void SyncDistributedObjects(MetadataSyncContext *context);
extern void SendNodeWideObjectsSyncCommands(MetadataSyncContext *context); extern void SendNodeWideObjectsSyncCommands(MetadataSyncContext *context);
@ -180,8 +181,10 @@ extern void SendInterTableRelationshipCommands(MetadataSyncContext *context);
#define REMOVE_ALL_CITUS_TABLES_COMMAND \ #define REMOVE_ALL_CITUS_TABLES_COMMAND \
"SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition" "SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition"
#define BREAK_CITUS_TABLE_SEQUENCE_DEPENDENCY_COMMAND \ #define BREAK_ALL_CITUS_TABLE_SEQUENCE_DEPENDENCY_COMMAND \
"SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition" "SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition"
#define BREAK_CITUS_TABLE_SEQUENCE_DEPENDENCY_COMMAND \
"SELECT pg_catalog.worker_drop_sequence_dependency(%s);"
#define DISABLE_DDL_PROPAGATION "SET citus.enable_ddl_propagation TO 'off'" #define DISABLE_DDL_PROPAGATION "SET citus.enable_ddl_propagation TO 'off'"
#define ENABLE_DDL_PROPAGATION "SET citus.enable_ddl_propagation TO 'on'" #define ENABLE_DDL_PROPAGATION "SET citus.enable_ddl_propagation TO 'on'"

View File

@ -326,6 +326,7 @@ extern void DeletePartitionRow(Oid distributedRelationId);
extern void DeleteShardRow(uint64 shardId); extern void DeleteShardRow(uint64 shardId);
extern void UpdatePlacementGroupId(uint64 placementId, int groupId); extern void UpdatePlacementGroupId(uint64 placementId, int groupId);
extern void DeleteShardPlacementRow(uint64 placementId); extern void DeleteShardPlacementRow(uint64 placementId);
extern void CreateSingleShardTable(Oid relationId, char *colocateWithTableName);
extern void CreateDistributedTable(Oid relationId, char *distributionColumnName, extern void CreateDistributedTable(Oid relationId, char *distributionColumnName,
char distributionMethod, int shardCount, char distributionMethod, int shardCount,
bool shardCountIsStrict, char *colocateWithTableName); bool shardCountIsStrict, char *colocateWithTableName);

View File

@ -200,6 +200,7 @@ extern bool IsCitusTableRTE(Node *node);
extern bool IsDistributedOrReferenceTableRTE(Node *node); extern bool IsDistributedOrReferenceTableRTE(Node *node);
extern bool IsDistributedTableRTE(Node *node); extern bool IsDistributedTableRTE(Node *node);
extern bool IsReferenceTableRTE(Node *node); extern bool IsReferenceTableRTE(Node *node);
extern bool IsTableWithDistKeyRTE(Node *node);
extern bool IsCitusExtraDataContainerRelation(RangeTblEntry *rte); extern bool IsCitusExtraDataContainerRelation(RangeTblEntry *rte);
extern bool ContainsReadIntermediateResultFunction(Node *node); extern bool ContainsReadIntermediateResultFunction(Node *node);
extern bool ContainsReadIntermediateResultArrayFunction(Node *node); extern bool ContainsReadIntermediateResultArrayFunction(Node *node);

View File

@ -36,6 +36,7 @@ extern DistributedPlan * CreateRouterPlan(Query *originalQuery, Query *query,
extern DistributedPlan * CreateModifyPlan(Query *originalQuery, Query *query, extern DistributedPlan * CreateModifyPlan(Query *originalQuery, Query *query,
PlannerRestrictionContext * PlannerRestrictionContext *
plannerRestrictionContext); plannerRestrictionContext);
extern void WrapRouterErrorForSingleShardTable(DeferredErrorMessage *planningError);
extern DeferredErrorMessage * PlanRouterQuery(Query *originalQuery, extern DeferredErrorMessage * PlanRouterQuery(Query *originalQuery,
PlannerRestrictionContext * PlannerRestrictionContext *
plannerRestrictionContext, plannerRestrictionContext,
@ -117,5 +118,6 @@ extern bool HasDangerousJoinUsing(List *rtableList, Node *jtnode);
extern Job * RouterJob(Query *originalQuery, extern Job * RouterJob(Query *originalQuery,
PlannerRestrictionContext *plannerRestrictionContext, PlannerRestrictionContext *plannerRestrictionContext,
DeferredErrorMessage **planningError); DeferredErrorMessage **planningError);
extern bool ContainsOnlyLocalTables(RTEListProperties *rteProperties);
#endif /* MULTI_ROUTER_PLANNER_H */ #endif /* MULTI_ROUTER_PLANNER_H */

View File

@ -19,4 +19,6 @@
#define Natts_pg_dist_node_metadata 1 #define Natts_pg_dist_node_metadata 1
#define Anum_pg_dist_node_metadata_metadata 1 #define Anum_pg_dist_node_metadata_metadata 1
extern int FindCoordinatorNodeId(void);
#endif /* PG_DIST_NODE_METADATA_H */ #endif /* PG_DIST_NODE_METADATA_H */

View File

@ -264,7 +264,7 @@ check-pytest:
pytest -n auto pytest -n auto
check-query-generator: all check-query-generator: all
${query_generator_check} --bindir=$(bindir) --pgxsdir=$(pgxsdir) ${query_generator_check} --bindir=$(bindir) --pgxsdir=$(pgxsdir) --seed=$(seed)
check-citus-upgrade: all check-citus-upgrade: all
$(citus_upgrade_check) \ $(citus_upgrade_check) \

View File

@ -76,6 +76,19 @@ def run_for_config(config, lock, sql_schedule_name):
cfg.SUPER_USER_NAME, cfg.SUPER_USER_NAME,
) )
common.save_regression_diff("postgres", config.output_dir) common.save_regression_diff("postgres", config.output_dir)
elif config.all_null_dist_key:
exitCode |= common.run_pg_regress_without_exit(
config.bindir,
config.pg_srcdir,
config.coordinator_port(),
cfg.SINGLE_SHARD_PREP_SCHEDULE,
config.output_dir,
config.input_dir,
cfg.SUPER_USER_NAME,
)
common.save_regression_diff(
"single_shard_table_prep_regression", config.output_dir
)
exitCode |= _run_pg_regress_on_port( exitCode |= _run_pg_regress_on_port(
config, config.coordinator_port(), cfg.CREATE_SCHEDULE config, config.coordinator_port(), cfg.CREATE_SCHEDULE

View File

@ -22,6 +22,7 @@ ARBITRARY_SCHEDULE_NAMES = [
"sql_schedule", "sql_schedule",
"sql_base_schedule", "sql_base_schedule",
"postgres_schedule", "postgres_schedule",
"single_shard_table_prep_schedule",
] ]
BEFORE_PG_UPGRADE_SCHEDULE = "./before_pg_upgrade_schedule" BEFORE_PG_UPGRADE_SCHEDULE = "./before_pg_upgrade_schedule"
@ -29,6 +30,7 @@ AFTER_PG_UPGRADE_SCHEDULE = "./after_pg_upgrade_schedule"
CREATE_SCHEDULE = "./create_schedule" CREATE_SCHEDULE = "./create_schedule"
POSTGRES_SCHEDULE = "./postgres_schedule" POSTGRES_SCHEDULE = "./postgres_schedule"
SINGLE_SHARD_PREP_SCHEDULE = "./single_shard_table_prep_schedule"
SQL_SCHEDULE = "./sql_schedule" SQL_SCHEDULE = "./sql_schedule"
SQL_BASE_SCHEDULE = "./sql_base_schedule" SQL_BASE_SCHEDULE = "./sql_base_schedule"
@ -101,6 +103,7 @@ class CitusBaseClusterConfig(object, metaclass=NewInitCaller):
self.dbname = DATABASE_NAME self.dbname = DATABASE_NAME
self.is_mx = True self.is_mx = True
self.is_citus = True self.is_citus = True
self.all_null_dist_key = False
self.name = type(self).__name__ self.name = type(self).__name__
self.settings = { self.settings = {
"shared_preload_libraries": "citus", "shared_preload_libraries": "citus",
@ -203,6 +206,43 @@ class PostgresConfig(CitusDefaultClusterConfig):
] ]
class AllSingleShardTableDefaultConfig(CitusDefaultClusterConfig):
def __init__(self, arguments):
super().__init__(arguments)
self.all_null_dist_key = True
self.skip_tests += [
# i) Skip the following tests because they require SQL support beyond
# router planner / supporting more DDL command types.
#
# group 1
"dropped_columns_create_load",
"dropped_columns_1",
# group 2
"distributed_planning_create_load",
"distributed_planning",
# group 4
"views_create",
"views",
# group 5
"intermediate_result_pruning_create",
"intermediate_result_pruning_queries_1",
"intermediate_result_pruning_queries_2",
# group 6
"local_dist_join_load",
"local_dist_join",
"arbitrary_configs_recurring_outer_join",
# group 7
"sequences_create",
"sequences",
# group 8
"function_create",
"functions",
#
# ii) Skip the following test as it requires support for create_distributed_function.
"nested_execution",
]
class CitusSingleNodeClusterConfig(CitusDefaultClusterConfig): class CitusSingleNodeClusterConfig(CitusDefaultClusterConfig):
def __init__(self, arguments): def __init__(self, arguments):
super().__init__(arguments) super().__init__(arguments)

View File

@ -20,10 +20,12 @@ citus_dev make testCluster --destroy
2. Run the test, 2. Run the test,
```bash ```bash
cd src/test/regress/citus_tests/query_generator/bin cd src/test/regress/citus_tests/query_generator/bin
bash citus_compare_dist_local_joins.sh <username> <dbname> <coordinator_port> bash citus_compare_dist_local_joins.sh <username> <dbname> <coordinator_port> Optional:<seed>
``` ```
3. See the diff content in `src/test/regress/citus_tests/query_generator/out/local_dist_diffs` 3. See the diff content in `src/test/regress/citus_tests/query_generator/out/local_dist_diffs`
Note: `seed` can be used to reproduce a run of Citus test by generating the same queries and results via the given seed.
### Configuration ### Configuration
You can configure 3 different parts: You can configure 3 different parts:

View File

@ -6,6 +6,7 @@ set -euo pipefail
psql_user=$1 psql_user=$1
psql_db=$2 psql_db=$2
psql_port=$3 psql_port=$3
seed=${4:-""}
runDDLs() runDDLs()
{ {
@ -39,7 +40,7 @@ showDiffs()
# run query generator and let it create output ddls and queries # run query generator and let it create output ddls and queries
script_folder=$(dirname "$0") script_folder=$(dirname "$0")
out_folder="${script_folder}"/../out out_folder="${script_folder}"/../out
pushd . && cd "${script_folder}"/.. && python3 generate_queries.py && popd pushd . && cd "${script_folder}"/.. && python3 generate_queries.py --seed="${seed}" && popd
# remove result files if exists # remove result files if exists
rm -rf "${out_folder}"/dist_queries.out "${out_folder}"/local_queries.out rm -rf "${out_folder}"/dist_queries.out "${out_folder}"/local_queries.out

View File

@ -2,11 +2,12 @@
"""query_gen_test """query_gen_test
Usage: Usage:
run_query_compare_test --bindir=<bindir> --pgxsdir=<pgxsdir> run_query_compare_test --bindir=<bindir> --pgxsdir=<pgxsdir> --seed=<seed>
Options: Options:
--bindir=<bindir> PostgreSQL executable directory(ex: '~/.pgenv/pgsql-10.4/bin') --bindir=<bindir> PostgreSQL executable directory(ex: '~/.pgenv/pgsql-10.4/bin')
--pgxsdir=<pgxsdir> Path to the PGXS directory(ex: ~/.pgenv/src/postgresql-11.3) --pgxsdir=<pgxsdir> Path to the PGXS directory(ex: ~/.pgenv/src/postgresql-11.3)
--seed=<seed> Seed number used by the query generator.(ex: 123)
""" """
import os import os
@ -27,7 +28,7 @@ import common # noqa: E402
import config as cfg # noqa: E402 import config as cfg # noqa: E402
def run_test(config): def run_test(config, seed):
# start cluster # start cluster
common.initialize_temp_dir(cfg.CITUS_ARBITRARY_TEST_DIR) common.initialize_temp_dir(cfg.CITUS_ARBITRARY_TEST_DIR)
common.initialize_citus_cluster( common.initialize_citus_cluster(
@ -36,8 +37,8 @@ def run_test(config):
# run test # run test
scriptDirPath = os.path.dirname(os.path.abspath(__file__)) scriptDirPath = os.path.dirname(os.path.abspath(__file__))
testRunCommand = "bash {}/citus_compare_dist_local_joins.sh {} {} {}".format( testRunCommand = "bash {}/citus_compare_dist_local_joins.sh {} {} {} {}".format(
scriptDirPath, config.user, config.dbname, config.coordinator_port() scriptDirPath, config.user, config.dbname, config.coordinator_port(), seed
) )
process = subprocess.Popen( process = subprocess.Popen(
testRunCommand.split(), stdout=subprocess.PIPE, stderr=subprocess.PIPE testRunCommand.split(), stdout=subprocess.PIPE, stderr=subprocess.PIPE
@ -56,8 +57,10 @@ def run_test(config):
if __name__ == "__main__": if __name__ == "__main__":
citusClusterConfig = cfg.CitusSuperUserDefaultClusterConfig( arguments = docopt(__doc__, version="run_query_compare_test")
docopt(__doc__, version="run_query_compare_test") citusClusterConfig = cfg.CitusSuperUserDefaultClusterConfig(arguments)
)
run_test(citusClusterConfig) seed = ""
if "--seed" in arguments and arguments["--seed"] != "":
seed = arguments["--seed"]
run_test(citusClusterConfig, seed)

View File

@ -1,4 +1,5 @@
import copy import copy
import os
import yaml import yaml
from config.config_parser import ( from config.config_parser import (
@ -13,7 +14,9 @@ from node_defs import CitusType
class Config: class Config:
def __init__(self): def __init__(self):
configObj = Config.parseConfigFile("config/config.yaml") configObj = Config.parseConfigFile(
f"{os.path.dirname(os.path.abspath(__file__))}/config.yaml"
)
self.targetTables = _distinctCopyTables( self.targetTables = _distinctCopyTables(
parseTableArray(configObj["targetTables"]) parseTableArray(configObj["targetTables"])

View File

@ -12,6 +12,7 @@ def getTableData():
for table in tables: for table in tables:
# generate base rows # generate base rows
dataGenerationSql += _genOverlappingData(table.name, fromVal, table.rowCount) dataGenerationSql += _genOverlappingData(table.name, fromVal, table.rowCount)
dataGenerationSql += "\n"
dataGenerationSql += _genNonOverlappingData(table.name, toVal, tableIdx) dataGenerationSql += _genNonOverlappingData(table.name, toVal, tableIdx)
dataGenerationSql += "\n" dataGenerationSql += "\n"

View File

@ -1,9 +1,23 @@
#!/usr/bin/env python3
"""generate_queries
Usage:
generate_queries --seed=<seed>
Options:
--seed=<seed> Seed number used by the query generator.(ex: 123)
"""
import os
import random
import signal import signal
import sys import sys
from data_gen import getTableData from data_gen import getTableData
from ddl_gen import getTableDDLs from ddl_gen import getTableDDLs
from docopt import docopt
from query_gen import newQuery from query_gen import newQuery
from random_selections import currentMilliSecs
from config.config import getConfig, resetConfig from config.config import getConfig, resetConfig
@ -29,12 +43,16 @@ def _interactiveMode(ddls, data):
def _fileMode(ddls, data): def _fileMode(ddls, data):
ddlFileName = "out/" + getConfig().ddlOutFile ddlFileName = (
f"{os.path.dirname(os.path.abspath(__file__))}/out/{getConfig().ddlOutFile}"
)
with open(ddlFileName, "w") as ddlFile: with open(ddlFileName, "w") as ddlFile:
ddlFile.writelines([ddls, data]) ddlFile.writelines([ddls, data])
queryCount = getConfig().queryCount queryCount = getConfig().queryCount
fileName = "out/" + getConfig().queryOutFile fileName = (
f"{os.path.dirname(os.path.abspath(__file__))}/out/{getConfig().queryOutFile}"
)
with open(fileName, "w") as f: with open(fileName, "w") as f:
# enable repartition joins due to https://github.com/citusdata/citus/issues/6865 # enable repartition joins due to https://github.com/citusdata/citus/issues/6865
enableRepartitionJoinCommand = "SET citus.enable_repartition_joins TO on;\n" enableRepartitionJoinCommand = "SET citus.enable_repartition_joins TO on;\n"
@ -54,6 +72,17 @@ def _fileMode(ddls, data):
if __name__ == "__main__": if __name__ == "__main__":
signal.signal(signal.SIGINT, _signal_handler) signal.signal(signal.SIGINT, _signal_handler)
arguments = docopt(__doc__, version="generate_queries")
seed = -1
if "--seed" in arguments and arguments["--seed"] != "":
seed = int(arguments["--seed"])
else:
seed = currentMilliSecs()
assert seed > 0
random.seed(seed)
print(f"---SEED: {seed} ---")
resetConfig() resetConfig()
ddls = getTableDDLs() ddls = getTableDDLs()

View File

@ -1,4 +1,5 @@
import random import random
import time
from node_defs import RestrictOp from node_defs import RestrictOp
@ -10,6 +11,11 @@ def shouldSelectThatBranch():
return random.randint(0, 1) return random.randint(0, 1)
def currentMilliSecs():
"""returns total milliseconds since epoch"""
return round(time.time() * 1000)
def randomRteType(): def randomRteType():
"""returns a randomly selected RteType given at config""" """returns a randomly selected RteType given at config"""
rtes = getConfig().targetRteTypes rtes = getConfig().targetRteTypes

View File

@ -128,6 +128,7 @@ DEPS = {
"multi_mx_copy_data": TestDeps(None, ["multi_mx_create_table"]), "multi_mx_copy_data": TestDeps(None, ["multi_mx_create_table"]),
"multi_mx_schema_support": TestDeps(None, ["multi_mx_copy_data"]), "multi_mx_schema_support": TestDeps(None, ["multi_mx_copy_data"]),
"multi_simple_queries": TestDeps("base_schedule"), "multi_simple_queries": TestDeps("base_schedule"),
"create_single_shard_table": TestDeps("minimal_schedule"),
} }
@ -154,6 +155,8 @@ def run_python_test(test_name, args):
def run_regress_test(test_name, args): def run_regress_test(test_name, args):
original_schedule, schedule_line = find_test_schedule_and_line(test_name, args) original_schedule, schedule_line = find_test_schedule_and_line(test_name, args)
print(f"SCHEDULE: {original_schedule}")
print(f"SCHEDULE_LINE: {schedule_line}")
dependencies = test_dependencies(test_name, original_schedule, schedule_line, args) dependencies = test_dependencies(test_name, original_schedule, schedule_line, args)
@ -287,7 +290,7 @@ def get_test_name(args):
def find_test_schedule_and_line(test_name, args): def find_test_schedule_and_line(test_name, args):
for schedule_file_path in sorted(REGRESS_DIR.glob("*_schedule")): for schedule_file_path in sorted(REGRESS_DIR.glob("*_schedule")):
for schedule_line in open(schedule_file_path, "r"): for schedule_line in open(schedule_file_path, "r"):
if re.search(r"\b" + test_name + r"\b", schedule_line): if re.search(r"^test:.*\b" + test_name + r"\b", schedule_line):
test_schedule = pathlib.Path(schedule_file_path).stem test_schedule = pathlib.Path(schedule_file_path).stem
if args["use_whole_schedule_line"]: if args["use_whole_schedule_line"]:
return test_schedule, schedule_line return test_schedule, schedule_line

View File

@ -0,0 +1,154 @@
CREATE SCHEMA alter_null_dist_key;
SET search_path TO alter_null_dist_key;
SET citus.next_shard_id TO 1720000;
SET citus.shard_count TO 32;
SET citus.shard_replication_factor TO 1;
CREATE SEQUENCE dist_seq;
CREATE TABLE null_dist_table(a bigint DEFAULT nextval('dist_seq') UNIQUE, "b" text, c bigint GENERATED BY DEFAULT AS IDENTITY);
INSERT INTO null_dist_table("b") VALUES ('test');
SELECT create_distributed_table('null_dist_table', null, colocate_with=>'none', distribution_type=>null);
NOTICE: Copying data from local table...
NOTICE: copying the data has completed
DETAIL: The local data in the table is no longer visible, but is still on disk.
HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$alter_null_dist_key.null_dist_table$$)
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- add column
ALTER TABLE null_dist_table ADD COLUMN d bigint DEFAULT 2;
SELECT * FROM null_dist_table ORDER BY c;
a | b | c | d
---------------------------------------------------------------------
1 | test | 1 | 2
(1 row)
-- alter default, set to 3
ALTER TABLE null_dist_table ALTER COLUMN d SET DEFAULT 3;
INSERT INTO null_dist_table("b") VALUES ('test');
SELECT * FROM null_dist_table ORDER BY c;
a | b | c | d
---------------------------------------------------------------------
1 | test | 1 | 2
2 | test | 2 | 3
(2 rows)
-- drop default, see null
ALTER TABLE null_dist_table ALTER COLUMN d DROP DEFAULT;
INSERT INTO null_dist_table("b") VALUES ('test');
SELECT * FROM null_dist_table ORDER BY c;
a | b | c | d
---------------------------------------------------------------------
1 | test | 1 | 2
2 | test | 2 | 3
3 | test | 3 |
(3 rows)
-- cleanup the rows that were added to test the default behavior
DELETE FROM null_dist_table WHERE "b" = 'test' AND a > 1;
-- alter column type
ALTER TABLE null_dist_table ALTER COLUMN d TYPE text;
UPDATE null_dist_table SET d = 'this is a text' WHERE d = '2';
SELECT * FROM null_dist_table ORDER BY c;
a | b | c | d
---------------------------------------------------------------------
1 | test | 1 | this is a text
(1 row)
-- drop seq column
ALTER TABLE null_dist_table DROP COLUMN a;
SELECT * FROM null_dist_table ORDER BY c;
b | c | d
---------------------------------------------------------------------
test | 1 | this is a text
(1 row)
-- add not null constraint
ALTER TABLE null_dist_table ALTER COLUMN b SET NOT NULL;
-- not null constraint violation, error out
INSERT INTO null_dist_table VALUES (NULL, 2, 'test');
ERROR: null value in column "b" violates not-null constraint
DETAIL: Failing row contains (null, 2, test).
CONTEXT: while executing command on localhost:xxxxx
-- drop not null constraint and try again
ALTER TABLE null_dist_table ALTER COLUMN b DROP NOT NULL;
INSERT INTO null_dist_table VALUES (NULL, 3, 'test');
SELECT * FROM null_dist_table ORDER BY c;
b | c | d
---------------------------------------------------------------------
test | 1 | this is a text
| 3 | test
(2 rows)
-- add exclusion constraint
ALTER TABLE null_dist_table ADD CONSTRAINT exc_b EXCLUDE USING btree (b with =);
-- rename the exclusion constraint, errors out
ALTER TABLE null_dist_table RENAME CONSTRAINT exc_b TO exc_b_1;
ERROR: renaming constraints belonging to distributed tables is currently unsupported
-- create exclusion constraint without a name
ALTER TABLE null_dist_table ADD EXCLUDE USING btree (b with =);
-- test setting autovacuum option
ALTER TABLE null_dist_table SET (autovacuum_enabled = false);
-- test multiple subcommands
ALTER TABLE null_dist_table ADD COLUMN int_column1 INTEGER,
DROP COLUMN d;
SELECT * FROM null_dist_table ORDER BY c;
b | c | int_column1
---------------------------------------------------------------------
test | 1 |
| 3 |
(2 rows)
-- test policy and row level security
CREATE TABLE null_dist_key_with_policy (table_user text);
INSERT INTO null_dist_key_with_policy VALUES ('user_1');
SELECT create_distributed_table('null_dist_key_with_policy', null);
NOTICE: Copying data from local table...
NOTICE: copying the data has completed
DETAIL: The local data in the table is no longer visible, but is still on disk.
HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$alter_null_dist_key.null_dist_key_with_policy$$)
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- enable rls
ALTER TABLE null_dist_key_with_policy ENABLE ROW LEVEL SECURITY;
-- user_1 will be allowed to see the inserted row
CREATE ROLE user_1 WITH LOGIN;
GRANT ALL ON SCHEMA alter_null_dist_key TO user_1;
GRANT ALL ON TABLE alter_null_dist_key.null_dist_key_with_policy TO user_1;
CREATE POLICY table_policy ON null_dist_key_with_policy TO user_1
USING (table_user = current_user);
-- user_2 will not be allowed to see the inserted row
CREATE ROLE user_2 WITH LOGIN;
GRANT ALL ON SCHEMA alter_null_dist_key TO user_2;
GRANT ALL ON TABLE alter_null_dist_key.null_dist_key_with_policy TO user_2;
CREATE POLICY table_policy_1 ON null_dist_key_with_policy TO user_2
USING (table_user = current_user);
\c - user_1 -
SELECT * FROM alter_null_dist_key.null_dist_key_with_policy;
table_user
---------------------------------------------------------------------
user_1
(1 row)
\c - user_2 -
SELECT * FROM alter_null_dist_key.null_dist_key_with_policy;
table_user
---------------------------------------------------------------------
(0 rows)
-- postgres will always be allowed to see the row as a superuser
\c - postgres -
SELECT * FROM alter_null_dist_key.null_dist_key_with_policy;
table_user
---------------------------------------------------------------------
user_1
(1 row)
-- cleanup
SET client_min_messages TO ERROR;
DROP SCHEMA alter_null_dist_key CASCADE;
DROP ROLE user_1, user_2;

File diff suppressed because it is too large Load Diff

View File

@ -19,10 +19,30 @@ SET client_min_messages TO ERROR;
-- Create roles -- Create roles
CREATE ROLE foo1; CREATE ROLE foo1;
CREATE ROLE foo2; CREATE ROLE foo2;
-- Create collation
CREATE COLLATION german_phonebook (provider = icu, locale = 'de-u-co-phonebk');
-- Create type
CREATE TYPE pair_type AS (a int, b int);
-- Create function
CREATE FUNCTION one_as_result() RETURNS INT LANGUAGE SQL AS
$$
SELECT 1;
$$;
-- Create text search dictionary
CREATE TEXT SEARCH DICTIONARY my_german_dict (
template = snowball,
language = german,
stopwords = german
);
-- Create text search config
CREATE TEXT SEARCH CONFIGURATION my_ts_config ( parser = default );
ALTER TEXT SEARCH CONFIGURATION my_ts_config ALTER MAPPING FOR asciiword WITH my_german_dict;
-- Create sequence -- Create sequence
CREATE SEQUENCE seq; CREATE SEQUENCE seq;
-- Create colocated distributed tables -- Create colocated distributed tables
CREATE TABLE dist1 (id int PRIMARY KEY default nextval('seq')); CREATE TABLE dist1 (id int PRIMARY KEY default nextval('seq'), col int default (one_as_result()), myserial serial, phone text COLLATE german_phonebook, initials pair_type);
CREATE SEQUENCE seq_owned OWNED BY dist1.id;
CREATE INDEX dist1_search_phone_idx ON dist1 USING gin (to_tsvector('my_ts_config'::regconfig, (COALESCE(phone, ''::text))::text));
SELECT create_distributed_table('dist1', 'id'); SELECT create_distributed_table('dist1', 'id');
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -52,6 +72,8 @@ CREATE TABLE loc1 (id int PRIMARY KEY);
INSERT INTO loc1 SELECT i FROM generate_series(1,100) i; INSERT INTO loc1 SELECT i FROM generate_series(1,100) i;
CREATE TABLE loc2 (id int REFERENCES loc1(id)); CREATE TABLE loc2 (id int REFERENCES loc1(id));
INSERT INTO loc2 SELECT i FROM generate_series(1,100) i; INSERT INTO loc2 SELECT i FROM generate_series(1,100) i;
-- Create publication
CREATE PUBLICATION pub_all;
-- citus_set_coordinator_host with wrong port -- citus_set_coordinator_host with wrong port
SELECT citus_set_coordinator_host('localhost', 9999); SELECT citus_set_coordinator_host('localhost', 9999);
citus_set_coordinator_host citus_set_coordinator_host
@ -168,8 +190,8 @@ SELECT citus.mitmproxy('conn.onQuery(query="INSERT INTO pg_dist_node").kill()');
SELECT citus_activate_node('localhost', :worker_2_proxy_port); SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open
-- Failure to drop sequence -- Failure to drop sequence dependency for all tables
SELECT citus.mitmproxy('conn.onQuery(query="SELECT pg_catalog.worker_drop_sequence_dependency").cancel(' || :pid || ')'); SELECT citus.mitmproxy('conn.onQuery(query="SELECT pg_catalog.worker_drop_sequence_dependency.*FROM pg_dist_partition").cancel(' || :pid || ')');
mitmproxy mitmproxy
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -177,7 +199,7 @@ SELECT citus.mitmproxy('conn.onQuery(query="SELECT pg_catalog.worker_drop_sequen
SELECT citus_activate_node('localhost', :worker_2_proxy_port); SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: canceling statement due to user request ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.onQuery(query="SELECT pg_catalog.worker_drop_sequence_dependency").kill()'); SELECT citus.mitmproxy('conn.onQuery(query="SELECT pg_catalog.worker_drop_sequence_dependency.*FROM pg_dist_partition").kill()');
mitmproxy mitmproxy
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -321,7 +343,7 @@ SELECT citus.mitmproxy('conn.onQuery(query="ALTER DATABASE.*OWNER TO").kill()');
SELECT citus_activate_node('localhost', :worker_2_proxy_port); SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open
-- Filure to create schema -- Failure to create schema
SELECT citus.mitmproxy('conn.onQuery(query="CREATE SCHEMA IF NOT EXISTS mx_metadata_sync_multi_trans AUTHORIZATION").cancel(' || :pid || ')'); SELECT citus.mitmproxy('conn.onQuery(query="CREATE SCHEMA IF NOT EXISTS mx_metadata_sync_multi_trans AUTHORIZATION").cancel(' || :pid || ')');
mitmproxy mitmproxy
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -336,6 +358,108 @@ SELECT citus.mitmproxy('conn.onQuery(query="CREATE SCHEMA IF NOT EXISTS mx_metad
(1 row) (1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open
-- Failure to create collation
SELECT citus.mitmproxy('conn.onQuery(query="SELECT worker_create_or_replace_object.*CREATE COLLATION mx_metadata_sync_multi_trans.german_phonebook").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.onQuery(query="SELECT worker_create_or_replace_object.*CREATE COLLATION mx_metadata_sync_multi_trans.german_phonebook").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open
-- Failure to create function
SELECT citus.mitmproxy('conn.onQuery(query="CREATE OR REPLACE FUNCTION mx_metadata_sync_multi_trans.one_as_result").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.onQuery(query="CREATE OR REPLACE FUNCTION mx_metadata_sync_multi_trans.one_as_result").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open
-- Failure to create text search dictionary
SELECT citus.mitmproxy('conn.onQuery(query="SELECT worker_create_or_replace_object.*my_german_dict").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.onQuery(query="SELECT worker_create_or_replace_object.*my_german_dict").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open
-- Failure to create text search config
SELECT citus.mitmproxy('conn.onQuery(query="SELECT worker_create_or_replace_object.*my_ts_config").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.onQuery(query="SELECT worker_create_or_replace_object.*my_ts_config").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open
-- Failure to create type
SELECT citus.mitmproxy('conn.onQuery(query="SELECT worker_create_or_replace_object.*pair_type").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.onQuery(query="SELECT worker_create_or_replace_object.*pair_type").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open
-- Failure to create publication
SELECT citus.mitmproxy('conn.onQuery(query="CREATE PUBLICATION.*pub_all").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.onQuery(query="CREATE PUBLICATION.*pub_all").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port); SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open
-- Failure to create sequence -- Failure to create sequence
@ -353,6 +477,40 @@ SELECT citus.mitmproxy('conn.onQuery(query="SELECT worker_apply_sequence_command
(1 row) (1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open
-- Failure to drop sequence dependency for distributed table
SELECT citus.mitmproxy('conn.onQuery(query="SELECT pg_catalog.worker_drop_sequence_dependency.*mx_metadata_sync_multi_trans.dist1").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.onQuery(query="SELECT pg_catalog.worker_drop_sequence_dependency.*mx_metadata_sync_multi_trans.dist1").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open
-- Failure to drop distributed table if exists
SELECT citus.mitmproxy('conn.onQuery(query="DROP TABLE IF EXISTS mx_metadata_sync_multi_trans.dist1").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.onQuery(query="DROP TABLE IF EXISTS mx_metadata_sync_multi_trans.dist1").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port); SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open
-- Failure to create distributed table -- Failure to create distributed table
@ -370,6 +528,40 @@ SELECT citus.mitmproxy('conn.onQuery(query="CREATE TABLE mx_metadata_sync_multi_
(1 row) (1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open
-- Failure to record sequence dependency for table
SELECT citus.mitmproxy('conn.onQuery(query="SELECT pg_catalog.worker_record_sequence_dependency").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.onQuery(query="SELECT pg_catalog.worker_record_sequence_dependency").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open
-- Failure to create index for table
SELECT citus.mitmproxy('conn.onQuery(query="CREATE INDEX dist1_search_phone_idx ON mx_metadata_sync_multi_trans.dist1 USING gin").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.onQuery(query="CREATE INDEX dist1_search_phone_idx ON mx_metadata_sync_multi_trans.dist1 USING gin").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port); SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open
-- Failure to create reference table -- Failure to create reference table
@ -540,6 +732,125 @@ SELECT citus.mitmproxy('conn.onQuery(query="SELECT citus_internal_add_object_met
(1 row) (1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open
-- Failure to mark function as distributed
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*one_as_result").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*one_as_result").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open
-- Failure to mark collation as distributed
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*german_phonebook").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*german_phonebook").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open
-- Failure to mark text search dictionary as distributed
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*my_german_dict").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*my_german_dict").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open
-- Failure to mark text search configuration as distributed
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*my_ts_config").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*my_ts_config").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open
-- Failure to mark type as distributed
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*pair_type").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*pair_type").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open
-- Failure to mark sequence as distributed
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*seq_owned").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*seq_owned").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open
-- Failure to mark publication as distributed
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*pub_all").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*pub_all").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_proxy_port); SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open ERROR: connection to the remote node localhost:xxxxx failed with the following error: connection not open
-- Failure to set isactive to true -- Failure to set isactive to true
@ -680,9 +991,10 @@ SELECT citus.mitmproxy('conn.allow()');
(1 row) (1 row)
RESET citus.metadata_sync_mode; RESET citus.metadata_sync_mode;
DROP PUBLICATION pub_all;
DROP SCHEMA dummy; DROP SCHEMA dummy;
DROP SCHEMA mx_metadata_sync_multi_trans CASCADE; DROP SCHEMA mx_metadata_sync_multi_trans CASCADE;
NOTICE: drop cascades to 10 other objects NOTICE: drop cascades to 15 other objects
DROP ROLE foo1; DROP ROLE foo1;
DROP ROLE foo2; DROP ROLE foo2;
SELECT citus_remove_node('localhost', :master_port); SELECT citus_remove_node('localhost', :master_port);

View File

@ -0,0 +1,814 @@
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;
SELECT 1 FROM citus_add_node('localhost', :master_port, groupid => 0);
?column?
---------------------------------------------------------------------
1
(1 row)
SET client_min_messages TO NOTICE;
CREATE TABLE nullkey_c1_t1(a int, b int);
CREATE TABLE nullkey_c1_t2(a int, b int);
SELECT create_distributed_table('nullkey_c1_t1', null, colocate_with=>'none');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table('nullkey_c1_t2', null, colocate_with=>'nullkey_c1_t1');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE nullkey_c2_t1(a int, b int);
SELECT create_distributed_table('nullkey_c2_t1', null, colocate_with=>'none');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE reference_table(a int, b int);
SELECT create_reference_table('reference_table');
create_reference_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE distributed_table_c1_t1(a int, b int);
SELECT create_distributed_table('distributed_table_c1_t1', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE distributed_table_c1_t2(a int, b int);
SELECT create_distributed_table('distributed_table_c1_t2', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE distributed_table_c2_t1(a int, b int);
SELECT create_distributed_table('distributed_table_c2_t1', 'a', colocate_with=>'none');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE citus_local_table(a int, b int);
SELECT citus_add_local_table_to_metadata('citus_local_table');
citus_add_local_table_to_metadata
---------------------------------------------------------------------
(1 row)
CREATE TABLE postgres_local_table(a int, b int);
CREATE FUNCTION reload_tables() RETURNS void AS $$
BEGIN
SET LOCAL client_min_messages TO WARNING;
TRUNCATE nullkey_c1_t1, nullkey_c1_t2, nullkey_c2_t1, reference_table, distributed_table_c1_t1,
distributed_table_c1_t2, distributed_table_c2_t1, citus_local_table, postgres_local_table;
INSERT INTO nullkey_c1_t1 SELECT i, i FROM generate_series(1, 8) i;
INSERT INTO nullkey_c1_t2 SELECT i, i FROM generate_series(2, 7) i;
INSERT INTO nullkey_c2_t1 SELECT i, i FROM generate_series(2, 7) i;
INSERT INTO reference_table SELECT i, i FROM generate_series(0, 5) i;
INSERT INTO distributed_table_c1_t1 SELECT i, i FROM generate_series(3, 8) i;
INSERT INTO distributed_table_c1_t2 SELECT i, i FROM generate_series(2, 9) i;
INSERT INTO distributed_table_c2_t1 SELECT i, i FROM generate_series(5, 10) i;
INSERT INTO citus_local_table SELECT i, i FROM generate_series(0, 10) i;
INSERT INTO postgres_local_table SELECT i, i FROM generate_series(5, 10) i;
END;
$$ LANGUAGE plpgsql;
SELECT reload_tables();
reload_tables
---------------------------------------------------------------------
(1 row)
CREATE TABLE append_table (a int, b int);
SELECT create_distributed_table('append_table', 'a', 'append');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT master_create_empty_shard('append_table') AS shardid1 \gset
SELECT master_create_empty_shard('append_table') AS shardid2 \gset
SELECT master_create_empty_shard('append_table') AS shardid3 \gset
COPY append_table (a, b) FROM STDIN WITH (format 'csv', append_to_shard :shardid1);
COPY append_table (a, b) FROM STDIN WITH (format 'csv', append_to_shard :shardid2);
CREATE TABLE range_table(a int, b int);
SELECT create_distributed_table('range_table', 'a', 'range');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CALL public.create_range_partitioned_shards('range_table', '{"0","25"}','{"24","49"}');
INSERT INTO range_table VALUES (0, 1), (1, 2), (2, 3), (3, 4), (4, 5), (5, 6), (6, 50);
CREATE MATERIALIZED VIEW matview AS SELECT b*2+a AS a, a*a AS b FROM nullkey_c1_t1;
SET client_min_messages TO DEBUG2;
-- Test inserting into a distributed table by selecting from a combination of
-- different table types together with 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
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- use a reference table
INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN reference_table USING (a);
DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 RIGHT JOIN reference_table USING (b) WHERE reference_table.a >= 1 AND reference_table.a <= 5;
DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 LEFT JOIN reference_table USING (b);
DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 INTERSECT SELECT * FROM reference_table;
DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- use a colocated 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
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 FULL JOIN nullkey_c1_t2 USING (a);
DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 FULL JOIN matview USING (a);
DEBUG: distributed INSERT ... SELECT cannot select from a local relation when inserting into a distributed table
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: Local tables cannot be used in distributed queries.
INSERT INTO distributed_table_c1_t1 SELECT * FROM nullkey_c1_t1 UNION SELECT * FROM nullkey_c1_t2;
DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- use a non-colocated 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
DETAIL: found no worker with all shard placements
INSERT INTO distributed_table_c1_t1 SELECT * FROM nullkey_c1_t1 UNION SELECT * FROM nullkey_c2_t1;
DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: found no worker with all shard placements
-- use a distributed table that is colocated with the target table
INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a);
DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: Router planner cannot handle multi-shard select queries
INSERT INTO distributed_table_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a);
DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: Router planner cannot handle multi-shard select queries
INSERT INTO distributed_table_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (b);
DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: Router planner cannot handle multi-shard select queries
INSERT INTO distributed_table_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a) WHERE distributed_table_c1_t2.a = 1;
DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables
DEBUG: Creating router plan
DEBUG: query has a single distribution column value: 1
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- use a distributed table that is not colocated with the target table
INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 JOIN distributed_table_c2_t1 USING (a);
DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: Router planner cannot handle multi-shard select queries
-- use a citus local table
INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN citus_local_table USING (a);
DEBUG: distributed INSERT ... SELECT cannot select from distributed tables and local tables at the same time
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: Local tables cannot be used in distributed queries.
-- use a postgres local table
INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 JOIN postgres_local_table USING (a);
DEBUG: distributed INSERT ... SELECT cannot select from distributed tables and local tables at the same time
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: Local tables cannot be used in distributed queries.
-- use append / range distributed tables
INSERT INTO range_table SELECT * FROM nullkey_c1_t1;
DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables
DEBUG: Distributed planning for a fast-path router query
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO append_table SELECT * FROM nullkey_c1_t1;
DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: INSERT ... SELECT into an append-distributed table is not supported
SELECT avg(a), avg(b) FROM distributed_table_c1_t1 ORDER BY 1, 2;
DEBUG: Router planner cannot handle multi-shard select queries
avg | avg
---------------------------------------------------------------------
4.2105263157894737 | 4.2105263157894737
(1 row)
TRUNCATE distributed_table_c1_t1;
INSERT INTO distributed_table_c1_t1 SELECT i, i FROM generate_series(3, 8) i;
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- Test inserting into a reference table by selecting from a combination of
-- different table types together with 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
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- use a reference table
INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN reference_table USING (a);
DEBUG: only reference tables may be queried when targeting a reference table with distributed INSERT ... SELECT
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO reference_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 LEFT JOIN reference_table USING (b);
DEBUG: only reference tables may be queried when targeting a reference table with distributed INSERT ... SELECT
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 UNION SELECT * FROM reference_table;
DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO reference_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 LEFT JOIN reference_table USING (b) WHERE b IN (SELECT b FROM matview);
DEBUG: only reference tables may be queried when targeting a reference table with distributed INSERT ... SELECT
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: Local tables cannot be used in distributed queries.
-- use a colocated 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
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 FULL JOIN nullkey_c1_t2 USING (a);
DEBUG: only reference tables may be queried when targeting a reference table with distributed INSERT ... SELECT
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- use a non-colocated 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
DETAIL: found no worker with all shard placements
-- use a distributed table
INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a);
DEBUG: only reference tables may be queried when targeting a reference table with distributed INSERT ... SELECT
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: Router planner cannot handle multi-shard select queries
INSERT INTO reference_table SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a);
DEBUG: only reference tables may be queried when targeting a reference table with distributed INSERT ... SELECT
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: Router planner cannot handle multi-shard select queries
INSERT INTO reference_table SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (b);
DEBUG: only reference tables may be queried when targeting a reference table with distributed INSERT ... SELECT
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: Router planner cannot handle multi-shard select queries
INSERT INTO reference_table SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a) WHERE distributed_table_c1_t2.a = 1;
DEBUG: only reference tables may be queried when targeting a reference table with distributed INSERT ... SELECT
DEBUG: Creating router plan
DEBUG: query has a single distribution column value: 1
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- use a citus local table
INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN citus_local_table USING (a);
DEBUG: distributed INSERT ... SELECT cannot select from distributed tables and local tables at the same time
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: Local tables cannot be used in distributed queries.
-- use a postgres local table
INSERT INTO reference_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 JOIN postgres_local_table USING (a);
DEBUG: distributed INSERT ... SELECT cannot select from distributed tables and local tables at the same time
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: Local tables cannot be used in distributed queries.
SELECT avg(a), avg(b) FROM reference_table ORDER BY 1, 2;
DEBUG: Distributed planning for a fast-path router query
DEBUG: Creating router plan
avg | avg
---------------------------------------------------------------------
4.0428571428571429 | 4.0428571428571429
(1 row)
TRUNCATE reference_table;
INSERT INTO reference_table SELECT i, i FROM generate_series(0, 5) i;
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- Test inserting into a citus local table by selecting from a combination of
-- different table types together with 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
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- use a reference table
INSERT INTO citus_local_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN reference_table USING (a);
DEBUG: distributed INSERT ... SELECT cannot insert into a local table that is added to metadata
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- use a colocated 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
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- use a distributed table
INSERT INTO citus_local_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a);
DEBUG: distributed INSERT ... SELECT cannot insert into a local table that is added to metadata
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: Router planner cannot handle multi-shard select queries
-- use a citus local table
INSERT INTO citus_local_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN citus_local_table USING (a);
DEBUG: distributed INSERT ... SELECT cannot select from distributed tables and local tables at the same time
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: Local tables cannot be used in distributed queries.
-- use a postgres local table
INSERT INTO citus_local_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 JOIN postgres_local_table USING (a);
DEBUG: distributed INSERT ... SELECT cannot select from distributed tables and local tables at the same time
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: Local tables cannot be used in distributed queries.
SELECT avg(a), avg(b) FROM citus_local_table ORDER BY 1, 2;
DEBUG: Distributed planning for a fast-path router query
DEBUG: Creating router plan
avg | avg
---------------------------------------------------------------------
4.4333333333333333 | 4.4333333333333333
(1 row)
TRUNCATE citus_local_table;
INSERT INTO citus_local_table SELECT i, i FROM generate_series(0, 10) i;
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- Test inserting into a 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
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO nullkey_c1_t1 SELECT postgres_local_table.a, postgres_local_table.b FROM postgres_local_table JOIN reference_table USING (a);
DEBUG: distributed INSERT ... SELECT cannot select from a local table
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO nullkey_c1_t1 SELECT postgres_local_table.a, postgres_local_table.b FROM postgres_local_table LEFT JOIN nullkey_c1_t1 USING (a);
DEBUG: distributed INSERT ... SELECT cannot select from distributed tables and local tables at the same time
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: Local tables cannot be used in distributed queries.
-- use a citus local table
INSERT INTO nullkey_c1_t1 SELECT citus_local_table.a, citus_local_table.b FROM citus_local_table;
DEBUG: distributed INSERT ... SELECT cannot select from a local relation when inserting into a distributed table
DEBUG: Distributed planning for a fast-path router query
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO nullkey_c1_t1 SELECT citus_local_table.a, citus_local_table.b FROM citus_local_table JOIN reference_table USING (a) JOIN postgres_local_table USING (a) ORDER BY 1,2 OFFSET 7;
DEBUG: distributed INSERT ... SELECT cannot select from a local table
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO nullkey_c1_t1 SELECT citus_local_table.a, citus_local_table.b FROM citus_local_table JOIN nullkey_c1_t1 USING (a);
DEBUG: distributed INSERT ... SELECT cannot select from distributed tables and local tables at the same time
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: Local tables cannot be used in distributed queries.
-- use a distributed table
INSERT INTO nullkey_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM distributed_table_c1_t2;
DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO nullkey_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM distributed_table_c1_t2 JOIN reference_table USING (a);
DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO nullkey_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM distributed_table_c1_t2 JOIN nullkey_c1_t1 USING (a);
DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: Router planner cannot handle multi-shard select queries
-- use a non-colocated 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
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- use a materialized view
INSERT INTO nullkey_c1_t1 SELECT * FROM matview;
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO nullkey_c1_t1 SELECT reference_table.a, reference_table.b FROM reference_table JOIN matview ON (reference_table.a = matview.a);
DEBUG: distributed INSERT ... SELECT cannot select from a local relation when inserting into a distributed table
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO nullkey_c1_t1 SELECT q.* FROM (SELECT reference_table.* FROM reference_table JOIN nullkey_c1_t1 USING (a)) q JOIN matview USING (a);
DEBUG: distributed INSERT ... SELECT cannot select from a local relation when inserting into a distributed table
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: Local tables cannot be used in distributed queries.
-- use append / range distributed tables
INSERT INTO nullkey_c1_t1 SELECT * FROM range_table;
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: Router planner cannot handle multi-shard select queries
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO nullkey_c1_t1 SELECT * FROM append_table;
DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables
DEBUG: Router planner does not support append-partitioned tables.
DEBUG: Collecting INSERT ... SELECT results on coordinator
SELECT avg(a), avg(b) FROM nullkey_c1_t1 ORDER BY 1, 2;
DEBUG: Distributed planning for a fast-path router query
DEBUG: Creating router plan
avg | avg
---------------------------------------------------------------------
5.8611111111111111 | 13.9305555555555556
(1 row)
SELECT avg(a), avg(b) FROM nullkey_c2_t1 ORDER BY 1, 2;
DEBUG: Distributed planning for a fast-path router query
DEBUG: Creating router plan
avg | avg
---------------------------------------------------------------------
3.8750000000000000 | 3.8750000000000000
(1 row)
TRUNCATE nullkey_c1_t1, nullkey_c2_t1;
INSERT INTO nullkey_c1_t1 SELECT i, i FROM generate_series(1, 8) i;
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO nullkey_c2_t1 SELECT i, i FROM generate_series(2, 7) i;
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- Test inserting into a local table by selecting from a combination of
-- different table types, together with or without 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;
DEBUG: Creating router plan
WITH cte_1 AS (
DELETE FROM nullkey_c1_t1 WHERE a >= 1 and a <= 4 RETURNING *
)
INSERT INTO postgres_local_table SELECT cte_1.* FROM cte_1 LEFT JOIN nullkey_c1_t2 USING (a) WHERE nullkey_c1_t2.a IS NULL;
DEBUG: Creating router plan
INSERT INTO postgres_local_table SELECT * FROM nullkey_c1_t1 EXCEPT SELECT * FROM postgres_local_table;
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: Local tables cannot be used in distributed queries.
SELECT avg(a), avg(b) FROM postgres_local_table ORDER BY 1, 2;
avg | avg
---------------------------------------------------------------------
5.0000000000000000 | 5.0000000000000000
(1 row)
TRUNCATE postgres_local_table;
INSERT INTO postgres_local_table SELECT i, i FROM generate_series(5, 10) i;
-- Try slightly more complex queries.
WITH cte_1 AS (
SELECT nullkey_c1_t1.a, reference_table.b FROM nullkey_c1_t1 JOIN reference_table USING (a)
),
cte_2 AS (
SELECT reference_table.a, postgres_local_table.b FROM postgres_local_table LEFT JOIN reference_table USING (b)
)
INSERT INTO distributed_table_c1_t1
SELECT cte_1.* FROM cte_1 JOIN cte_2 USING (a) JOIN distributed_table_c1_t2 USING (a) ORDER BY 1,2;
DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match
DETAIL: The target table's partition column should correspond to a partition column in the subquery.
DEBUG: CTE cte_1 is going to be inlined via distributed planning
DEBUG: CTE cte_2 is going to be inlined via distributed planning
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: Local tables cannot be used in distributed queries.
WITH cte_1 AS (
SELECT nullkey_c1_t1.a, reference_table.b FROM nullkey_c1_t1 JOIN reference_table USING (a)
),
cte_2 AS (
SELECT * FROM nullkey_c1_t2 WHERE EXISTS (
SELECT 1 FROM reference_table WHERE reference_table.a = nullkey_c1_t2.a
)
ORDER BY 1,2 OFFSET 1 LIMIT 4
)
INSERT INTO distributed_table_c1_t1
SELECT * FROM cte_1 UNION SELECT * FROM cte_2 EXCEPT SELECT * FROM reference_table;
DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries
DEBUG: CTE cte_1 is going to be inlined via distributed planning
DEBUG: CTE cte_2 is going to be inlined via distributed planning
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO distributed_table_c1_t1 (a, b)
SELECT t1.a, t2.b
FROM nullkey_c1_t1 t1
JOIN (
SELECT b FROM nullkey_c1_t2 ORDER BY b DESC LIMIT 1
) t2
ON t1.b < t2.b;
DEBUG: LIMIT clauses are not allowed in distributed INSERT ... SELECT queries
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO distributed_table_c1_t1 (a, b)
WITH cte AS (
SELECT a, b,
(SELECT a FROM nullkey_c1_t2 WHERE b = t.b) AS d1,
(SELECT a FROM reference_table WHERE b = t.b) AS d2
FROM nullkey_c1_t1 t
)
SELECT d1, COALESCE(d2, a) FROM cte WHERE d1 IS NOT NULL AND d2 IS NOT NULL;
DEBUG: CTE cte is going to be inlined via distributed planning
DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO citus_local_table (a, b)
SELECT t1.a, t2.b
FROM nullkey_c1_t1 t1
CROSS JOIN (
SELECT b FROM nullkey_c2_t1 ORDER BY b LIMIT 1
) t2;
DEBUG: distributed INSERT ... SELECT cannot insert into a local table that is added to metadata
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: found no worker with all shard placements
INSERT INTO distributed_table_c1_t1 (a, b)
SELECT t1.a, t2.b
FROM reference_table t1
LEFT JOIN (
SELECT b, ROW_NUMBER() OVER (ORDER BY b DESC) AS rn
FROM nullkey_c1_t1
) t2 ON t1.b = t2.b
WHERE t2.rn > 0;
DEBUG: Window functions without PARTITION BY on distribution column is currently unsupported
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO nullkey_c1_t1 (a, b)
SELECT t1.a, t2.b
FROM nullkey_c1_t1 t1
JOIN (
SELECT rn, b
FROM (
SELECT b, ROW_NUMBER() OVER (ORDER BY b DESC) AS rn
FROM distributed_table_c2_t1
) q
) t2 ON t1.b = t2.b
WHERE t2.rn > 2;
DEBUG: Window functions without PARTITION BY on distribution column is currently unsupported
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: Router planner cannot handle multi-shard select queries
INSERT INTO distributed_table_c1_t1 (a, b)
SELECT t1.a, t2.b
FROM nullkey_c1_t1 t1
JOIN (
SELECT sum_val, b
FROM (
SELECT b, SUM(a) OVER (PARTITION BY b) AS sum_val
FROM nullkey_c1_t1
) q
) t2 ON t1.b = t2.b
WHERE t2.sum_val > 2;
DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- MultiTaskRouterSelectQuerySupported() is unnecessarily restrictive
-- about pushing down queries with DISTINCT ON clause even if the table
-- doesn't have a shard key. See https://github.com/citusdata/citus/pull/6752.
INSERT INTO nullkey_c1_t1 SELECT DISTINCT ON (a) a, b FROM nullkey_c1_t2;
DEBUG: DISTINCT ON (non-partition column) clauses are not allowed in distributed INSERT ... SELECT queries
DEBUG: Distributed planning for a fast-path router query
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- Similarly, we could push down the following query as well. see
-- https://github.com/citusdata/citus/pull/6831.
INSERT INTO nullkey_c1_t1 SELECT b, SUM(a) OVER (ORDER BY b) AS sum_val FROM nullkey_c1_t1;
DEBUG: Window functions without PARTITION BY on distribution column is currently unsupported
DEBUG: Distributed planning for a fast-path router query
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO nullkey_c2_t1
SELECT t2.a, t2.b
FROM nullkey_c1_t1 AS t2
JOIN reference_table AS t3 ON (t2.a = t3.a)
WHERE NOT EXISTS (
SELECT 1 FROM nullkey_c1_t2 AS t1 WHERE t1.b = t3.b
);
DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO distributed_table_c1_t1
SELECT t1.a, t1.b
FROM nullkey_c1_t1 AS t1
WHERE t1.a NOT IN (
SELECT DISTINCT t2.a FROM distributed_table_c1_t2 AS t2
);
DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: Router planner cannot handle multi-shard select queries
INSERT INTO distributed_table_c1_t1
SELECT t1.a, t1.b
FROM reference_table AS t1
JOIN (
SELECT t2.a FROM (
SELECT a FROM nullkey_c1_t1
UNION
SELECT a FROM nullkey_c1_t2
) AS t2
) AS t3 ON t1.a = t3.a;
DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries
DEBUG: Creating router plan
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- Temporaryly reduce the verbosity to avoid noise
-- in the output of the next query.
SET client_min_messages TO DEBUG1;
INSERT INTO nullkey_c1_t1
SELECT t1.a, t1.b
FROM reference_table AS t1
WHERE t1.a IN (
SELECT t2.a FROM (
SELECT t3.a FROM (
SELECT a FROM distributed_table_c1_t1 WHERE b > 4
) AS t3
JOIN (
SELECT a FROM distributed_table_c1_t2 WHERE b < 7
) AS t4 ON t3.a = t4.a
) AS t2
);
DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables
DEBUG: generating subplan XXX_1 for subquery SELECT a FROM (SELECT t3.a FROM ((SELECT distributed_table_c1_t1.a FROM insert_select_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
CREATE TABLE upsert_test_1
(
unique_col int UNIQUE,
other_col int,
third_col int
);
DEBUG: CREATE TABLE / UNIQUE will create implicit index "upsert_test_1_unique_col_key" for table "upsert_test_1"
SELECT create_distributed_table('upsert_test_1', null);
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE upsert_test_2(key int primary key, value text);
DEBUG: CREATE TABLE / PRIMARY KEY will create implicit index "upsert_test_2_pkey" for table "upsert_test_2"
SELECT create_distributed_table('upsert_test_2', null);
create_distributed_table
---------------------------------------------------------------------
(1 row)
INSERT INTO upsert_test_2 AS upsert_test_2_alias (key, value) VALUES (1, '5') ON CONFLICT(key)
DO UPDATE SET value = (upsert_test_2_alias.value::int * 2)::text;
DEBUG: Creating router plan
INSERT INTO upsert_test_2 (key, value) VALUES (1, '5') ON CONFLICT(key)
DO UPDATE SET value = (upsert_test_2.value::int * 3)::text;
DEBUG: Creating router plan
INSERT INTO upsert_test_1 (unique_col, other_col) VALUES (1, 1) ON CONFLICT (unique_col)
DO UPDATE SET other_col = (SELECT count(*) from upsert_test_1);
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: subqueries are not supported within INSERT queries
HINT: Try rewriting your queries with 'INSERT INTO ... SELECT' syntax.
INSERT INTO upsert_test_1 (unique_col, other_col) VALUES (1, 1) ON CONFLICT (unique_col)
DO UPDATE SET other_col = random()::int;
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: functions used in the DO UPDATE SET clause of INSERTs on distributed tables must be marked IMMUTABLE
INSERT INTO upsert_test_1 (unique_col, other_col) VALUES (1, 1) ON CONFLICT (unique_col)
DO UPDATE SET other_col = 5 WHERE upsert_test_1.other_col = random()::int;
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: functions used in the WHERE clause of the ON CONFLICT clause of INSERTs on distributed tables must be marked IMMUTABLE
INSERT INTO upsert_test_1 VALUES (3, 5, 7);
DEBUG: Creating router plan
INSERT INTO upsert_test_1 (unique_col, other_col) VALUES (1, 1) ON CONFLICT (unique_col) WHERE unique_col = random()::int
DO UPDATE SET other_col = 5;
ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables
DETAIL: functions used in the WHERE clause of the ON CONFLICT clause of INSERTs on distributed tables must be marked IMMUTABLE
CREATE TABLE upsert_test_3 (key_1 int, key_2 bigserial, value text DEFAULT 'default_value', PRIMARY KEY (key_1, key_2));
DEBUG: CREATE TABLE will create implicit sequence "upsert_test_3_key_2_seq" for serial column "upsert_test_3.key_2"
DEBUG: CREATE TABLE / PRIMARY KEY will create implicit index "upsert_test_3_pkey" for table "upsert_test_3"
SELECT create_distributed_table('upsert_test_3', null);
create_distributed_table
---------------------------------------------------------------------
(1 row)
INSERT INTO upsert_test_3 VALUES (1, DEFAULT, '1') RETURNING *;
DEBUG: Creating router plan
key_1 | key_2 | value
---------------------------------------------------------------------
1 | 1 | 1
(1 row)
INSERT INTO upsert_test_3 VALUES (5, DEFAULT, DEFAULT) RETURNING *;
DEBUG: Creating router plan
key_1 | key_2 | value
---------------------------------------------------------------------
5 | 2 | default_value
(1 row)
SET client_min_messages TO DEBUG1;
INSERT INTO upsert_test_3 SELECT 7, other_col, 'harcoded_text_value' FROM upsert_test_1 RETURNING *;
key_1 | key_2 | value
---------------------------------------------------------------------
7 | 5 | harcoded_text_value
(1 row)
SET client_min_messages TO DEBUG2;
-- test upsert with INSERT .. SELECT queries
SET client_min_messages TO DEBUG1;
INSERT INTO upsert_test_1 (unique_col, other_col) SELECT unique_col, other_col FROM upsert_test_1 ON CONFLICT (unique_col)
DO UPDATE SET other_col = upsert_test_1.other_col + 1;
-- Fails due to https://github.com/citusdata/citus/issues/6826.
INSERT INTO upsert_test_1 (unique_col, other_col) SELECT unique_col, other_col FROM upsert_test_1 ON CONFLICT (unique_col)
DO UPDATE SET other_col = (SELECT count(*) from upsert_test_1);
ERROR: cannot execute a distributed query from a query on a shard
DETAIL: Executing a distributed query in a function call that may be pushed to a remote node can lead to incorrect results.
HINT: Avoid nesting of distributed queries or use alter user current_user set citus.allow_nested_distributed_execution to on to allow it with possible incorrectness.
CONTEXT: while executing command on localhost:xxxxx
SET client_min_messages TO DEBUG2;
INSERT INTO upsert_test_1 (unique_col, other_col) SELECT unique_col, other_col FROM upsert_test_1 ON CONFLICT (unique_col)
DO UPDATE SET other_col = random()::int;
ERROR: functions used in the DO UPDATE SET clause of INSERTs on distributed tables must be marked IMMUTABLE
INSERT INTO upsert_test_1 (unique_col, other_col) SELECT unique_col, other_col FROM upsert_test_1 ON CONFLICT (unique_col)
DO UPDATE SET other_col = 5 WHERE upsert_test_1.other_col = random()::int;
ERROR: functions used in the WHERE clause of the ON CONFLICT clause of INSERTs on distributed tables must be marked IMMUTABLE
SELECT reload_tables();
DEBUG: function does not have co-located tables
reload_tables
---------------------------------------------------------------------
(1 row)
ALTER TABLE nullkey_c1_t1 ADD PRIMARY KEY (a);
DEBUG: ALTER TABLE / ADD PRIMARY KEY will create implicit index "nullkey_c1_t1_pkey" for table "nullkey_c1_t1"
DEBUG: verifying table "nullkey_c1_t1"
ALTER TABLE distributed_table_c1_t1 ADD PRIMARY KEY (a,b);
DEBUG: ALTER TABLE / ADD PRIMARY KEY will create implicit index "distributed_table_c1_t1_pkey" for table "distributed_table_c1_t1"
DEBUG: verifying table "distributed_table_c1_t1"
INSERT INTO nullkey_c1_t1 AS t1 (a, b) SELECT t3.a, t3.b FROM nullkey_c1_t2 t2 JOIN reference_table t3 ON (t2.a = t3.a) ON CONFLICT (a)
DO UPDATE SET a = t1.a + 10;
DEBUG: distributed statement: INSERT INTO insert_select_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;
DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator
INSERT INTO nullkey_c1_t1 AS t1 (a, b) SELECT t3.a, t3.b FROM distributed_table_c1_t1 t2 JOIN reference_table t3 ON (t2.a = t3.a) ON CONFLICT (a)
DO UPDATE SET a = t1.a + 10;
DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator
-- This also fails due to https://github.com/citusdata/citus/issues/6826.
INSERT INTO nullkey_c1_t1 AS t1 (a, b) SELECT t3.a, t3.b FROM distributed_table_c1_t1 t2 JOIN reference_table t3 ON (t2.a = t3.a) WHERE t2.a = 3 ON CONFLICT (a)
DO UPDATE SET a = (SELECT max(b)+1 FROM distributed_table_c1_t1 WHERE a = 3);
DEBUG: distributed INSERT ... SELECT cannot reference a distributed table without a shard key together with non-colocated distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator
ERROR: cannot execute a distributed query from a query on a shard
DETAIL: Executing a distributed query in a function call that may be pushed to a remote node can lead to incorrect results.
HINT: Avoid nesting of distributed queries or use alter user current_user set citus.allow_nested_distributed_execution to on to allow it with possible incorrectness.
CONTEXT: while executing command on localhost:xxxxx
SET client_min_messages TO DEBUG2;
SELECT avg(a), avg(b) FROM distributed_table_c1_t1;
DEBUG: Router planner cannot handle multi-shard select queries
avg | avg
---------------------------------------------------------------------
5.0000000000000000 | 9.2857142857142857
(1 row)
SELECT avg(a), avg(b) FROM nullkey_c1_t1;
DEBUG: Distributed planning for a fast-path router query
DEBUG: Creating router plan
avg | avg
---------------------------------------------------------------------
7.5000000000000000 | 4.1666666666666667
(1 row)
SELECT avg(a), avg(b) FROM nullkey_c1_t2;
DEBUG: Distributed planning for a fast-path router query
DEBUG: Creating router plan
avg | avg
---------------------------------------------------------------------
4.5000000000000000 | 4.5000000000000000
(1 row)
SELECT * FROM upsert_test_1 ORDER BY unique_col;
DEBUG: Distributed planning for a fast-path router query
DEBUG: Creating router plan
unique_col | other_col | third_col
---------------------------------------------------------------------
3 | 6 | 7
(1 row)
SELECT * FROM upsert_test_2 ORDER BY key;
DEBUG: Distributed planning for a fast-path router query
DEBUG: Creating router plan
key | value
---------------------------------------------------------------------
1 | 15
(1 row)
SELECT * FROM upsert_test_3 ORDER BY key_1, key_2;
DEBUG: Distributed planning for a fast-path router query
DEBUG: Creating router plan
key_1 | key_2 | value
---------------------------------------------------------------------
1 | 1 | 1
5 | 2 | default_value
7 | 5 | harcoded_text_value
(3 rows)
SET client_min_messages TO WARNING;
DROP SCHEMA insert_select_single_shard_table CASCADE;
SELECT citus_remove_node('localhost', :master_port);
citus_remove_node
---------------------------------------------------------------------
(1 row)

View File

@ -2416,9 +2416,134 @@ SELECT * FROM target_set ORDER BY 1, 2;
2 | 2 |
(2 rows) (2 rows)
--
-- Reference as a source
--
CREATE TABLE reftarget_local(t1 int, t2 int);
CREATE TABLE refsource_ref(s1 int, s2 int);
INSERT INTO reftarget_local VALUES(1, 0);
INSERT INTO reftarget_local VALUES(3, 100);
INSERT INTO refsource_ref VALUES(1, 1);
INSERT INTO refsource_ref VALUES(2, 2);
INSERT INTO refsource_ref VALUES(3, 3);
MERGE INTO reftarget_local
USING (SELECT * FROM refsource_ref UNION SELECT * FROM refsource_ref) AS foo ON reftarget_local.t1 = foo.s1
WHEN MATCHED AND reftarget_local.t2 = 100 THEN
DELETE
WHEN MATCHED THEN
UPDATE SET t2 = t2 + 100
WHEN NOT MATCHED THEN
INSERT VALUES(foo.s1);
DROP TABLE IF EXISTS pg_result;
SELECT * INTO pg_result FROM reftarget_local ORDER BY 1, 2;
-- Make source table as reference (target is Postgres)
TRUNCATE reftarget_local;
TRUNCATE refsource_ref;
INSERT INTO reftarget_local VALUES(1, 0);
INSERT INTO reftarget_local VALUES(3, 100);
INSERT INTO refsource_ref VALUES(1, 1);
INSERT INTO refsource_ref VALUES(2, 2);
INSERT INTO refsource_ref VALUES(3, 3);
SELECT create_reference_table('refsource_ref');
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($$merge_schema.refsource_ref$$)
create_reference_table
---------------------------------------------------------------------
(1 row)
MERGE INTO reftarget_local
USING (SELECT * FROM refsource_ref UNION SELECT * FROM refsource_ref) AS foo ON reftarget_local.t1 = foo.s1
WHEN MATCHED AND reftarget_local.t2 = 100 THEN
DELETE
WHEN MATCHED THEN
UPDATE SET t2 = t2 + 100
WHEN NOT MATCHED THEN
INSERT VALUES(foo.s1);
SELECT * INTO pg_ref FROM reftarget_local ORDER BY 1, 2;
-- Should be equal
SELECT c.*, p.*
FROM pg_ref c, pg_result p
WHERE c.t1 = p.t1
ORDER BY 1,2;
t1 | t2 | t1 | t2
---------------------------------------------------------------------
1 | 100 | 1 | 100
2 | | 2 |
(2 rows)
-- Must return zero rows
SELECT count(*)
FROM pg_result FULL OUTER JOIN pg_ref ON pg_result.t1 = pg_ref.t1
WHERE pg_result.t1 IS NULL OR pg_ref.t1 IS NULL;
count
---------------------------------------------------------------------
0
(1 row)
-- Now make both Citus tables, reference as source, local as target
TRUNCATE reftarget_local;
TRUNCATE refsource_ref;
INSERT INTO reftarget_local VALUES(1, 0);
INSERT INTO reftarget_local VALUES(3, 100);
INSERT INTO refsource_ref VALUES(1, 1);
INSERT INTO refsource_ref VALUES(2, 2);
INSERT INTO refsource_ref VALUES(3, 3);
SELECT citus_add_local_table_to_metadata('reftarget_local');
citus_add_local_table_to_metadata
---------------------------------------------------------------------
(1 row)
MERGE INTO reftarget_local
USING (SELECT * FROM refsource_ref UNION SELECT * FROM refsource_ref) AS foo ON reftarget_local.t1 = foo.s1
WHEN MATCHED AND reftarget_local.t2 = 100 THEN
DELETE
WHEN MATCHED THEN
UPDATE SET t2 = t2 + 100
WHEN NOT MATCHED THEN
INSERT VALUES(foo.s1);
SELECT * INTO local_ref FROM reftarget_local ORDER BY 1, 2;
-- Should be equal
SELECT c.*, p.*
FROM local_ref c, pg_result p
WHERE c.t1 = p.t1
ORDER BY 1,2;
t1 | t2 | t1 | t2
---------------------------------------------------------------------
1 | 100 | 1 | 100
2 | | 2 |
(2 rows)
-- Must return zero rows
SELECT count(*)
FROM pg_result FULL OUTER JOIN local_ref ON pg_result.t1 = local_ref.t1
WHERE pg_result.t1 IS NULL OR local_ref.t1 IS NULL;
count
---------------------------------------------------------------------
0
(1 row)
-- --
-- Error and Unsupported scenarios -- Error and Unsupported scenarios
-- --
-- Reference as a target and local as source
MERGE INTO refsource_ref
USING (SELECT * FROM reftarget_local UNION SELECT * FROM reftarget_local) AS foo ON refsource_ref.s1 = foo.t1
WHEN MATCHED THEN
UPDATE SET s2 = s2 + 100
WHEN NOT MATCHED THEN
INSERT VALUES(foo.t1);
ERROR: Reference table as target is not allowed in MERGE command
-- Reference as a source and distributed as target
MERGE INTO target_set t
USING refsource_ref AS s ON t.t1 = s.s1
WHEN MATCHED THEN
DO NOTHING;
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 target_set MERGE INTO target_set
USING source_set AS foo ON target_set.t1 = foo.s1 USING source_set AS foo ON target_set.t1 = foo.s1
WHEN MATCHED THEN WHEN MATCHED THEN
@ -2735,7 +2860,7 @@ MERGE INTO t1
UPDATE SET val = t1.val + 1 UPDATE SET val = t1.val + 1
WHEN NOT MATCHED THEN WHEN NOT MATCHED THEN
INSERT (id, val) VALUES (s1.id, s1.val); INSERT (id, val) VALUES (s1.id, s1.val);
ERROR: MERGE command is not supported on reference tables yet ERROR: Reference table as target is not allowed in MERGE command
-- --
-- Postgres + Citus-Distributed table -- Postgres + Citus-Distributed table
-- --
@ -3103,6 +3228,154 @@ WHEN NOT MATCHED THEN
INSERT VALUES(dist_source.id, dist_source.val); 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 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 HINT: Consider using hash distribution instead
-- 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);
SELECT create_distributed_table('nullkey_c1_t1', null, colocate_with=>'none');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table('nullkey_c1_t2', null, colocate_with=>'nullkey_c1_t1');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE nullkey_c2_t1(a int, b int);
CREATE TABLE nullkey_c2_t2(a int, b int);
SELECT create_distributed_table('nullkey_c2_t1', null, colocate_with=>'none');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table('nullkey_c2_t2', null, colocate_with=>'nullkey_c2_t1', distribution_type=>null);
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE reference_table(a int, b int);
SELECT create_reference_table('reference_table');
create_reference_table
---------------------------------------------------------------------
(1 row)
INSERT INTO reference_table SELECT i, i FROM generate_series(0, 5) i;
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator
CREATE TABLE distributed_table(a int, b int);
SELECT create_distributed_table('distributed_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
INSERT INTO distributed_table SELECT i, i FROM generate_series(3, 8) i;
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator
CREATE TABLE citus_local_table(a int, b int);
SELECT citus_add_local_table_to_metadata('citus_local_table');
citus_add_local_table_to_metadata
---------------------------------------------------------------------
(1 row)
INSERT INTO citus_local_table SELECT i, i FROM generate_series(0, 10) i;
DEBUG: distributed INSERT ... SELECT can only select from distributed tables
DEBUG: Collecting INSERT ... SELECT results on coordinator
CREATE TABLE postgres_local_table(a int, b int);
INSERT INTO postgres_local_table SELECT i, i FROM generate_series(5, 10) i;
-- with a colocated table
MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a)
WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b;
DEBUG: <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_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_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_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 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
MERGE INTO nullkey_c1_t1 USING nullkey_c2_t1 ON (nullkey_c1_t1.a = nullkey_c2_t1.a)
WHEN MATCHED THEN UPDATE SET b = nullkey_c2_t1.b
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c2_t1.a, nullkey_c2_t1.b);
ERROR: For MERGE command, all the distributed tables must be colocated
-- with a distributed table
MERGE INTO nullkey_c1_t1 USING distributed_table ON (nullkey_c1_t1.a = distributed_table.a)
WHEN MATCHED THEN UPDATE SET b = distributed_table.b
WHEN NOT MATCHED THEN INSERT VALUES (distributed_table.a, distributed_table.b);
ERROR: For MERGE command, all the distributed tables must be colocated
MERGE INTO distributed_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = distributed_table.a)
WHEN MATCHED THEN DELETE
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b);
ERROR: For MERGE command, all the distributed tables must be colocated
-- with a reference table
MERGE INTO nullkey_c1_t1 USING reference_table ON (nullkey_c1_t1.a = reference_table.a)
WHEN MATCHED THEN UPDATE SET b = reference_table.b;
ERROR: MERGE command is not supported 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: 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;
ERROR: MERGE command is not supported with combination of distributed/local tables yet
MERGE INTO citus_local_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = citus_local_table.a)
WHEN MATCHED THEN DELETE;
ERROR: MERGE command is not supported with combination of distributed/local tables yet
-- with a postgres table
MERGE INTO nullkey_c1_t1 USING postgres_local_table ON (nullkey_c1_t1.a = postgres_local_table.a)
WHEN MATCHED THEN UPDATE SET b = postgres_local_table.b;
ERROR: MERGE command is not supported with combination of distributed/local tables yet
MERGE INTO postgres_local_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = postgres_local_table.a)
WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t1.b
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b);
ERROR: MERGE command is not supported with combination of distributed/local tables yet
-- using ctes
WITH cte AS (
SELECT * FROM nullkey_c1_t1
)
MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a)
WHEN MATCHED THEN UPDATE SET b = cte.b;
DEBUG: <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
)
MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a)
WHEN MATCHED THEN UPDATE SET b = cte.b;
ERROR: For MERGE command, all the distributed tables must be colocated
WITH cte AS materialized (
SELECT * FROM distributed_table
)
MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a)
WHEN MATCHED THEN UPDATE SET b = cte.b;
ERROR: For MERGE command, all the distributed tables must be colocated
SET client_min_messages TO WARNING;
DROP SCHEMA query_single_shard_table CASCADE;
RESET client_min_messages;
SET search_path TO merge_schema;
DROP SERVER foreign_server CASCADE; DROP SERVER foreign_server CASCADE;
NOTICE: drop cascades to 3 other objects NOTICE: drop cascades to 3 other objects
DETAIL: drop cascades to user mapping for postgres on server foreign_server DETAIL: drop cascades to user mapping for postgres on server foreign_server
@ -3113,9 +3386,8 @@ CONTEXT: SQL statement "SELECT citus_drop_all_shards(v_obj.objid, v_obj.schema_
PL/pgSQL function citus_drop_trigger() line XX at PERFORM PL/pgSQL function citus_drop_trigger() line XX at PERFORM
DROP FUNCTION merge_when_and_write(); DROP FUNCTION merge_when_and_write();
DROP SCHEMA merge_schema CASCADE; DROP SCHEMA merge_schema CASCADE;
NOTICE: drop cascades to 84 other objects NOTICE: drop cascades to 90 other objects
DETAIL: drop cascades to function insert_data() DETAIL: drop cascades to function insert_data()
drop cascades to table pg_result
drop cascades to table local_local drop cascades to table local_local
drop cascades to table target drop cascades to table target
drop cascades to table source drop cascades to table source
@ -3188,10 +3460,17 @@ drop cascades to table source_serial
drop cascades to table target_serial drop cascades to table target_serial
drop cascades to table target_set drop cascades to table target_set
drop cascades to table source_set drop cascades to table source_set
drop cascades to table reftarget_local_4000113
drop cascades to table refsource_ref
drop cascades to table pg_result
drop cascades to table refsource_ref_4000112
drop cascades to table pg_ref
drop cascades to table reftarget_local
drop cascades to table local_ref
drop cascades to function add_s(integer,integer) drop cascades to function add_s(integer,integer)
drop cascades to table pg drop cascades to table pg
drop cascades to table t1_4000131 drop cascades to table t1_4000133
drop cascades to table s1_4000132 drop cascades to table s1_4000134
drop cascades to table t1 drop cascades to table t1
drop cascades to table s1 drop cascades to table s1
drop cascades to table dist_colocated drop cascades to table dist_colocated

View File

@ -612,10 +612,10 @@ CREATE TABLE table_postgresql( id int );
CREATE TABLE table_failing ( id int ); CREATE TABLE table_failing ( id int );
SELECT create_distributed_table('table_failing', 'id', colocate_with => 'table_append'); SELECT create_distributed_table('table_failing', 'id', colocate_with => 'table_append');
ERROR: cannot distribute relation ERROR: cannot distribute relation
DETAIL: Currently, colocate_with option is only supported for hash distributed tables. DETAIL: Currently, colocate_with option is not supported with append / range distributed tables and local tables added to metadata.
SELECT create_distributed_table('table_failing', 'id', 'append', 'table1_groupE'); SELECT create_distributed_table('table_failing', 'id', 'append', 'table1_groupE');
ERROR: cannot distribute relation ERROR: cannot distribute relation
DETAIL: Currently, colocate_with option is only supported for hash distributed tables. DETAIL: Currently, colocate_with option is not supported for append / range distributed tables.
SELECT create_distributed_table('table_failing', 'id', colocate_with => 'table_postgresql'); SELECT create_distributed_table('table_failing', 'id', colocate_with => 'table_postgresql');
ERROR: relation table_postgresql is not distributed ERROR: relation table_postgresql is not distributed
SELECT create_distributed_table('table_failing', 'id', colocate_with => 'no_table'); SELECT create_distributed_table('table_failing', 'id', colocate_with => 'no_table');

View File

@ -1735,6 +1735,33 @@ HINT: To remove the local data, run: SELECT truncate_local_data_after_distribut
DROP TABLE test; DROP TABLE test;
TRUNCATE pg_dist_node; TRUNCATE pg_dist_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;
SELECT create_distributed_table('test', null, colocate_with=>'none', distribution_type=>null);
NOTICE: Copying data from local table...
NOTICE: copying the data has completed
DETAIL: The local data in the table is no longer visible, but is still on disk.
HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$public.test$$)
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- and make sure that we can't remove the coordinator due to "test"
SELECT citus_remove_node('localhost', :master_port);
ERROR: cannot remove or disable the node localhost:xxxxx because because it contains the only shard placement for shard xxxxx
DETAIL: One of the table(s) that prevents the operation complete successfully is public.test
HINT: To proceed, either drop the tables or use undistribute_table() function to convert them to local tables
DROP TABLE test;
-- and now we should be able to remove the coordinator
SELECT citus_remove_node('localhost', :master_port);
citus_remove_node
---------------------------------------------------------------------
(1 row)
-- confirm that we can create a reference table on an empty node -- confirm that we can create a reference table on an empty node
CREATE TABLE test (x int, y int); CREATE TABLE test (x int, y int);
INSERT INTO test VALUES (1,2); INSERT INTO test VALUES (1,2);

View File

@ -650,7 +650,7 @@ NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing SET citus.enable_ddl_propagation TO 'off' NOTICE: issuing SET citus.enable_ddl_propagation TO 'off'
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing CREATE EXTENSION IF NOT EXISTS citus_columnar WITH SCHEMA pg_catalog VERSION "11.2-1"; NOTICE: issuing CREATE EXTENSION IF NOT EXISTS citus_columnar WITH SCHEMA pg_catalog VERSION "11.3-1";
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing COMMIT NOTICE: issuing COMMIT
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
@ -658,7 +658,7 @@ NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing SET citus.enable_ddl_propagation TO 'off' NOTICE: issuing SET citus.enable_ddl_propagation TO 'off'
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing CREATE EXTENSION IF NOT EXISTS citus_columnar WITH SCHEMA pg_catalog VERSION "11.2-1"; NOTICE: issuing CREATE EXTENSION IF NOT EXISTS citus_columnar WITH SCHEMA pg_catalog VERSION "11.3-1";
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing COMMIT NOTICE: issuing COMMIT
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx

View File

@ -149,6 +149,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
RESET ROLE RESET ROLE
SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''') SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''')
SELECT citus_internal_add_partition_metadata ('public.mx_test_table'::regclass, 'h', 'col_1', 2, 's') SELECT citus_internal_add_partition_metadata ('public.mx_test_table'::regclass, 'h', 'col_1', 2, 's')
SELECT pg_catalog.worker_drop_sequence_dependency('public.mx_test_table');
SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition
SELECT pg_catalog.worker_record_sequence_dependency('public.mx_test_table_col_3_seq'::regclass,'public.mx_test_table'::regclass,'col_3') SELECT pg_catalog.worker_record_sequence_dependency('public.mx_test_table_col_3_seq'::regclass,'public.mx_test_table'::regclass,'col_3')
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint') SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint')
@ -174,7 +175,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['public', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data; WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['public', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data; WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
(49 rows) (50 rows)
-- Show that CREATE INDEX commands are included in the activate node snapshot -- Show that CREATE INDEX commands are included in the activate node snapshot
CREATE INDEX mx_index ON mx_test_table(col_2); CREATE INDEX mx_index ON mx_test_table(col_2);
@ -206,6 +207,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
RESET ROLE RESET ROLE
SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''') SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''')
SELECT citus_internal_add_partition_metadata ('public.mx_test_table'::regclass, 'h', 'col_1', 2, 's') SELECT citus_internal_add_partition_metadata ('public.mx_test_table'::regclass, 'h', 'col_1', 2, 's')
SELECT pg_catalog.worker_drop_sequence_dependency('public.mx_test_table');
SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition
SELECT pg_catalog.worker_record_sequence_dependency('public.mx_test_table_col_3_seq'::regclass,'public.mx_test_table'::regclass,'col_3') SELECT pg_catalog.worker_record_sequence_dependency('public.mx_test_table_col_3_seq'::regclass,'public.mx_test_table'::regclass,'col_3')
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint') SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint')
@ -231,7 +233,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['public', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data; WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['public', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data; WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
(50 rows) (51 rows)
-- Show that schema changes are included in the activate node snapshot -- Show that schema changes are included in the activate node snapshot
CREATE SCHEMA mx_testing_schema; CREATE SCHEMA mx_testing_schema;
@ -265,6 +267,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
RESET ROLE RESET ROLE
SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''') SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''')
SELECT citus_internal_add_partition_metadata ('mx_testing_schema.mx_test_table'::regclass, 'h', 'col_1', 2, 's') SELECT citus_internal_add_partition_metadata ('mx_testing_schema.mx_test_table'::regclass, 'h', 'col_1', 2, 's')
SELECT pg_catalog.worker_drop_sequence_dependency('mx_testing_schema.mx_test_table');
SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition
SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3') SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3')
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint') SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint')
@ -291,7 +294,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['mx_testing_schema', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data; WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['mx_testing_schema', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data; WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
(52 rows) (53 rows)
-- Show that append distributed tables are not included in the activate node snapshot -- Show that append distributed tables are not included in the activate node snapshot
CREATE TABLE non_mx_test_table (col_1 int, col_2 text); CREATE TABLE non_mx_test_table (col_1 int, col_2 text);
@ -331,6 +334,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
RESET ROLE RESET ROLE
SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''') SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''')
SELECT citus_internal_add_partition_metadata ('mx_testing_schema.mx_test_table'::regclass, 'h', 'col_1', 2, 's') SELECT citus_internal_add_partition_metadata ('mx_testing_schema.mx_test_table'::regclass, 'h', 'col_1', 2, 's')
SELECT pg_catalog.worker_drop_sequence_dependency('mx_testing_schema.mx_test_table');
SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition
SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3') SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3')
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint') SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint')
@ -357,7 +361,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['mx_testing_schema', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data; WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['mx_testing_schema', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data; WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
(52 rows) (53 rows)
-- Show that range distributed tables are not included in the activate node snapshot -- Show that range distributed tables are not included in the activate node snapshot
UPDATE pg_dist_partition SET partmethod='r' WHERE logicalrelid='non_mx_test_table'::regclass; UPDATE pg_dist_partition SET partmethod='r' WHERE logicalrelid='non_mx_test_table'::regclass;
@ -390,6 +394,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
RESET ROLE RESET ROLE
SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''') SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''')
SELECT citus_internal_add_partition_metadata ('mx_testing_schema.mx_test_table'::regclass, 'h', 'col_1', 2, 's') SELECT citus_internal_add_partition_metadata ('mx_testing_schema.mx_test_table'::regclass, 'h', 'col_1', 2, 's')
SELECT pg_catalog.worker_drop_sequence_dependency('mx_testing_schema.mx_test_table');
SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition
SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3') SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3')
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint') SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint')
@ -416,7 +421,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['mx_testing_schema', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data; WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['mx_testing_schema', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data; WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
(52 rows) (53 rows)
-- Test start_metadata_sync_to_node and citus_activate_node UDFs -- Test start_metadata_sync_to_node and citus_activate_node UDFs
-- Ensure that hasmetadata=false for all nodes -- Ensure that hasmetadata=false for all nodes
@ -1943,6 +1948,12 @@ SELECT unnest(activate_node_snapshot()) order by 1;
SELECT citus_internal_add_partition_metadata ('public.dist_table_1'::regclass, 'h', 'a', 10010, 's') SELECT citus_internal_add_partition_metadata ('public.dist_table_1'::regclass, 'h', 'a', 10010, 's')
SELECT citus_internal_add_partition_metadata ('public.mx_ref'::regclass, 'n', NULL, 10009, 't') SELECT citus_internal_add_partition_metadata ('public.mx_ref'::regclass, 'n', NULL, 10009, 't')
SELECT citus_internal_add_partition_metadata ('public.test_table'::regclass, 'h', 'id', 10010, 's') SELECT citus_internal_add_partition_metadata ('public.test_table'::regclass, 'h', 'id', 10010, 's')
SELECT pg_catalog.worker_drop_sequence_dependency('mx_test_schema_1.mx_table_1');
SELECT pg_catalog.worker_drop_sequence_dependency('mx_test_schema_2.mx_table_2');
SELECT pg_catalog.worker_drop_sequence_dependency('mx_testing_schema.mx_test_table');
SELECT pg_catalog.worker_drop_sequence_dependency('public.dist_table_1');
SELECT pg_catalog.worker_drop_sequence_dependency('public.mx_ref');
SELECT pg_catalog.worker_drop_sequence_dependency('public.test_table');
SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition
SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3') SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3')
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint') SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint')
@ -1997,7 +2008,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.dist_table_1'::regclass, 1310074, 't'::"char", '-2147483648', '-1073741825'), ('public.dist_table_1'::regclass, 1310075, 't'::"char", '-1073741824', '-1'), ('public.dist_table_1'::regclass, 1310076, 't'::"char", '0', '1073741823'), ('public.dist_table_1'::regclass, 1310077, 't'::"char", '1073741824', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.dist_table_1'::regclass, 1310074, 't'::"char", '-2147483648', '-1073741825'), ('public.dist_table_1'::regclass, 1310075, 't'::"char", '-1073741824', '-1'), ('public.dist_table_1'::regclass, 1310076, 't'::"char", '0', '1073741823'), ('public.dist_table_1'::regclass, 1310077, 't'::"char", '1073741824', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_ref'::regclass, 1310073, 't'::"char", NULL, NULL)) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_ref'::regclass, 1310073, 't'::"char", NULL, NULL)) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.test_table'::regclass, 1310083, 't'::"char", '-2147483648', '-1073741825'), ('public.test_table'::regclass, 1310084, 't'::"char", '-1073741824', '-1'), ('public.test_table'::regclass, 1310085, 't'::"char", '0', '1073741823'), ('public.test_table'::regclass, 1310086, 't'::"char", '1073741824', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.test_table'::regclass, 1310083, 't'::"char", '-2147483648', '-1073741825'), ('public.test_table'::regclass, 1310084, 't'::"char", '-1073741824', '-1'), ('public.test_table'::regclass, 1310085, 't'::"char", '0', '1073741823'), ('public.test_table'::regclass, 1310086, 't'::"char", '1073741824', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
(111 rows) (117 rows)
-- shouldn't work since test_table is MX -- shouldn't work since test_table is MX
ALTER TABLE test_table ADD COLUMN id3 bigserial; ALTER TABLE test_table ADD COLUMN id3 bigserial;

View File

@ -149,6 +149,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
RESET ROLE RESET ROLE
SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''') SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''')
SELECT citus_internal_add_partition_metadata ('public.mx_test_table'::regclass, 'h', 'col_1', 2, 's') SELECT citus_internal_add_partition_metadata ('public.mx_test_table'::regclass, 'h', 'col_1', 2, 's')
SELECT pg_catalog.worker_drop_sequence_dependency('public.mx_test_table');
SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition
SELECT pg_catalog.worker_record_sequence_dependency('public.mx_test_table_col_3_seq'::regclass,'public.mx_test_table'::regclass,'col_3') SELECT pg_catalog.worker_record_sequence_dependency('public.mx_test_table_col_3_seq'::regclass,'public.mx_test_table'::regclass,'col_3')
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint') SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint')
@ -174,7 +175,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['public', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data; WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['public', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data; WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
(49 rows) (50 rows)
-- Show that CREATE INDEX commands are included in the activate node snapshot -- Show that CREATE INDEX commands are included in the activate node snapshot
CREATE INDEX mx_index ON mx_test_table(col_2); CREATE INDEX mx_index ON mx_test_table(col_2);
@ -206,6 +207,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
RESET ROLE RESET ROLE
SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''') SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''')
SELECT citus_internal_add_partition_metadata ('public.mx_test_table'::regclass, 'h', 'col_1', 2, 's') SELECT citus_internal_add_partition_metadata ('public.mx_test_table'::regclass, 'h', 'col_1', 2, 's')
SELECT pg_catalog.worker_drop_sequence_dependency('public.mx_test_table');
SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition
SELECT pg_catalog.worker_record_sequence_dependency('public.mx_test_table_col_3_seq'::regclass,'public.mx_test_table'::regclass,'col_3') SELECT pg_catalog.worker_record_sequence_dependency('public.mx_test_table_col_3_seq'::regclass,'public.mx_test_table'::regclass,'col_3')
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint') SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint')
@ -231,7 +233,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['public', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data; WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['public', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data; WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('public.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('public.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('public.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('public.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('public.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('public.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('public.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
(50 rows) (51 rows)
-- Show that schema changes are included in the activate node snapshot -- Show that schema changes are included in the activate node snapshot
CREATE SCHEMA mx_testing_schema; CREATE SCHEMA mx_testing_schema;
@ -265,6 +267,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
RESET ROLE RESET ROLE
SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''') SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''')
SELECT citus_internal_add_partition_metadata ('mx_testing_schema.mx_test_table'::regclass, 'h', 'col_1', 2, 's') SELECT citus_internal_add_partition_metadata ('mx_testing_schema.mx_test_table'::regclass, 'h', 'col_1', 2, 's')
SELECT pg_catalog.worker_drop_sequence_dependency('mx_testing_schema.mx_test_table');
SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition
SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3') SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3')
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint') SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint')
@ -291,7 +294,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['mx_testing_schema', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data; WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['mx_testing_schema', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data; WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
(52 rows) (53 rows)
-- Show that append distributed tables are not included in the activate node snapshot -- Show that append distributed tables are not included in the activate node snapshot
CREATE TABLE non_mx_test_table (col_1 int, col_2 text); CREATE TABLE non_mx_test_table (col_1 int, col_2 text);
@ -331,6 +334,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
RESET ROLE RESET ROLE
SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''') SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''')
SELECT citus_internal_add_partition_metadata ('mx_testing_schema.mx_test_table'::regclass, 'h', 'col_1', 2, 's') SELECT citus_internal_add_partition_metadata ('mx_testing_schema.mx_test_table'::regclass, 'h', 'col_1', 2, 's')
SELECT pg_catalog.worker_drop_sequence_dependency('mx_testing_schema.mx_test_table');
SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition
SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3') SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3')
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint') SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint')
@ -357,7 +361,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['mx_testing_schema', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data; WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['mx_testing_schema', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data; WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
(52 rows) (53 rows)
-- Show that range distributed tables are not included in the activate node snapshot -- Show that range distributed tables are not included in the activate node snapshot
UPDATE pg_dist_partition SET partmethod='r' WHERE logicalrelid='non_mx_test_table'::regclass; UPDATE pg_dist_partition SET partmethod='r' WHERE logicalrelid='non_mx_test_table'::regclass;
@ -390,6 +394,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
RESET ROLE RESET ROLE
SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''') SELECT alter_role_if_exists('postgres', 'ALTER ROLE postgres SET lc_messages = ''C''')
SELECT citus_internal_add_partition_metadata ('mx_testing_schema.mx_test_table'::regclass, 'h', 'col_1', 2, 's') SELECT citus_internal_add_partition_metadata ('mx_testing_schema.mx_test_table'::regclass, 'h', 'col_1', 2, 's')
SELECT pg_catalog.worker_drop_sequence_dependency('mx_testing_schema.mx_test_table');
SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition
SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3') SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3')
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint') SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint')
@ -416,7 +421,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['mx_testing_schema', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data; WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('table', ARRAY['mx_testing_schema', 'mx_test_table']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data; WITH placement_data(shardid, shardlength, groupid, placementid) AS (VALUES (1310000, 0, 1, 100000), (1310001, 0, 2, 100001), (1310002, 0, 1, 100002), (1310003, 0, 2, 100003), (1310004, 0, 1, 100004), (1310005, 0, 2, 100005), (1310006, 0, 1, 100006), (1310007, 0, 2, 100007)) SELECT citus_internal_add_placement_metadata(shardid, shardlength, groupid, placementid) FROM placement_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't'::"char", '-2147483648', '-1610612737'), ('mx_testing_schema.mx_test_table'::regclass, 1310001, 't'::"char", '-1610612736', '-1073741825'), ('mx_testing_schema.mx_test_table'::regclass, 1310002, 't'::"char", '-1073741824', '-536870913'), ('mx_testing_schema.mx_test_table'::regclass, 1310003, 't'::"char", '-536870912', '-1'), ('mx_testing_schema.mx_test_table'::regclass, 1310004, 't'::"char", '0', '536870911'), ('mx_testing_schema.mx_test_table'::regclass, 1310005, 't'::"char", '536870912', '1073741823'), ('mx_testing_schema.mx_test_table'::regclass, 1310006, 't'::"char", '1073741824', '1610612735'), ('mx_testing_schema.mx_test_table'::regclass, 1310007, 't'::"char", '1610612736', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
(52 rows) (53 rows)
-- Test start_metadata_sync_to_node and citus_activate_node UDFs -- Test start_metadata_sync_to_node and citus_activate_node UDFs
-- Ensure that hasmetadata=false for all nodes -- Ensure that hasmetadata=false for all nodes
@ -1943,6 +1948,12 @@ SELECT unnest(activate_node_snapshot()) order by 1;
SELECT citus_internal_add_partition_metadata ('public.dist_table_1'::regclass, 'h', 'a', 10010, 's') SELECT citus_internal_add_partition_metadata ('public.dist_table_1'::regclass, 'h', 'a', 10010, 's')
SELECT citus_internal_add_partition_metadata ('public.mx_ref'::regclass, 'n', NULL, 10009, 't') SELECT citus_internal_add_partition_metadata ('public.mx_ref'::regclass, 'n', NULL, 10009, 't')
SELECT citus_internal_add_partition_metadata ('public.test_table'::regclass, 'h', 'id', 10010, 's') SELECT citus_internal_add_partition_metadata ('public.test_table'::regclass, 'h', 'id', 10010, 's')
SELECT pg_catalog.worker_drop_sequence_dependency('mx_test_schema_1.mx_table_1');
SELECT pg_catalog.worker_drop_sequence_dependency('mx_test_schema_2.mx_table_2');
SELECT pg_catalog.worker_drop_sequence_dependency('mx_testing_schema.mx_test_table');
SELECT pg_catalog.worker_drop_sequence_dependency('public.dist_table_1');
SELECT pg_catalog.worker_drop_sequence_dependency('public.mx_ref');
SELECT pg_catalog.worker_drop_sequence_dependency('public.test_table');
SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition SELECT pg_catalog.worker_drop_sequence_dependency(logicalrelid::regclass::text) FROM pg_dist_partition
SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3') SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3')
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint') SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq AS bigint INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 CACHE 1 NO CYCLE','bigint')
@ -1997,7 +2008,7 @@ SELECT unnest(activate_node_snapshot()) order by 1;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.dist_table_1'::regclass, 1310074, 't'::"char", '-2147483648', '-1073741825'), ('public.dist_table_1'::regclass, 1310075, 't'::"char", '-1073741824', '-1'), ('public.dist_table_1'::regclass, 1310076, 't'::"char", '0', '1073741823'), ('public.dist_table_1'::regclass, 1310077, 't'::"char", '1073741824', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.dist_table_1'::regclass, 1310074, 't'::"char", '-2147483648', '-1073741825'), ('public.dist_table_1'::regclass, 1310075, 't'::"char", '-1073741824', '-1'), ('public.dist_table_1'::regclass, 1310076, 't'::"char", '0', '1073741823'), ('public.dist_table_1'::regclass, 1310077, 't'::"char", '1073741824', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_ref'::regclass, 1310073, 't'::"char", NULL, NULL)) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.mx_ref'::regclass, 1310073, 't'::"char", NULL, NULL)) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.test_table'::regclass, 1310083, 't'::"char", '-2147483648', '-1073741825'), ('public.test_table'::regclass, 1310084, 't'::"char", '-1073741824', '-1'), ('public.test_table'::regclass, 1310085, 't'::"char", '0', '1073741823'), ('public.test_table'::regclass, 1310086, 't'::"char", '1073741824', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data; WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) AS (VALUES ('public.test_table'::regclass, 1310083, 't'::"char", '-2147483648', '-1073741825'), ('public.test_table'::regclass, 1310084, 't'::"char", '-1073741824', '-1'), ('public.test_table'::regclass, 1310085, 't'::"char", '0', '1073741823'), ('public.test_table'::regclass, 1310086, 't'::"char", '1073741824', '2147483647')) SELECT citus_internal_add_shard_metadata(relationname, shardid, storagetype, shardminvalue, shardmaxvalue) FROM shard_data;
(111 rows) (117 rows)
-- shouldn't work since test_table is MX -- shouldn't work since test_table is MX
ALTER TABLE test_table ADD COLUMN id3 bigserial; ALTER TABLE test_table ADD COLUMN id3 bigserial;

View File

@ -306,7 +306,7 @@ SELECT citus_add_local_table_to_metadata('tbl2');
MERGE INTO tbl1 USING tbl2 ON (true) MERGE INTO tbl1 USING tbl2 ON (true)
WHEN MATCHED THEN DELETE; WHEN MATCHED THEN DELETE;
-- one table is reference, the other local, not supported -- source table is reference, the target is local, supported
SELECT create_reference_table('tbl2'); SELECT create_reference_table('tbl2');
create_reference_table create_reference_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -315,8 +315,7 @@ SELECT create_reference_table('tbl2');
MERGE INTO tbl1 USING tbl2 ON (true) MERGE INTO tbl1 USING tbl2 ON (true)
WHEN MATCHED THEN DELETE; WHEN MATCHED THEN DELETE;
ERROR: MERGE command is not supported on reference tables yet -- now, both are reference, not supported
-- now, both are reference, still not supported
SELECT create_reference_table('tbl1'); SELECT create_reference_table('tbl1');
create_reference_table create_reference_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -325,7 +324,7 @@ SELECT create_reference_table('tbl1');
MERGE INTO tbl1 USING tbl2 ON (true) MERGE INTO tbl1 USING tbl2 ON (true)
WHEN MATCHED THEN DELETE; WHEN MATCHED THEN DELETE;
ERROR: MERGE command is not supported on reference tables yet ERROR: Reference table as target is not allowed in MERGE command
-- now, both distributed, not works -- now, both distributed, not works
SELECT undistribute_table('tbl1'); SELECT undistribute_table('tbl1');
NOTICE: creating a new table for pg15.tbl1 NOTICE: creating a new table for pg15.tbl1

File diff suppressed because it is too large Load Diff

View File

@ -20,13 +20,14 @@ SELECT create_distributed_table('dist_table_test', 'a');
CREATE TABLE postgres_table_test(a int primary key); CREATE TABLE postgres_table_test(a int primary key);
-- make sure that all rebalance operations works fine when -- make sure that all rebalance operations works fine when
-- reference tables are replicated to the coordinator -- reference tables are replicated to the coordinator
SET client_min_messages TO ERROR;
SELECT 1 FROM master_add_node('localhost', :master_port, groupId=>0); SELECT 1 FROM master_add_node('localhost', :master_port, groupId=>0);
NOTICE: localhost:xxxxx is the coordinator and already contains metadata, skipping syncing the metadata
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
RESET client_min_messages;
-- should just be noops even if we add the coordinator to the pg_dist_node -- should just be noops even if we add the coordinator to the pg_dist_node
SELECT rebalance_table_shards('dist_table_test'); SELECT rebalance_table_shards('dist_table_test');
rebalance_table_shards rebalance_table_shards
@ -2713,6 +2714,113 @@ SELECT sh.logicalrelid, pl.nodeport
(5 rows) (5 rows)
DROP TABLE single_shard_colocation_1a, single_shard_colocation_1b, single_shard_colocation_1c, single_shard_colocation_2a, single_shard_colocation_2b CASCADE; DROP TABLE single_shard_colocation_1a, single_shard_colocation_1b, single_shard_colocation_1c, single_shard_colocation_2a, single_shard_colocation_2b CASCADE;
-- test the same with coordinator shouldhaveshards = false and shard_count = 2
-- so that the shard allowed node count would be 2 when rebalancing
-- for such cases, we only count the nodes that are allowed for shard placements
UPDATE pg_dist_node SET shouldhaveshards=false WHERE nodeport = :master_port;
create table two_shard_colocation_1a (a int primary key);
create table two_shard_colocation_1b (a int primary key);
SET citus.shard_replication_factor = 1;
select create_distributed_table('two_shard_colocation_1a','a', colocate_with => 'none', shard_count => 2);
create_distributed_table
---------------------------------------------------------------------
(1 row)
select create_distributed_table('two_shard_colocation_1b','a',colocate_with=>'two_shard_colocation_1a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
create table two_shard_colocation_2a (a int primary key);
create table two_shard_colocation_2b (a int primary key);
select create_distributed_table('two_shard_colocation_2a','a', colocate_with => 'none', shard_count => 2);
create_distributed_table
---------------------------------------------------------------------
(1 row)
select create_distributed_table('two_shard_colocation_2b','a',colocate_with=>'two_shard_colocation_2a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- move shards of colocation group 1 to worker1
SELECT citus_move_shard_placement(sh.shardid, 'localhost', :worker_2_port, 'localhost', :worker_1_port)
FROM pg_dist_shard sh JOIN pg_dist_shard_placement pl ON sh.shardid = pl.shardid
WHERE sh.logicalrelid = 'two_shard_colocation_1a'::regclass
AND pl.nodeport = :worker_2_port
LIMIT 1;
citus_move_shard_placement
---------------------------------------------------------------------
(1 row)
-- move shards of colocation group 2 to worker2
SELECT citus_move_shard_placement(sh.shardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port)
FROM pg_dist_shard sh JOIN pg_dist_shard_placement pl ON sh.shardid = pl.shardid
WHERE sh.logicalrelid = 'two_shard_colocation_2a'::regclass
AND pl.nodeport = :worker_1_port
LIMIT 1;
citus_move_shard_placement
---------------------------------------------------------------------
(1 row)
-- current state:
-- coordinator: []
-- worker 1: [1_1, 1_2]
-- worker 2: [2_1, 2_2]
SELECT sh.logicalrelid, pl.nodeport
FROM pg_dist_shard sh JOIN pg_dist_shard_placement pl ON sh.shardid = pl.shardid
WHERE sh.logicalrelid::text IN ('two_shard_colocation_1a', 'two_shard_colocation_1b', 'two_shard_colocation_2a', 'two_shard_colocation_2b')
ORDER BY sh.logicalrelid, pl.nodeport;
logicalrelid | nodeport
---------------------------------------------------------------------
two_shard_colocation_1a | 57637
two_shard_colocation_1a | 57637
two_shard_colocation_1b | 57637
two_shard_colocation_1b | 57637
two_shard_colocation_2a | 57638
two_shard_colocation_2a | 57638
two_shard_colocation_2b | 57638
two_shard_colocation_2b | 57638
(8 rows)
-- If we take the coordinator into account, the rebalancer considers this as balanced and does nothing (shard_count < worker_count)
-- but because the coordinator is not allowed for shards, rebalancer will distribute each colocation group to both workers
select rebalance_table_shards(shard_transfer_mode:='block_writes');
NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ...
NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ...
rebalance_table_shards
---------------------------------------------------------------------
(1 row)
-- final state:
-- coordinator: []
-- worker 1: [1_1, 2_1]
-- worker 2: [1_2, 2_2]
SELECT sh.logicalrelid, pl.nodeport
FROM pg_dist_shard sh JOIN pg_dist_shard_placement pl ON sh.shardid = pl.shardid
WHERE sh.logicalrelid::text IN ('two_shard_colocation_1a', 'two_shard_colocation_1b', 'two_shard_colocation_2a', 'two_shard_colocation_2b')
ORDER BY sh.logicalrelid, pl.nodeport;
logicalrelid | nodeport
---------------------------------------------------------------------
two_shard_colocation_1a | 57637
two_shard_colocation_1a | 57638
two_shard_colocation_1b | 57637
two_shard_colocation_1b | 57638
two_shard_colocation_2a | 57637
two_shard_colocation_2a | 57638
two_shard_colocation_2b | 57637
two_shard_colocation_2b | 57638
(8 rows)
-- cleanup
DROP TABLE two_shard_colocation_1a, two_shard_colocation_1b, two_shard_colocation_2a, two_shard_colocation_2b CASCADE;
-- verify we detect if one of the tables do not have a replica identity or primary key -- verify we detect if one of the tables do not have a replica identity or primary key
-- and error out in case of shard transfer mode = auto -- and error out in case of shard transfer mode = auto
SELECT 1 FROM citus_remove_node('localhost', :worker_2_port); SELECT 1 FROM citus_remove_node('localhost', :worker_2_port);

View File

@ -101,8 +101,63 @@ SELECT pg_reload_conf();
t t
(1 row) (1 row)
CREATE TABLE single_node_nullkey_c1(a int, b int);
SELECT create_distributed_table('single_node_nullkey_c1', null, colocate_with=>'none', distribution_type=>null);
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE single_node_nullkey_c2(a int, b int);
SELECT create_distributed_table('single_node_nullkey_c2', null, colocate_with=>'none', distribution_type=>null);
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- created on different colocation groups ..
SELECT
(
SELECT colocationid FROM pg_dist_partition
WHERE logicalrelid = 'single_node.single_node_nullkey_c1'::regclass
)
!=
(
SELECT colocationid FROM pg_dist_partition
WHERE logicalrelid = 'single_node.single_node_nullkey_c2'::regclass
);
?column?
---------------------------------------------------------------------
t
(1 row)
-- .. but both are associated to coordinator
SELECT groupid = 0 FROM pg_dist_placement
WHERE shardid = (
SELECT shardid FROM pg_dist_shard
WHERE logicalrelid = 'single_node.single_node_nullkey_c1'::regclass
);
?column?
---------------------------------------------------------------------
t
(1 row)
SELECT groupid = 0 FROM pg_dist_placement
WHERE shardid = (
SELECT shardid FROM pg_dist_shard
WHERE logicalrelid = 'single_node.single_node_nullkey_c2'::regclass
);
?column?
---------------------------------------------------------------------
t
(1 row)
-- try creating a 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
SET client_min_messages TO WARNING; SET client_min_messages TO WARNING;
DROP TABLE failover_to_local; DROP TABLE failover_to_local, single_node_nullkey_c1, single_node_nullkey_c2;
RESET client_min_messages; RESET client_min_messages;
-- so that we don't have to update rest of the test output -- so that we don't have to update rest of the test output
SET citus.next_shard_id TO 90630500; SET citus.next_shard_id TO 90630500;

View File

@ -101,8 +101,63 @@ SELECT pg_reload_conf();
t t
(1 row) (1 row)
CREATE TABLE single_node_nullkey_c1(a int, b int);
SELECT create_distributed_table('single_node_nullkey_c1', null, colocate_with=>'none', distribution_type=>null);
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE single_node_nullkey_c2(a int, b int);
SELECT create_distributed_table('single_node_nullkey_c2', null, colocate_with=>'none', distribution_type=>null);
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- created on different colocation groups ..
SELECT
(
SELECT colocationid FROM pg_dist_partition
WHERE logicalrelid = 'single_node.single_node_nullkey_c1'::regclass
)
!=
(
SELECT colocationid FROM pg_dist_partition
WHERE logicalrelid = 'single_node.single_node_nullkey_c2'::regclass
);
?column?
---------------------------------------------------------------------
t
(1 row)
-- .. but both are associated to coordinator
SELECT groupid = 0 FROM pg_dist_placement
WHERE shardid = (
SELECT shardid FROM pg_dist_shard
WHERE logicalrelid = 'single_node.single_node_nullkey_c1'::regclass
);
?column?
---------------------------------------------------------------------
t
(1 row)
SELECT groupid = 0 FROM pg_dist_placement
WHERE shardid = (
SELECT shardid FROM pg_dist_shard
WHERE logicalrelid = 'single_node.single_node_nullkey_c2'::regclass
);
?column?
---------------------------------------------------------------------
t
(1 row)
-- try creating a 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
SET client_min_messages TO WARNING; SET client_min_messages TO WARNING;
DROP TABLE failover_to_local; DROP TABLE failover_to_local, single_node_nullkey_c1, single_node_nullkey_c2;
RESET client_min_messages; RESET client_min_messages;
-- so that we don't have to update rest of the test output -- so that we don't have to update rest of the test output
SET citus.next_shard_id TO 90630500; SET citus.next_shard_id TO 90630500;

View File

@ -0,0 +1,13 @@
ALTER FUNCTION create_distributed_table RENAME TO create_distributed_table_internal;
CREATE OR REPLACE FUNCTION pg_catalog.create_distributed_table(table_name regclass,
distribution_column text,
distribution_type citus.distribution_type DEFAULT 'hash',
colocate_with text DEFAULT 'default',
shard_count int DEFAULT NULL)
RETURNS void
LANGUAGE plpgsql
AS $function$
BEGIN
PERFORM create_distributed_table_internal(table_name, NULL, NULL, colocate_with, NULL);
END;
$function$;

View File

@ -32,6 +32,7 @@ test: escape_extension_name
test: ref_citus_local_fkeys test: ref_citus_local_fkeys
test: alter_database_owner test: alter_database_owner
test: distributed_triggers test: distributed_triggers
test: create_single_shard_table
test: multi_test_catalog_views test: multi_test_catalog_views
test: multi_table_ddl test: multi_table_ddl
@ -67,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_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: multi_shard_update_delete recursive_dml_with_different_planners_executors
test: insert_select_repartition window_functions dml_recursive multi_insert_select_window test: insert_select_repartition window_functions dml_recursive multi_insert_select_window
test: multi_insert_select_conflict citus_table_triggers test: multi_insert_select_conflict citus_table_triggers alter_table_single_shard_table
test: multi_row_insert insert_select_into_local_table alter_index 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 # following should not run in parallel because it relies on connection counts to workers
@ -199,6 +200,8 @@ test: local_table_join
test: local_dist_join_mixed test: local_dist_join_mixed
test: citus_local_dist_joins test: citus_local_dist_joins
test: recurring_outer_join test: recurring_outer_join
test: query_single_shard_table
test: insert_select_single_shard_table
test: pg_dump test: pg_dump
# --------- # ---------

View File

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

View File

@ -0,0 +1,98 @@
CREATE SCHEMA alter_null_dist_key;
SET search_path TO alter_null_dist_key;
SET citus.next_shard_id TO 1720000;
SET citus.shard_count TO 32;
SET citus.shard_replication_factor TO 1;
CREATE SEQUENCE dist_seq;
CREATE TABLE null_dist_table(a bigint DEFAULT nextval('dist_seq') UNIQUE, "b" text, c bigint GENERATED BY DEFAULT AS IDENTITY);
INSERT INTO null_dist_table("b") VALUES ('test');
SELECT create_distributed_table('null_dist_table', null, colocate_with=>'none', distribution_type=>null);
-- add column
ALTER TABLE null_dist_table ADD COLUMN d bigint DEFAULT 2;
SELECT * FROM null_dist_table ORDER BY c;
-- alter default, set to 3
ALTER TABLE null_dist_table ALTER COLUMN d SET DEFAULT 3;
INSERT INTO null_dist_table("b") VALUES ('test');
SELECT * FROM null_dist_table ORDER BY c;
-- drop default, see null
ALTER TABLE null_dist_table ALTER COLUMN d DROP DEFAULT;
INSERT INTO null_dist_table("b") VALUES ('test');
SELECT * FROM null_dist_table ORDER BY c;
-- cleanup the rows that were added to test the default behavior
DELETE FROM null_dist_table WHERE "b" = 'test' AND a > 1;
-- alter column type
ALTER TABLE null_dist_table ALTER COLUMN d TYPE text;
UPDATE null_dist_table SET d = 'this is a text' WHERE d = '2';
SELECT * FROM null_dist_table ORDER BY c;
-- drop seq column
ALTER TABLE null_dist_table DROP COLUMN a;
SELECT * FROM null_dist_table ORDER BY c;
-- add not null constraint
ALTER TABLE null_dist_table ALTER COLUMN b SET NOT NULL;
-- not null constraint violation, error out
INSERT INTO null_dist_table VALUES (NULL, 2, 'test');
-- drop not null constraint and try again
ALTER TABLE null_dist_table ALTER COLUMN b DROP NOT NULL;
INSERT INTO null_dist_table VALUES (NULL, 3, 'test');
SELECT * FROM null_dist_table ORDER BY c;
-- add exclusion constraint
ALTER TABLE null_dist_table ADD CONSTRAINT exc_b EXCLUDE USING btree (b with =);
-- rename the exclusion constraint, errors out
ALTER TABLE null_dist_table RENAME CONSTRAINT exc_b TO exc_b_1;
-- create exclusion constraint without a name
ALTER TABLE null_dist_table ADD EXCLUDE USING btree (b with =);
-- test setting autovacuum option
ALTER TABLE null_dist_table SET (autovacuum_enabled = false);
-- test multiple subcommands
ALTER TABLE null_dist_table ADD COLUMN int_column1 INTEGER,
DROP COLUMN d;
SELECT * FROM null_dist_table ORDER BY c;
-- test policy and row level security
CREATE TABLE null_dist_key_with_policy (table_user text);
INSERT INTO null_dist_key_with_policy VALUES ('user_1');
SELECT create_distributed_table('null_dist_key_with_policy', null);
-- enable rls
ALTER TABLE null_dist_key_with_policy ENABLE ROW LEVEL SECURITY;
-- user_1 will be allowed to see the inserted row
CREATE ROLE user_1 WITH LOGIN;
GRANT ALL ON SCHEMA alter_null_dist_key TO user_1;
GRANT ALL ON TABLE alter_null_dist_key.null_dist_key_with_policy TO user_1;
CREATE POLICY table_policy ON null_dist_key_with_policy TO user_1
USING (table_user = current_user);
-- user_2 will not be allowed to see the inserted row
CREATE ROLE user_2 WITH LOGIN;
GRANT ALL ON SCHEMA alter_null_dist_key TO user_2;
GRANT ALL ON TABLE alter_null_dist_key.null_dist_key_with_policy TO user_2;
CREATE POLICY table_policy_1 ON null_dist_key_with_policy TO user_2
USING (table_user = current_user);
\c - user_1 -
SELECT * FROM alter_null_dist_key.null_dist_key_with_policy;
\c - user_2 -
SELECT * FROM alter_null_dist_key.null_dist_key_with_policy;
-- postgres will always be allowed to see the row as a superuser
\c - postgres -
SELECT * FROM alter_null_dist_key.null_dist_key_with_policy;
-- cleanup
SET client_min_messages TO ERROR;
DROP SCHEMA alter_null_dist_key CASCADE;
DROP ROLE user_1, user_2;

File diff suppressed because it is too large Load Diff

View File

@ -18,11 +18,36 @@ SET client_min_messages TO ERROR;
CREATE ROLE foo1; CREATE ROLE foo1;
CREATE ROLE foo2; CREATE ROLE foo2;
-- Create collation
CREATE COLLATION german_phonebook (provider = icu, locale = 'de-u-co-phonebk');
-- Create type
CREATE TYPE pair_type AS (a int, b int);
-- Create function
CREATE FUNCTION one_as_result() RETURNS INT LANGUAGE SQL AS
$$
SELECT 1;
$$;
-- Create text search dictionary
CREATE TEXT SEARCH DICTIONARY my_german_dict (
template = snowball,
language = german,
stopwords = german
);
-- Create text search config
CREATE TEXT SEARCH CONFIGURATION my_ts_config ( parser = default );
ALTER TEXT SEARCH CONFIGURATION my_ts_config ALTER MAPPING FOR asciiword WITH my_german_dict;
-- Create sequence -- Create sequence
CREATE SEQUENCE seq; CREATE SEQUENCE seq;
-- Create colocated distributed tables -- Create colocated distributed tables
CREATE TABLE dist1 (id int PRIMARY KEY default nextval('seq')); CREATE TABLE dist1 (id int PRIMARY KEY default nextval('seq'), col int default (one_as_result()), myserial serial, phone text COLLATE german_phonebook, initials pair_type);
CREATE SEQUENCE seq_owned OWNED BY dist1.id;
CREATE INDEX dist1_search_phone_idx ON dist1 USING gin (to_tsvector('my_ts_config'::regconfig, (COALESCE(phone, ''::text))::text));
SELECT create_distributed_table('dist1', 'id'); SELECT create_distributed_table('dist1', 'id');
INSERT INTO dist1 SELECT i FROM generate_series(1,100) i; INSERT INTO dist1 SELECT i FROM generate_series(1,100) i;
@ -42,6 +67,9 @@ INSERT INTO loc1 SELECT i FROM generate_series(1,100) i;
CREATE TABLE loc2 (id int REFERENCES loc1(id)); CREATE TABLE loc2 (id int REFERENCES loc1(id));
INSERT INTO loc2 SELECT i FROM generate_series(1,100) i; INSERT INTO loc2 SELECT i FROM generate_series(1,100) i;
-- Create publication
CREATE PUBLICATION pub_all;
-- citus_set_coordinator_host with wrong port -- citus_set_coordinator_host with wrong port
SELECT citus_set_coordinator_host('localhost', 9999); SELECT citus_set_coordinator_host('localhost', 9999);
-- citus_set_coordinator_host with correct port -- citus_set_coordinator_host with correct port
@ -88,10 +116,10 @@ SELECT citus_activate_node('localhost', :worker_2_proxy_port);
SELECT citus.mitmproxy('conn.onQuery(query="INSERT INTO pg_dist_node").kill()'); SELECT citus.mitmproxy('conn.onQuery(query="INSERT INTO pg_dist_node").kill()');
SELECT citus_activate_node('localhost', :worker_2_proxy_port); SELECT citus_activate_node('localhost', :worker_2_proxy_port);
-- Failure to drop sequence -- Failure to drop sequence dependency for all tables
SELECT citus.mitmproxy('conn.onQuery(query="SELECT pg_catalog.worker_drop_sequence_dependency").cancel(' || :pid || ')'); SELECT citus.mitmproxy('conn.onQuery(query="SELECT pg_catalog.worker_drop_sequence_dependency.*FROM pg_dist_partition").cancel(' || :pid || ')');
SELECT citus_activate_node('localhost', :worker_2_proxy_port); SELECT citus_activate_node('localhost', :worker_2_proxy_port);
SELECT citus.mitmproxy('conn.onQuery(query="SELECT pg_catalog.worker_drop_sequence_dependency").kill()'); SELECT citus.mitmproxy('conn.onQuery(query="SELECT pg_catalog.worker_drop_sequence_dependency.*FROM pg_dist_partition").kill()');
SELECT citus_activate_node('localhost', :worker_2_proxy_port); SELECT citus_activate_node('localhost', :worker_2_proxy_port);
-- Failure to drop shell table -- Failure to drop shell table
@ -142,24 +170,84 @@ SELECT citus_activate_node('localhost', :worker_2_proxy_port);
SELECT citus.mitmproxy('conn.onQuery(query="ALTER DATABASE.*OWNER TO").kill()'); SELECT citus.mitmproxy('conn.onQuery(query="ALTER DATABASE.*OWNER TO").kill()');
SELECT citus_activate_node('localhost', :worker_2_proxy_port); SELECT citus_activate_node('localhost', :worker_2_proxy_port);
-- Filure to create schema -- Failure to create schema
SELECT citus.mitmproxy('conn.onQuery(query="CREATE SCHEMA IF NOT EXISTS mx_metadata_sync_multi_trans AUTHORIZATION").cancel(' || :pid || ')'); SELECT citus.mitmproxy('conn.onQuery(query="CREATE SCHEMA IF NOT EXISTS mx_metadata_sync_multi_trans AUTHORIZATION").cancel(' || :pid || ')');
SELECT citus_activate_node('localhost', :worker_2_proxy_port); SELECT citus_activate_node('localhost', :worker_2_proxy_port);
SELECT citus.mitmproxy('conn.onQuery(query="CREATE SCHEMA IF NOT EXISTS mx_metadata_sync_multi_trans AUTHORIZATION").kill()'); SELECT citus.mitmproxy('conn.onQuery(query="CREATE SCHEMA IF NOT EXISTS mx_metadata_sync_multi_trans AUTHORIZATION").kill()');
SELECT citus_activate_node('localhost', :worker_2_proxy_port); SELECT citus_activate_node('localhost', :worker_2_proxy_port);
-- Failure to create collation
SELECT citus.mitmproxy('conn.onQuery(query="SELECT worker_create_or_replace_object.*CREATE COLLATION mx_metadata_sync_multi_trans.german_phonebook").cancel(' || :pid || ')');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
SELECT citus.mitmproxy('conn.onQuery(query="SELECT worker_create_or_replace_object.*CREATE COLLATION mx_metadata_sync_multi_trans.german_phonebook").kill()');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
-- Failure to create function
SELECT citus.mitmproxy('conn.onQuery(query="CREATE OR REPLACE FUNCTION mx_metadata_sync_multi_trans.one_as_result").cancel(' || :pid || ')');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
SELECT citus.mitmproxy('conn.onQuery(query="CREATE OR REPLACE FUNCTION mx_metadata_sync_multi_trans.one_as_result").kill()');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
-- Failure to create text search dictionary
SELECT citus.mitmproxy('conn.onQuery(query="SELECT worker_create_or_replace_object.*my_german_dict").cancel(' || :pid || ')');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
SELECT citus.mitmproxy('conn.onQuery(query="SELECT worker_create_or_replace_object.*my_german_dict").kill()');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
-- Failure to create text search config
SELECT citus.mitmproxy('conn.onQuery(query="SELECT worker_create_or_replace_object.*my_ts_config").cancel(' || :pid || ')');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
SELECT citus.mitmproxy('conn.onQuery(query="SELECT worker_create_or_replace_object.*my_ts_config").kill()');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
-- Failure to create type
SELECT citus.mitmproxy('conn.onQuery(query="SELECT worker_create_or_replace_object.*pair_type").cancel(' || :pid || ')');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
SELECT citus.mitmproxy('conn.onQuery(query="SELECT worker_create_or_replace_object.*pair_type").kill()');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
-- Failure to create publication
SELECT citus.mitmproxy('conn.onQuery(query="CREATE PUBLICATION.*pub_all").cancel(' || :pid || ')');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
SELECT citus.mitmproxy('conn.onQuery(query="CREATE PUBLICATION.*pub_all").kill()');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
-- Failure to create sequence -- Failure to create sequence
SELECT citus.mitmproxy('conn.onQuery(query="SELECT worker_apply_sequence_command").cancel(' || :pid || ')'); SELECT citus.mitmproxy('conn.onQuery(query="SELECT worker_apply_sequence_command").cancel(' || :pid || ')');
SELECT citus_activate_node('localhost', :worker_2_proxy_port); SELECT citus_activate_node('localhost', :worker_2_proxy_port);
SELECT citus.mitmproxy('conn.onQuery(query="SELECT worker_apply_sequence_command").kill()'); SELECT citus.mitmproxy('conn.onQuery(query="SELECT worker_apply_sequence_command").kill()');
SELECT citus_activate_node('localhost', :worker_2_proxy_port); SELECT citus_activate_node('localhost', :worker_2_proxy_port);
-- Failure to drop sequence dependency for distributed table
SELECT citus.mitmproxy('conn.onQuery(query="SELECT pg_catalog.worker_drop_sequence_dependency.*mx_metadata_sync_multi_trans.dist1").cancel(' || :pid || ')');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
SELECT citus.mitmproxy('conn.onQuery(query="SELECT pg_catalog.worker_drop_sequence_dependency.*mx_metadata_sync_multi_trans.dist1").kill()');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
-- Failure to drop distributed table if exists
SELECT citus.mitmproxy('conn.onQuery(query="DROP TABLE IF EXISTS mx_metadata_sync_multi_trans.dist1").cancel(' || :pid || ')');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
SELECT citus.mitmproxy('conn.onQuery(query="DROP TABLE IF EXISTS mx_metadata_sync_multi_trans.dist1").kill()');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
-- Failure to create distributed table -- Failure to create distributed table
SELECT citus.mitmproxy('conn.onQuery(query="CREATE TABLE mx_metadata_sync_multi_trans.dist1").cancel(' || :pid || ')'); SELECT citus.mitmproxy('conn.onQuery(query="CREATE TABLE mx_metadata_sync_multi_trans.dist1").cancel(' || :pid || ')');
SELECT citus_activate_node('localhost', :worker_2_proxy_port); SELECT citus_activate_node('localhost', :worker_2_proxy_port);
SELECT citus.mitmproxy('conn.onQuery(query="CREATE TABLE mx_metadata_sync_multi_trans.dist1").kill()'); SELECT citus.mitmproxy('conn.onQuery(query="CREATE TABLE mx_metadata_sync_multi_trans.dist1").kill()');
SELECT citus_activate_node('localhost', :worker_2_proxy_port); SELECT citus_activate_node('localhost', :worker_2_proxy_port);
-- Failure to record sequence dependency for table
SELECT citus.mitmproxy('conn.onQuery(query="SELECT pg_catalog.worker_record_sequence_dependency").cancel(' || :pid || ')');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
SELECT citus.mitmproxy('conn.onQuery(query="SELECT pg_catalog.worker_record_sequence_dependency").kill()');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
-- Failure to create index for table
SELECT citus.mitmproxy('conn.onQuery(query="CREATE INDEX dist1_search_phone_idx ON mx_metadata_sync_multi_trans.dist1 USING gin").cancel(' || :pid || ')');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
SELECT citus.mitmproxy('conn.onQuery(query="CREATE INDEX dist1_search_phone_idx ON mx_metadata_sync_multi_trans.dist1 USING gin").kill()');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
-- Failure to create reference table -- Failure to create reference table
SELECT citus.mitmproxy('conn.onQuery(query="CREATE TABLE mx_metadata_sync_multi_trans.ref").cancel(' || :pid || ')'); SELECT citus.mitmproxy('conn.onQuery(query="CREATE TABLE mx_metadata_sync_multi_trans.ref").cancel(' || :pid || ')');
SELECT citus_activate_node('localhost', :worker_2_proxy_port); SELECT citus_activate_node('localhost', :worker_2_proxy_port);
@ -220,6 +308,48 @@ SELECT citus_activate_node('localhost', :worker_2_proxy_port);
SELECT citus.mitmproxy('conn.onQuery(query="SELECT citus_internal_add_object_metadata").kill()'); SELECT citus.mitmproxy('conn.onQuery(query="SELECT citus_internal_add_object_metadata").kill()');
SELECT citus_activate_node('localhost', :worker_2_proxy_port); SELECT citus_activate_node('localhost', :worker_2_proxy_port);
-- Failure to mark function as distributed
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*one_as_result").cancel(' || :pid || ')');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*one_as_result").kill()');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
-- Failure to mark collation as distributed
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*german_phonebook").cancel(' || :pid || ')');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*german_phonebook").kill()');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
-- Failure to mark text search dictionary as distributed
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*my_german_dict").cancel(' || :pid || ')');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*my_german_dict").kill()');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
-- Failure to mark text search configuration as distributed
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*my_ts_config").cancel(' || :pid || ')');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*my_ts_config").kill()');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
-- Failure to mark type as distributed
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*pair_type").cancel(' || :pid || ')');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*pair_type").kill()');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
-- Failure to mark sequence as distributed
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*seq_owned").cancel(' || :pid || ')');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*seq_owned").kill()');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
-- Failure to mark publication as distributed
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*pub_all").cancel(' || :pid || ')');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
SELECT citus.mitmproxy('conn.onQuery(query="WITH distributed_object_data.*pub_all").kill()');
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
-- Failure to set isactive to true -- Failure to set isactive to true
SELECT citus.mitmproxy('conn.onQuery(query="UPDATE pg_dist_node SET isactive = TRUE").cancel(' || :pid || ')'); SELECT citus.mitmproxy('conn.onQuery(query="UPDATE pg_dist_node SET isactive = TRUE").cancel(' || :pid || ')');
SELECT citus_activate_node('localhost', :worker_2_proxy_port); SELECT citus_activate_node('localhost', :worker_2_proxy_port);
@ -277,6 +407,7 @@ SELECT * FROM pg_dist_node ORDER BY nodeport;
SELECT citus.mitmproxy('conn.allow()'); SELECT citus.mitmproxy('conn.allow()');
RESET citus.metadata_sync_mode; RESET citus.metadata_sync_mode;
DROP PUBLICATION pub_all;
DROP SCHEMA dummy; DROP SCHEMA dummy;
DROP SCHEMA mx_metadata_sync_multi_trans CASCADE; DROP SCHEMA mx_metadata_sync_multi_trans CASCADE;
DROP ROLE foo1; DROP ROLE foo1;

View File

@ -0,0 +1,470 @@
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;
SELECT 1 FROM citus_add_node('localhost', :master_port, groupid => 0);
SET client_min_messages TO NOTICE;
CREATE TABLE nullkey_c1_t1(a int, b int);
CREATE TABLE nullkey_c1_t2(a int, b int);
SELECT create_distributed_table('nullkey_c1_t1', null, colocate_with=>'none');
SELECT create_distributed_table('nullkey_c1_t2', null, colocate_with=>'nullkey_c1_t1');
CREATE TABLE nullkey_c2_t1(a int, b int);
SELECT create_distributed_table('nullkey_c2_t1', null, colocate_with=>'none');
CREATE TABLE reference_table(a int, b int);
SELECT create_reference_table('reference_table');
CREATE TABLE distributed_table_c1_t1(a int, b int);
SELECT create_distributed_table('distributed_table_c1_t1', 'a');
CREATE TABLE distributed_table_c1_t2(a int, b int);
SELECT create_distributed_table('distributed_table_c1_t2', 'a');
CREATE TABLE distributed_table_c2_t1(a int, b int);
SELECT create_distributed_table('distributed_table_c2_t1', 'a', colocate_with=>'none');
CREATE TABLE citus_local_table(a int, b int);
SELECT citus_add_local_table_to_metadata('citus_local_table');
CREATE TABLE postgres_local_table(a int, b int);
CREATE FUNCTION reload_tables() RETURNS void AS $$
BEGIN
SET LOCAL client_min_messages TO WARNING;
TRUNCATE nullkey_c1_t1, nullkey_c1_t2, nullkey_c2_t1, reference_table, distributed_table_c1_t1,
distributed_table_c1_t2, distributed_table_c2_t1, citus_local_table, postgres_local_table;
INSERT INTO nullkey_c1_t1 SELECT i, i FROM generate_series(1, 8) i;
INSERT INTO nullkey_c1_t2 SELECT i, i FROM generate_series(2, 7) i;
INSERT INTO nullkey_c2_t1 SELECT i, i FROM generate_series(2, 7) i;
INSERT INTO reference_table SELECT i, i FROM generate_series(0, 5) i;
INSERT INTO distributed_table_c1_t1 SELECT i, i FROM generate_series(3, 8) i;
INSERT INTO distributed_table_c1_t2 SELECT i, i FROM generate_series(2, 9) i;
INSERT INTO distributed_table_c2_t1 SELECT i, i FROM generate_series(5, 10) i;
INSERT INTO citus_local_table SELECT i, i FROM generate_series(0, 10) i;
INSERT INTO postgres_local_table SELECT i, i FROM generate_series(5, 10) i;
END;
$$ LANGUAGE plpgsql;
SELECT reload_tables();
CREATE TABLE append_table (a int, b int);
SELECT create_distributed_table('append_table', 'a', 'append');
SELECT master_create_empty_shard('append_table') AS shardid1 \gset
SELECT master_create_empty_shard('append_table') AS shardid2 \gset
SELECT master_create_empty_shard('append_table') AS shardid3 \gset
COPY append_table (a, b) FROM STDIN WITH (format 'csv', append_to_shard :shardid1);
1, 40
2, 42
3, 44
4, 46
5, 48
\.
COPY append_table (a, b) FROM STDIN WITH (format 'csv', append_to_shard :shardid2);
6, 50
7, 52
8, 54
9, 56
10, 58
\.
CREATE TABLE range_table(a int, b int);
SELECT create_distributed_table('range_table', 'a', 'range');
CALL public.create_range_partitioned_shards('range_table', '{"0","25"}','{"24","49"}');
INSERT INTO range_table VALUES (0, 1), (1, 2), (2, 3), (3, 4), (4, 5), (5, 6), (6, 50);
CREATE MATERIALIZED VIEW matview AS SELECT b*2+a AS a, a*a AS b FROM nullkey_c1_t1;
SET client_min_messages TO DEBUG2;
-- Test inserting into a distributed table by selecting from a combination of
-- different table types together with 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;
-- use a reference table
INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN reference_table USING (a);
INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 RIGHT JOIN reference_table USING (b) WHERE reference_table.a >= 1 AND reference_table.a <= 5;
INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 LEFT JOIN reference_table USING (b);
INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 INTERSECT SELECT * FROM reference_table;
-- use a colocated 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 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;
-- use a distributed table that is colocated with the target table
INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a);
INSERT INTO distributed_table_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a);
INSERT INTO distributed_table_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (b);
INSERT INTO distributed_table_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a) WHERE distributed_table_c1_t2.a = 1;
-- use a distributed table that is not colocated with the target table
INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 JOIN distributed_table_c2_t1 USING (a);
-- use a citus local table
INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN citus_local_table USING (a);
-- use a postgres local table
INSERT INTO distributed_table_c1_t1 SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 JOIN postgres_local_table USING (a);
-- use append / range distributed tables
INSERT INTO range_table SELECT * FROM nullkey_c1_t1;
INSERT INTO append_table SELECT * FROM nullkey_c1_t1;
SELECT avg(a), avg(b) FROM distributed_table_c1_t1 ORDER BY 1, 2;
TRUNCATE distributed_table_c1_t1;
INSERT INTO distributed_table_c1_t1 SELECT i, i FROM generate_series(3, 8) i;
-- Test inserting into a reference table by selecting from a combination of
-- different table types together with 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;
-- use a reference table
INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN reference_table USING (a);
INSERT INTO reference_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 LEFT JOIN reference_table USING (b);
INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 UNION SELECT * FROM reference_table;
INSERT INTO reference_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 LEFT JOIN reference_table USING (b) WHERE b IN (SELECT b FROM matview);
-- use a colocated 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 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
INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a);
INSERT INTO reference_table SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a);
INSERT INTO reference_table SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (b);
INSERT INTO reference_table SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a) WHERE distributed_table_c1_t2.a = 1;
-- use a citus local table
INSERT INTO reference_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN citus_local_table USING (a);
-- use a postgres local table
INSERT INTO reference_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 JOIN postgres_local_table USING (a);
SELECT avg(a), avg(b) FROM reference_table ORDER BY 1, 2;
TRUNCATE reference_table;
INSERT INTO reference_table SELECT i, i FROM generate_series(0, 5) i;
-- Test inserting into a citus local table by selecting from a combination of
-- different table types together with 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;
-- 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 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
INSERT INTO citus_local_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN distributed_table_c1_t2 USING (a);
-- use a citus local table
INSERT INTO citus_local_table SELECT nullkey_c1_t1.a, nullkey_c1_t1.b FROM nullkey_c1_t1 JOIN citus_local_table USING (a);
-- use a postgres local table
INSERT INTO citus_local_table SELECT nullkey_c1_t2.a, nullkey_c1_t2.b FROM nullkey_c1_t2 JOIN postgres_local_table USING (a);
SELECT avg(a), avg(b) FROM citus_local_table ORDER BY 1, 2;
TRUNCATE citus_local_table;
INSERT INTO citus_local_table SELECT i, i FROM generate_series(0, 10) i;
-- Test inserting into a 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;
INSERT INTO nullkey_c1_t1 SELECT postgres_local_table.a, postgres_local_table.b FROM postgres_local_table JOIN reference_table USING (a);
INSERT INTO nullkey_c1_t1 SELECT postgres_local_table.a, postgres_local_table.b FROM postgres_local_table LEFT JOIN nullkey_c1_t1 USING (a);
-- use a citus local table
INSERT INTO nullkey_c1_t1 SELECT citus_local_table.a, citus_local_table.b FROM citus_local_table;
INSERT INTO nullkey_c1_t1 SELECT citus_local_table.a, citus_local_table.b FROM citus_local_table JOIN reference_table USING (a) JOIN postgres_local_table USING (a) ORDER BY 1,2 OFFSET 7;
INSERT INTO nullkey_c1_t1 SELECT citus_local_table.a, citus_local_table.b FROM citus_local_table JOIN nullkey_c1_t1 USING (a);
-- use a distributed table
INSERT INTO nullkey_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM distributed_table_c1_t2;
INSERT INTO nullkey_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM distributed_table_c1_t2 JOIN reference_table USING (a);
INSERT INTO nullkey_c1_t1 SELECT distributed_table_c1_t2.a, distributed_table_c1_t2.b FROM distributed_table_c1_t2 JOIN nullkey_c1_t1 USING (a);
-- use a non-colocated 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
INSERT INTO nullkey_c1_t1 SELECT * FROM matview;
INSERT INTO nullkey_c1_t1 SELECT reference_table.a, reference_table.b FROM reference_table JOIN matview ON (reference_table.a = matview.a);
INSERT INTO nullkey_c1_t1 SELECT q.* FROM (SELECT reference_table.* FROM reference_table JOIN nullkey_c1_t1 USING (a)) q JOIN matview USING (a);
-- use append / range distributed tables
INSERT INTO nullkey_c1_t1 SELECT * FROM range_table;
INSERT INTO nullkey_c1_t1 SELECT * FROM append_table;
SELECT avg(a), avg(b) FROM nullkey_c1_t1 ORDER BY 1, 2;
SELECT avg(a), avg(b) FROM nullkey_c2_t1 ORDER BY 1, 2;
TRUNCATE nullkey_c1_t1, nullkey_c2_t1;
INSERT INTO nullkey_c1_t1 SELECT i, i FROM generate_series(1, 8) i;
INSERT INTO nullkey_c2_t1 SELECT i, i FROM generate_series(2, 7) i;
-- Test inserting into a local table by selecting from a combination of
-- different table types, together with or without 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);
INSERT INTO postgres_local_table SELECT * FROM nullkey_c1_t1 ORDER BY 1,2 OFFSET 3 LIMIT 2;
WITH cte_1 AS (
DELETE FROM nullkey_c1_t1 WHERE a >= 1 and a <= 4 RETURNING *
)
INSERT INTO postgres_local_table SELECT cte_1.* FROM cte_1 LEFT JOIN nullkey_c1_t2 USING (a) WHERE nullkey_c1_t2.a IS NULL;
INSERT INTO postgres_local_table SELECT * FROM nullkey_c1_t1 EXCEPT SELECT * FROM postgres_local_table;
SELECT avg(a), avg(b) FROM postgres_local_table ORDER BY 1, 2;
TRUNCATE postgres_local_table;
INSERT INTO postgres_local_table SELECT i, i FROM generate_series(5, 10) i;
-- Try slightly more complex queries.
WITH cte_1 AS (
SELECT nullkey_c1_t1.a, reference_table.b FROM nullkey_c1_t1 JOIN reference_table USING (a)
),
cte_2 AS (
SELECT reference_table.a, postgres_local_table.b FROM postgres_local_table LEFT JOIN reference_table USING (b)
)
INSERT INTO distributed_table_c1_t1
SELECT cte_1.* FROM cte_1 JOIN cte_2 USING (a) JOIN distributed_table_c1_t2 USING (a) ORDER BY 1,2;
WITH cte_1 AS (
SELECT nullkey_c1_t1.a, reference_table.b FROM nullkey_c1_t1 JOIN reference_table USING (a)
),
cte_2 AS (
SELECT * FROM nullkey_c1_t2 WHERE EXISTS (
SELECT 1 FROM reference_table WHERE reference_table.a = nullkey_c1_t2.a
)
ORDER BY 1,2 OFFSET 1 LIMIT 4
)
INSERT INTO distributed_table_c1_t1
SELECT * FROM cte_1 UNION SELECT * FROM cte_2 EXCEPT SELECT * FROM reference_table;
INSERT INTO distributed_table_c1_t1 (a, b)
SELECT t1.a, t2.b
FROM nullkey_c1_t1 t1
JOIN (
SELECT b FROM nullkey_c1_t2 ORDER BY b DESC LIMIT 1
) t2
ON t1.b < t2.b;
INSERT INTO distributed_table_c1_t1 (a, b)
WITH cte AS (
SELECT a, b,
(SELECT a FROM nullkey_c1_t2 WHERE b = t.b) AS d1,
(SELECT a FROM reference_table WHERE b = t.b) AS d2
FROM nullkey_c1_t1 t
)
SELECT d1, COALESCE(d2, a) FROM cte WHERE d1 IS NOT NULL AND d2 IS NOT NULL;
INSERT INTO citus_local_table (a, b)
SELECT t1.a, t2.b
FROM nullkey_c1_t1 t1
CROSS JOIN (
SELECT b FROM nullkey_c2_t1 ORDER BY b LIMIT 1
) t2;
INSERT INTO distributed_table_c1_t1 (a, b)
SELECT t1.a, t2.b
FROM reference_table t1
LEFT JOIN (
SELECT b, ROW_NUMBER() OVER (ORDER BY b DESC) AS rn
FROM nullkey_c1_t1
) t2 ON t1.b = t2.b
WHERE t2.rn > 0;
INSERT INTO nullkey_c1_t1 (a, b)
SELECT t1.a, t2.b
FROM nullkey_c1_t1 t1
JOIN (
SELECT rn, b
FROM (
SELECT b, ROW_NUMBER() OVER (ORDER BY b DESC) AS rn
FROM distributed_table_c2_t1
) q
) t2 ON t1.b = t2.b
WHERE t2.rn > 2;
INSERT INTO distributed_table_c1_t1 (a, b)
SELECT t1.a, t2.b
FROM nullkey_c1_t1 t1
JOIN (
SELECT sum_val, b
FROM (
SELECT b, SUM(a) OVER (PARTITION BY b) AS sum_val
FROM nullkey_c1_t1
) q
) t2 ON t1.b = t2.b
WHERE t2.sum_val > 2;
-- MultiTaskRouterSelectQuerySupported() is unnecessarily restrictive
-- about pushing down queries with DISTINCT ON clause even if the table
-- doesn't have a shard key. See https://github.com/citusdata/citus/pull/6752.
INSERT INTO nullkey_c1_t1 SELECT DISTINCT ON (a) a, b FROM nullkey_c1_t2;
-- Similarly, we could push down the following query as well. see
-- https://github.com/citusdata/citus/pull/6831.
INSERT INTO nullkey_c1_t1 SELECT b, SUM(a) OVER (ORDER BY b) AS sum_val FROM nullkey_c1_t1;
INSERT INTO nullkey_c2_t1
SELECT t2.a, t2.b
FROM nullkey_c1_t1 AS t2
JOIN reference_table AS t3 ON (t2.a = t3.a)
WHERE NOT EXISTS (
SELECT 1 FROM nullkey_c1_t2 AS t1 WHERE t1.b = t3.b
);
INSERT INTO distributed_table_c1_t1
SELECT t1.a, t1.b
FROM nullkey_c1_t1 AS t1
WHERE t1.a NOT IN (
SELECT DISTINCT t2.a FROM distributed_table_c1_t2 AS t2
);
INSERT INTO distributed_table_c1_t1
SELECT t1.a, t1.b
FROM reference_table AS t1
JOIN (
SELECT t2.a FROM (
SELECT a FROM nullkey_c1_t1
UNION
SELECT a FROM nullkey_c1_t2
) AS t2
) AS t3 ON t1.a = t3.a;
-- Temporaryly reduce the verbosity to avoid noise
-- in the output of the next query.
SET client_min_messages TO DEBUG1;
INSERT INTO nullkey_c1_t1
SELECT t1.a, t1.b
FROM reference_table AS t1
WHERE t1.a IN (
SELECT t2.a FROM (
SELECT t3.a FROM (
SELECT a FROM distributed_table_c1_t1 WHERE b > 4
) AS t3
JOIN (
SELECT a FROM distributed_table_c1_t2 WHERE b < 7
) AS t4 ON t3.a = t4.a
) AS t2
);
SET client_min_messages TO DEBUG2;
-- test upsert with plain INSERT query
CREATE TABLE upsert_test_1
(
unique_col int UNIQUE,
other_col int,
third_col int
);
SELECT create_distributed_table('upsert_test_1', null);
CREATE TABLE upsert_test_2(key int primary key, value text);
SELECT create_distributed_table('upsert_test_2', null);
INSERT INTO upsert_test_2 AS upsert_test_2_alias (key, value) VALUES (1, '5') ON CONFLICT(key)
DO UPDATE SET value = (upsert_test_2_alias.value::int * 2)::text;
INSERT INTO upsert_test_2 (key, value) VALUES (1, '5') ON CONFLICT(key)
DO UPDATE SET value = (upsert_test_2.value::int * 3)::text;
INSERT INTO upsert_test_1 (unique_col, other_col) VALUES (1, 1) ON CONFLICT (unique_col)
DO UPDATE SET other_col = (SELECT count(*) from upsert_test_1);
INSERT INTO upsert_test_1 (unique_col, other_col) VALUES (1, 1) ON CONFLICT (unique_col)
DO UPDATE SET other_col = random()::int;
INSERT INTO upsert_test_1 (unique_col, other_col) VALUES (1, 1) ON CONFLICT (unique_col)
DO UPDATE SET other_col = 5 WHERE upsert_test_1.other_col = random()::int;
INSERT INTO upsert_test_1 VALUES (3, 5, 7);
INSERT INTO upsert_test_1 (unique_col, other_col) VALUES (1, 1) ON CONFLICT (unique_col) WHERE unique_col = random()::int
DO UPDATE SET other_col = 5;
CREATE TABLE upsert_test_3 (key_1 int, key_2 bigserial, value text DEFAULT 'default_value', PRIMARY KEY (key_1, key_2));
SELECT create_distributed_table('upsert_test_3', null);
INSERT INTO upsert_test_3 VALUES (1, DEFAULT, '1') RETURNING *;
INSERT INTO upsert_test_3 VALUES (5, DEFAULT, DEFAULT) RETURNING *;
SET client_min_messages TO DEBUG1;
INSERT INTO upsert_test_3 SELECT 7, other_col, 'harcoded_text_value' FROM upsert_test_1 RETURNING *;
SET client_min_messages TO DEBUG2;
-- test upsert with INSERT .. SELECT queries
SET client_min_messages TO DEBUG1;
INSERT INTO upsert_test_1 (unique_col, other_col) SELECT unique_col, other_col FROM upsert_test_1 ON CONFLICT (unique_col)
DO UPDATE SET other_col = upsert_test_1.other_col + 1;
-- Fails due to https://github.com/citusdata/citus/issues/6826.
INSERT INTO upsert_test_1 (unique_col, other_col) SELECT unique_col, other_col FROM upsert_test_1 ON CONFLICT (unique_col)
DO UPDATE SET other_col = (SELECT count(*) from upsert_test_1);
SET client_min_messages TO DEBUG2;
INSERT INTO upsert_test_1 (unique_col, other_col) SELECT unique_col, other_col FROM upsert_test_1 ON CONFLICT (unique_col)
DO UPDATE SET other_col = random()::int;
INSERT INTO upsert_test_1 (unique_col, other_col) SELECT unique_col, other_col FROM upsert_test_1 ON CONFLICT (unique_col)
DO UPDATE SET other_col = 5 WHERE upsert_test_1.other_col = random()::int;
SELECT reload_tables();
ALTER TABLE nullkey_c1_t1 ADD PRIMARY KEY (a);
ALTER TABLE distributed_table_c1_t1 ADD PRIMARY KEY (a,b);
INSERT INTO nullkey_c1_t1 AS t1 (a, b) SELECT t3.a, t3.b FROM nullkey_c1_t2 t2 JOIN reference_table t3 ON (t2.a = t3.a) ON CONFLICT (a)
DO UPDATE SET a = t1.a + 10;
SET client_min_messages TO DEBUG1;
INSERT INTO distributed_table_c1_t1 AS t1 (a, b) SELECT t3.a, t3.b FROM nullkey_c1_t2 t2 JOIN reference_table t3 ON (t2.a = t3.a) ON CONFLICT (a, b)
DO UPDATE SET b = t1.b + 10;
INSERT INTO nullkey_c1_t1 AS t1 (a, b) SELECT t3.a, t3.b FROM distributed_table_c1_t1 t2 JOIN reference_table t3 ON (t2.a = t3.a) ON CONFLICT (a)
DO UPDATE SET a = t1.a + 10;
-- This also fails due to https://github.com/citusdata/citus/issues/6826.
INSERT INTO nullkey_c1_t1 AS t1 (a, b) SELECT t3.a, t3.b FROM distributed_table_c1_t1 t2 JOIN reference_table t3 ON (t2.a = t3.a) WHERE t2.a = 3 ON CONFLICT (a)
DO UPDATE SET a = (SELECT max(b)+1 FROM distributed_table_c1_t1 WHERE a = 3);
SET client_min_messages TO DEBUG2;
SELECT avg(a), avg(b) FROM distributed_table_c1_t1;
SELECT avg(a), avg(b) FROM nullkey_c1_t1;
SELECT avg(a), avg(b) FROM nullkey_c1_t2;
SELECT * FROM upsert_test_1 ORDER BY unique_col;
SELECT * FROM upsert_test_2 ORDER BY key;
SELECT * FROM upsert_test_3 ORDER BY key_1, key_2;
SET client_min_messages TO WARNING;
DROP SCHEMA insert_select_single_shard_table CASCADE;
SELECT citus_remove_node('localhost', :master_port);

View File

@ -1536,10 +1536,111 @@ WHEN NOT MATCHED THEN
INSERT VALUES(foo.s1); INSERT VALUES(foo.s1);
SELECT * FROM target_set ORDER BY 1, 2; SELECT * FROM target_set ORDER BY 1, 2;
--
-- Reference as a source
--
CREATE TABLE reftarget_local(t1 int, t2 int);
CREATE TABLE refsource_ref(s1 int, s2 int);
INSERT INTO reftarget_local VALUES(1, 0);
INSERT INTO reftarget_local VALUES(3, 100);
INSERT INTO refsource_ref VALUES(1, 1);
INSERT INTO refsource_ref VALUES(2, 2);
INSERT INTO refsource_ref VALUES(3, 3);
MERGE INTO reftarget_local
USING (SELECT * FROM refsource_ref UNION SELECT * FROM refsource_ref) AS foo ON reftarget_local.t1 = foo.s1
WHEN MATCHED AND reftarget_local.t2 = 100 THEN
DELETE
WHEN MATCHED THEN
UPDATE SET t2 = t2 + 100
WHEN NOT MATCHED THEN
INSERT VALUES(foo.s1);
DROP TABLE IF EXISTS pg_result;
SELECT * INTO pg_result FROM reftarget_local ORDER BY 1, 2;
-- Make source table as reference (target is Postgres)
TRUNCATE reftarget_local;
TRUNCATE refsource_ref;
INSERT INTO reftarget_local VALUES(1, 0);
INSERT INTO reftarget_local VALUES(3, 100);
INSERT INTO refsource_ref VALUES(1, 1);
INSERT INTO refsource_ref VALUES(2, 2);
INSERT INTO refsource_ref VALUES(3, 3);
SELECT create_reference_table('refsource_ref');
MERGE INTO reftarget_local
USING (SELECT * FROM refsource_ref UNION SELECT * FROM refsource_ref) AS foo ON reftarget_local.t1 = foo.s1
WHEN MATCHED AND reftarget_local.t2 = 100 THEN
DELETE
WHEN MATCHED THEN
UPDATE SET t2 = t2 + 100
WHEN NOT MATCHED THEN
INSERT VALUES(foo.s1);
SELECT * INTO pg_ref FROM reftarget_local ORDER BY 1, 2;
-- Should be equal
SELECT c.*, p.*
FROM pg_ref c, pg_result p
WHERE c.t1 = p.t1
ORDER BY 1,2;
-- Must return zero rows
SELECT count(*)
FROM pg_result FULL OUTER JOIN pg_ref ON pg_result.t1 = pg_ref.t1
WHERE pg_result.t1 IS NULL OR pg_ref.t1 IS NULL;
-- Now make both Citus tables, reference as source, local as target
TRUNCATE reftarget_local;
TRUNCATE refsource_ref;
INSERT INTO reftarget_local VALUES(1, 0);
INSERT INTO reftarget_local VALUES(3, 100);
INSERT INTO refsource_ref VALUES(1, 1);
INSERT INTO refsource_ref VALUES(2, 2);
INSERT INTO refsource_ref VALUES(3, 3);
SELECT citus_add_local_table_to_metadata('reftarget_local');
MERGE INTO reftarget_local
USING (SELECT * FROM refsource_ref UNION SELECT * FROM refsource_ref) AS foo ON reftarget_local.t1 = foo.s1
WHEN MATCHED AND reftarget_local.t2 = 100 THEN
DELETE
WHEN MATCHED THEN
UPDATE SET t2 = t2 + 100
WHEN NOT MATCHED THEN
INSERT VALUES(foo.s1);
SELECT * INTO local_ref FROM reftarget_local ORDER BY 1, 2;
-- Should be equal
SELECT c.*, p.*
FROM local_ref c, pg_result p
WHERE c.t1 = p.t1
ORDER BY 1,2;
-- Must return zero rows
SELECT count(*)
FROM pg_result FULL OUTER JOIN local_ref ON pg_result.t1 = local_ref.t1
WHERE pg_result.t1 IS NULL OR local_ref.t1 IS NULL;
-- --
-- Error and Unsupported scenarios -- Error and Unsupported scenarios
-- --
-- Reference as a target and local as source
MERGE INTO refsource_ref
USING (SELECT * FROM reftarget_local UNION SELECT * FROM reftarget_local) AS foo ON refsource_ref.s1 = foo.t1
WHEN MATCHED THEN
UPDATE SET s2 = s2 + 100
WHEN NOT MATCHED THEN
INSERT VALUES(foo.t1);
-- Reference as a source and distributed as target
MERGE INTO target_set t
USING refsource_ref AS s ON t.t1 = s.s1
WHEN MATCHED THEN
DO NOTHING;
MERGE INTO target_set MERGE INTO target_set
USING source_set AS foo ON target_set.t1 = foo.s1 USING source_set AS foo ON target_set.t1 = foo.s1
WHEN MATCHED THEN WHEN MATCHED THEN
@ -1950,6 +2051,118 @@ UPDATE SET val = dist_source.val
WHEN NOT MATCHED THEN WHEN NOT MATCHED THEN
INSERT VALUES(dist_source.id, dist_source.val); INSERT VALUES(dist_source.id, dist_source.val);
-- 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);
SELECT create_distributed_table('nullkey_c1_t1', null, colocate_with=>'none');
SELECT create_distributed_table('nullkey_c1_t2', null, colocate_with=>'nullkey_c1_t1');
CREATE TABLE nullkey_c2_t1(a int, b int);
CREATE TABLE nullkey_c2_t2(a int, b int);
SELECT create_distributed_table('nullkey_c2_t1', null, colocate_with=>'none');
SELECT create_distributed_table('nullkey_c2_t2', null, colocate_with=>'nullkey_c2_t1', distribution_type=>null);
CREATE TABLE reference_table(a int, b int);
SELECT create_reference_table('reference_table');
INSERT INTO reference_table SELECT i, i FROM generate_series(0, 5) i;
CREATE TABLE distributed_table(a int, b int);
SELECT create_distributed_table('distributed_table', 'a');
INSERT INTO distributed_table SELECT i, i FROM generate_series(3, 8) i;
CREATE TABLE citus_local_table(a int, b int);
SELECT citus_add_local_table_to_metadata('citus_local_table');
INSERT INTO citus_local_table SELECT i, i FROM generate_series(0, 10) i;
CREATE TABLE postgres_local_table(a int, b int);
INSERT INTO postgres_local_table SELECT i, i FROM generate_series(5, 10) i;
-- with a colocated table
MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a)
WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b;
MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a)
WHEN MATCHED THEN DELETE;
MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a)
WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t2.b
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b);
MERGE INTO nullkey_c1_t1 USING nullkey_c1_t2 ON (nullkey_c1_t1.a = nullkey_c1_t2.a)
WHEN MATCHED THEN DELETE
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t2.a, nullkey_c1_t2.b);
-- with non-colocated 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;
MERGE INTO nullkey_c1_t1 USING nullkey_c2_t1 ON (nullkey_c1_t1.a = nullkey_c2_t1.a)
WHEN MATCHED THEN UPDATE SET b = nullkey_c2_t1.b
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c2_t1.a, nullkey_c2_t1.b);
-- with a distributed table
MERGE INTO nullkey_c1_t1 USING distributed_table ON (nullkey_c1_t1.a = distributed_table.a)
WHEN MATCHED THEN UPDATE SET b = distributed_table.b
WHEN NOT MATCHED THEN INSERT VALUES (distributed_table.a, distributed_table.b);
MERGE INTO distributed_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = distributed_table.a)
WHEN MATCHED THEN DELETE
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b);
-- with a reference table
MERGE INTO nullkey_c1_t1 USING reference_table ON (nullkey_c1_t1.a = reference_table.a)
WHEN MATCHED THEN UPDATE SET b = reference_table.b;
MERGE INTO reference_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = reference_table.a)
WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t1.b
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b);
-- with a citus local table
MERGE INTO nullkey_c1_t1 USING citus_local_table ON (nullkey_c1_t1.a = citus_local_table.a)
WHEN MATCHED THEN UPDATE SET b = citus_local_table.b;
MERGE INTO citus_local_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = citus_local_table.a)
WHEN MATCHED THEN DELETE;
-- with a postgres table
MERGE INTO nullkey_c1_t1 USING postgres_local_table ON (nullkey_c1_t1.a = postgres_local_table.a)
WHEN MATCHED THEN UPDATE SET b = postgres_local_table.b;
MERGE INTO postgres_local_table USING nullkey_c1_t1 ON (nullkey_c1_t1.a = postgres_local_table.a)
WHEN MATCHED THEN UPDATE SET b = nullkey_c1_t1.b
WHEN NOT MATCHED THEN INSERT VALUES (nullkey_c1_t1.a, nullkey_c1_t1.b);
-- using ctes
WITH cte AS (
SELECT * FROM nullkey_c1_t1
)
MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a)
WHEN MATCHED THEN UPDATE SET b = cte.b;
WITH cte AS (
SELECT * FROM distributed_table
)
MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a)
WHEN MATCHED THEN UPDATE SET b = cte.b;
WITH cte AS materialized (
SELECT * FROM distributed_table
)
MERGE INTO nullkey_c1_t1 USING cte ON (nullkey_c1_t1.a = cte.a)
WHEN MATCHED THEN UPDATE SET b = cte.b;
SET client_min_messages TO WARNING;
DROP SCHEMA query_single_shard_table CASCADE;
RESET client_min_messages;
SET search_path TO merge_schema;
DROP SERVER foreign_server CASCADE; DROP SERVER foreign_server CASCADE;
DROP FUNCTION merge_when_and_write(); DROP FUNCTION merge_when_and_write();
DROP SCHEMA merge_schema CASCADE; DROP SCHEMA merge_schema CASCADE;

View File

@ -904,6 +904,20 @@ SELECT create_distributed_table('test','x');
DROP TABLE test; DROP TABLE test;
TRUNCATE pg_dist_node; TRUNCATE pg_dist_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;
SELECT create_distributed_table('test', null, colocate_with=>'none', distribution_type=>null);
-- and make sure that we can't remove the coordinator due to "test"
SELECT citus_remove_node('localhost', :master_port);
DROP TABLE test;
-- and now we should be able to remove the coordinator
SELECT citus_remove_node('localhost', :master_port);
-- confirm that we can create a reference table on an empty node -- confirm that we can create a reference table on an empty node
CREATE TABLE test (x int, y int); CREATE TABLE test (x int, y int);
INSERT INTO test VALUES (1,2); INSERT INTO test VALUES (1,2);

View File

@ -198,13 +198,13 @@ SELECT citus_add_local_table_to_metadata('tbl2');
MERGE INTO tbl1 USING tbl2 ON (true) MERGE INTO tbl1 USING tbl2 ON (true)
WHEN MATCHED THEN DELETE; WHEN MATCHED THEN DELETE;
-- one table is reference, the other local, not supported -- source table is reference, the target is local, supported
SELECT create_reference_table('tbl2'); SELECT create_reference_table('tbl2');
MERGE INTO tbl1 USING tbl2 ON (true) MERGE INTO tbl1 USING tbl2 ON (true)
WHEN MATCHED THEN DELETE; WHEN MATCHED THEN DELETE;
-- now, both are reference, still not supported -- now, both are reference, not supported
SELECT create_reference_table('tbl1'); SELECT create_reference_table('tbl1');
MERGE INTO tbl1 USING tbl2 ON (true) MERGE INTO tbl1 USING tbl2 ON (true)

File diff suppressed because it is too large Load Diff

View File

@ -13,7 +13,9 @@ CREATE TABLE postgres_table_test(a int primary key);
-- make sure that all rebalance operations works fine when -- make sure that all rebalance operations works fine when
-- reference tables are replicated to the coordinator -- reference tables are replicated to the coordinator
SET client_min_messages TO ERROR;
SELECT 1 FROM master_add_node('localhost', :master_port, groupId=>0); SELECT 1 FROM master_add_node('localhost', :master_port, groupId=>0);
RESET client_min_messages;
-- should just be noops even if we add the coordinator to the pg_dist_node -- should just be noops even if we add the coordinator to the pg_dist_node
SELECT rebalance_table_shards('dist_table_test'); SELECT rebalance_table_shards('dist_table_test');
@ -1497,6 +1499,61 @@ SELECT sh.logicalrelid, pl.nodeport
DROP TABLE single_shard_colocation_1a, single_shard_colocation_1b, single_shard_colocation_1c, single_shard_colocation_2a, single_shard_colocation_2b CASCADE; DROP TABLE single_shard_colocation_1a, single_shard_colocation_1b, single_shard_colocation_1c, single_shard_colocation_2a, single_shard_colocation_2b CASCADE;
-- test the same with coordinator shouldhaveshards = false and shard_count = 2
-- so that the shard allowed node count would be 2 when rebalancing
-- for such cases, we only count the nodes that are allowed for shard placements
UPDATE pg_dist_node SET shouldhaveshards=false WHERE nodeport = :master_port;
create table two_shard_colocation_1a (a int primary key);
create table two_shard_colocation_1b (a int primary key);
SET citus.shard_replication_factor = 1;
select create_distributed_table('two_shard_colocation_1a','a', colocate_with => 'none', shard_count => 2);
select create_distributed_table('two_shard_colocation_1b','a',colocate_with=>'two_shard_colocation_1a');
create table two_shard_colocation_2a (a int primary key);
create table two_shard_colocation_2b (a int primary key);
select create_distributed_table('two_shard_colocation_2a','a', colocate_with => 'none', shard_count => 2);
select create_distributed_table('two_shard_colocation_2b','a',colocate_with=>'two_shard_colocation_2a');
-- move shards of colocation group 1 to worker1
SELECT citus_move_shard_placement(sh.shardid, 'localhost', :worker_2_port, 'localhost', :worker_1_port)
FROM pg_dist_shard sh JOIN pg_dist_shard_placement pl ON sh.shardid = pl.shardid
WHERE sh.logicalrelid = 'two_shard_colocation_1a'::regclass
AND pl.nodeport = :worker_2_port
LIMIT 1;
-- move shards of colocation group 2 to worker2
SELECT citus_move_shard_placement(sh.shardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port)
FROM pg_dist_shard sh JOIN pg_dist_shard_placement pl ON sh.shardid = pl.shardid
WHERE sh.logicalrelid = 'two_shard_colocation_2a'::regclass
AND pl.nodeport = :worker_1_port
LIMIT 1;
-- current state:
-- coordinator: []
-- worker 1: [1_1, 1_2]
-- worker 2: [2_1, 2_2]
SELECT sh.logicalrelid, pl.nodeport
FROM pg_dist_shard sh JOIN pg_dist_shard_placement pl ON sh.shardid = pl.shardid
WHERE sh.logicalrelid::text IN ('two_shard_colocation_1a', 'two_shard_colocation_1b', 'two_shard_colocation_2a', 'two_shard_colocation_2b')
ORDER BY sh.logicalrelid, pl.nodeport;
-- If we take the coordinator into account, the rebalancer considers this as balanced and does nothing (shard_count < worker_count)
-- but because the coordinator is not allowed for shards, rebalancer will distribute each colocation group to both workers
select rebalance_table_shards(shard_transfer_mode:='block_writes');
-- final state:
-- coordinator: []
-- worker 1: [1_1, 2_1]
-- worker 2: [1_2, 2_2]
SELECT sh.logicalrelid, pl.nodeport
FROM pg_dist_shard sh JOIN pg_dist_shard_placement pl ON sh.shardid = pl.shardid
WHERE sh.logicalrelid::text IN ('two_shard_colocation_1a', 'two_shard_colocation_1b', 'two_shard_colocation_2a', 'two_shard_colocation_2b')
ORDER BY sh.logicalrelid, pl.nodeport;
-- cleanup
DROP TABLE two_shard_colocation_1a, two_shard_colocation_1b, two_shard_colocation_2a, two_shard_colocation_2b CASCADE;
-- verify we detect if one of the tables do not have a replica identity or primary key -- verify we detect if one of the tables do not have a replica identity or primary key
-- and error out in case of shard transfer mode = auto -- and error out in case of shard transfer mode = auto
SELECT 1 FROM citus_remove_node('localhost', :worker_2_port); SELECT 1 FROM citus_remove_node('localhost', :worker_2_port);

View File

@ -63,8 +63,43 @@ ALTER SYSTEM RESET citus.local_shared_pool_size;
ALTER SYSTEM RESET citus.max_cached_conns_per_worker; ALTER SYSTEM RESET citus.max_cached_conns_per_worker;
SELECT pg_reload_conf(); SELECT pg_reload_conf();
CREATE TABLE single_node_nullkey_c1(a int, b int);
SELECT create_distributed_table('single_node_nullkey_c1', null, colocate_with=>'none', distribution_type=>null);
CREATE TABLE single_node_nullkey_c2(a int, b int);
SELECT create_distributed_table('single_node_nullkey_c2', null, colocate_with=>'none', distribution_type=>null);
-- created on different colocation groups ..
SELECT
(
SELECT colocationid FROM pg_dist_partition
WHERE logicalrelid = 'single_node.single_node_nullkey_c1'::regclass
)
!=
(
SELECT colocationid FROM pg_dist_partition
WHERE logicalrelid = 'single_node.single_node_nullkey_c2'::regclass
);
-- .. but both are associated to coordinator
SELECT groupid = 0 FROM pg_dist_placement
WHERE shardid = (
SELECT shardid FROM pg_dist_shard
WHERE logicalrelid = 'single_node.single_node_nullkey_c1'::regclass
);
SELECT groupid = 0 FROM pg_dist_placement
WHERE shardid = (
SELECT shardid FROM pg_dist_shard
WHERE logicalrelid = 'single_node.single_node_nullkey_c2'::regclass
);
-- try creating a 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);
SET client_min_messages TO WARNING; SET client_min_messages TO WARNING;
DROP TABLE failover_to_local; DROP TABLE failover_to_local, single_node_nullkey_c1, single_node_nullkey_c2;
RESET client_min_messages; RESET client_min_messages;
-- so that we don't have to update rest of the test output -- so that we don't have to update rest of the test output

View File

@ -0,0 +1,14 @@
ALTER FUNCTION create_distributed_table RENAME TO create_distributed_table_internal;
CREATE OR REPLACE FUNCTION pg_catalog.create_distributed_table(table_name regclass,
distribution_column text,
distribution_type citus.distribution_type DEFAULT 'hash',
colocate_with text DEFAULT 'default',
shard_count int DEFAULT NULL)
RETURNS void
LANGUAGE plpgsql
AS $function$
BEGIN
PERFORM create_distributed_table_internal(table_name, NULL, NULL, colocate_with, NULL);
END;
$function$;