Refactor/Improve PreprocessAlterTableStmtAttachPartition (#5366)

* Refactor/Improve PreprocessAlterTableStmtAttachPartition
pull/5375/head
Ahmet Gedemenli 2021-10-14 11:39:39 +03:00 committed by GitHub
parent de61a89083
commit 35f6fe5f9f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
5 changed files with 288 additions and 56 deletions

View File

@ -58,6 +58,15 @@ bool EnableLocalReferenceForeignKeys = true;
static void PostprocessCreateTableStmtForeignKeys(CreateStmt *createStatement);
static void PostprocessCreateTableStmtPartitionOf(CreateStmt *createStatement,
const char *queryString);
static void PreprocessAttachPartitionToCitusTable(Oid parentRelationId,
Oid partitionRelationId);
static void PreprocessAttachCitusPartitionToCitusTable(Oid parentCitusRelationId,
Oid partitionRelationId);
static void DistributePartitionUsingParent(Oid parentRelationId,
Oid partitionRelationId);
static void ErrorIfMultiLevelPartitioning(Oid parentRelationId, Oid partitionRelationId);
static void ErrorIfAttachCitusTableToPgLocalTable(Oid parentRelationId,
Oid partitionRelationId);
static bool AlterTableDefinesFKeyBetweenPostgresAndNonDistTable(
AlterTableStmt *alterTableStatement);
static bool RelationIdListContainsCitusTableType(List *relationIdList,
@ -420,68 +429,34 @@ PreprocessAlterTableStmtAttachPartition(AlterTableStmt *alterTableStatement,
* and want to ensure we acquire the locks in the same order with Postgres
*/
LOCKMODE lockmode = AlterTableGetLockLevel(alterTableStatement->cmds);
Oid relationId = AlterTableLookupRelation(alterTableStatement, lockmode);
Oid parentRelationId = AlterTableLookupRelation(alterTableStatement,
lockmode);
PartitionCmd *partitionCommand = (PartitionCmd *) alterTableCommand->def;
bool partitionMissingOk = false;
Oid partitionRelationId = RangeVarGetRelid(partitionCommand->name, lockmode,
partitionMissingOk);
/*
* If user first distributes the table then tries to attach it to non
* distributed table, we error out.
*/
if (!IsCitusTable(relationId) &&
IsCitusTable(partitionRelationId))
if (!IsCitusTable(parentRelationId))
{
char *parentRelationName = get_rel_name(relationId);
/*
* If the parent is a regular Postgres table, but the partition is a
* Citus table, we error out.
*/
ErrorIfAttachCitusTableToPgLocalTable(parentRelationId,
partitionRelationId);
ereport(ERROR, (errmsg("non-distributed tables cannot have "
"distributed partitions"),
errhint("Distribute the partitioned table \"%s\" "
"instead", parentRelationName)));
/*
* If both the parent and the child table are Postgres tables,
* we can just skip preprocessing this command.
*/
continue;
}
/* if parent of this table is distributed, distribute this table too */
if (IsCitusTable(relationId) &&
!IsCitusTable(partitionRelationId))
{
if (IsCitusTableType(relationId, CITUS_LOCAL_TABLE))
{
if (PartitionedTable(partitionRelationId))
{
char *relationName = get_rel_name(partitionRelationId);
char *parentRelationName = get_rel_name(relationId);
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("distributing multi-level partitioned "
"tables is not supported"),
errdetail("Relation \"%s\" is partitioned table "
"itself and it is also partition of "
"relation \"%s\".",
relationName, parentRelationName)));
}
CreateCitusLocalTable(partitionRelationId, false);
return NIL;
}
/* Citus doesn't support multi-level partitioned tables */
ErrorIfMultiLevelPartitioning(parentRelationId, partitionRelationId);
Var *distributionColumn = DistPartitionKeyOrError(relationId);
char *distributionColumnName = ColumnToColumnName(relationId,
nodeToString(
distributionColumn));
distributionColumn = FindColumnWithNameOnTargetRelation(relationId,
distributionColumnName,
partitionRelationId);
char distributionMethod = DISTRIBUTE_BY_HASH;
char *parentRelationName = generate_qualified_relation_name(relationId);
bool viaDeprecatedAPI = false;
SwitchToSequentialAndLocalExecutionIfPartitionNameTooLong(
relationId, partitionRelationId);
CreateDistributedTable(partitionRelationId, distributionColumn,
distributionMethod, ShardCount, false,
parentRelationName, viaDeprecatedAPI);
}
/* attaching to a Citus table */
PreprocessAttachPartitionToCitusTable(parentRelationId, partitionRelationId);
}
}
@ -489,6 +464,172 @@ PreprocessAlterTableStmtAttachPartition(AlterTableStmt *alterTableStatement,
}
/*
* PreprocessAttachPartitionToCitusTable takes a parent relation, which is a Citus table,
* and a partition to be attached to it.
* If the partition table is a regular Postgres table:
* - Converts the partition to Citus Local Table, if the parent is a Citus Local Table.
* - Distributes the partition, if the parent is a distributed table.
* If not, calls PreprocessAttachCitusPartitionToCitusTable to attach given partition to
* the parent relation.
*/
static void
PreprocessAttachPartitionToCitusTable(Oid parentRelationId, Oid partitionRelationId)
{
Assert(IsCitusTable(parentRelationId));
/* reference tables cannot be partitioned */
Assert(!IsCitusTableType(parentRelationId, REFERENCE_TABLE));
/* if parent of this table is distributed, distribute this table too */
if (!IsCitusTable(partitionRelationId))
{
if (IsCitusTableType(parentRelationId, CITUS_LOCAL_TABLE))
{
/*
* We pass the cascade option as false, since Citus Local Table partitions
* cannot have non-inherited foreign keys.
*/
bool cascadeViaForeignKeys = false;
CreateCitusLocalTable(partitionRelationId, cascadeViaForeignKeys);
}
else if (IsCitusTableType(parentRelationId, DISTRIBUTED_TABLE))
{
DistributePartitionUsingParent(parentRelationId, partitionRelationId);
}
}
else
{
/* both the parent and child are Citus tables */
PreprocessAttachCitusPartitionToCitusTable(parentRelationId, partitionRelationId);
}
}
/*
* PreprocessAttachCitusPartitionToCitusTable takes a parent relation, and a partition
* to be attached to it. Both of them are Citus tables.
* Errors out if the partition is a reference table.
* Errors out if the partition is distributed and the parent is a Citus Local Table.
* Distributes the partition, if it's a Citus Local Table, and the parent is distributed.
*/
static void
PreprocessAttachCitusPartitionToCitusTable(Oid parentCitusRelationId, Oid
partitionRelationId)
{
if (IsCitusTableType(partitionRelationId, REFERENCE_TABLE))
{
ereport(ERROR, (errmsg("partitioned reference tables are not supported")));
}
else if (IsCitusTableType(partitionRelationId, DISTRIBUTED_TABLE) &&
IsCitusTableType(parentCitusRelationId, CITUS_LOCAL_TABLE))
{
ereport(ERROR, (errmsg("non-distributed partitioned tables cannot have "
"distributed partitions")));
}
else if (IsCitusTableType(partitionRelationId, CITUS_LOCAL_TABLE) &&
IsCitusTableType(parentCitusRelationId, DISTRIBUTED_TABLE))
{
/* if the parent is a distributed table, distribute the partition too */
DistributePartitionUsingParent(parentCitusRelationId, partitionRelationId);
}
else if (IsCitusTableType(partitionRelationId, CITUS_LOCAL_TABLE) &&
IsCitusTableType(parentCitusRelationId, CITUS_LOCAL_TABLE))
{
/*
* We should ensure that the partition relation has no foreign keys,
* as Citus Local Table partitions can only have inherited foreign keys.
*/
if (TableHasExternalForeignKeys(partitionRelationId))
{
ereport(ERROR, (errmsg("partition local tables added to citus metadata "
"cannot have non-inherited foreign keys")));
}
}
/*
* We don't need to add other cases here, like distributed - distributed and
* citus_local - citus_local, as PreprocessAlterTableStmt and standard process
* utility would do the work to attach partitions to shell and shard relations.
*/
}
/*
* DistributePartitionUsingParent takes a parent and a partition relation and
* distributes the partition, using the same distribution column as the parent.
* It creates a *hash* distributed table by default, as partitioned tables can only be
* distributed by hash.
*/
static void
DistributePartitionUsingParent(Oid parentCitusRelationId, Oid partitionRelationId)
{
Var *distributionColumn = DistPartitionKeyOrError(parentCitusRelationId);
char *distributionColumnName =
ColumnToColumnName(parentCitusRelationId,
nodeToString(distributionColumn));
distributionColumn =
FindColumnWithNameOnTargetRelation(parentCitusRelationId,
distributionColumnName,
partitionRelationId);
char distributionMethod = DISTRIBUTE_BY_HASH;
char *parentRelationName = generate_qualified_relation_name(parentCitusRelationId);
bool viaDeprecatedAPI = false;
SwitchToSequentialAndLocalExecutionIfPartitionNameTooLong(
parentCitusRelationId, partitionRelationId);
CreateDistributedTable(partitionRelationId, distributionColumn,
distributionMethod, ShardCount, false,
parentRelationName, viaDeprecatedAPI);
}
/*
* ErrorIfMultiLevelPartitioning takes a parent, and a partition relation to be attached
* and errors out if the partition is also a partitioned table, which means we are
* trying to build a multi-level partitioned table.
*/
static void
ErrorIfMultiLevelPartitioning(Oid parentRelationId, Oid partitionRelationId)
{
if (PartitionedTable(partitionRelationId))
{
char *relationName = get_rel_name(partitionRelationId);
char *parentRelationName = get_rel_name(parentRelationId);
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("Citus doesn't support multi-level "
"partitioned tables"),
errdetail("Relation \"%s\" is partitioned table "
"itself and it is also partition of "
"relation \"%s\".",
relationName, parentRelationName)));
}
}
/*
* ErrorIfAttachCitusTableToPgLocalTable takes a parent, and a partition relation
* to be attached. Errors out if the partition is a Citus table, and the parent is a
* regular Postgres table.
*/
static void
ErrorIfAttachCitusTableToPgLocalTable(Oid parentRelationId, Oid partitionRelationId)
{
if (!IsCitusTable(parentRelationId) &&
IsCitusTable(partitionRelationId))
{
char *parentRelationName = get_rel_name(parentRelationId);
ereport(ERROR, (errmsg("non-citus partitioned tables cannot have "
"citus table partitions"),
errhint("Distribute the partitioned table \"%s\" "
"instead, or add it to metadata", parentRelationName)));
}
}
/*
* PostprocessAlterTableSchemaStmt is executed after the change has been applied
* locally, we can now use the new dependencies of the table to ensure all its

View File

@ -296,7 +296,7 @@ ERROR: distributing multi-level partitioned tables is not supported
BEGIN;
CREATE TABLE partitioned_mx_4(a INT UNIQUE) PARTITION BY RANGE (a);
alter table partitioned_mx attach partition partitioned_mx_4 FOR VALUES FROM (13) TO (16);
ERROR: distributing multi-level partitioned tables is not supported
ERROR: Citus doesn't support multi-level partitioned tables
END;
CREATE TABLE multi_level_p (a INT UNIQUE) PARTITION BY RANGE (a);
CREATE TABLE IF NOT EXISTS multi_level_c PARTITION OF multi_level_p FOR VALUES FROM (13) TO (16) PARTITION BY RANGE (a);
@ -645,6 +645,67 @@ select logicalrelid from pg_dist_partition where logicalrelid::text like 'citus_
---------------------------------------------------------------------
(0 rows)
-- verify attaching partition with a foreign key errors out
CREATE TABLE citus_local_parent (b TEXT, a INT UNIQUE REFERENCES ref(a), d INT) PARTITION BY RANGE(a);
CREATE TABLE citus_local_with_fkey (d INT);
ALTER TABLE citus_local_with_fkey ADD CONSTRAINT cl_to_cl FOREIGN KEY(d) REFERENCES citus_local_parent(a);
-- error out
ALTER TABLE citus_local_parent ATTACH PARTITION citus_local_with_fkey DEFAULT;
ERROR: partition local tables added to citus metadata cannot have non-inherited foreign keys
DROP TABLE citus_local_parent CASCADE;
-- test attaching citus local table to distributed table
-- citus local table should be distributed
CREATE TABLE dist_table_parent (a INT UNIQUE) PARTITION BY RANGE(a);
SELECT create_distributed_table('dist_table_parent','a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE citus_local_child (a int unique);
select citus_add_local_table_to_metadata('citus_local_child', false);
citus_add_local_table_to_metadata
---------------------------------------------------------------------
(1 row)
alter table dist_table_parent attach partition citus_local_child default ;
select logicalrelid, partmethod from pg_dist_partition where logicalrelid::text in ('dist_table_parent', 'citus_local_child');
logicalrelid | partmethod
---------------------------------------------------------------------
dist_table_parent | h
citus_local_child | h
(2 rows)
-- test attaching distributed table to citus local table
CREATE TABLE dist_table_child (a INT UNIQUE);
SELECT create_distributed_table('dist_table_child','a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE citus_local_parent (a INT UNIQUE) PARTITION BY RANGE(a);
select citus_add_local_table_to_metadata('citus_local_parent', false);
citus_add_local_table_to_metadata
---------------------------------------------------------------------
(1 row)
-- this should error out
alter table citus_local_parent attach partition dist_table_child default ;
ERROR: non-distributed partitioned tables cannot have distributed partitions
-- error out attaching
CREATE TABLE pg_local (a INT UNIQUE) PARTITION BY RANGE(a);
CREATE TABLE citus_local_attach_to_pglocal (a INT UNIQUE) PARTITION BY RANGE(a);
select citus_add_local_table_to_metadata('citus_local_attach_to_pglocal', false);
citus_add_local_table_to_metadata
---------------------------------------------------------------------
(1 row)
alter table citus_local_attach_to_pglocal attach partition pg_local default ;
ERROR: Citus doesn't support multi-level partitioned tables
SELECT master_remove_distributed_table_metadata_from_workers('citus_local_table_4'::regclass::oid, 'citus_local_tables_mx', 'citus_local_table_4');
master_remove_distributed_table_metadata_from_workers
---------------------------------------------------------------------

View File

@ -310,8 +310,8 @@ SELECT create_distributed_table('partitioning_test_failure_2009', 'id');
(1 row)
ALTER TABLE partitioning_test_failure ATTACH PARTITION partitioning_test_failure_2009 FOR VALUES FROM ('2009-01-01') TO ('2010-01-01');
ERROR: non-distributed tables cannot have distributed partitions
HINT: Distribute the partitioned table "partitioning_test_failure" instead
ERROR: non-citus partitioned tables cannot have citus table partitions
HINT: Distribute the partitioned table "partitioning_test_failure" instead, or add it to metadata
-- multi-level partitioning is not allowed
DROP TABLE partitioning_test_failure_2009;
CREATE TABLE partitioning_test_failure_2009 PARTITION OF partitioning_test_failure FOR VALUES FROM ('2009-01-01') TO ('2010-01-01') PARTITION BY RANGE (time);

View File

@ -342,6 +342,37 @@ set client_min_messages to error;
alter table citus_local_parent_1 drop constraint fkey_to_drop_test;
select logicalrelid from pg_dist_partition where logicalrelid::text like 'citus_local_parent%';
-- verify attaching partition with a foreign key errors out
CREATE TABLE citus_local_parent (b TEXT, a INT UNIQUE REFERENCES ref(a), d INT) PARTITION BY RANGE(a);
CREATE TABLE citus_local_with_fkey (d INT);
ALTER TABLE citus_local_with_fkey ADD CONSTRAINT cl_to_cl FOREIGN KEY(d) REFERENCES citus_local_parent(a);
-- error out
ALTER TABLE citus_local_parent ATTACH PARTITION citus_local_with_fkey DEFAULT;
DROP TABLE citus_local_parent CASCADE;
-- test attaching citus local table to distributed table
-- citus local table should be distributed
CREATE TABLE dist_table_parent (a INT UNIQUE) PARTITION BY RANGE(a);
SELECT create_distributed_table('dist_table_parent','a');
CREATE TABLE citus_local_child (a int unique);
select citus_add_local_table_to_metadata('citus_local_child', false);
alter table dist_table_parent attach partition citus_local_child default ;
select logicalrelid, partmethod from pg_dist_partition where logicalrelid::text in ('dist_table_parent', 'citus_local_child');
-- test attaching distributed table to citus local table
CREATE TABLE dist_table_child (a INT UNIQUE);
SELECT create_distributed_table('dist_table_child','a');
CREATE TABLE citus_local_parent (a INT UNIQUE) PARTITION BY RANGE(a);
select citus_add_local_table_to_metadata('citus_local_parent', false);
-- this should error out
alter table citus_local_parent attach partition dist_table_child default ;
-- error out attaching
CREATE TABLE pg_local (a INT UNIQUE) PARTITION BY RANGE(a);
CREATE TABLE citus_local_attach_to_pglocal (a INT UNIQUE) PARTITION BY RANGE(a);
select citus_add_local_table_to_metadata('citus_local_attach_to_pglocal', false);
alter table citus_local_attach_to_pglocal attach partition pg_local default ;
SELECT master_remove_distributed_table_metadata_from_workers('citus_local_table_4'::regclass::oid, 'citus_local_tables_mx', 'citus_local_table_4');
-- both workers should print 0 as master_remove_distributed_table_metadata_from_workers

View File

@ -2004,7 +2004,6 @@ WHERE schemaname = 'partitioning_schema' AND tablename ilike '%part_table_with_%
\c - - - :master_port
SELECT stop_metadata_sync_to_node('localhost', :worker_1_port);
DROP SCHEMA partitioning_schema CASCADE;
RESET search_path;
DROP TABLE IF EXISTS