From cdf54ff4b1ead8b4a77dc995f0ddd2695a4c2ff1 Mon Sep 17 00:00:00 2001 From: Ahmet Gedemenli Date: Tue, 21 Mar 2023 12:24:16 +0300 Subject: [PATCH] Add DDL support null-shard-key tables(#6778/#6784/#6787/#6859) Add tests for ddl coverage: * indexes * partitioned tables + indexes with long names * triggers * foreign keys * statistics * grant & revoke statements * truncate & vacuum * create/test/drop view that depends on a dist table with no shard key * policy & rls test * alter table add/drop/alter_type column (using sequences/different data types/identity columns) * alter table add constraint (not null, check, exclusion constraint) * alter table add column with a default value / set default / drop default * alter table set option (autovacuum) * indexes / constraints without names * multiple subcommands Adds support for * Creating new partitions after distributing (with null key) the parent table * Attaching partitions to a distributed table with null distribution key (and automatically distribute the new partition with null key as well) * Detaching partitions from it --- .../commands/create_distributed_table.c | 3 +- src/backend/distributed/commands/table.c | 35 +- .../distributed/planner/multi_join_order.c | 2 +- src/include/distributed/metadata_utility.h | 1 + .../expected/alter_table_null_dist_key.out | 154 +++++++ .../regress/expected/create_null_dist_key.out | 410 +++++++++++++++++- src/test/regress/multi_1_schedule | 2 +- .../regress/sql/alter_table_null_dist_key.sql | 98 +++++ src/test/regress/sql/create_null_dist_key.sql | 280 +++++++++++- 9 files changed, 956 insertions(+), 29 deletions(-) create mode 100644 src/test/regress/expected/alter_table_null_dist_key.out create mode 100644 src/test/regress/sql/alter_table_null_dist_key.sql diff --git a/src/backend/distributed/commands/create_distributed_table.c b/src/backend/distributed/commands/create_distributed_table.c index 12bfcf9a5..544d8f04e 100644 --- a/src/backend/distributed/commands/create_distributed_table.c +++ b/src/backend/distributed/commands/create_distributed_table.c @@ -134,7 +134,6 @@ static List * HashSplitPointsForShardList(List *shardList); static List * HashSplitPointsForShardCount(int shardCount); static List * WorkerNodesForShardList(List *shardList); static List * RoundRobinWorkerNodeList(List *workerNodeList, int listLength); -static void CreateNullShardKeyDistTable(Oid relationId, char *colocateWithTableName); static CitusTableParams DecideCitusTableParams(CitusTableType tableType, DistributedTableParams * distributedTableParams); @@ -1031,7 +1030,7 @@ CreateReferenceTable(Oid relationId) * CreateNullShardKeyDistTable is a wrapper around CreateCitusTable that creates a * single shard distributed table that doesn't have a shard key. */ -static void +void CreateNullShardKeyDistTable(Oid relationId, char *colocateWithTableName) { DistributedTableParams distributedTableParams = { diff --git a/src/backend/distributed/commands/table.c b/src/backend/distributed/commands/table.c index 6d5fcda3f..af3439ab5 100644 --- a/src/backend/distributed/commands/table.c +++ b/src/backend/distributed/commands/table.c @@ -384,6 +384,11 @@ PostprocessCreateTableStmtPartitionOf(CreateStmt *createStatement, const */ if (IsCitusTable(parentRelationId)) { + /* + * We can create Citus local tables and distributed tables with null shard keys + * right away, without switching to sequential mode, because they are going to + * have only one shard. + */ if (IsCitusTableType(parentRelationId, CITUS_LOCAL_TABLE)) { CreateCitusLocalTablePartitionOf(createStatement, relationId, @@ -391,11 +396,18 @@ PostprocessCreateTableStmtPartitionOf(CreateStmt *createStatement, const return; } + char *parentRelationName = generate_qualified_relation_name(parentRelationId); + + if (IsCitusTableType(parentRelationId, NULL_KEY_DISTRIBUTED_TABLE)) + { + CreateNullShardKeyDistTable(relationId, parentRelationName); + return; + } + Var *parentDistributionColumn = DistPartitionKeyOrError(parentRelationId); char *distributionColumnName = ColumnToColumnName(parentRelationId, (Node *) parentDistributionColumn); char parentDistributionMethod = DISTRIBUTE_BY_HASH; - char *parentRelationName = generate_qualified_relation_name(parentRelationId); SwitchToSequentialAndLocalExecutionIfPartitionNameTooLong(parentRelationId, relationId); @@ -589,19 +601,32 @@ PreprocessAttachCitusPartitionToCitusTable(Oid parentCitusRelationId, Oid /* * DistributePartitionUsingParent takes a parent and a partition relation and - * distributes the partition, using the same distribution column as the parent. - * It creates a *hash* distributed table by default, as partitioned tables can only be - * distributed by hash. + * distributes the partition, using the same distribution column as the parent, if the + * parent has a distribution column. It creates a *hash* distributed table by default, as + * partitioned tables can only be distributed by hash, unless it's null key distributed. + * + * If the parent has no distribution key, we distribute the partition with null key too. */ static void DistributePartitionUsingParent(Oid parentCitusRelationId, Oid partitionRelationId) { + char *parentRelationName = generate_qualified_relation_name(parentCitusRelationId); + + if (!HasDistributionKey(parentCitusRelationId)) + { + /* + * If the parent is null key distributed, we should distribute the partition + * with null distribution key as well. + */ + CreateNullShardKeyDistTable(partitionRelationId, parentRelationName); + return; + } + Var *distributionColumn = DistPartitionKeyOrError(parentCitusRelationId); char *distributionColumnName = ColumnToColumnName(parentCitusRelationId, (Node *) distributionColumn); char distributionMethod = DISTRIBUTE_BY_HASH; - char *parentRelationName = generate_qualified_relation_name(parentCitusRelationId); SwitchToSequentialAndLocalExecutionIfPartitionNameTooLong( parentCitusRelationId, partitionRelationId); diff --git a/src/backend/distributed/planner/multi_join_order.c b/src/backend/distributed/planner/multi_join_order.c index b1195c664..0fff79ed8 100644 --- a/src/backend/distributed/planner/multi_join_order.c +++ b/src/backend/distributed/planner/multi_join_order.c @@ -1404,7 +1404,7 @@ DistPartitionKeyOrError(Oid relationId) if (partitionKey == NULL) { ereport(ERROR, (errmsg( - "no distribution column found for relation %d, because it is a reference table", + "no distribution column found for relation %d", relationId))); } diff --git a/src/include/distributed/metadata_utility.h b/src/include/distributed/metadata_utility.h index e27f3df22..fe404acf8 100644 --- a/src/include/distributed/metadata_utility.h +++ b/src/include/distributed/metadata_utility.h @@ -326,6 +326,7 @@ extern void DeletePartitionRow(Oid distributedRelationId); extern void DeleteShardRow(uint64 shardId); extern void UpdatePlacementGroupId(uint64 placementId, int groupId); extern void DeleteShardPlacementRow(uint64 placementId); +extern void CreateNullShardKeyDistTable(Oid relationId, char *colocateWithTableName); extern void CreateDistributedTable(Oid relationId, char *distributionColumnName, char distributionMethod, int shardCount, bool shardCountIsStrict, char *colocateWithTableName); diff --git a/src/test/regress/expected/alter_table_null_dist_key.out b/src/test/regress/expected/alter_table_null_dist_key.out new file mode 100644 index 000000000..1812c33cb --- /dev/null +++ b/src/test/regress/expected/alter_table_null_dist_key.out @@ -0,0 +1,154 @@ +CREATE SCHEMA alter_null_dist_key; +SET search_path TO alter_null_dist_key; +SET citus.next_shard_id TO 1720000; +SET citus.shard_count TO 32; +SET citus.shard_replication_factor TO 1; +CREATE SEQUENCE dist_seq; +CREATE TABLE null_dist_table(a bigint DEFAULT nextval('dist_seq') UNIQUE, "b" text, c bigint GENERATED BY DEFAULT AS IDENTITY); +INSERT INTO null_dist_table("b") VALUES ('test'); +SELECT create_distributed_table('null_dist_table', null, colocate_with=>'none', distribution_type=>null); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$alter_null_dist_key.null_dist_table$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- add column +ALTER TABLE null_dist_table ADD COLUMN d bigint DEFAULT 2; +SELECT * FROM null_dist_table ORDER BY c; + a | b | c | d +--------------------------------------------------------------------- + 1 | test | 1 | 2 +(1 row) + +-- alter default, set to 3 +ALTER TABLE null_dist_table ALTER COLUMN d SET DEFAULT 3; +INSERT INTO null_dist_table("b") VALUES ('test'); +SELECT * FROM null_dist_table ORDER BY c; + a | b | c | d +--------------------------------------------------------------------- + 1 | test | 1 | 2 + 2 | test | 2 | 3 +(2 rows) + +-- drop default, see null +ALTER TABLE null_dist_table ALTER COLUMN d DROP DEFAULT; +INSERT INTO null_dist_table("b") VALUES ('test'); +SELECT * FROM null_dist_table ORDER BY c; + a | b | c | d +--------------------------------------------------------------------- + 1 | test | 1 | 2 + 2 | test | 2 | 3 + 3 | test | 3 | +(3 rows) + +-- cleanup the rows that were added to test the default behavior +DELETE FROM null_dist_table WHERE "b" = 'test' AND a > 1; +-- alter column type +ALTER TABLE null_dist_table ALTER COLUMN d TYPE text; +UPDATE null_dist_table SET d = 'this is a text' WHERE d = '2'; +SELECT * FROM null_dist_table ORDER BY c; + a | b | c | d +--------------------------------------------------------------------- + 1 | test | 1 | this is a text +(1 row) + +-- drop seq column +ALTER TABLE null_dist_table DROP COLUMN a; +SELECT * FROM null_dist_table ORDER BY c; + b | c | d +--------------------------------------------------------------------- + test | 1 | this is a text +(1 row) + +-- add not null constraint +ALTER TABLE null_dist_table ALTER COLUMN b SET NOT NULL; +-- not null constraint violation, error out +INSERT INTO null_dist_table VALUES (NULL, 2, 'test'); +ERROR: null value in column "b" violates not-null constraint +DETAIL: Failing row contains (null, 2, test). +CONTEXT: while executing command on localhost:xxxxx +-- drop not null constraint and try again +ALTER TABLE null_dist_table ALTER COLUMN b DROP NOT NULL; +INSERT INTO null_dist_table VALUES (NULL, 3, 'test'); +SELECT * FROM null_dist_table ORDER BY c; + b | c | d +--------------------------------------------------------------------- + test | 1 | this is a text + | 3 | test +(2 rows) + +-- add exclusion constraint +ALTER TABLE null_dist_table ADD CONSTRAINT exc_b EXCLUDE USING btree (b with =); +-- rename the exclusion constraint, errors out +ALTER TABLE null_dist_table RENAME CONSTRAINT exc_b TO exc_b_1; +ERROR: renaming constraints belonging to distributed tables is currently unsupported +-- create exclusion constraint without a name +ALTER TABLE null_dist_table ADD EXCLUDE USING btree (b with =); +-- test setting autovacuum option +ALTER TABLE null_dist_table SET (autovacuum_enabled = false); +-- test multiple subcommands +ALTER TABLE null_dist_table ADD COLUMN int_column1 INTEGER, + DROP COLUMN d; +SELECT * FROM null_dist_table ORDER BY c; + b | c | int_column1 +--------------------------------------------------------------------- + test | 1 | + | 3 | +(2 rows) + +-- test policy and row level security +CREATE TABLE null_dist_key_with_policy (table_user text); +INSERT INTO null_dist_key_with_policy VALUES ('user_1'); +SELECT create_distributed_table('null_dist_key_with_policy', null); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$alter_null_dist_key.null_dist_key_with_policy$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- enable rls +ALTER TABLE null_dist_key_with_policy ENABLE ROW LEVEL SECURITY; +-- user_1 will be allowed to see the inserted row +CREATE ROLE user_1 WITH LOGIN; +GRANT ALL ON SCHEMA alter_null_dist_key TO user_1; +GRANT ALL ON TABLE alter_null_dist_key.null_dist_key_with_policy TO user_1; +CREATE POLICY table_policy ON null_dist_key_with_policy TO user_1 + USING (table_user = current_user); +-- user_2 will not be allowed to see the inserted row +CREATE ROLE user_2 WITH LOGIN; +GRANT ALL ON SCHEMA alter_null_dist_key TO user_2; +GRANT ALL ON TABLE alter_null_dist_key.null_dist_key_with_policy TO user_2; +CREATE POLICY table_policy_1 ON null_dist_key_with_policy TO user_2 + USING (table_user = current_user); +\c - user_1 - +SELECT * FROM alter_null_dist_key.null_dist_key_with_policy; + table_user +--------------------------------------------------------------------- + user_1 +(1 row) + +\c - user_2 - +SELECT * FROM alter_null_dist_key.null_dist_key_with_policy; + table_user +--------------------------------------------------------------------- +(0 rows) + +-- postgres will always be allowed to see the row as a superuser +\c - postgres - +SELECT * FROM alter_null_dist_key.null_dist_key_with_policy; + table_user +--------------------------------------------------------------------- + user_1 +(1 row) + +-- cleanup +SET client_min_messages TO ERROR; +DROP SCHEMA alter_null_dist_key CASCADE; +DROP ROLE user_1, user_2; diff --git a/src/test/regress/expected/create_null_dist_key.out b/src/test/regress/expected/create_null_dist_key.out index e24ff1e91..af6e66f62 100644 --- a/src/test/regress/expected/create_null_dist_key.out +++ b/src/test/regress/expected/create_null_dist_key.out @@ -497,7 +497,8 @@ SELECT * FROM null_dist_key_table_2 ORDER BY a; DROP TABLE null_dist_key_table_1, null_dist_key_table_2; -- create indexes before creating the null dist key tables -- .. for an initially empty table -CREATE TABLE null_dist_key_table_1(a int); +CREATE TABLE null_dist_key_table_1(a int, b int); +CREATE STATISTICS s1 (dependencies) ON a, b FROM null_dist_key_table_1; CREATE INDEX null_dist_key_table_1_idx ON null_dist_key_table_1(a); SELECT create_distributed_table('null_dist_key_table_1', null, colocate_with=>'none'); create_distributed_table @@ -505,6 +506,7 @@ SELECT create_distributed_table('null_dist_key_table_1', null, colocate_with=>'n (1 row) +CREATE STATISTICS s2 (dependencies) ON a, b FROM null_dist_key_table_1; -- .. and for another table having data in it before creating null dist key table CREATE TABLE null_dist_key_table_2(a int); INSERT INTO null_dist_key_table_2 VALUES(1); @@ -515,6 +517,11 @@ SELECT create_distributed_table('null_dist_key_table_2', null, colocate_with=>'n (1 row) +-- test create index concurrently, then reindex +CREATE INDEX CONCURRENTLY ind_conc ON null_dist_key_table_2(a); +REINDEX INDEX ind_conc; +REINDEX INDEX CONCURRENTLY ind_conc; +DROP INDEX ind_conc; SELECT * FROM null_dist_key_table_2 ORDER BY a; a --------------------------------------------------------------------- @@ -536,15 +543,23 @@ BEGIN; CREATE ROLE table_users; CREATE POLICY table_policy ON null_dist_key_table_3 TO table_users USING (table_user = current_user); + GRANT ALL ON TABLE null_dist_key_table_3 TO table_users; + ALTER TABLE null_dist_key_table_3 OWNER TO table_users; SELECT create_distributed_table('null_dist_key_table_3', null, colocate_with=>'none'); create_distributed_table --------------------------------------------------------------------- (1 row) + REVOKE ALL ON TABLE null_dist_key_table_3 FROM table_users; + ALTER TABLE null_dist_key_table_3 OWNER TO postgres; + GRANT ALL ON TABLE null_dist_key_table_3 TO table_users; ROLLBACK; +ALTER STATISTICS s2 SET STATISTICS 46; +ALTER TABLE null_dist_key_table_1 SET SCHEMA public; +DROP STATISTICS s1, s2; -- drop them for next tests -DROP TABLE null_dist_key_table_1, null_dist_key_table_2, distributed_table; +DROP TABLE public.null_dist_key_table_1, null_dist_key_table_2, distributed_table; -- tests for object names that should be escaped properly CREATE SCHEMA "NULL_!_dist_key"; CREATE TABLE "NULL_!_dist_key"."my_TABLE.1!?!"(id int, "Second_Id" int); @@ -643,6 +658,348 @@ SELECT create_distributed_table('sensors', NULL, distribution_type=>null); (1 row) +-- verify we can create new partitions after distributing the parent table +CREATE TABLE sensors_2001 PARTITION OF sensors FOR VALUES FROM ('2001-01-01') TO ('2002-01-01'); +-- verify we can attach to a null dist key table +CREATE TABLE sensors_2002 (measureid integer, eventdatetime date, measure_data jsonb, PRIMARY KEY (measureid, eventdatetime, measure_data)); +ALTER TABLE sensors ATTACH PARTITION sensors_2002 FOR VALUES FROM ('2002-01-01') TO ('2003-01-01'); +-- verify we can detach from a null dist key table +ALTER TABLE sensors DETACH PARTITION sensors_2001; +-- error out when attaching a noncolocated partition +CREATE TABLE sensors_2003 (measureid integer, eventdatetime date, measure_data jsonb, PRIMARY KEY (measureid, eventdatetime, measure_data)); +SELECT create_distributed_table('sensors_2003', NULL, distribution_type=>null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +ALTER TABLE sensors ATTACH PARTITION sensors_2003 FOR VALUES FROM ('2003-01-01') TO ('2004-01-01'); +ERROR: distributed tables cannot have non-colocated distributed tables as a partition +DROP TABLE sensors_2003; +-- verify we can attach after distributing, if the parent and partition are colocated +CREATE TABLE sensors_2004 (measureid integer, eventdatetime date, measure_data jsonb, PRIMARY KEY (measureid, eventdatetime, measure_data)); +SELECT create_distributed_table('sensors_2004', NULL, distribution_type=>null, colocate_with=>'sensors'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +ALTER TABLE sensors ATTACH PARTITION sensors_2004 FOR VALUES FROM ('2004-01-01') TO ('2005-01-01'); +-- verify we can attach a citus local table +CREATE TABLE sensors_2005 (measureid integer, eventdatetime date, measure_data jsonb, PRIMARY KEY (measureid, eventdatetime, measure_data)); +SELECT citus_add_local_table_to_metadata('sensors_2005'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +ALTER TABLE sensors ATTACH PARTITION sensors_2005 FOR VALUES FROM ('2005-01-01') TO ('2006-01-01'); +-- check metadata +-- check all partitions and the parent on pg_dist_partition +SELECT logicalrelid::text FROM pg_dist_partition WHERE logicalrelid::text IN ('sensors', 'sensors_2000', 'sensors_2001', 'sensors_2002', 'sensors_2004', 'sensors_2005') ORDER BY logicalrelid::text; + logicalrelid +--------------------------------------------------------------------- + sensors + sensors_2000 + sensors_2001 + sensors_2002 + sensors_2004 + sensors_2005 +(6 rows) + +-- verify they are all colocated +SELECT COUNT(DISTINCT(colocationid)) FROM pg_dist_partition WHERE logicalrelid::text IN ('sensors', 'sensors_2000', 'sensors_2001', 'sensors_2002', 'sensors_2004', 'sensors_2005'); + count +--------------------------------------------------------------------- + 1 +(1 row) + +-- verify all partitions are placed on the same node +SELECT COUNT(DISTINCT(groupid)) FROM pg_dist_placement WHERE shardid IN + (SELECT shardid FROM pg_dist_shard WHERE logicalrelid::text IN ('sensors', 'sensors_2000', 'sensors_2001', 'sensors_2002', 'sensors_2004', 'sensors_2005')); + count +--------------------------------------------------------------------- + 1 +(1 row) + +-- verify the shard of sensors_2000 is attached to the parent shard, on the worker node +SELECT COUNT(*) FROM run_command_on_workers($$ + SELECT relpartbound FROM pg_class WHERE relname LIKE 'sensors_2000_1______';$$) + WHERE length(result) > 0; + count +--------------------------------------------------------------------- + 1 +(1 row) + +-- verify the shard of sensors_2001 is detached from the parent shard, on the worker node +SELECT COUNT(*) FROM run_command_on_workers($$ + SELECT relpartbound FROM pg_class WHERE relname LIKE 'sensors_2001_1______';$$) + WHERE length(result) > 0; + count +--------------------------------------------------------------------- + 0 +(1 row) + +-- verify the shard of sensors_2002 is attached to the parent shard, on the worker node +SELECT COUNT(*) FROM run_command_on_workers($$ + SELECT relpartbound FROM pg_class WHERE relname LIKE 'sensors_2002_1______';$$) + WHERE length(result) > 0; + count +--------------------------------------------------------------------- + 1 +(1 row) + +-- create a partitioned citus local table and verify we error out when attaching a partition with null dist key +CREATE TABLE partitioned_citus_local_tbl( + measureid integer, + eventdatetime date, + measure_data jsonb, +PRIMARY KEY (measureid, eventdatetime, measure_data)) +PARTITION BY RANGE(eventdatetime); +SELECT citus_add_local_table_to_metadata('partitioned_citus_local_tbl'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +CREATE TABLE partition_with_null_key (measureid integer, eventdatetime date, measure_data jsonb, PRIMARY KEY (measureid, eventdatetime, measure_data)); +SELECT create_distributed_table('partition_with_null_key', NULL, distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +ALTER TABLE partitioned_citus_local_tbl ATTACH PARTITION partition_with_null_key FOR VALUES FROM ('2004-01-01') TO ('2005-01-01'); +ERROR: non-distributed partitioned tables cannot have distributed partitions +-- test partitioned tables + indexes with long names +CREATE TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"( + id int PRIMARY KEY, + "TeNANt_Id" int, + "jsondata" jsonb NOT NULL, + name text, + price numeric CHECK (price > 0), + serial_data bigserial, UNIQUE (id, price)) + PARTITION BY LIST(id); +CREATE TABLE "NULL_!_dist_key"."partition1_nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + PARTITION OF "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + FOR VALUES IN (1); +CREATE TABLE "NULL_!_dist_key"."partition2_nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + PARTITION OF "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + FOR VALUES IN (2); +CREATE TABLE "NULL_!_dist_key"."partition100_nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + PARTITION OF "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + FOR VALUES IN (100); +-- create some objects before create_distributed_table +CREATE INDEX "my!Index1New" ON "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id) WITH ( fillfactor = 80 ) WHERE id > 10; +CREATE UNIQUE INDEX uniqueIndexNew ON "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" (id); +-- ingest some data before create_distributed_table +set client_min_messages to ERROR; +INSERT INTO "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" VALUES (1, 1, row_to_json(row(1,1), true)), + (2, 1, row_to_json(row(2,2), 'false')); +reset client_min_messages; +-- create a replica identity before create_distributed_table +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" REPLICA IDENTITY USING INDEX uniqueIndexNew; +NOTICE: identifier "nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" will be truncated to "nullKeyTable.1!?!9012345678901234567890123456789012345678901234" +-- test triggers +SET client_min_messages TO ERROR; +CREATE FUNCTION insert_id_100() RETURNS trigger AS $insert_100$ +BEGIN + INSERT INTO "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" VALUES (100, 1, row_to_json(row(1,1), true)); + RETURN NEW; +END; +$insert_100$ LANGUAGE plpgsql; +CREATE TABLE null_key_table_with_trigger(a INT); +SELECT create_distributed_table('null_key_table_with_trigger', null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- try to add a trigger after distributing the table, fails +CREATE TRIGGER insert_100_trigger + AFTER UPDATE ON null_key_table_with_trigger + FOR EACH STATEMENT EXECUTE FUNCTION insert_id_100(); +ERROR: triggers are not supported on distributed tables +-- now try to distribute a table that already has a trigger on it +CREATE TRIGGER insert_100_trigger + AFTER UPDATE ON "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + FOR EACH STATEMENT EXECUTE FUNCTION insert_id_100(); +-- error out because of the trigger +SELECT create_distributed_table('"NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"', null); +ERROR: cannot distribute relation "nullKeyTable.1!?!9012345678901234567890123456789012345678901234" because it has triggers +HINT: Consider dropping all the triggers on "nullKeyTable.1!?!9012345678901234567890123456789012345678901234" and retry. +SET citus.enable_unsafe_triggers TO ON; +RESET client_min_messages; +-- this shouldn't give any syntax errors +SELECT create_distributed_table('"NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"', null); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$"NULL_!_dist_key"."partition1_nullKeyTable.1!?!90123456789012345678901234567890123"$$) +NOTICE: Copying data from local table... +NOTICE: copying the data has completed +DETAIL: The local data in the table is no longer visible, but is still on disk. +HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$"NULL_!_dist_key"."partition2_nullKeyTable.1!?!90123456789012345678901234567890123"$$) + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +-- now we can add triggers on distributed tables, because we set the GUC to on +CREATE TRIGGER insert_100_trigger_2 + AFTER UPDATE ON null_key_table_with_trigger + FOR EACH STATEMENT EXECUTE FUNCTION insert_id_100(); +SET client_min_messages TO ERROR; +UPDATE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" SET "TeNANt_Id"="TeNANt_Id"+1; +-- we should see one row with id = 100 +SELECT COUNT(*) FROM "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" WHERE id = 100; + count +--------------------------------------------------------------------- + 1 +(1 row) + +-- create some objects after create_distributed_table +CREATE INDEX "my!Index2New" ON "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id) WITH ( fillfactor = 90 ) WHERE id < 20; +CREATE UNIQUE INDEX uniqueIndex2New ON "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id); +-- error out for already existing, because of the unique index +INSERT INTO "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" VALUES (1, 1, row_to_json(row(1,1), true)); +ERROR: duplicate key value violates unique constraint "partition1_nullKeyTable.1!?!901234567890123456_bf4a8ac1_1730056" +DETAIL: Key (id)=(X) already exists. +CONTEXT: while executing command on localhost:xxxxx +-- verify all 4 shard indexes are created on the same node +SELECT result FROM run_command_on_workers($$ + SELECT COUNT(*) FROM pg_indexes WHERE indexname LIKE '%my!Index_New_1%' OR indexname LIKE '%uniqueindex%new_1%';$$) + ORDER BY nodeport; + result +--------------------------------------------------------------------- + 4 + 0 +(2 rows) + +-- foreign key to a ref table +CREATE TABLE dummy_reference_table (a INT PRIMARY KEY); +SELECT create_reference_table('dummy_reference_table'); + create_reference_table +--------------------------------------------------------------------- + +(1 row) + +TRUNCATE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"; +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + ADD CONSTRAINT fkey_to_dummy_ref FOREIGN KEY (id) REFERENCES dummy_reference_table(a); +BEGIN; -- try to add the same fkey, reversed + ALTER TABLE dummy_reference_table + ADD CONSTRAINT fkey_to_dummy_ref FOREIGN KEY (a) REFERENCES "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id); +ERROR: cannot create foreign key constraint since foreign keys from reference tables and local tables to distributed tables are not supported +DETAIL: Reference tables and local tables can only have foreign keys to reference tables and local tables +ROLLBACK; +-- errors out because of foreign key violation +INSERT INTO "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" VALUES (100, 1, row_to_json(row(1,1), true)); +ERROR: insert or update on table "partition100_nullKeyTable.1!?!9012345678901234_0aba0bf3_1730058" violates foreign key constraint "fkey_to_dummy_ref_1730055" +DETAIL: Key (id)=(X) is not present in table "dummy_reference_table_1730059". +CONTEXT: while executing command on localhost:xxxxx +-- now inserts successfully +INSERT INTO dummy_reference_table VALUES (100); +INSERT INTO "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" VALUES (100, 1, row_to_json(row(1,1), true)); +DELETE FROM "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" WHERE id = 100; +-- foreign key to a local table, errors out +CREATE TABLE local_table_for_fkey (a INT PRIMARY KEY); +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + ADD CONSTRAINT fkey_to_dummy_local FOREIGN KEY (id) REFERENCES local_table_for_fkey(a); +ERROR: referenced table "local_table_for_fkey" must be a distributed table or a reference table +DETAIL: To enforce foreign keys, the referencing and referenced rows need to be stored on the same node. +HINT: You could use SELECT create_reference_table('local_table_for_fkey') to replicate the referenced table to all nodes or consider dropping the foreign key +-- Normally, we support foreign keys from Postgres tables to distributed +-- tables assuming that the user will soon distribute the local table too +-- anyway. However, this is not the case for null-shard-key tables before +-- we improve SQL support. +ALTER TABLE local_table_for_fkey + ADD CONSTRAINT fkey_from_dummy_local FOREIGN KEY (a) REFERENCES "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id); +ERROR: queries that reference a distributed table without a shard key can only reference colocated distributed tables or reference tables +DETAIL: Local tables cannot be used in distributed queries. +CONTEXT: SQL statement "SELECT fk."a" FROM ONLY "create_null_dist_key"."local_table_for_fkey" fk LEFT OUTER JOIN "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234" pk ON ( pk."id" OPERATOR(pg_catalog.=) fk."a") WHERE pk."id" IS NULL AND (fk."a" IS NOT NULL)" +-- foreign key to a citus local table, errors out +CREATE TABLE citus_local_table_for_fkey (a INT PRIMARY KEY); +SELECT citus_add_local_table_to_metadata('citus_local_table_for_fkey'); + citus_add_local_table_to_metadata +--------------------------------------------------------------------- + +(1 row) + +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + ADD CONSTRAINT fkey_to_dummy_citus_local FOREIGN KEY (id) REFERENCES citus_local_table_for_fkey(a); +ERROR: cannot create foreign key constraint since relations are not colocated or not referencing a reference table +DETAIL: A distributed table can only have foreign keys if it is referencing another colocated hash distributed table or a reference table +-- reversed, still fails +ALTER TABLE citus_local_table_for_fkey + ADD CONSTRAINT fkey_from_dummy_citus_local FOREIGN KEY (a) REFERENCES "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id); +ERROR: cannot create foreign key constraint since foreign keys from reference tables and local tables to distributed tables are not supported +DETAIL: Reference tables and local tables can only have foreign keys to reference tables and local tables +-- foreign key to a distributed table, errors out because not colocated +CREATE TABLE dist_table_for_fkey (a INT PRIMARY KEY); +SELECT create_distributed_table('dist_table_for_fkey', 'a'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + ADD CONSTRAINT fkey_to_dummy_dist FOREIGN KEY (id) REFERENCES dist_table_for_fkey(a); +ERROR: cannot create foreign key constraint since relations are not colocated or not referencing a reference table +DETAIL: A distributed table can only have foreign keys if it is referencing another colocated hash distributed table or a reference table +-- reversed, still fails +ALTER TABLE dist_table_for_fkey + ADD CONSTRAINT fkey_to_dummy_dist FOREIGN KEY (a) REFERENCES "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id); +ERROR: cannot create foreign key constraint since relations are not colocated or not referencing a reference table +DETAIL: A distributed table can only have foreign keys if it is referencing another colocated hash distributed table or a reference table +-- create a null key distributed table, not colocated with the partitioned table, and then try to create a fkey +CREATE TABLE null_key_dist_not_colocated (a INT PRIMARY KEY); +SELECT create_distributed_table('null_key_dist_not_colocated', null, colocate_with=>'none'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + ADD CONSTRAINT fkey_to_dummy_dist FOREIGN KEY (id) REFERENCES null_key_dist_not_colocated(a); +ERROR: cannot create foreign key constraint since relations are not colocated or not referencing a reference table +DETAIL: A distributed table can only have foreign keys if it is referencing another colocated hash distributed table or a reference table +-- create a null key distributed table, colocated with the partitioned table, and then create a fkey +CREATE TABLE null_key_dist (a INT PRIMARY KEY); +SELECT create_distributed_table('null_key_dist', null, colocate_with=>'"NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + ADD CONSTRAINT fkey_to_dummy_dist FOREIGN KEY (id) REFERENCES null_key_dist(a); +-- check supported ON DELETE and ON UPDATE commands +ALTER TABLE null_key_dist ADD CONSTRAINT fkey_add_test_1 FOREIGN KEY(a) + REFERENCES "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id) ON DELETE SET DEFAULT; +ALTER TABLE null_key_dist ADD CONSTRAINT fkey_add_test_2 FOREIGN KEY(a) + REFERENCES "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id) ON UPDATE CASCADE; +ALTER TABLE null_key_dist ADD CONSTRAINT fkey_add_test_3 FOREIGN KEY(a) + REFERENCES dummy_reference_table(a) ON DELETE SET DEFAULT; +ALTER TABLE null_key_dist ADD CONSTRAINT fkey_add_test_4 FOREIGN KEY(a) + REFERENCES dummy_reference_table(a) ON UPDATE CASCADE; +ALTER TABLE null_key_dist DROP CONSTRAINT fkey_add_test_1; +ALTER TABLE null_key_dist DROP CONSTRAINT fkey_add_test_2; +ALTER TABLE null_key_dist DROP CONSTRAINT fkey_add_test_3; +ALTER TABLE null_key_dist DROP CONSTRAINT fkey_add_test_4; +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" DROP CONSTRAINT fkey_to_dummy_dist; +-- create a view that depends on the null shard key table +CREATE VIEW public.v1 AS SELECT * FROM null_key_dist; +SELECT * FROM public.v1; + a +--------------------------------------------------------------------- +(0 rows) + +DELETE FROM null_key_dist; +VACUUM null_key_dist; +TRUNCATE null_key_dist; +DROP TABLE null_key_dist CASCADE; +RESET client_min_messages; CREATE TABLE multi_level_partitioning_parent( measureid integer, eventdatetime date, @@ -771,7 +1128,7 @@ BEGIN; INSERT INTO referencing_table VALUES (1, 2); -- fails INSERT INTO referencing_table VALUES (2, 2); -ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730049" +ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730098" DETAIL: Key (a)=(2) is not present in table "referenced_table_xxxxxxx". CONTEXT: while executing command on localhost:xxxxx ROLLBACK; @@ -817,7 +1174,7 @@ BEGIN; INSERT INTO referencing_table VALUES (1, 2); -- fails INSERT INTO referencing_table VALUES (2, 2); -ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730085" +ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730134" DETAIL: Key (a)=(2) is not present in table "referenced_table_xxxxxxx". CONTEXT: while executing command on localhost:xxxxx ROLLBACK; @@ -935,8 +1292,8 @@ SELECT result, success FROM run_command_on_workers($$ $$); result | success --------------------------------------------------------------------- - ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730102" | f - ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730102" | f + ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730151" | f + ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730151" | f (2 rows) DROP TABLE referencing_table, referenced_table; @@ -951,8 +1308,8 @@ SELECT create_distributed_table('self_fkey_test', NULL, distribution_type=>null) INSERT INTO self_fkey_test VALUES (1, 1); -- ok INSERT INTO self_fkey_test VALUES (2, 3); -- fails -ERROR: insert or update on table "self_fkey_test_1730103" violates foreign key constraint "self_fkey_test_b_fkey_1730103" -DETAIL: Key (b)=(3) is not present in table "self_fkey_test_1730103". +ERROR: insert or update on table "self_fkey_test_1730152" violates foreign key constraint "self_fkey_test_b_fkey_1730152" +DETAIL: Key (b)=(3) is not present in table "self_fkey_test_1730152". CONTEXT: while executing command on localhost:xxxxx -- similar foreign key tests but this time create the referencing table later on -- referencing table is a null shard key table @@ -976,7 +1333,7 @@ BEGIN; INSERT INTO referencing_table VALUES (1, 2); -- fails INSERT INTO referencing_table VALUES (2, 2); -ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730105" +ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730154" DETAIL: Key (a)=(2) is not present in table "referenced_table_xxxxxxx". CONTEXT: while executing command on localhost:xxxxx ROLLBACK; @@ -999,7 +1356,7 @@ BEGIN; INSERT INTO referencing_table VALUES (2, 1); -- fails INSERT INTO referencing_table VALUES (1, 2); -ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_b_fkey_1730107" +ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_b_fkey_1730156" DETAIL: Key (a, b)=(1, 2) is not present in table "referenced_table_xxxxxxx". CONTEXT: while executing command on localhost:xxxxx ROLLBACK; @@ -1040,6 +1397,24 @@ BEGIN; SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'referenced_table'); ERROR: cannot create foreign key constraint since Citus does not support ON DELETE / UPDATE SET DEFAULT actions on the columns that default to sequences ROLLBACK; +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + CREATE TABLE referencing_table(a serial, b int); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'referenced_table'); + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + + ALTER TABLE referencing_table ADD CONSTRAINT fkey_to_dummy_ref_on_update FOREIGN KEY (a) REFERENCES referenced_table(a) ON UPDATE SET DEFAULT; +ERROR: cannot create foreign key constraint since Citus does not support ON DELETE / UPDATE SET DEFAULT actions on the columns that default to sequences +ROLLBACK; -- to a non-colocated null dist key table BEGIN; CREATE TABLE referenced_table(a int UNIQUE, b int); @@ -1088,7 +1463,7 @@ BEGIN; INSERT INTO referencing_table VALUES (1, 2); -- fails INSERT INTO referencing_table VALUES (2, 2); -ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730146" +ERROR: insert or update on table "referencing_table_xxxxxxx" violates foreign key constraint "referencing_table_a_fkey_1730197" DETAIL: Key (a)=(2) is not present in table "referenced_table_xxxxxxx". CONTEXT: while executing command on localhost:xxxxx ROLLBACK; @@ -1339,10 +1714,6 @@ CREATE TABLE trigger_table_1 (value int); CREATE TRIGGER trigger_1 BEFORE INSERT ON trigger_table_1 FOR EACH ROW EXECUTE FUNCTION increment_value(); -SELECT create_distributed_table('trigger_table_1', NULL, distribution_type=>null); -ERROR: cannot distribute relation "trigger_table_1" because it has triggers -HINT: Consider dropping all the triggers on "trigger_table_1" and retry. -SET citus.enable_unsafe_triggers TO ON; SELECT create_distributed_table('trigger_table_1', NULL, distribution_type=>null); create_distributed_table --------------------------------------------------------------------- @@ -1423,10 +1794,19 @@ TRUNCATE trigger_table_3; NOTICE: notice_truncate() CONTEXT: PL/pgSQL function notice_truncate() line XX at RAISE SET client_min_messages TO WARNING; +-- test rename, disable and drop trigger +ALTER TRIGGER trigger_4 ON trigger_table_3 RENAME TO trigger_new_name; +ALTER TABLE trigger_table_3 DISABLE TRIGGER ALL; +DROP TRIGGER trigger_new_name ON trigger_table_3; +-- enable the remaining triggers +ALTER TABLE trigger_table_3 ENABLE TRIGGER ALL; -- try a few simple queries at least to make sure that we don't crash BEGIN; INSERT INTO nullkey_c1_t1 SELECT * FROM nullkey_c2_t1; ROLLBACK; +DROP TRIGGER IF EXISTS trigger_1 ON trigger_table_1; +DROP TRIGGER trigger_2 ON trigger_table_2 CASCADE; +DROP TRIGGER trigger_3 ON trigger_table_3 RESTRICT; -- cleanup at exit SET client_min_messages TO ERROR; DROP SCHEMA create_null_dist_key, "NULL_!_dist_key" CASCADE; diff --git a/src/test/regress/multi_1_schedule b/src/test/regress/multi_1_schedule index a673a71d0..5d6fbb068 100644 --- a/src/test/regress/multi_1_schedule +++ b/src/test/regress/multi_1_schedule @@ -68,7 +68,7 @@ test: multi_master_protocol multi_load_data multi_load_data_superuser multi_beha test: multi_behavioral_analytics_basics multi_behavioral_analytics_single_shard_queries multi_insert_select_non_pushable_queries multi_insert_select multi_behavioral_analytics_create_table_superuser test: multi_shard_update_delete recursive_dml_with_different_planners_executors test: insert_select_repartition window_functions dml_recursive multi_insert_select_window -test: multi_insert_select_conflict citus_table_triggers +test: multi_insert_select_conflict citus_table_triggers alter_table_null_dist_key test: multi_row_insert insert_select_into_local_table alter_index # following should not run in parallel because it relies on connection counts to workers diff --git a/src/test/regress/sql/alter_table_null_dist_key.sql b/src/test/regress/sql/alter_table_null_dist_key.sql new file mode 100644 index 000000000..bcf0b4f74 --- /dev/null +++ b/src/test/regress/sql/alter_table_null_dist_key.sql @@ -0,0 +1,98 @@ +CREATE SCHEMA alter_null_dist_key; +SET search_path TO alter_null_dist_key; + +SET citus.next_shard_id TO 1720000; +SET citus.shard_count TO 32; +SET citus.shard_replication_factor TO 1; + +CREATE SEQUENCE dist_seq; +CREATE TABLE null_dist_table(a bigint DEFAULT nextval('dist_seq') UNIQUE, "b" text, c bigint GENERATED BY DEFAULT AS IDENTITY); +INSERT INTO null_dist_table("b") VALUES ('test'); +SELECT create_distributed_table('null_dist_table', null, colocate_with=>'none', distribution_type=>null); + +-- add column +ALTER TABLE null_dist_table ADD COLUMN d bigint DEFAULT 2; +SELECT * FROM null_dist_table ORDER BY c; + +-- alter default, set to 3 +ALTER TABLE null_dist_table ALTER COLUMN d SET DEFAULT 3; +INSERT INTO null_dist_table("b") VALUES ('test'); +SELECT * FROM null_dist_table ORDER BY c; + +-- drop default, see null +ALTER TABLE null_dist_table ALTER COLUMN d DROP DEFAULT; +INSERT INTO null_dist_table("b") VALUES ('test'); +SELECT * FROM null_dist_table ORDER BY c; + +-- cleanup the rows that were added to test the default behavior +DELETE FROM null_dist_table WHERE "b" = 'test' AND a > 1; + +-- alter column type +ALTER TABLE null_dist_table ALTER COLUMN d TYPE text; +UPDATE null_dist_table SET d = 'this is a text' WHERE d = '2'; +SELECT * FROM null_dist_table ORDER BY c; + +-- drop seq column +ALTER TABLE null_dist_table DROP COLUMN a; +SELECT * FROM null_dist_table ORDER BY c; + +-- add not null constraint +ALTER TABLE null_dist_table ALTER COLUMN b SET NOT NULL; + +-- not null constraint violation, error out +INSERT INTO null_dist_table VALUES (NULL, 2, 'test'); +-- drop not null constraint and try again +ALTER TABLE null_dist_table ALTER COLUMN b DROP NOT NULL; +INSERT INTO null_dist_table VALUES (NULL, 3, 'test'); +SELECT * FROM null_dist_table ORDER BY c; + +-- add exclusion constraint +ALTER TABLE null_dist_table ADD CONSTRAINT exc_b EXCLUDE USING btree (b with =); +-- rename the exclusion constraint, errors out +ALTER TABLE null_dist_table RENAME CONSTRAINT exc_b TO exc_b_1; +-- create exclusion constraint without a name +ALTER TABLE null_dist_table ADD EXCLUDE USING btree (b with =); + +-- test setting autovacuum option +ALTER TABLE null_dist_table SET (autovacuum_enabled = false); + +-- test multiple subcommands +ALTER TABLE null_dist_table ADD COLUMN int_column1 INTEGER, + DROP COLUMN d; + +SELECT * FROM null_dist_table ORDER BY c; + +-- test policy and row level security +CREATE TABLE null_dist_key_with_policy (table_user text); +INSERT INTO null_dist_key_with_policy VALUES ('user_1'); +SELECT create_distributed_table('null_dist_key_with_policy', null); + +-- enable rls +ALTER TABLE null_dist_key_with_policy ENABLE ROW LEVEL SECURITY; + +-- user_1 will be allowed to see the inserted row +CREATE ROLE user_1 WITH LOGIN; +GRANT ALL ON SCHEMA alter_null_dist_key TO user_1; +GRANT ALL ON TABLE alter_null_dist_key.null_dist_key_with_policy TO user_1; +CREATE POLICY table_policy ON null_dist_key_with_policy TO user_1 + USING (table_user = current_user); + +-- user_2 will not be allowed to see the inserted row +CREATE ROLE user_2 WITH LOGIN; +GRANT ALL ON SCHEMA alter_null_dist_key TO user_2; +GRANT ALL ON TABLE alter_null_dist_key.null_dist_key_with_policy TO user_2; +CREATE POLICY table_policy_1 ON null_dist_key_with_policy TO user_2 + USING (table_user = current_user); + +\c - user_1 - +SELECT * FROM alter_null_dist_key.null_dist_key_with_policy; +\c - user_2 - +SELECT * FROM alter_null_dist_key.null_dist_key_with_policy; +-- postgres will always be allowed to see the row as a superuser +\c - postgres - +SELECT * FROM alter_null_dist_key.null_dist_key_with_policy; + +-- cleanup +SET client_min_messages TO ERROR; +DROP SCHEMA alter_null_dist_key CASCADE; +DROP ROLE user_1, user_2; diff --git a/src/test/regress/sql/create_null_dist_key.sql b/src/test/regress/sql/create_null_dist_key.sql index b03cdde4d..9ca943d75 100644 --- a/src/test/regress/sql/create_null_dist_key.sql +++ b/src/test/regress/sql/create_null_dist_key.sql @@ -317,9 +317,11 @@ DROP TABLE null_dist_key_table_1, null_dist_key_table_2; -- create indexes before creating the null dist key tables -- .. for an initially empty table -CREATE TABLE null_dist_key_table_1(a int); +CREATE TABLE null_dist_key_table_1(a int, b int); +CREATE STATISTICS s1 (dependencies) ON a, b FROM null_dist_key_table_1; CREATE INDEX null_dist_key_table_1_idx ON null_dist_key_table_1(a); SELECT create_distributed_table('null_dist_key_table_1', null, colocate_with=>'none'); +CREATE STATISTICS s2 (dependencies) ON a, b FROM null_dist_key_table_1; -- .. and for another table having data in it before creating null dist key table CREATE TABLE null_dist_key_table_2(a int); @@ -327,6 +329,12 @@ INSERT INTO null_dist_key_table_2 VALUES(1); CREATE INDEX null_dist_key_table_2_idx ON null_dist_key_table_2(a); SELECT create_distributed_table('null_dist_key_table_2', null, colocate_with=>'none'); +-- test create index concurrently, then reindex +CREATE INDEX CONCURRENTLY ind_conc ON null_dist_key_table_2(a); +REINDEX INDEX ind_conc; +REINDEX INDEX CONCURRENTLY ind_conc; +DROP INDEX ind_conc; + SELECT * FROM null_dist_key_table_2 ORDER BY a; -- show that we do not support inheritance relationships @@ -347,11 +355,22 @@ BEGIN; CREATE POLICY table_policy ON null_dist_key_table_3 TO table_users USING (table_user = current_user); + GRANT ALL ON TABLE null_dist_key_table_3 TO table_users; + ALTER TABLE null_dist_key_table_3 OWNER TO table_users; + SELECT create_distributed_table('null_dist_key_table_3', null, colocate_with=>'none'); + + REVOKE ALL ON TABLE null_dist_key_table_3 FROM table_users; + ALTER TABLE null_dist_key_table_3 OWNER TO postgres; + GRANT ALL ON TABLE null_dist_key_table_3 TO table_users; ROLLBACK; +ALTER STATISTICS s2 SET STATISTICS 46; +ALTER TABLE null_dist_key_table_1 SET SCHEMA public; +DROP STATISTICS s1, s2; + -- drop them for next tests -DROP TABLE null_dist_key_table_1, null_dist_key_table_2, distributed_table; +DROP TABLE public.null_dist_key_table_1, null_dist_key_table_2, distributed_table; -- tests for object names that should be escaped properly @@ -438,6 +457,240 @@ SELECT create_distributed_table('sensors_2000', NULL, distribution_type=>null); SELECT create_distributed_table('sensors', NULL, distribution_type=>null); +-- verify we can create new partitions after distributing the parent table +CREATE TABLE sensors_2001 PARTITION OF sensors FOR VALUES FROM ('2001-01-01') TO ('2002-01-01'); + +-- verify we can attach to a null dist key table +CREATE TABLE sensors_2002 (measureid integer, eventdatetime date, measure_data jsonb, PRIMARY KEY (measureid, eventdatetime, measure_data)); +ALTER TABLE sensors ATTACH PARTITION sensors_2002 FOR VALUES FROM ('2002-01-01') TO ('2003-01-01'); + +-- verify we can detach from a null dist key table +ALTER TABLE sensors DETACH PARTITION sensors_2001; + +-- error out when attaching a noncolocated partition +CREATE TABLE sensors_2003 (measureid integer, eventdatetime date, measure_data jsonb, PRIMARY KEY (measureid, eventdatetime, measure_data)); +SELECT create_distributed_table('sensors_2003', NULL, distribution_type=>null, colocate_with=>'none'); +ALTER TABLE sensors ATTACH PARTITION sensors_2003 FOR VALUES FROM ('2003-01-01') TO ('2004-01-01'); +DROP TABLE sensors_2003; + +-- verify we can attach after distributing, if the parent and partition are colocated +CREATE TABLE sensors_2004 (measureid integer, eventdatetime date, measure_data jsonb, PRIMARY KEY (measureid, eventdatetime, measure_data)); +SELECT create_distributed_table('sensors_2004', NULL, distribution_type=>null, colocate_with=>'sensors'); +ALTER TABLE sensors ATTACH PARTITION sensors_2004 FOR VALUES FROM ('2004-01-01') TO ('2005-01-01'); + +-- verify we can attach a citus local table +CREATE TABLE sensors_2005 (measureid integer, eventdatetime date, measure_data jsonb, PRIMARY KEY (measureid, eventdatetime, measure_data)); +SELECT citus_add_local_table_to_metadata('sensors_2005'); +ALTER TABLE sensors ATTACH PARTITION sensors_2005 FOR VALUES FROM ('2005-01-01') TO ('2006-01-01'); + +-- check metadata +-- check all partitions and the parent on pg_dist_partition +SELECT logicalrelid::text FROM pg_dist_partition WHERE logicalrelid::text IN ('sensors', 'sensors_2000', 'sensors_2001', 'sensors_2002', 'sensors_2004', 'sensors_2005') ORDER BY logicalrelid::text; +-- verify they are all colocated +SELECT COUNT(DISTINCT(colocationid)) FROM pg_dist_partition WHERE logicalrelid::text IN ('sensors', 'sensors_2000', 'sensors_2001', 'sensors_2002', 'sensors_2004', 'sensors_2005'); +-- verify all partitions are placed on the same node +SELECT COUNT(DISTINCT(groupid)) FROM pg_dist_placement WHERE shardid IN + (SELECT shardid FROM pg_dist_shard WHERE logicalrelid::text IN ('sensors', 'sensors_2000', 'sensors_2001', 'sensors_2002', 'sensors_2004', 'sensors_2005')); + +-- verify the shard of sensors_2000 is attached to the parent shard, on the worker node +SELECT COUNT(*) FROM run_command_on_workers($$ + SELECT relpartbound FROM pg_class WHERE relname LIKE 'sensors_2000_1______';$$) + WHERE length(result) > 0; + +-- verify the shard of sensors_2001 is detached from the parent shard, on the worker node +SELECT COUNT(*) FROM run_command_on_workers($$ + SELECT relpartbound FROM pg_class WHERE relname LIKE 'sensors_2001_1______';$$) + WHERE length(result) > 0; + +-- verify the shard of sensors_2002 is attached to the parent shard, on the worker node +SELECT COUNT(*) FROM run_command_on_workers($$ + SELECT relpartbound FROM pg_class WHERE relname LIKE 'sensors_2002_1______';$$) + WHERE length(result) > 0; + +-- create a partitioned citus local table and verify we error out when attaching a partition with null dist key +CREATE TABLE partitioned_citus_local_tbl( + measureid integer, + eventdatetime date, + measure_data jsonb, +PRIMARY KEY (measureid, eventdatetime, measure_data)) +PARTITION BY RANGE(eventdatetime); +SELECT citus_add_local_table_to_metadata('partitioned_citus_local_tbl'); +CREATE TABLE partition_with_null_key (measureid integer, eventdatetime date, measure_data jsonb, PRIMARY KEY (measureid, eventdatetime, measure_data)); +SELECT create_distributed_table('partition_with_null_key', NULL, distribution_type=>null); +ALTER TABLE partitioned_citus_local_tbl ATTACH PARTITION partition_with_null_key FOR VALUES FROM ('2004-01-01') TO ('2005-01-01'); + +-- test partitioned tables + indexes with long names +CREATE TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"( + id int PRIMARY KEY, + "TeNANt_Id" int, + "jsondata" jsonb NOT NULL, + name text, + price numeric CHECK (price > 0), + serial_data bigserial, UNIQUE (id, price)) + PARTITION BY LIST(id); + +CREATE TABLE "NULL_!_dist_key"."partition1_nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + PARTITION OF "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + FOR VALUES IN (1); +CREATE TABLE "NULL_!_dist_key"."partition2_nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + PARTITION OF "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + FOR VALUES IN (2); +CREATE TABLE "NULL_!_dist_key"."partition100_nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + PARTITION OF "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + FOR VALUES IN (100); + +-- create some objects before create_distributed_table +CREATE INDEX "my!Index1New" ON "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id) WITH ( fillfactor = 80 ) WHERE id > 10; +CREATE UNIQUE INDEX uniqueIndexNew ON "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" (id); + +-- ingest some data before create_distributed_table +set client_min_messages to ERROR; +INSERT INTO "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" VALUES (1, 1, row_to_json(row(1,1), true)), + (2, 1, row_to_json(row(2,2), 'false')); +reset client_min_messages; +-- create a replica identity before create_distributed_table +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" REPLICA IDENTITY USING INDEX uniqueIndexNew; + +-- test triggers +SET client_min_messages TO ERROR; +CREATE FUNCTION insert_id_100() RETURNS trigger AS $insert_100$ +BEGIN + INSERT INTO "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" VALUES (100, 1, row_to_json(row(1,1), true)); + RETURN NEW; +END; +$insert_100$ LANGUAGE plpgsql; + +CREATE TABLE null_key_table_with_trigger(a INT); +SELECT create_distributed_table('null_key_table_with_trigger', null); +-- try to add a trigger after distributing the table, fails +CREATE TRIGGER insert_100_trigger + AFTER UPDATE ON null_key_table_with_trigger + FOR EACH STATEMENT EXECUTE FUNCTION insert_id_100(); + +-- now try to distribute a table that already has a trigger on it +CREATE TRIGGER insert_100_trigger + AFTER UPDATE ON "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + FOR EACH STATEMENT EXECUTE FUNCTION insert_id_100(); + +-- error out because of the trigger +SELECT create_distributed_table('"NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"', null); + +SET citus.enable_unsafe_triggers TO ON; +RESET client_min_messages; + +-- this shouldn't give any syntax errors +SELECT create_distributed_table('"NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"', null); + +-- now we can add triggers on distributed tables, because we set the GUC to on +CREATE TRIGGER insert_100_trigger_2 + AFTER UPDATE ON null_key_table_with_trigger + FOR EACH STATEMENT EXECUTE FUNCTION insert_id_100(); + +SET client_min_messages TO ERROR; +UPDATE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" SET "TeNANt_Id"="TeNANt_Id"+1; +-- we should see one row with id = 100 +SELECT COUNT(*) FROM "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" WHERE id = 100; + +-- create some objects after create_distributed_table +CREATE INDEX "my!Index2New" ON "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id) WITH ( fillfactor = 90 ) WHERE id < 20; +CREATE UNIQUE INDEX uniqueIndex2New ON "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id); + +-- error out for already existing, because of the unique index +INSERT INTO "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" VALUES (1, 1, row_to_json(row(1,1), true)); + +-- verify all 4 shard indexes are created on the same node +SELECT result FROM run_command_on_workers($$ + SELECT COUNT(*) FROM pg_indexes WHERE indexname LIKE '%my!Index_New_1%' OR indexname LIKE '%uniqueindex%new_1%';$$) + ORDER BY nodeport; + +-- foreign key to a ref table +CREATE TABLE dummy_reference_table (a INT PRIMARY KEY); +SELECT create_reference_table('dummy_reference_table'); +TRUNCATE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"; +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + ADD CONSTRAINT fkey_to_dummy_ref FOREIGN KEY (id) REFERENCES dummy_reference_table(a); +BEGIN; -- try to add the same fkey, reversed + ALTER TABLE dummy_reference_table + ADD CONSTRAINT fkey_to_dummy_ref FOREIGN KEY (a) REFERENCES "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id); +ROLLBACK; + +-- errors out because of foreign key violation +INSERT INTO "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" VALUES (100, 1, row_to_json(row(1,1), true)); + +-- now inserts successfully +INSERT INTO dummy_reference_table VALUES (100); +INSERT INTO "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" VALUES (100, 1, row_to_json(row(1,1), true)); +DELETE FROM "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" WHERE id = 100; + +-- foreign key to a local table, errors out +CREATE TABLE local_table_for_fkey (a INT PRIMARY KEY); +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + ADD CONSTRAINT fkey_to_dummy_local FOREIGN KEY (id) REFERENCES local_table_for_fkey(a); + +-- Normally, we support foreign keys from Postgres tables to distributed +-- tables assuming that the user will soon distribute the local table too +-- anyway. However, this is not the case for null-shard-key tables before +-- we improve SQL support. +ALTER TABLE local_table_for_fkey + ADD CONSTRAINT fkey_from_dummy_local FOREIGN KEY (a) REFERENCES "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id); + +-- foreign key to a citus local table, errors out +CREATE TABLE citus_local_table_for_fkey (a INT PRIMARY KEY); +SELECT citus_add_local_table_to_metadata('citus_local_table_for_fkey'); +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + ADD CONSTRAINT fkey_to_dummy_citus_local FOREIGN KEY (id) REFERENCES citus_local_table_for_fkey(a); +-- reversed, still fails +ALTER TABLE citus_local_table_for_fkey + ADD CONSTRAINT fkey_from_dummy_citus_local FOREIGN KEY (a) REFERENCES "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id); + +-- foreign key to a distributed table, errors out because not colocated +CREATE TABLE dist_table_for_fkey (a INT PRIMARY KEY); +SELECT create_distributed_table('dist_table_for_fkey', 'a'); +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + ADD CONSTRAINT fkey_to_dummy_dist FOREIGN KEY (id) REFERENCES dist_table_for_fkey(a); +-- reversed, still fails +ALTER TABLE dist_table_for_fkey + ADD CONSTRAINT fkey_to_dummy_dist FOREIGN KEY (a) REFERENCES "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id); + +-- create a null key distributed table, not colocated with the partitioned table, and then try to create a fkey +CREATE TABLE null_key_dist_not_colocated (a INT PRIMARY KEY); +SELECT create_distributed_table('null_key_dist_not_colocated', null, colocate_with=>'none'); +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + ADD CONSTRAINT fkey_to_dummy_dist FOREIGN KEY (id) REFERENCES null_key_dist_not_colocated(a); + +-- create a null key distributed table, colocated with the partitioned table, and then create a fkey +CREATE TABLE null_key_dist (a INT PRIMARY KEY); +SELECT create_distributed_table('null_key_dist', null, colocate_with=>'"NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"'); +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" + ADD CONSTRAINT fkey_to_dummy_dist FOREIGN KEY (id) REFERENCES null_key_dist(a); + +-- check supported ON DELETE and ON UPDATE commands +ALTER TABLE null_key_dist ADD CONSTRAINT fkey_add_test_1 FOREIGN KEY(a) + REFERENCES "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id) ON DELETE SET DEFAULT; +ALTER TABLE null_key_dist ADD CONSTRAINT fkey_add_test_2 FOREIGN KEY(a) + REFERENCES "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789"(id) ON UPDATE CASCADE; +ALTER TABLE null_key_dist ADD CONSTRAINT fkey_add_test_3 FOREIGN KEY(a) + REFERENCES dummy_reference_table(a) ON DELETE SET DEFAULT; +ALTER TABLE null_key_dist ADD CONSTRAINT fkey_add_test_4 FOREIGN KEY(a) + REFERENCES dummy_reference_table(a) ON UPDATE CASCADE; + +ALTER TABLE null_key_dist DROP CONSTRAINT fkey_add_test_1; +ALTER TABLE null_key_dist DROP CONSTRAINT fkey_add_test_2; +ALTER TABLE null_key_dist DROP CONSTRAINT fkey_add_test_3; +ALTER TABLE null_key_dist DROP CONSTRAINT fkey_add_test_4; +ALTER TABLE "NULL_!_dist_key"."nullKeyTable.1!?!9012345678901234567890123456789012345678901234567890123456789" DROP CONSTRAINT fkey_to_dummy_dist; + +-- create a view that depends on the null shard key table +CREATE VIEW public.v1 AS SELECT * FROM null_key_dist; +SELECT * FROM public.v1; + +DELETE FROM null_key_dist; +VACUUM null_key_dist; +TRUNCATE null_key_dist; +DROP TABLE null_key_dist CASCADE; + +RESET client_min_messages; + CREATE TABLE multi_level_partitioning_parent( measureid integer, eventdatetime date, @@ -685,6 +938,15 @@ BEGIN; SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'referenced_table'); ROLLBACK; +BEGIN; + CREATE TABLE referenced_table(a int UNIQUE, b int); + SELECT create_distributed_table('referenced_table', NULL, distribution_type=>null); + + CREATE TABLE referencing_table(a serial, b int); + SELECT create_distributed_table('referencing_table', NULL, distribution_type=>null, colocate_with=>'referenced_table'); + ALTER TABLE referencing_table ADD CONSTRAINT fkey_to_dummy_ref_on_update FOREIGN KEY (a) REFERENCES referenced_table(a) ON UPDATE SET DEFAULT; +ROLLBACK; + -- to a non-colocated null dist key table BEGIN; CREATE TABLE referenced_table(a int UNIQUE, b int); @@ -888,9 +1150,6 @@ FOR EACH ROW EXECUTE FUNCTION increment_value(); SELECT create_distributed_table('trigger_table_1', NULL, distribution_type=>null); -SET citus.enable_unsafe_triggers TO ON; -SELECT create_distributed_table('trigger_table_1', NULL, distribution_type=>null); - INSERT INTO trigger_table_1 VALUES(1), (2); SELECT * FROM trigger_table_1 ORDER BY value; @@ -952,11 +1211,22 @@ SET client_min_messages TO NOTICE; TRUNCATE trigger_table_3; SET client_min_messages TO WARNING; +-- test rename, disable and drop trigger +ALTER TRIGGER trigger_4 ON trigger_table_3 RENAME TO trigger_new_name; +ALTER TABLE trigger_table_3 DISABLE TRIGGER ALL; +DROP TRIGGER trigger_new_name ON trigger_table_3; +-- enable the remaining triggers +ALTER TABLE trigger_table_3 ENABLE TRIGGER ALL; + -- try a few simple queries at least to make sure that we don't crash BEGIN; INSERT INTO nullkey_c1_t1 SELECT * FROM nullkey_c2_t1; ROLLBACK; +DROP TRIGGER IF EXISTS trigger_1 ON trigger_table_1; +DROP TRIGGER trigger_2 ON trigger_table_2 CASCADE; +DROP TRIGGER trigger_3 ON trigger_table_3 RESTRICT; + -- cleanup at exit SET client_min_messages TO ERROR; DROP SCHEMA create_null_dist_key, "NULL_!_dist_key" CASCADE;