mirror of https://github.com/citusdata/citus.git
Introduce UDFs for fixing partitioned table constraint names
parent
92cf49b7e9
commit
cab17afce9
|
@ -67,3 +67,6 @@ DROP FUNCTION pg_catalog.master_create_worker_shards(text, integer, integer);
|
|||
DROP FUNCTION pg_catalog.mark_tables_colocated(regclass, regclass[]);
|
||||
#include "udfs/citus_shard_sizes/10.0-1.sql"
|
||||
#include "udfs/citus_shards/10.0-1.sql"
|
||||
|
||||
#include "udfs/fix_pre_citus10_partitioned_table_constraint_names/10.0-1.sql"
|
||||
#include "udfs/worker_fix_pre_citus10_partitioned_table_constraint_names/10.0-1.sql"
|
||||
|
|
|
@ -108,3 +108,7 @@ DROP FUNCTION pg_catalog.remove_local_tables_from_metadata();
|
|||
#include "../udfs/create_citus_local_table/9.5-1.sql"
|
||||
DROP VIEW pg_catalog.citus_shards CASCADE;
|
||||
DROP FUNCTION pg_catalog.citus_shard_sizes(OUT table_name text, OUT size bigint);
|
||||
|
||||
DROP FUNCTION pg_catalog.fix_pre_citus10_partitioned_table_constraint_names();
|
||||
DROP FUNCTION pg_catalog.fix_pre_citus10_partitioned_table_constraint_names(regclass);
|
||||
DROP FUNCTION pg_catalog.worker_fix_pre_citus10_partitioned_table_constraint_names(regclass,bigint,text);
|
||||
|
|
29
src/backend/distributed/sql/udfs/fix_pre_citus10_partitioned_table_constraint_names/10.0-1.sql
generated
Normal file
29
src/backend/distributed/sql/udfs/fix_pre_citus10_partitioned_table_constraint_names/10.0-1.sql
generated
Normal file
|
@ -0,0 +1,29 @@
|
|||
CREATE FUNCTION pg_catalog.fix_pre_citus10_partitioned_table_constraint_names(table_name regclass)
|
||||
RETURNS void
|
||||
LANGUAGE C STRICT
|
||||
AS 'MODULE_PATHNAME', $$fix_pre_citus10_partitioned_table_constraint_names$$;
|
||||
COMMENT ON FUNCTION pg_catalog.fix_pre_citus10_partitioned_table_constraint_names(table_name regclass)
|
||||
IS 'fix constraint names on partition shards';
|
||||
|
||||
CREATE OR REPLACE FUNCTION pg_catalog.fix_pre_citus10_partitioned_table_constraint_names()
|
||||
RETURNS SETOF regclass
|
||||
LANGUAGE plpgsql
|
||||
AS $$
|
||||
DECLARE
|
||||
oid regclass;
|
||||
BEGIN
|
||||
FOR oid IN SELECT c.oid
|
||||
FROM pg_dist_partition p
|
||||
JOIN pg_class c ON p.logicalrelid = c.oid
|
||||
JOIN pg_namespace n ON c.relnamespace = n.oid
|
||||
WHERE c.relkind = 'p'
|
||||
ORDER BY n.nspname, c.relname
|
||||
LOOP
|
||||
EXECUTE 'SELECT fix_pre_citus10_partitioned_table_constraint_names( ' || quote_literal(oid) || ' )';
|
||||
RETURN NEXT oid;
|
||||
END LOOP;
|
||||
RETURN;
|
||||
END;
|
||||
$$;
|
||||
COMMENT ON FUNCTION pg_catalog.fix_pre_citus10_partitioned_table_constraint_names()
|
||||
IS 'fix constraint names on all partition shards';
|
|
@ -0,0 +1,29 @@
|
|||
CREATE FUNCTION pg_catalog.fix_pre_citus10_partitioned_table_constraint_names(table_name regclass)
|
||||
RETURNS void
|
||||
LANGUAGE C STRICT
|
||||
AS 'MODULE_PATHNAME', $$fix_pre_citus10_partitioned_table_constraint_names$$;
|
||||
COMMENT ON FUNCTION pg_catalog.fix_pre_citus10_partitioned_table_constraint_names(table_name regclass)
|
||||
IS 'fix constraint names on partition shards';
|
||||
|
||||
CREATE OR REPLACE FUNCTION pg_catalog.fix_pre_citus10_partitioned_table_constraint_names()
|
||||
RETURNS SETOF regclass
|
||||
LANGUAGE plpgsql
|
||||
AS $$
|
||||
DECLARE
|
||||
oid regclass;
|
||||
BEGIN
|
||||
FOR oid IN SELECT c.oid
|
||||
FROM pg_dist_partition p
|
||||
JOIN pg_class c ON p.logicalrelid = c.oid
|
||||
JOIN pg_namespace n ON c.relnamespace = n.oid
|
||||
WHERE c.relkind = 'p'
|
||||
ORDER BY n.nspname, c.relname
|
||||
LOOP
|
||||
EXECUTE 'SELECT fix_pre_citus10_partitioned_table_constraint_names( ' || quote_literal(oid) || ' )';
|
||||
RETURN NEXT oid;
|
||||
END LOOP;
|
||||
RETURN;
|
||||
END;
|
||||
$$;
|
||||
COMMENT ON FUNCTION pg_catalog.fix_pre_citus10_partitioned_table_constraint_names()
|
||||
IS 'fix constraint names on all partition shards';
|
|
@ -0,0 +1,10 @@
|
|||
CREATE FUNCTION pg_catalog.worker_fix_pre_citus10_partitioned_table_constraint_names(table_name regclass,
|
||||
shardid bigint,
|
||||
constraint_name text)
|
||||
RETURNS void
|
||||
LANGUAGE C STRICT
|
||||
AS 'MODULE_PATHNAME', $$worker_fix_pre_citus10_partitioned_table_constraint_names$$;
|
||||
COMMENT ON FUNCTION pg_catalog.worker_fix_pre_citus10_partitioned_table_constraint_names(table_name regclass,
|
||||
shardid bigint,
|
||||
constraint_name text)
|
||||
IS 'fix constraint names on partition shards on worker nodes';
|
|
@ -0,0 +1,10 @@
|
|||
CREATE FUNCTION pg_catalog.worker_fix_pre_citus10_partitioned_table_constraint_names(table_name regclass,
|
||||
shardid bigint,
|
||||
constraint_name text)
|
||||
RETURNS void
|
||||
LANGUAGE C STRICT
|
||||
AS 'MODULE_PATHNAME', $$worker_fix_pre_citus10_partitioned_table_constraint_names$$;
|
||||
COMMENT ON FUNCTION pg_catalog.worker_fix_pre_citus10_partitioned_table_constraint_names(table_name regclass,
|
||||
shardid bigint,
|
||||
constraint_name text)
|
||||
IS 'fix constraint names on partition shards on worker nodes';
|
|
@ -14,13 +14,22 @@
|
|||
#include "catalog/indexing.h"
|
||||
#include "catalog/partition.h"
|
||||
#include "catalog/pg_class.h"
|
||||
#include "catalog/pg_constraint.h"
|
||||
#include "catalog/pg_inherits.h"
|
||||
#include "common/string.h"
|
||||
#include "distributed/citus_nodes.h"
|
||||
#include "distributed/adaptive_executor.h"
|
||||
#include "distributed/citus_ruleutils.h"
|
||||
#include "distributed/colocation_utils.h"
|
||||
#include "distributed/commands.h"
|
||||
#include "distributed/coordinator_protocol.h"
|
||||
#include "distributed/deparse_shard_query.h"
|
||||
#include "distributed/listutils.h"
|
||||
#include "distributed/metadata_utility.h"
|
||||
#include "distributed/coordinator_protocol.h"
|
||||
#include "distributed/multi_partitioning_utils.h"
|
||||
#include "distributed/multi_physical_planner.h"
|
||||
#include "distributed/relay_utility.h"
|
||||
#include "distributed/resource_lock.h"
|
||||
#include "distributed/shardinterval_utils.h"
|
||||
#include "distributed/version_compat.h"
|
||||
#include "lib/stringinfo.h"
|
||||
|
@ -37,6 +46,267 @@
|
|||
|
||||
static char * PartitionBound(Oid partitionId);
|
||||
static Relation try_relation_open_nolock(Oid relationId);
|
||||
static List * CreateFixPartitionConstraintsTaskList(Oid relationId);
|
||||
static List * WorkerFixPartitionConstraintCommandList(Oid relationId, uint64 shardId,
|
||||
List *checkConstraintList);
|
||||
static List * CheckConstraintNameListForRelation(Oid relationId);
|
||||
static bool RelationHasConstraint(Oid relationId, char *constraintName);
|
||||
static char * RenameConstraintCommand(Oid relationId, char *constraintName,
|
||||
char *newConstraintName);
|
||||
|
||||
|
||||
PG_FUNCTION_INFO_V1(fix_pre_citus10_partitioned_table_constraint_names);
|
||||
PG_FUNCTION_INFO_V1(worker_fix_pre_citus10_partitioned_table_constraint_names);
|
||||
|
||||
|
||||
/*
|
||||
* fix_pre_citus10_partitioned_table_constraint_names fixes the constraint names of
|
||||
* partitioned table shards on workers.
|
||||
*
|
||||
* Constraint names for partitioned table shards should have shardId suffixes if and only
|
||||
* if they are unique or foreign key constraints. We mistakenly appended shardIds to
|
||||
* constraint names on ALTER TABLE dist_part_table ADD CONSTRAINT .. queries prior to
|
||||
* Citus 10. fix_pre_citus10_partitioned_table_constraint_names determines if this is the
|
||||
* case, and renames constraints back to their original names on shards.
|
||||
*/
|
||||
Datum
|
||||
fix_pre_citus10_partitioned_table_constraint_names(PG_FUNCTION_ARGS)
|
||||
{
|
||||
Oid relationId = PG_GETARG_OID(0);
|
||||
EnsureCoordinator();
|
||||
|
||||
if (!PartitionedTable(relationId))
|
||||
{
|
||||
ereport(ERROR, (errmsg("could not fix partition constraints: "
|
||||
"relation does not exist or is not partitioned")));
|
||||
}
|
||||
if (!IsCitusTable(relationId))
|
||||
{
|
||||
ereport(ERROR, (errmsg("fix_pre_citus10_partitioned_table_constraint_names can "
|
||||
"only be called for distributed partitioned tables")));
|
||||
}
|
||||
|
||||
List *taskList = CreateFixPartitionConstraintsTaskList(relationId);
|
||||
bool localExecutionSupported = true;
|
||||
ExecuteUtilityTaskList(taskList, localExecutionSupported);
|
||||
|
||||
PG_RETURN_VOID();
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* worker_fix_pre_citus10_partitioned_table_constraint_names fixes the constraint names on a worker given a shell
|
||||
* table name and shard id.
|
||||
*/
|
||||
Datum
|
||||
worker_fix_pre_citus10_partitioned_table_constraint_names(PG_FUNCTION_ARGS)
|
||||
{
|
||||
Oid relationId = PG_GETARG_OID(0);
|
||||
int64 shardId = PG_GETARG_INT32(1);
|
||||
text *constraintNameText = PG_GETARG_TEXT_P(2);
|
||||
|
||||
if (!PartitionedTable(relationId))
|
||||
{
|
||||
ereport(ERROR, (errmsg("could not fix partition constraints: "
|
||||
"relation does not exist or is not partitioned")));
|
||||
}
|
||||
|
||||
char *constraintName = text_to_cstring(constraintNameText);
|
||||
char *shardIdAppendedConstraintName = pstrdup(constraintName);
|
||||
AppendShardIdToName(&shardIdAppendedConstraintName, shardId);
|
||||
|
||||
/* if shardId was appended to the constraint name, rename back to original */
|
||||
if (RelationHasConstraint(relationId, shardIdAppendedConstraintName))
|
||||
{
|
||||
char *renameConstraintDDLCommand =
|
||||
RenameConstraintCommand(relationId, shardIdAppendedConstraintName,
|
||||
constraintName);
|
||||
ExecuteAndLogUtilityCommand(renameConstraintDDLCommand);
|
||||
}
|
||||
PG_RETURN_VOID();
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* CreateFixPartitionConstraintsTaskList goes over all the partitions of a distributed
|
||||
* partitioned table, and creates the list of tasks to execute
|
||||
* worker_fix_pre_citus10_partitioned_table_constraint_names UDF on worker nodes.
|
||||
*/
|
||||
static List *
|
||||
CreateFixPartitionConstraintsTaskList(Oid relationId)
|
||||
{
|
||||
List *taskList = NIL;
|
||||
|
||||
/* enumerate the tasks when putting them to the taskList */
|
||||
int taskId = 1;
|
||||
List *checkConstraintList = CheckConstraintNameListForRelation(relationId);
|
||||
|
||||
/* early exit if the relation does not have any check constraints */
|
||||
if (checkConstraintList == NIL)
|
||||
{
|
||||
return NIL;
|
||||
}
|
||||
|
||||
List *shardIntervalList = LoadShardIntervalList(relationId);
|
||||
|
||||
/* lock metadata before getting placement lists */
|
||||
LockShardListMetadata(shardIntervalList, ShareLock);
|
||||
|
||||
ShardInterval *shardInterval = NULL;
|
||||
foreach_ptr(shardInterval, shardIntervalList)
|
||||
{
|
||||
uint64 shardId = shardInterval->shardId;
|
||||
|
||||
List *queryStringList = WorkerFixPartitionConstraintCommandList(relationId,
|
||||
shardId,
|
||||
checkConstraintList);
|
||||
|
||||
Task *task = CitusMakeNode(Task);
|
||||
task->jobId = INVALID_JOB_ID;
|
||||
task->taskId = taskId++;
|
||||
|
||||
task->taskType = DDL_TASK;
|
||||
SetTaskQueryStringList(task, queryStringList);
|
||||
task->dependentTaskList = NULL;
|
||||
task->replicationModel = REPLICATION_MODEL_INVALID;
|
||||
task->anchorShardId = shardId;
|
||||
task->taskPlacementList = ActiveShardPlacementList(shardId);
|
||||
|
||||
taskList = lappend(taskList, task);
|
||||
}
|
||||
|
||||
return taskList;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* CheckConstraintNameListForRelation returns a list of names of CHECK constraints
|
||||
* for a relation.
|
||||
*/
|
||||
static List *
|
||||
CheckConstraintNameListForRelation(Oid relationId)
|
||||
{
|
||||
List *constraintNameList = NIL;
|
||||
|
||||
int scanKeyCount = 2;
|
||||
ScanKeyData scanKey[2];
|
||||
|
||||
Relation pgConstraint = table_open(ConstraintRelationId, AccessShareLock);
|
||||
|
||||
ScanKeyInit(&scanKey[0], Anum_pg_constraint_conrelid,
|
||||
BTEqualStrategyNumber, F_OIDEQ, relationId);
|
||||
ScanKeyInit(&scanKey[1], Anum_pg_constraint_contype,
|
||||
BTEqualStrategyNumber, F_CHAREQ, CONSTRAINT_CHECK);
|
||||
|
||||
bool useIndex = false;
|
||||
SysScanDesc scanDescriptor = systable_beginscan(pgConstraint, InvalidOid, useIndex,
|
||||
NULL, scanKeyCount, scanKey);
|
||||
|
||||
HeapTuple heapTuple = systable_getnext(scanDescriptor);
|
||||
while (HeapTupleIsValid(heapTuple))
|
||||
{
|
||||
Form_pg_constraint constraintForm = (Form_pg_constraint) GETSTRUCT(heapTuple);
|
||||
char *constraintName = NameStr(constraintForm->conname);
|
||||
constraintNameList = lappend(constraintNameList, pstrdup(constraintName));
|
||||
|
||||
heapTuple = systable_getnext(scanDescriptor);
|
||||
}
|
||||
|
||||
systable_endscan(scanDescriptor);
|
||||
table_close(pgConstraint, NoLock);
|
||||
|
||||
return constraintNameList;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* WorkerFixPartitionConstraintCommandList creates a list of queries that will fix
|
||||
* all check constraint names of a shard.
|
||||
*/
|
||||
static List *
|
||||
WorkerFixPartitionConstraintCommandList(Oid relationId, uint64 shardId,
|
||||
List *checkConstraintList)
|
||||
{
|
||||
List *commandList = NIL;
|
||||
Oid schemaId = get_rel_namespace(relationId);
|
||||
char *schemaName = get_namespace_name(schemaId);
|
||||
char *relationName = get_rel_name(relationId);
|
||||
char *shardRelationName = pstrdup(relationName);
|
||||
|
||||
/* build shard relation name */
|
||||
AppendShardIdToName(&shardRelationName, shardId);
|
||||
|
||||
char *quotedShardName = quote_qualified_identifier(schemaName, shardRelationName);
|
||||
|
||||
char *constraintName = NULL;
|
||||
foreach_ptr(constraintName, checkConstraintList)
|
||||
{
|
||||
StringInfo shardQueryString = makeStringInfo();
|
||||
appendStringInfo(shardQueryString,
|
||||
"SELECT worker_fix_pre_citus10_partitioned_table_constraint_names(%s::regclass, "
|
||||
UINT64_FORMAT ", %s::text)",
|
||||
quote_literal_cstr(quotedShardName), shardId,
|
||||
quote_literal_cstr(constraintName));
|
||||
commandList = lappend(commandList, shardQueryString->data);
|
||||
}
|
||||
|
||||
return commandList;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* RelationHasConstraint checks if a relation has a constraint with a given name.
|
||||
*/
|
||||
static bool
|
||||
RelationHasConstraint(Oid relationId, char *constraintName)
|
||||
{
|
||||
bool found = false;
|
||||
|
||||
int scanKeyCount = 2;
|
||||
ScanKeyData scanKey[2];
|
||||
|
||||
Relation pgConstraint = table_open(ConstraintRelationId, AccessShareLock);
|
||||
|
||||
ScanKeyInit(&scanKey[0], Anum_pg_constraint_conrelid,
|
||||
BTEqualStrategyNumber, F_OIDEQ, ObjectIdGetDatum(relationId));
|
||||
ScanKeyInit(&scanKey[1], Anum_pg_constraint_conname,
|
||||
BTEqualStrategyNumber, F_NAMEEQ, CStringGetDatum(constraintName));
|
||||
|
||||
bool useIndex = false;
|
||||
SysScanDesc scanDescriptor = systable_beginscan(pgConstraint, InvalidOid, useIndex,
|
||||
NULL, scanKeyCount, scanKey);
|
||||
|
||||
HeapTuple heapTuple = systable_getnext(scanDescriptor);
|
||||
if (HeapTupleIsValid(heapTuple))
|
||||
{
|
||||
found = true;
|
||||
}
|
||||
|
||||
systable_endscan(scanDescriptor);
|
||||
table_close(pgConstraint, NoLock);
|
||||
|
||||
return found;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* RenameConstraintCommand creates the query string that will rename a constraint
|
||||
*/
|
||||
static char *
|
||||
RenameConstraintCommand(Oid relationId, char *constraintName, char *newConstraintName)
|
||||
{
|
||||
char *qualifiedRelationName = generate_qualified_relation_name(relationId);
|
||||
const char *quotedConstraintName = quote_identifier(constraintName);
|
||||
const char *quotedNewConstraintName = quote_identifier(newConstraintName);
|
||||
|
||||
StringInfo renameCommand = makeStringInfo();
|
||||
appendStringInfo(renameCommand, "ALTER TABLE %s RENAME CONSTRAINT %s TO %s",
|
||||
qualifiedRelationName, quotedConstraintName,
|
||||
quotedNewConstraintName);
|
||||
|
||||
return renameCommand->data;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* Returns true if the given relation is a partitioned table.
|
||||
|
|
|
@ -25,6 +25,8 @@ extern char * GenerateDetachPartitionCommand(Oid partitionTableId);
|
|||
extern char * GenerateAttachShardPartitionCommand(ShardInterval *shardInterval);
|
||||
extern char * GenerateAlterTableAttachPartitionCommand(Oid partitionTableId);
|
||||
extern char * GeneratePartitioningInformation(Oid tableId);
|
||||
extern void FixPartitionConstraintsOnWorkers(Oid relationId);
|
||||
extern void FixLocalPartitionConstraints(Oid relationId, int64 shardId);
|
||||
|
||||
|
||||
#endif /* MULTI_PARTITIONING_UTILS_H_ */
|
||||
|
|
|
@ -1,4 +1,5 @@
|
|||
# this schedule is to be run only on coordinators
|
||||
|
||||
test: upgrade_basic_after
|
||||
test: upgrade_partition_constraints_after
|
||||
test: upgrade_pg_dist_object_test_after
|
||||
|
|
|
@ -1,4 +1,5 @@
|
|||
# this schedule is to be run on only coordinators
|
||||
|
||||
test: upgrade_basic_before
|
||||
test: upgrade_partition_constraints_before
|
||||
test: upgrade_pg_dist_object_test_before
|
||||
|
|
|
@ -204,3 +204,6 @@ s/ERROR: cannot append to shardId [0-9]+/ERROR: cannot append to shardId xxxxx
|
|||
# hide warning/hint message that we get when executing create_citus_local_table
|
||||
/local tables that are added to metadata but not chained with reference tables via foreign keys might be automatically converted back to postgres tables$/d
|
||||
/Consider setting citus.enable_local_reference_table_foreign_keys to 'off' to disable this behavior$/d
|
||||
|
||||
# normalize partitioned table shard constraint name errors for upgrade_partition_constraints_(before|after)
|
||||
s/^(ERROR: child table is missing constraint "\w+)_([0-9])+"/\1_xxxxxx"/g
|
||||
|
|
|
@ -448,7 +448,7 @@ SELECT * FROM print_extension_changes();
|
|||
---------------------------------------------------------------------
|
||||
function citus_total_relation_size(regclass) |
|
||||
function create_citus_local_table(regclass) |
|
||||
function mark_tables_colocated(regclass, regclass[]) |
|
||||
function mark_tables_colocated(regclass,regclass[]) |
|
||||
function master_conninfo_cache_invalidate() |
|
||||
function master_create_distributed_table(regclass,text,citus.distribution_type) |
|
||||
function master_create_worker_shards(text,integer,integer) |
|
||||
|
@ -496,11 +496,14 @@ SELECT * FROM print_extension_changes();
|
|||
| function citus_update_shard_statistics(bigint)
|
||||
| function citus_update_table_statistics(regclass)
|
||||
| function columnar.columnar_handler(internal)
|
||||
| function fix_pre_citus10_partitioned_table_constraint_names()
|
||||
| function fix_pre_citus10_partitioned_table_constraint_names(regclass)
|
||||
| function notify_constraint_dropped()
|
||||
| function remove_local_tables_from_metadata()
|
||||
| function time_partition_range(regclass)
|
||||
| function undistribute_table(regclass,boolean)
|
||||
| function worker_change_sequence_dependency(regclass,regclass,regclass)
|
||||
| function worker_fix_pre_citus10_partitioned_table_constraint_names(regclass,bigint,text)
|
||||
| schema columnar
|
||||
| sequence columnar.storageid_seq
|
||||
| table columnar.chunk
|
||||
|
@ -509,7 +512,7 @@ SELECT * FROM print_extension_changes();
|
|||
| view citus_shards
|
||||
| view citus_tables
|
||||
| view time_partitions
|
||||
(63 rows)
|
||||
(66 rows)
|
||||
|
||||
DROP TABLE prev_objects, extension_diff;
|
||||
-- show running version
|
||||
|
|
|
@ -448,7 +448,7 @@ SELECT * FROM print_extension_changes();
|
|||
---------------------------------------------------------------------
|
||||
function citus_total_relation_size(regclass) |
|
||||
function create_citus_local_table(regclass) |
|
||||
function mark_tables_colocated(regclass, regclass[]) |
|
||||
function mark_tables_colocated(regclass,regclass[]) |
|
||||
function master_conninfo_cache_invalidate() |
|
||||
function master_create_distributed_table(regclass,text,citus.distribution_type) |
|
||||
function master_create_worker_shards(text,integer,integer) |
|
||||
|
@ -492,11 +492,14 @@ SELECT * FROM print_extension_changes();
|
|||
| function citus_update_node(integer,text,integer,boolean,integer)
|
||||
| function citus_update_shard_statistics(bigint)
|
||||
| function citus_update_table_statistics(regclass)
|
||||
| function fix_pre_citus10_partitioned_table_constraint_names()
|
||||
| function fix_pre_citus10_partitioned_table_constraint_names(regclass)
|
||||
| function notify_constraint_dropped()
|
||||
| function remove_local_tables_from_metadata()
|
||||
| function time_partition_range(regclass)
|
||||
| function undistribute_table(regclass,boolean)
|
||||
| function worker_change_sequence_dependency(regclass,regclass,regclass)
|
||||
| function worker_fix_pre_citus10_partitioned_table_constraint_names(regclass,bigint,text)
|
||||
| schema columnar
|
||||
| sequence columnar.storageid_seq
|
||||
| table columnar.chunk
|
||||
|
@ -505,7 +508,7 @@ SELECT * FROM print_extension_changes();
|
|||
| view citus_shards
|
||||
| view citus_tables
|
||||
| view time_partitions
|
||||
(59 rows)
|
||||
(62 rows)
|
||||
|
||||
DROP TABLE prev_objects, extension_diff;
|
||||
-- show running version
|
||||
|
|
|
@ -1988,6 +1988,28 @@ CREATE TABLE IF NOT EXISTS partition_of_other_table PARTITION OF partitioning_te
|
|||
NOTICE: relation "partition_of_other_table" already exists, skipping
|
||||
ALTER TABLE another_table DETACH PARTITION partition_of_other_table;
|
||||
DROP TABLE another_table, partition_of_other_table;
|
||||
-- test fix_pre_citus10_partitioned_table_constraint_names udf
|
||||
SELECT fix_pre_citus10_partitioned_table_constraint_names('partitioning_test');
|
||||
fix_pre_citus10_partitioned_table_constraint_names
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT fix_pre_citus10_partitioned_table_constraint_names();
|
||||
fix_pre_citus10_partitioned_table_constraint_names
|
||||
---------------------------------------------------------------------
|
||||
partitioning_test
|
||||
"schema-test"
|
||||
public.partitioning_hash_join_test
|
||||
public.partitioning_hash_test
|
||||
public.partitioning_test_failure
|
||||
(5 rows)
|
||||
|
||||
-- the following should fail
|
||||
SELECT fix_pre_citus10_partitioned_table_constraint_names('public.non_distributed_partitioned_table');
|
||||
ERROR: fix_pre_citus10_partitioned_table_constraint_names can only be called for distributed partitioned tables
|
||||
SELECT fix_pre_citus10_partitioned_table_constraint_names('reference_table');
|
||||
ERROR: could not fix partition constraints: relation does not exist or is not partitioned
|
||||
ALTER TABLE partitioning_test DETACH PARTITION partitioning_test_2008;
|
||||
ALTER TABLE partitioning_test DETACH PARTITION partitioning_test_2009;
|
||||
ALTER TABLE partitioning_test DETACH PARTITION partitioning_test_2010;
|
||||
|
|
|
@ -276,7 +276,7 @@ SELECT public.generate_partition_information('partition_child_1_schema.child_1')
|
|||
ERROR: "child_1" is not a parent table
|
||||
SELECT public.print_partitions('partition_child_1_schema.child_1');
|
||||
ERROR: "child_1" is not a parent table
|
||||
-- now pring the partitions
|
||||
-- now print the partitions
|
||||
SELECT public.print_partitions('parent_table');
|
||||
print_partitions
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -112,6 +112,8 @@ ORDER BY 1;
|
|||
function dump_global_wait_edges()
|
||||
function dump_local_wait_edges()
|
||||
function fetch_intermediate_results(text[],text,integer)
|
||||
function fix_pre_citus10_partitioned_table_constraint_names()
|
||||
function fix_pre_citus10_partitioned_table_constraint_names(regclass)
|
||||
function get_all_active_transactions()
|
||||
function get_colocated_shard_array(bigint)
|
||||
function get_colocated_table_array(regclass)
|
||||
|
@ -188,6 +190,7 @@ ORDER BY 1;
|
|||
function worker_drop_distributed_table(text)
|
||||
function worker_fetch_foreign_file(text,text,bigint,text[],integer[])
|
||||
function worker_fetch_partition_file(bigint,integer,integer,integer,text,integer)
|
||||
function worker_fix_pre_citus10_partitioned_table_constraint_names(regclass,bigint,text)
|
||||
function worker_hash("any")
|
||||
function worker_hash_partition_table(bigint,integer,text,text,oid,anyarray)
|
||||
function worker_last_saved_explain_analyze()
|
||||
|
@ -238,5 +241,5 @@ ORDER BY 1;
|
|||
view citus_worker_stat_activity
|
||||
view pg_dist_shard_placement
|
||||
view time_partitions
|
||||
(222 rows)
|
||||
(225 rows)
|
||||
|
||||
|
|
|
@ -108,6 +108,8 @@ ORDER BY 1;
|
|||
function dump_global_wait_edges()
|
||||
function dump_local_wait_edges()
|
||||
function fetch_intermediate_results(text[],text,integer)
|
||||
function fix_pre_citus10_partitioned_table_constraint_names()
|
||||
function fix_pre_citus10_partitioned_table_constraint_names(regclass)
|
||||
function get_all_active_transactions()
|
||||
function get_colocated_shard_array(bigint)
|
||||
function get_colocated_table_array(regclass)
|
||||
|
@ -184,6 +186,7 @@ ORDER BY 1;
|
|||
function worker_drop_distributed_table(text)
|
||||
function worker_fetch_foreign_file(text,text,bigint,text[],integer[])
|
||||
function worker_fetch_partition_file(bigint,integer,integer,integer,text,integer)
|
||||
function worker_fix_pre_citus10_partitioned_table_constraint_names(regclass,bigint,text)
|
||||
function worker_hash("any")
|
||||
function worker_hash_partition_table(bigint,integer,text,text,oid,anyarray)
|
||||
function worker_last_saved_explain_analyze()
|
||||
|
@ -234,5 +237,5 @@ ORDER BY 1;
|
|||
view citus_worker_stat_activity
|
||||
view pg_dist_shard_placement
|
||||
view time_partitions
|
||||
(218 rows)
|
||||
(221 rows)
|
||||
|
||||
|
|
|
@ -0,0 +1,19 @@
|
|||
-- test cases for #3970
|
||||
SET search_path = test_3970;
|
||||
--5. add a partition
|
||||
-- This command will fail as the child table has a wrong constraint name
|
||||
CREATE TABLE part_table_p202009 PARTITION OF part_table FOR VALUES FROM ('2020-09-01 00:00:00') TO ('2020-10-01 00:00:00');
|
||||
ERROR: child table is missing constraint "ck_01234567890123456789012345678901234567890123_8478db72_xxxxxx"
|
||||
CONTEXT: while executing command on localhost:xxxxx
|
||||
-- fix constraint names on partitioned table shards
|
||||
SELECT fix_pre_citus10_partitioned_table_constraint_names('part_table'::regclass);
|
||||
fix_pre_citus10_partitioned_table_constraint_names
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
--5. add a partition
|
||||
CREATE TABLE part_table_p202009 PARTITION OF part_table FOR VALUES FROM ('2020-09-01 00:00:00') TO ('2020-10-01 00:00:00');
|
||||
RESET search_path;
|
||||
DROP SCHEMA test_3970 CASCADE;
|
||||
NOTICE: drop cascades to table test_3970.part_table
|
|
@ -0,0 +1,30 @@
|
|||
-- test cases for #3970
|
||||
CREATE SCHEMA test_3970;
|
||||
SET search_path = test_3970;
|
||||
--1. create a partitioned table
|
||||
CREATE TABLE part_table (
|
||||
work_ymdt timestamp without time zone NOT NULL,
|
||||
seq bigint NOT NULL,
|
||||
my_seq bigint NOT NULL,
|
||||
work_memo character varying(150),
|
||||
CONSTRAINT work_memo_check CHECK ((octet_length((work_memo)::text) <= 150))
|
||||
)
|
||||
PARTITION BY RANGE (work_ymdt);
|
||||
--2. perform create_distributed_table
|
||||
SELECT create_distributed_table('part_table', 'seq');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
--3. add a partition
|
||||
CREATE TABLE part_table_p202008 PARTITION OF part_table FOR VALUES FROM ('2020-08-01 00:00:00') TO ('2020-09-01 00:00:00');
|
||||
--4. add a check constraint
|
||||
ALTER TABLE part_table ADD CONSTRAINT my_seq CHECK (my_seq > 0);
|
||||
--5. add a partition
|
||||
-- This command will fail as the child table has a wrong constraint name
|
||||
CREATE TABLE part_table_p202009 PARTITION OF part_table FOR VALUES FROM ('2020-09-01 00:00:00') TO ('2020-10-01 00:00:00');
|
||||
ERROR: child table is missing constraint "my_seq_xxxxxx"
|
||||
CONTEXT: while executing command on localhost:xxxxx
|
||||
-- Add another constraint with a long name that will get truncated with a hash
|
||||
ALTER TABLE part_table ADD CONSTRAINT ck_012345678901234567890123456789012345678901234567890123456789 CHECK (my_seq > 0);
|
|
@ -62,7 +62,7 @@ test: ensure_no_intermediate_data_leak
|
|||
# ----------
|
||||
# Tests for partitioning support
|
||||
# ----------
|
||||
test: multi_partitioning_utils multi_partitioning replicated_partitioned_table
|
||||
test: multi_partitioning_utils multi_partitioning partitioning_issue_3970 replicated_partitioned_table
|
||||
|
||||
# ----------
|
||||
# Tests for foreign data wrapper support
|
||||
|
|
|
@ -1177,6 +1177,14 @@ CREATE TABLE IF NOT EXISTS partition_of_other_table PARTITION OF partitioning_te
|
|||
ALTER TABLE another_table DETACH PARTITION partition_of_other_table;
|
||||
DROP TABLE another_table, partition_of_other_table;
|
||||
|
||||
-- test fix_pre_citus10_partitioned_table_constraint_names udf
|
||||
SELECT fix_pre_citus10_partitioned_table_constraint_names('partitioning_test');
|
||||
SELECT fix_pre_citus10_partitioned_table_constraint_names();
|
||||
|
||||
-- the following should fail
|
||||
SELECT fix_pre_citus10_partitioned_table_constraint_names('public.non_distributed_partitioned_table');
|
||||
SELECT fix_pre_citus10_partitioned_table_constraint_names('reference_table');
|
||||
|
||||
ALTER TABLE partitioning_test DETACH PARTITION partitioning_test_2008;
|
||||
ALTER TABLE partitioning_test DETACH PARTITION partitioning_test_2009;
|
||||
ALTER TABLE partitioning_test DETACH PARTITION partitioning_test_2010;
|
||||
|
|
|
@ -183,7 +183,7 @@ SELECT public.generate_alter_table_attach_partition_command('parent_table');
|
|||
SELECT public.generate_partition_information('partition_child_1_schema.child_1');
|
||||
SELECT public.print_partitions('partition_child_1_schema.child_1');
|
||||
|
||||
-- now pring the partitions
|
||||
-- now print the partitions
|
||||
SELECT public.print_partitions('parent_table');
|
||||
|
||||
SET search_path = 'public';
|
||||
|
|
|
@ -0,0 +1,15 @@
|
|||
-- test cases for #3970
|
||||
SET search_path = test_3970;
|
||||
|
||||
--5. add a partition
|
||||
-- This command will fail as the child table has a wrong constraint name
|
||||
CREATE TABLE part_table_p202009 PARTITION OF part_table FOR VALUES FROM ('2020-09-01 00:00:00') TO ('2020-10-01 00:00:00');
|
||||
|
||||
-- fix constraint names on partitioned table shards
|
||||
SELECT fix_pre_citus10_partitioned_table_constraint_names('part_table'::regclass);
|
||||
|
||||
--5. add a partition
|
||||
CREATE TABLE part_table_p202009 PARTITION OF part_table FOR VALUES FROM ('2020-09-01 00:00:00') TO ('2020-10-01 00:00:00');
|
||||
|
||||
RESET search_path;
|
||||
DROP SCHEMA test_3970 CASCADE;
|
|
@ -0,0 +1,29 @@
|
|||
-- test cases for #3970
|
||||
CREATE SCHEMA test_3970;
|
||||
SET search_path = test_3970;
|
||||
|
||||
--1. create a partitioned table
|
||||
CREATE TABLE part_table (
|
||||
work_ymdt timestamp without time zone NOT NULL,
|
||||
seq bigint NOT NULL,
|
||||
my_seq bigint NOT NULL,
|
||||
work_memo character varying(150),
|
||||
CONSTRAINT work_memo_check CHECK ((octet_length((work_memo)::text) <= 150))
|
||||
)
|
||||
PARTITION BY RANGE (work_ymdt);
|
||||
|
||||
--2. perform create_distributed_table
|
||||
SELECT create_distributed_table('part_table', 'seq');
|
||||
|
||||
--3. add a partition
|
||||
CREATE TABLE part_table_p202008 PARTITION OF part_table FOR VALUES FROM ('2020-08-01 00:00:00') TO ('2020-09-01 00:00:00');
|
||||
|
||||
--4. add a check constraint
|
||||
ALTER TABLE part_table ADD CONSTRAINT my_seq CHECK (my_seq > 0);
|
||||
|
||||
--5. add a partition
|
||||
-- This command will fail as the child table has a wrong constraint name
|
||||
CREATE TABLE part_table_p202009 PARTITION OF part_table FOR VALUES FROM ('2020-09-01 00:00:00') TO ('2020-10-01 00:00:00');
|
||||
|
||||
-- Add another constraint with a long name that will get truncated with a hash
|
||||
ALTER TABLE part_table ADD CONSTRAINT ck_012345678901234567890123456789012345678901234567890123456789 CHECK (my_seq > 0);
|
Loading…
Reference in New Issue