Merge pull request #4203 from citusdata/fix/sequence-drop

pull/4215/head
Marco Slot 2020-10-06 11:22:21 +02:00 committed by GitHub
commit bce11514b9
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
17 changed files with 313 additions and 196 deletions

View File

@ -22,6 +22,7 @@
#include "distributed/citus_ruleutils.h"
#include "distributed/colocation_utils.h"
#include "distributed/commands.h"
#include "distributed/commands/sequence.h"
#include "distributed/commands/utility_hook.h"
#include "distributed/listutils.h"
#include "distributed/metadata_sync.h"
@ -60,8 +61,6 @@ static char * GetDropTriggerCommand(Oid relationId, char *triggerName);
static List * GetExplicitIndexNameList(Oid relationId);
static void DropAndMoveDefaultSequenceOwnerships(Oid sourceRelationId,
Oid targetRelationId);
static void ExtractColumnsOwningSequences(Oid relationId, List **columnNameList,
List **ownedSequenceIdList);
static void DropDefaultColumnDefinition(Oid relationId, char *columnName);
static void TransferSequenceOwnership(Oid ownedSequenceId, Oid targetRelationId,
char *columnName);
@ -724,60 +723,6 @@ DropAndMoveDefaultSequenceOwnerships(Oid sourceRelationId, Oid targetRelationId)
}
/*
* ExtractColumnsOwningSequences finds each column of relation with relationId
* defaulting to an owned sequence. Then, appends the column name and id of the
* owned sequence -that the column defaults- to the lists passed as NIL initially.
*/
static void
ExtractColumnsOwningSequences(Oid relationId, List **columnNameList,
List **ownedSequenceIdList)
{
Assert(*columnNameList == NIL && *ownedSequenceIdList == NIL);
Relation relation = relation_open(relationId, AccessShareLock);
TupleDesc tupleDescriptor = RelationGetDescr(relation);
for (int attributeIndex = 0; attributeIndex < tupleDescriptor->natts;
attributeIndex++)
{
Form_pg_attribute attributeForm = TupleDescAttr(tupleDescriptor, attributeIndex);
if (attributeForm->attisdropped || !attributeForm->atthasdef)
{
/*
* If this column has already been dropped or it has no DEFAULT
* definition, skip it.
*/
continue;
}
char *columnName = NameStr(attributeForm->attname);
*columnNameList = lappend(*columnNameList, columnName);
List *columnOwnedSequences =
GetSequencesOwnedByColumn(relationId, attributeIndex + 1);
Oid ownedSequenceId = InvalidOid;
if (list_length(columnOwnedSequences) != 0)
{
/*
* A column might only own one sequence. We intentionally use
* GetSequencesOwnedByColumn macro and pick initial oid from the
* list instead of using getOwnedSequence. This is both because
* getOwnedSequence is removed in pg13 and is also because it
* errors out if column does not have any sequences.
*/
Assert(list_length(columnOwnedSequences) == 1);
ownedSequenceId = linitial_oid(columnOwnedSequences);
}
*ownedSequenceIdList = lappend_oid(*ownedSequenceIdList, ownedSequenceId);
}
relation_close(relation, NoLock);
}
/*
* DropDefaultColumnDefinition drops the DEFAULT definiton of the column with
* columnName of the relation with relationId via process utility.

View File

@ -15,6 +15,7 @@
#include "catalog/namespace.h"
#include "commands/defrem.h"
#include "distributed/commands.h"
#include "distributed/commands/sequence.h"
#include "distributed/listutils.h"
#include "distributed/metadata_cache.h"
#include "nodes/parsenodes.h"
@ -147,3 +148,57 @@ OptionsSpecifyOwnedBy(List *optionList, Oid *ownedByTableId)
return false;
}
/*
* ExtractColumnsOwningSequences finds each column of relation with relationId
* defaulting to an owned sequence. Then, appends the column name and id of the
* owned sequence -that the column defaults- to the lists passed as NIL initially.
*/
void
ExtractColumnsOwningSequences(Oid relationId, List **columnNameList,
List **ownedSequenceIdList)
{
Assert(*columnNameList == NIL && *ownedSequenceIdList == NIL);
Relation relation = relation_open(relationId, AccessShareLock);
TupleDesc tupleDescriptor = RelationGetDescr(relation);
for (int attributeIndex = 0; attributeIndex < tupleDescriptor->natts;
attributeIndex++)
{
Form_pg_attribute attributeForm = TupleDescAttr(tupleDescriptor, attributeIndex);
if (attributeForm->attisdropped || !attributeForm->atthasdef)
{
/*
* If this column has already been dropped or it has no DEFAULT
* definition, skip it.
*/
continue;
}
char *columnName = NameStr(attributeForm->attname);
*columnNameList = lappend(*columnNameList, columnName);
List *columnOwnedSequences =
GetSequencesOwnedByColumn(relationId, attributeIndex + 1);
Oid ownedSequenceId = InvalidOid;
if (list_length(columnOwnedSequences) != 0)
{
/*
* A column might only own one sequence. We intentionally use
* GetSequencesOwnedByColumn macro and pick initial oid from the
* list instead of using getOwnedSequence. This is both because
* getOwnedSequence is removed in pg13 and is also because it
* errors out if column does not have any sequences.
*/
Assert(list_length(columnOwnedSequences) == 1);
ownedSequenceId = linitial_oid(columnOwnedSequences);
}
*ownedSequenceIdList = lappend_oid(*ownedSequenceIdList, ownedSequenceId);
}
relation_close(relation, NoLock);
}

View File

@ -24,6 +24,7 @@
#include "access/xact.h"
#include "catalog/dependency.h"
#include "catalog/indexing.h"
#include "catalog/pg_depend.h"
#include "catalog/pg_foreign_server.h"
#include "catalog/pg_namespace.h"
#include "catalog/pg_type.h"
@ -60,11 +61,14 @@ static char * LocalGroupIdUpdateCommand(int32 groupId);
static void UpdateDistNodeBoolAttr(const char *nodeName, int32 nodePort,
int attrNum, bool value);
static List * SequenceDDLCommandsForTable(Oid relationId);
static List * SequenceDependencyCommandList(Oid relationId);
static char * TruncateTriggerCreateCommand(Oid relationId);
static char * SchemaOwnerName(Oid objectId);
static bool HasMetadataWorkers(void);
static List * DetachPartitionCommandList(void);
static bool SyncMetadataSnapshotToNode(WorkerNode *workerNode, bool raiseOnError);
static char * CreateSequenceDependencyCommand(Oid relationId, Oid sequenceId,
char *columnName);
static List * GenerateGrantOnSchemaQueriesFromAclItem(Oid schemaOid,
AclItem *aclItem);
static GrantStmt * GenerateGrantOnSchemaStmtForRights(Oid roleOid,
@ -75,6 +79,7 @@ static char * GenerateSetRoleQuery(Oid roleOid);
PG_FUNCTION_INFO_V1(start_metadata_sync_to_node);
PG_FUNCTION_INFO_V1(stop_metadata_sync_to_node);
PG_FUNCTION_INFO_V1(worker_record_sequence_dependency);
/*
@ -395,6 +400,7 @@ MetadataCreateCommands(void)
List *workerSequenceDDLCommands = SequenceDDLCommandsForTable(relationId);
List *ddlCommandList = GetTableDDLEvents(relationId, includeSequenceDefaults);
char *tableOwnerResetCommand = TableOwnerResetCommand(relationId);
List *sequenceDependencyCommandList = SequenceDependencyCommandList(relationId);
/*
* Tables might have dependencies on different objects, since we create shards for
@ -410,6 +416,8 @@ MetadataCreateCommands(void)
ddlCommandList);
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
tableOwnerResetCommand);
metadataSnapshotCommandList = list_concat(metadataSnapshotCommandList,
sequenceDependencyCommandList);
}
/* construct the foreign key constraints after all tables are created */
@ -508,6 +516,10 @@ GetDistributedTableDDLEvents(Oid relationId)
/* command to reset the table owner */
char *tableOwnerResetCommand = TableOwnerResetCommand(relationId);
commandList = lappend(commandList, tableOwnerResetCommand);
/* command to associate sequences with table */
List *sequenceDependencyCommandList = SequenceDependencyCommandList(relationId);
commandList = list_concat(commandList, sequenceDependencyCommandList);
}
/* command to insert pg_dist_partition entry */
@ -1101,6 +1113,123 @@ SequenceDDLCommandsForTable(Oid relationId)
}
/*
* SequenceDependencyCommandList generates commands to record the dependency
* of sequences on tables on the worker. This dependency does not exist by
* default since the sequences and table are created separately, but it is
* necessary to ensure that the sequence is dropped when the table is
* dropped.
*/
static List *
SequenceDependencyCommandList(Oid relationId)
{
List *sequenceCommandList = NIL;
List *columnNameList = NIL;
List *sequenceIdList = NIL;
ExtractColumnsOwningSequences(relationId, &columnNameList, &sequenceIdList);
ListCell *columnNameCell = NULL;
ListCell *sequenceIdCell = NULL;
forboth(columnNameCell, columnNameList, sequenceIdCell, sequenceIdList)
{
char *columnName = lfirst(columnNameCell);
Oid sequenceId = lfirst_oid(sequenceIdCell);
if (!OidIsValid(sequenceId))
{
/*
* ExtractColumnsOwningSequences returns entries for all columns,
* but with 0 sequence ID unless there is default nextval(..).
*/
continue;
}
char *sequenceDependencyCommand =
CreateSequenceDependencyCommand(relationId, sequenceId, columnName);
sequenceCommandList = lappend(sequenceCommandList,
sequenceDependencyCommand);
}
return sequenceCommandList;
}
/*
* CreateSequenceDependencyCommand generates a query string for calling
* worker_record_sequence_dependency on the worker to recreate a sequence->table
* dependency.
*/
static char *
CreateSequenceDependencyCommand(Oid relationId, Oid sequenceId, char *columnName)
{
char *relationName = generate_qualified_relation_name(relationId);
char *sequenceName = generate_qualified_relation_name(sequenceId);
StringInfo sequenceDependencyCommand = makeStringInfo();
appendStringInfo(sequenceDependencyCommand,
"SELECT pg_catalog.worker_record_sequence_dependency"
"(%s::regclass,%s::regclass,%s)",
quote_literal_cstr(sequenceName),
quote_literal_cstr(relationName),
quote_literal_cstr(columnName));
return sequenceDependencyCommand->data;
}
/*
* worker_record_sequence_dependency records the fact that the sequence depends on
* the table in pg_depend, such that it will be automatically dropped.
*/
Datum
worker_record_sequence_dependency(PG_FUNCTION_ARGS)
{
Oid sequenceOid = PG_GETARG_OID(0);
Oid relationOid = PG_GETARG_OID(1);
Name columnName = PG_GETARG_NAME(2);
const char *columnNameStr = NameStr(*columnName);
/* lookup column definition */
HeapTuple columnTuple = SearchSysCacheAttName(relationOid, columnNameStr);
if (!HeapTupleIsValid(columnTuple))
{
ereport(ERROR, (errcode(ERRCODE_UNDEFINED_COLUMN),
errmsg("column \"%s\" does not exist",
columnNameStr)));
}
Form_pg_attribute columnForm = (Form_pg_attribute) GETSTRUCT(columnTuple);
if (columnForm->attnum <= 0)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot create dependency on system column \"%s\"",
columnNameStr)));
}
ObjectAddress sequenceAddr = {
.classId = RelationRelationId,
.objectId = sequenceOid,
.objectSubId = 0
};
ObjectAddress relationAddr = {
.classId = RelationRelationId,
.objectId = relationOid,
.objectSubId = columnForm->attnum
};
/* dependency from sequence to table */
recordDependencyOn(&sequenceAddr, &relationAddr, DEPENDENCY_AUTO);
ReleaseSysCache(columnTuple);
PG_RETURN_VOID();
}
/*
* CreateSchemaDDLCommand returns a "CREATE SCHEMA..." SQL string for creating the given
* schema if not exists and with proper authorization.

View File

@ -252,86 +252,15 @@ master_drop_all_shards(PG_FUNCTION_ARGS)
/*
* master_drop_sequences attempts to drop a list of sequences on worker nodes.
* The "IF EXISTS" clause is used to permit dropping sequences even if they may not
* exist. If the commands fail on the workers, the operation is rolled back.
* If ddl propagation (citus.enable_ddl_propagation) is set to off, then the function
* returns without doing anything.
* master_drop_sequences was previously used to drop sequences on workers
* when using metadata syncing.
*
* It may still be called when dropping objects during CREATE EXTENSION,
* hence the function remains in place.
*/
Datum
master_drop_sequences(PG_FUNCTION_ARGS)
{
ArrayType *sequenceNamesArray = PG_GETARG_ARRAYTYPE_P(0);
Datum sequenceNameDatum = 0;
bool isNull = false;
StringInfo dropSeqCommand = makeStringInfo();
if (!CitusHasBeenLoaded())
{
/* ignore calls during CREATE EXTENSION citus */
PG_RETURN_VOID();
}
CheckCitusVersion(ERROR);
/*
* Do nothing if DDL propagation is switched off or we're not on
* the coordinator. Here we prefer to not error out on the workers
* because this function is called on every dropped sequence and
* we don't want to mess up the sequences that are not associated
* with distributed tables.
*/
if (!EnableDDLPropagation || !IsCoordinator())
{
PG_RETURN_VOID();
}
/* iterate over sequence names to build single command to DROP them all */
ArrayIterator sequenceIterator = array_create_iterator(sequenceNamesArray, 0, NULL);
while (array_iterate(sequenceIterator, &sequenceNameDatum, &isNull))
{
if (isNull)
{
ereport(ERROR, (errmsg("unexpected NULL sequence name"),
errcode(ERRCODE_INVALID_PARAMETER_VALUE)));
}
text *sequenceNameText = DatumGetTextP(sequenceNameDatum);
Oid sequenceOid = ResolveRelationId(sequenceNameText, true);
if (OidIsValid(sequenceOid))
{
/*
* This case (e.g., OID is valid) could only happen when a user manually calls
* the UDF. So, ensure that the user has right to drop the sequence.
*
* In case the UDF is called via the DROP trigger, the OID wouldn't be valid since
* the trigger is called after DROP happens.
*/
EnsureSequenceOwner(sequenceOid);
}
/* append command portion if we haven't added any sequence names yet */
if (dropSeqCommand->len == 0)
{
appendStringInfoString(dropSeqCommand, "DROP SEQUENCE IF EXISTS");
}
else
{
/* otherwise, add a comma to separate subsequent sequence names */
appendStringInfoChar(dropSeqCommand, ',');
}
appendStringInfo(dropSeqCommand, " %s", TextDatumGetCString(sequenceNameText));
}
if (dropSeqCommand->len != 0)
{
appendStringInfoString(dropSeqCommand, " CASCADE");
SendCommandToWorkersWithMetadata(DISABLE_DDL_PROPAGATION);
SendCommandToWorkersWithMetadata(dropSeqCommand->data);
}
PG_RETURN_VOID();
}

View File

@ -3,6 +3,8 @@
-- bump version to 9.5-1
#include "udfs/undistribute_table/9.5-1.sql"
#include "udfs/create_citus_local_table/9.5-1.sql"
#include "udfs/citus_drop_trigger/9.5-1.sql"
#include "udfs/worker_record_sequence_dependency/9.5-1.sql"
SET search_path = 'pg_catalog';
@ -14,5 +16,6 @@ DROP FUNCTION worker_execute_sql_task(bigint, integer, text, bool);
DROP TRIGGER dist_authinfo_task_tracker_cache_invalidate ON pg_catalog.pg_dist_authinfo;
DROP TRIGGER dist_poolinfo_task_tracker_cache_invalidate ON pg_catalog.pg_dist_poolinfo;
DROP FUNCTION task_tracker_conninfo_cache_invalidate();
DROP FUNCTION master_drop_sequences(text[]);
RESET search_path;

View File

@ -2,6 +2,8 @@
SET search_path = 'pg_catalog';
#include "../udfs/citus_drop_trigger/9.0-1.sql"
-- Check if user has any citus local tables.
-- If not, DROP create_citus_local_table UDF and continue safely.
-- Otherwise, raise an exception to stop the downgrade process.
@ -24,6 +26,8 @@ BEGIN
END;
$$ LANGUAGE plpgsql;
DROP FUNCTION worker_record_sequence_dependency(regclass, regclass, name);
-- task_tracker_* functions
CREATE FUNCTION task_tracker_assign_task(bigint, integer, text)
@ -78,6 +82,13 @@ CREATE TRIGGER dist_authinfo_task_tracker_cache_invalidate
ON pg_catalog.pg_dist_authinfo
FOR EACH STATEMENT EXECUTE PROCEDURE task_tracker_conninfo_cache_invalidate();
CREATE FUNCTION master_drop_sequences(sequence_names text[])
RETURNS void
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$master_drop_sequences$$;
COMMENT ON FUNCTION master_drop_sequences(text[])
IS 'drop specified sequences from the cluster';
RESET search_path;
DROP FUNCTION pg_catalog.undistribute_table(table_name regclass);

View File

@ -0,0 +1,33 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_drop_trigger()
RETURNS event_trigger
LANGUAGE plpgsql
SET search_path = pg_catalog
AS $cdbdt$
DECLARE
v_obj record;
sequence_names text[] := '{}';
table_colocation_id integer;
propagate_drop boolean := false;
BEGIN
FOR v_obj IN SELECT * FROM pg_event_trigger_dropped_objects()
WHERE object_type IN ('table', 'foreign table')
LOOP
-- first drop the table and metadata on the workers
-- then drop all the shards on the workers
-- finally remove the pg_dist_partition entry on the coordinator
PERFORM master_remove_distributed_table_metadata_from_workers(v_obj.objid, v_obj.schema_name, v_obj.object_name);
PERFORM master_drop_all_shards(v_obj.objid, v_obj.schema_name, v_obj.object_name);
PERFORM master_remove_partition_metadata(v_obj.objid, v_obj.schema_name, v_obj.object_name);
END LOOP;
-- remove entries from citus.pg_dist_object for all dropped root (objsubid = 0) objects
FOR v_obj IN SELECT * FROM pg_event_trigger_dropped_objects()
LOOP
PERFORM master_unmark_object_distributed(v_obj.classid, v_obj.objid, v_obj.objsubid);
END LOOP;
END;
$cdbdt$;
COMMENT ON FUNCTION pg_catalog.citus_drop_trigger()
IS 'perform checks and actions at the end of DROP actions';

View File

@ -9,11 +9,6 @@ DECLARE
table_colocation_id integer;
propagate_drop boolean := false;
BEGIN
-- collect set of dropped sequences to drop on workers later
SELECT array_agg(object_identity) INTO sequence_names
FROM pg_event_trigger_dropped_objects()
WHERE object_type = 'sequence';
FOR v_obj IN SELECT * FROM pg_event_trigger_dropped_objects()
WHERE object_type IN ('table', 'foreign table')
LOOP
@ -25,10 +20,6 @@ BEGIN
PERFORM master_remove_partition_metadata(v_obj.objid, v_obj.schema_name, v_obj.object_name);
END LOOP;
IF cardinality(sequence_names) > 0 THEN
PERFORM master_drop_sequences(sequence_names);
END IF;
-- remove entries from citus.pg_dist_object for all dropped root (objsubid = 0) objects
FOR v_obj IN SELECT * FROM pg_event_trigger_dropped_objects()
LOOP

View File

@ -0,0 +1,9 @@
CREATE FUNCTION pg_catalog.worker_record_sequence_dependency(seq_name regclass, table_name regclass, column_name name)
RETURNS VOID
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', 'worker_record_sequence_dependency';
COMMENT ON FUNCTION pg_catalog.worker_record_sequence_dependency(regclass,regclass,name)
IS 'record the fact that the sequence depends on the table in pg_depend';
REVOKE ALL ON FUNCTION pg_catalog.worker_record_sequence_dependency(regclass,regclass,name)
FROM PUBLIC;

View File

@ -0,0 +1,9 @@
CREATE FUNCTION pg_catalog.worker_record_sequence_dependency(seq_name regclass, table_name regclass, column_name name)
RETURNS VOID
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', 'worker_record_sequence_dependency';
COMMENT ON FUNCTION pg_catalog.worker_record_sequence_dependency(regclass,regclass,name)
IS 'record the fact that the sequence depends on the table in pg_depend';
REVOKE ALL ON FUNCTION pg_catalog.worker_record_sequence_dependency(regclass,regclass,name)
FROM PUBLIC;

View File

@ -0,0 +1,20 @@
/*-------------------------------------------------------------------------
*
* sequence.h
* Functions for dealing with sequences
*
* Copyright (c) Citus Data, Inc.
*-------------------------------------------------------------------------
*/
#ifndef CITUS_SEQUENCE_H
#define CITUS_SEQUENCE_H
#include "nodes/pg_list.h"
extern void ExtractColumnsOwningSequences(Oid relationId, List **columnNameList,
List **ownedSequenceIdList);
#endif /* CITUS_SEQUENCE_H */

View File

@ -17,6 +17,7 @@
#include "commands/explain.h"
#include "catalog/namespace.h"
#include "distributed/citus_ruleutils.h"
#include "distributed/citus_safe_lib.h"
#include "nodes/parsenodes.h"
#include "parser/parse_func.h"

View File

@ -439,8 +439,9 @@ SELECT * FROM print_extension_changes();
-- Snapshot of state at 9.5-1
ALTER EXTENSION citus UPDATE TO '9.5-1';
SELECT * FROM print_extension_changes();
previous_object | current_object
previous_object | current_object
---------------------------------------------------------------------
function master_drop_sequences(text[]) |
function task_tracker_assign_task(bigint,integer,text) |
function task_tracker_cleanup_job(bigint) |
function task_tracker_conninfo_cache_invalidate() |
@ -449,7 +450,8 @@ SELECT * FROM print_extension_changes();
function worker_merge_files_and_run_query(bigint,integer,text,text) |
| function create_citus_local_table(regclass)
| function undistribute_table(regclass)
(8 rows)
| function worker_record_sequence_dependency(regclass,regclass,name)
(10 rows)
DROP TABLE prev_objects, extension_diff;
-- show running version

View File

@ -63,11 +63,12 @@ SELECT unnest(master_metadata_snapshot()) order by 1;
INSERT INTO pg_dist_partition (logicalrelid, partmethod, partkey, colocationid, repmodel) VALUES ('public.mx_test_table'::regclass, 'h', column_name_to_column('public.mx_test_table','col_1'), 0, 's')
INSERT INTO pg_dist_placement (shardid, shardstate, shardlength, groupid, placementid) VALUES (1310000, 1, 0, 1, 100000),(1310001, 1, 0, 2, 100001),(1310002, 1, 0, 1, 100002),(1310003, 1, 0, 2, 100003),(1310004, 1, 0, 1, 100004),(1310005, 1, 0, 2, 100005),(1310006, 1, 0, 1, 100006),(1310007, 1, 0, 2, 100007)
INSERT INTO pg_dist_shard (logicalrelid, shardid, shardstorage, shardminvalue, shardmaxvalue) VALUES ('public.mx_test_table'::regclass, 1310000, 't', '-2147483648', '-1610612737'),('public.mx_test_table'::regclass, 1310001, 't', '-1610612736', '-1073741825'),('public.mx_test_table'::regclass, 1310002, 't', '-1073741824', '-536870913'),('public.mx_test_table'::regclass, 1310003, 't', '-536870912', '-1'),('public.mx_test_table'::regclass, 1310004, 't', '0', '536870911'),('public.mx_test_table'::regclass, 1310005, 't', '536870912', '1073741823'),('public.mx_test_table'::regclass, 1310006, 't', '1073741824', '1610612735'),('public.mx_test_table'::regclass, 1310007, 't', '1610612736', '2147483647')
SELECT pg_catalog.worker_record_sequence_dependency('public.mx_test_table_col_3_seq'::regclass,'public.mx_test_table'::regclass,'col_3')
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.mx_test_table_col_3_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
SELECT worker_create_truncate_trigger('public.mx_test_table')
SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition
TRUNCATE pg_dist_node CASCADE
(13 rows)
(14 rows)
-- Show that CREATE INDEX commands are included in the metadata snapshot
CREATE INDEX mx_index ON mx_test_table(col_2);
@ -84,11 +85,12 @@ SELECT unnest(master_metadata_snapshot()) order by 1;
INSERT INTO pg_dist_partition (logicalrelid, partmethod, partkey, colocationid, repmodel) VALUES ('public.mx_test_table'::regclass, 'h', column_name_to_column('public.mx_test_table','col_1'), 0, 's')
INSERT INTO pg_dist_placement (shardid, shardstate, shardlength, groupid, placementid) VALUES (1310000, 1, 0, 1, 100000),(1310001, 1, 0, 2, 100001),(1310002, 1, 0, 1, 100002),(1310003, 1, 0, 2, 100003),(1310004, 1, 0, 1, 100004),(1310005, 1, 0, 2, 100005),(1310006, 1, 0, 1, 100006),(1310007, 1, 0, 2, 100007)
INSERT INTO pg_dist_shard (logicalrelid, shardid, shardstorage, shardminvalue, shardmaxvalue) VALUES ('public.mx_test_table'::regclass, 1310000, 't', '-2147483648', '-1610612737'),('public.mx_test_table'::regclass, 1310001, 't', '-1610612736', '-1073741825'),('public.mx_test_table'::regclass, 1310002, 't', '-1073741824', '-536870913'),('public.mx_test_table'::regclass, 1310003, 't', '-536870912', '-1'),('public.mx_test_table'::regclass, 1310004, 't', '0', '536870911'),('public.mx_test_table'::regclass, 1310005, 't', '536870912', '1073741823'),('public.mx_test_table'::regclass, 1310006, 't', '1073741824', '1610612735'),('public.mx_test_table'::regclass, 1310007, 't', '1610612736', '2147483647')
SELECT pg_catalog.worker_record_sequence_dependency('public.mx_test_table_col_3_seq'::regclass,'public.mx_test_table'::regclass,'col_3')
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS public.mx_test_table_col_3_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
SELECT worker_create_truncate_trigger('public.mx_test_table')
SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition
TRUNCATE pg_dist_node CASCADE
(14 rows)
(15 rows)
-- Show that schema changes are included in the metadata snapshot
CREATE SCHEMA mx_testing_schema;
@ -106,11 +108,12 @@ SELECT unnest(master_metadata_snapshot()) order by 1;
INSERT INTO pg_dist_partition (logicalrelid, partmethod, partkey, colocationid, repmodel) VALUES ('mx_testing_schema.mx_test_table'::regclass, 'h', column_name_to_column('mx_testing_schema.mx_test_table','col_1'), 0, 's')
INSERT INTO pg_dist_placement (shardid, shardstate, shardlength, groupid, placementid) VALUES (1310000, 1, 0, 1, 100000),(1310001, 1, 0, 2, 100001),(1310002, 1, 0, 1, 100002),(1310003, 1, 0, 2, 100003),(1310004, 1, 0, 1, 100004),(1310005, 1, 0, 2, 100005),(1310006, 1, 0, 1, 100006),(1310007, 1, 0, 2, 100007)
INSERT INTO pg_dist_shard (logicalrelid, shardid, shardstorage, shardminvalue, shardmaxvalue) VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't', '-2147483648', '-1610612737'),('mx_testing_schema.mx_test_table'::regclass, 1310001, 't', '-1610612736', '-1073741825'),('mx_testing_schema.mx_test_table'::regclass, 1310002, 't', '-1073741824', '-536870913'),('mx_testing_schema.mx_test_table'::regclass, 1310003, 't', '-536870912', '-1'),('mx_testing_schema.mx_test_table'::regclass, 1310004, 't', '0', '536870911'),('mx_testing_schema.mx_test_table'::regclass, 1310005, 't', '536870912', '1073741823'),('mx_testing_schema.mx_test_table'::regclass, 1310006, 't', '1073741824', '1610612735'),('mx_testing_schema.mx_test_table'::regclass, 1310007, 't', '1610612736', '2147483647')
SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3')
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
SELECT worker_create_truncate_trigger('mx_testing_schema.mx_test_table')
SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition
TRUNCATE pg_dist_node CASCADE
(14 rows)
(15 rows)
-- Show that append distributed tables are not included in the metadata snapshot
CREATE TABLE non_mx_test_table (col_1 int, col_2 text);
@ -134,11 +137,12 @@ SELECT unnest(master_metadata_snapshot()) order by 1;
INSERT INTO pg_dist_partition (logicalrelid, partmethod, partkey, colocationid, repmodel) VALUES ('mx_testing_schema.mx_test_table'::regclass, 'h', column_name_to_column('mx_testing_schema.mx_test_table','col_1'), 0, 's')
INSERT INTO pg_dist_placement (shardid, shardstate, shardlength, groupid, placementid) VALUES (1310000, 1, 0, 1, 100000),(1310001, 1, 0, 2, 100001),(1310002, 1, 0, 1, 100002),(1310003, 1, 0, 2, 100003),(1310004, 1, 0, 1, 100004),(1310005, 1, 0, 2, 100005),(1310006, 1, 0, 1, 100006),(1310007, 1, 0, 2, 100007)
INSERT INTO pg_dist_shard (logicalrelid, shardid, shardstorage, shardminvalue, shardmaxvalue) VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't', '-2147483648', '-1610612737'),('mx_testing_schema.mx_test_table'::regclass, 1310001, 't', '-1610612736', '-1073741825'),('mx_testing_schema.mx_test_table'::regclass, 1310002, 't', '-1073741824', '-536870913'),('mx_testing_schema.mx_test_table'::regclass, 1310003, 't', '-536870912', '-1'),('mx_testing_schema.mx_test_table'::regclass, 1310004, 't', '0', '536870911'),('mx_testing_schema.mx_test_table'::regclass, 1310005, 't', '536870912', '1073741823'),('mx_testing_schema.mx_test_table'::regclass, 1310006, 't', '1073741824', '1610612735'),('mx_testing_schema.mx_test_table'::regclass, 1310007, 't', '1610612736', '2147483647')
SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3')
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
SELECT worker_create_truncate_trigger('mx_testing_schema.mx_test_table')
SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition
TRUNCATE pg_dist_node CASCADE
(14 rows)
(15 rows)
-- Show that range distributed tables are not included in the metadata snapshot
UPDATE pg_dist_partition SET partmethod='r' WHERE logicalrelid='non_mx_test_table'::regclass;
@ -155,11 +159,12 @@ SELECT unnest(master_metadata_snapshot()) order by 1;
INSERT INTO pg_dist_partition (logicalrelid, partmethod, partkey, colocationid, repmodel) VALUES ('mx_testing_schema.mx_test_table'::regclass, 'h', column_name_to_column('mx_testing_schema.mx_test_table','col_1'), 0, 's')
INSERT INTO pg_dist_placement (shardid, shardstate, shardlength, groupid, placementid) VALUES (1310000, 1, 0, 1, 100000),(1310001, 1, 0, 2, 100001),(1310002, 1, 0, 1, 100002),(1310003, 1, 0, 2, 100003),(1310004, 1, 0, 1, 100004),(1310005, 1, 0, 2, 100005),(1310006, 1, 0, 1, 100006),(1310007, 1, 0, 2, 100007)
INSERT INTO pg_dist_shard (logicalrelid, shardid, shardstorage, shardminvalue, shardmaxvalue) VALUES ('mx_testing_schema.mx_test_table'::regclass, 1310000, 't', '-2147483648', '-1610612737'),('mx_testing_schema.mx_test_table'::regclass, 1310001, 't', '-1610612736', '-1073741825'),('mx_testing_schema.mx_test_table'::regclass, 1310002, 't', '-1073741824', '-536870913'),('mx_testing_schema.mx_test_table'::regclass, 1310003, 't', '-536870912', '-1'),('mx_testing_schema.mx_test_table'::regclass, 1310004, 't', '0', '536870911'),('mx_testing_schema.mx_test_table'::regclass, 1310005, 't', '536870912', '1073741823'),('mx_testing_schema.mx_test_table'::regclass, 1310006, 't', '1073741824', '1610612735'),('mx_testing_schema.mx_test_table'::regclass, 1310007, 't', '1610612736', '2147483647')
SELECT pg_catalog.worker_record_sequence_dependency('mx_testing_schema.mx_test_table_col_3_seq'::regclass,'mx_testing_schema.mx_test_table'::regclass,'col_3')
SELECT worker_apply_sequence_command ('CREATE SEQUENCE IF NOT EXISTS mx_testing_schema.mx_test_table_col_3_seq INCREMENT BY 1 MINVALUE 1 MAXVALUE 9223372036854775807 START WITH 1 NO CYCLE','bigint')
SELECT worker_create_truncate_trigger('mx_testing_schema.mx_test_table')
SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition
TRUNCATE pg_dist_node CASCADE
(14 rows)
(15 rows)
-- Test start_metadata_sync_to_node UDF
-- Ensure that hasmetadata=false for all nodes
@ -336,7 +341,7 @@ SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
-- Check that foreign key metadata exists on the worker
\c - - - :worker_1_port
SELECT "Constraint", "Definition" FROM table_fkeys WHERE relid='mx_testing_schema_2.fk_test_2'::regclass;
Constraint | Definition
Constraint | Definition
---------------------------------------------------------------------
fk_test_2_col1_fkey | FOREIGN KEY (col1, col2) REFERENCES mx_testing_schema.fk_test_1(col1, col3)
(1 row)
@ -1014,6 +1019,21 @@ SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='mx_table_with_
public | mx_table_with_sequence_c_seq | sequence | postgres
(1 row)
-- check that pg_depend records exist on the worker
SELECT refobjsubid FROM pg_depend
WHERE objid = 'mx_table_with_sequence_b_seq'::regclass AND refobjid = 'mx_table_with_sequence'::regclass;
refobjsubid
---------------------------------------------------------------------
2
(1 row)
SELECT refobjsubid FROM pg_depend
WHERE objid = 'mx_table_with_sequence_c_seq'::regclass AND refobjid = 'mx_table_with_sequence'::regclass;
refobjsubid
---------------------------------------------------------------------
3
(1 row)
-- Check that the sequences on the worker have their own space
SELECT nextval('mx_table_with_sequence_b_seq');
nextval

View File

@ -334,42 +334,9 @@ SELECT raise_failed_aclcheck($$
$$);
ERROR: must be owner of the object
CONTEXT: PL/pgSQL function raise_failed_aclcheck(text) line 6 at RAISE
SELECT raise_failed_aclcheck($$
SELECT master_drop_sequences(ARRAY['public.distributed_mx_table_some_val_seq']);
$$);
ERROR: must be owner of the object
CONTEXT: PL/pgSQL function raise_failed_aclcheck(text) line 6 at RAISE
SELECT raise_failed_aclcheck($$
SELECT master_drop_sequences(ARRAY['distributed_mx_table_some_val_seq']);
$$);
ERROR: must be owner of the object
CONTEXT: PL/pgSQL function raise_failed_aclcheck(text) line 6 at RAISE
SELECT master_drop_sequences(ARRAY['non_existing_schema.distributed_mx_table_some_val_seq']);
master_drop_sequences
---------------------------------------------------------------------
(1 row)
SELECT master_drop_sequences(ARRAY['']);
ERROR: invalid name syntax
SELECT master_drop_sequences(ARRAY['public.']);
ERROR: invalid name syntax
SELECT master_drop_sequences(ARRAY['public.distributed_mx_table_some_val_seq_not_existing']);
master_drop_sequences
---------------------------------------------------------------------
(1 row)
-- make sure that we can drop unrelated tables/sequences
CREATE TABLE unrelated_table(key serial);
DROP TABLE unrelated_table;
-- doesn't error out but it has no effect, so no need to error out
SELECT master_drop_sequences(NULL);
master_drop_sequences
---------------------------------------------------------------------
(1 row)
\c - postgres - :master_port
-- finally make sure that the sequence remains
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='distributed_mx_table'::regclass;

View File

@ -464,6 +464,13 @@ SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='mx_table_with_
\ds mx_table_with_sequence_b_seq
\ds mx_table_with_sequence_c_seq
-- check that pg_depend records exist on the worker
SELECT refobjsubid FROM pg_depend
WHERE objid = 'mx_table_with_sequence_b_seq'::regclass AND refobjid = 'mx_table_with_sequence'::regclass;
SELECT refobjsubid FROM pg_depend
WHERE objid = 'mx_table_with_sequence_c_seq'::regclass AND refobjid = 'mx_table_with_sequence'::regclass;
-- Check that the sequences on the worker have their own space
SELECT nextval('mx_table_with_sequence_b_seq');
SELECT nextval('mx_table_with_sequence_c_seq');

View File

@ -210,25 +210,11 @@ $$);
SELECT raise_failed_aclcheck($$
SELECT master_remove_partition_metadata('distributed_mx_table'::regclass, 'public', 'distributed_mx_table');
$$);
SELECT raise_failed_aclcheck($$
SELECT master_drop_sequences(ARRAY['public.distributed_mx_table_some_val_seq']);
$$);
SELECT raise_failed_aclcheck($$
SELECT master_drop_sequences(ARRAY['distributed_mx_table_some_val_seq']);
$$);
SELECT master_drop_sequences(ARRAY['non_existing_schema.distributed_mx_table_some_val_seq']);
SELECT master_drop_sequences(ARRAY['']);
SELECT master_drop_sequences(ARRAY['public.']);
SELECT master_drop_sequences(ARRAY['public.distributed_mx_table_some_val_seq_not_existing']);
-- make sure that we can drop unrelated tables/sequences
CREATE TABLE unrelated_table(key serial);
DROP TABLE unrelated_table;
-- doesn't error out but it has no effect, so no need to error out
SELECT master_drop_sequences(NULL);
\c - postgres - :master_port
-- finally make sure that the sequence remains