diff --git a/src/backend/distributed/metadata/metadata_sync.c b/src/backend/distributed/metadata/metadata_sync.c index 7549fba7d..1d90b3f1b 100644 --- a/src/backend/distributed/metadata/metadata_sync.c +++ b/src/backend/distributed/metadata/metadata_sync.c @@ -86,7 +86,6 @@ char *EnableManualMetadataChangesForUser = ""; static List * GetDistributedTableMetadataEvents(Oid relationId); static void EnsureObjectMetadataIsSane(int distributionArgumentIndex, int colocationId); -static char * LocalGroupIdUpdateCommand(int32 groupId); static char * SchemaOwnerName(Oid objectId); static bool HasMetadataWorkers(void); static bool ShouldSyncTableMetadataInternal(bool hashDistributed, @@ -1188,7 +1187,7 @@ PlacementUpsertCommand(uint64 shardId, uint64 placementId, int shardState, * LocalGroupIdUpdateCommand creates the SQL command required to set the local group id * of a worker and returns the command in a string. */ -static char * +char * LocalGroupIdUpdateCommand(int32 groupId) { StringInfo updateCommand = makeStringInfo(); diff --git a/src/backend/distributed/metadata/node_metadata.c b/src/backend/distributed/metadata/node_metadata.c index 0a30fdd82..89d081827 100644 --- a/src/backend/distributed/metadata/node_metadata.c +++ b/src/backend/distributed/metadata/node_metadata.c @@ -110,9 +110,9 @@ static void InsertNodeRow(int nodeid, char *nodename, int32 nodeport, NodeMetada *nodeMetadata); static void DeleteNodeRow(char *nodename, int32 nodeport); static void SetUpObjectMetadata(WorkerNode *workerNode); -static void AdjustSequenceLimits(WorkerNode *workerNode); static void ClearDistributedObjectsFromNode(WorkerNode *workerNode); static void ClearDistributedTablesFromNode(WorkerNode *workerNode); +static void UpdatePgDistLocalGroupOnNode(WorkerNode *workerNode); static void SetUpDistributedTableWithDependencies(WorkerNode *workerNode); static void SetUpMultipleDistributedTableIntegrations(WorkerNode *workerNode); static WorkerNode * TupleToWorkerNode(TupleDesc tupleDescriptor, HeapTuple heapTuple); @@ -247,12 +247,6 @@ citus_add_node(PG_FUNCTION_ARGS) bool nodeAlreadyExists = false; nodeMetadata.groupId = PG_GETARG_INT32(2); - if (!EnableDependencyCreation) - { - ereport(ERROR, (errmsg("citus.enable_object_propagation must be on to " - "add node"))); - } - /* * During tests this function is called before nodeRole and nodeCluster have been * created. @@ -734,49 +728,6 @@ SetUpObjectMetadata(WorkerNode *workerNode) } -/* - * AdjustSequenceLimits adjusts the limits of sequences on the given node - */ -static void -AdjustSequenceLimits(WorkerNode *workerNode) -{ - List *distributedTableList = CitusTableList(); - List *propagatedTableList = NIL; - List *metadataSnapshotCommandList = NIL; - - /* create the list of tables whose metadata will be created */ - CitusTableCacheEntry *cacheEntry = NULL; - foreach_ptr(cacheEntry, distributedTableList) - { - if (ShouldSyncTableMetadata(cacheEntry->relationId)) - { - propagatedTableList = lappend(propagatedTableList, cacheEntry); - } - } - - /* after all tables are created, create the metadata */ - foreach_ptr(cacheEntry, propagatedTableList) - { - Oid relationId = cacheEntry->relationId; - - List *workerSequenceDDLCommands = SequenceDDLCommandsForTable(relationId); - metadataSnapshotCommandList = list_concat(metadataSnapshotCommandList, - workerSequenceDDLCommands); - } - - metadataSnapshotCommandList = lcons(DISABLE_DDL_PROPAGATION, - metadataSnapshotCommandList); - metadataSnapshotCommandList = lappend(metadataSnapshotCommandList, - ENABLE_DDL_PROPAGATION); - - char *currentUser = CurrentUserName(); - SendMetadataCommandListToWorkerInCoordinatedTransaction(workerNode->workerName, - workerNode->workerPort, - currentUser, - metadataSnapshotCommandList); -} - - /* * DistributedObjectMetadataSyncCommandList returns the necessary commands to create * pg_dist_object entries on the new node. @@ -866,6 +817,24 @@ DistributedObjectMetadataSyncCommandList(void) } +/* + * UpdatePgDistLocalGroupOnNode updates the pg_dist_local_group on the given node + */ +static void +UpdatePgDistLocalGroupOnNode(WorkerNode *workerNode) +{ + /* generate and add the local group id's update query */ + char *localGroupIdUpdateCommand = LocalGroupIdUpdateCommand(workerNode->groupId); + + List *localGroupIdUpdateCommandList = list_make1(localGroupIdUpdateCommand); + + SendMetadataCommandListToWorkerInCoordinatedTransaction(workerNode->workerName, + workerNode->workerPort, + CitusExtensionOwnerName(), + localGroupIdUpdateCommandList); +} + + /* * ClearDistributedTablesFromNode clear (shell) distributed tables from the given node. */ @@ -951,6 +920,7 @@ SetUpDistributedTableWithDependencies(WorkerNode *newWorkerNode) { ClearDistributedTablesFromNode(newWorkerNode); PropagateNodeWideObjects(newWorkerNode); + UpdatePgDistLocalGroupOnNode(newWorkerNode); ReplicateAllDependenciesToNode(newWorkerNode->workerName, newWorkerNode->workerPort); SetUpMultipleDistributedTableIntegrations(newWorkerNode); @@ -1035,12 +1005,6 @@ citus_activate_node(PG_FUNCTION_ARGS) text *nodeNameText = PG_GETARG_TEXT_P(0); int32 nodePort = PG_GETARG_INT32(1); - if (!EnableDependencyCreation) - { - ereport(ERROR, (errmsg("citus.enable_object_propagation must be on to " - "activate node"))); - } - WorkerNode *workerNode = ModifiableWorkerNode(text_to_cstring(nodeNameText), nodePort); ActivateNode(workerNode->workerName, workerNode->workerPort); @@ -1222,6 +1186,12 @@ ActivateNode(char *nodeName, int nodePort) */ EnsureSuperUser(); + if (!EnableDependencyCreation) + { + ereport(ERROR, (errmsg("citus.enable_object_propagation must be on to " + "add an active node"))); + } + /* take an exclusive lock on pg_dist_node to serialize pg_dist_node changes */ LockRelationOid(DistNodeRelationId(), ExclusiveLock); @@ -1255,6 +1225,7 @@ ActivateNode(char *nodeName, int nodePort) SetWorkerColumnLocalOnly(workerNode, Anum_pg_dist_node_isactive, BoolGetDatum(isActive)); + // TODO: Once all tests will be enabled for MX, we can remove sync by default check bool syncMetadata = EnableMetadataSyncByDefault && NodeIsPrimary(workerNode); if (syncMetadata) @@ -1273,11 +1244,10 @@ ActivateNode(char *nodeName, int nodePort) { StartMetadataSyncToNode(nodeName, nodePort); - if (!NodeIsCoordinator(workerNode) && NodeIsPrimary(workerNode)) + if (!NodeIsCoordinator(workerNode)) { ClearDistributedObjectsFromNode(workerNode); SetUpObjectMetadata(workerNode); - AdjustSequenceLimits(workerNode); } } diff --git a/src/backend/distributed/worker/worker_data_fetch_protocol.c b/src/backend/distributed/worker/worker_data_fetch_protocol.c index efdc2da85..881619774 100644 --- a/src/backend/distributed/worker/worker_data_fetch_protocol.c +++ b/src/backend/distributed/worker/worker_data_fetch_protocol.c @@ -35,6 +35,7 @@ #include "distributed/intermediate_results.h" #include "distributed/listutils.h" #include "distributed/metadata_cache.h" +#include "distributed/metadata_sync.h" #include "distributed/multi_client_executor.h" #include "distributed/multi_logical_optimizer.h" #include "distributed/multi_partitioning_utils.h" @@ -45,6 +46,7 @@ #include "distributed/worker_protocol.h" #include "distributed/version_compat.h" +#include "executor/spi.h" #include "nodes/makefuncs.h" #include "parser/parse_relation.h" #include "storage/lmgr.h" @@ -461,18 +463,38 @@ worker_apply_sequence_command(PG_FUNCTION_ARGS) " SEQUENCE command string"))); } + /* + * If sequence with the same name exist for different type, it must have been + * stayed on that node after a rollbacked create_distributed_table operation. + * We must drop it first to create the sequence with the correct type. + */ + CreateSeqStmt *createSequenceStatement = (CreateSeqStmt *) commandNode; + char *sequenceName = createSequenceStatement->sequence->relname; + char *sequenceSchema = createSequenceStatement->sequence->schemaname; + RangeVar *sequenceRange = makeRangeVar(sequenceSchema, sequenceName, -1); + + Oid sequenceRelationId = RangeVarGetRelid(sequenceRange, AccessShareLock, true); + + if (sequenceRelationId != InvalidOid) + { + Form_pg_sequence pgSequenceForm = pg_get_sequencedef(sequenceRelationId); + if (pgSequenceForm->seqtypid != sequenceTypeId) + { + StringInfo dropSequenceString = makeStringInfo(); + char *qualifiedSequenceName = quote_qualified_identifier(sequenceSchema, sequenceName); + appendStringInfoString(dropSequenceString, "DROP SEQUENCE "); + appendStringInfoString(dropSequenceString, qualifiedSequenceName); + appendStringInfoString(dropSequenceString, ";"); + ExecuteQueryViaSPI(dropSequenceString->data, SPI_OK_UTILITY); + } + } + /* run the CREATE SEQUENCE command */ ProcessUtilityParseTree(commandNode, commandString, PROCESS_UTILITY_QUERY, NULL, None_Receiver, NULL); CommandCounterIncrement(); - CreateSeqStmt *createSequenceStatement = (CreateSeqStmt *) commandNode; - - char *sequenceName = createSequenceStatement->sequence->relname; - char *sequenceSchema = createSequenceStatement->sequence->schemaname; - createSequenceStatement = (CreateSeqStmt *) commandNode; - - Oid sequenceRelationId = RangeVarGetRelid(createSequenceStatement->sequence, + sequenceRelationId = RangeVarGetRelid(createSequenceStatement->sequence, AccessShareLock, false); Assert(sequenceRelationId != InvalidOid); diff --git a/src/include/distributed/metadata_sync.h b/src/include/distributed/metadata_sync.h index 96f4325ea..4814e7e70 100644 --- a/src/include/distributed/metadata_sync.h +++ b/src/include/distributed/metadata_sync.h @@ -31,6 +31,7 @@ typedef enum extern void StartMetadataSyncToNode(const char *nodeNameString, int32 nodePort); extern void EnsureSequentialModeMetadataOperations(void); extern bool ClusterHasKnownMetadataWorkers(void); +extern char * LocalGroupIdUpdateCommand(int32 groupId); extern bool ShouldSyncTableMetadata(Oid relationId); extern bool ShouldSyncTableMetadataViaCatalog(Oid relationId); extern List * MetadataCreateCommands(void); diff --git a/src/test/regress/expected/multi_cluster_management.out b/src/test/regress/expected/multi_cluster_management.out index 8d9907c34..fd289945b 100644 --- a/src/test/regress/expected/multi_cluster_management.out +++ b/src/test/regress/expected/multi_cluster_management.out @@ -256,7 +256,6 @@ SELECT master_update_node(nodeid, 'localhost', :worker_2_port + 3) FROM pg_dist_ ERROR: permission denied for function master_update_node -- try to manipulate node metadata via privileged user SET ROLE node_metadata_user; -SET citus.enable_object_propagation TO off; -- prevent master activate node to actually connect for this test SELECT 1 FROM master_add_node('localhost', :worker_2_port); ERROR: operation is not allowed HINT: Run the command with a superuser. @@ -474,8 +473,7 @@ SELECT run_command_on_workers('DELETE FROM pg_dist_node WHERE nodeport=' || :'wo (1 row) SELECT * FROM cluster_management_test; -NOTICE: there is a shard placement in node group 6 but there are no nodes in that group -ERROR: no active placements were found for shard 1220001 +ERROR: there is a shard placement in node group 6 but there are no nodes in that group -- clean-up SELECT * INTO old_placements FROM pg_dist_placement WHERE groupid = :worker_2_group; DELETE FROM pg_dist_placement WHERE groupid = :worker_2_group; diff --git a/src/test/regress/expected/multi_fix_partition_shard_index_names.out b/src/test/regress/expected/multi_fix_partition_shard_index_names.out index 395785d9e..e4023c412 100644 --- a/src/test/regress/expected/multi_fix_partition_shard_index_names.out +++ b/src/test/regress/expected/multi_fix_partition_shard_index_names.out @@ -669,15 +669,15 @@ DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx NOTICE: issuing ALTER TABLE fix_idx_names.p2 OWNER TO postgres DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx NOTICE: issuing ALTER TABLE fix_idx_names.p2 OWNER TO postgres -+NOTICE: issuing WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid) AS (VALUES ('table', ARRAY['fix_idx_names', 'p2']::text[], ARRAY[]::text[], -1, 0)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int) FROM distributed_object_data; -+DETAIL: on server postgres@localhost:57638 connectionId: 1 -+NOTICE: issuing WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid) AS (VALUES ('table', ARRAY['fix_idx_names', 'p2']::text[], ARRAY[]::text[], -1, 0)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int) FROM distributed_object_data; -+DETAIL: on server postgres@localhost:57637 connectionId: 2 -+NOTICE: issuing SET citus.enable_ddl_propagation TO 'off' -+DETAIL: on server postgres@localhost:57638 connectionId: 1 -+NOTICE: issuing SET citus.enable_ddl_propagation TO 'off' -+DETAIL: on server postgres@localhost:57637 connectionId: 2 DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx +NOTICE: issuing WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid) AS (VALUES ('table', ARRAY['fix_idx_names', 'p2']::text[], ARRAY[]::text[], -1, 0)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int) FROM distributed_object_data; +DETAIL: on server postgres@localhost:57638 connectionId: xxxxxxx +NOTICE: issuing WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid) AS (VALUES ('table', ARRAY['fix_idx_names', 'p2']::text[], ARRAY[]::text[], -1, 0)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int) FROM distributed_object_data; +DETAIL: on server postgres@localhost:57637 connectionId: xxxxxxx +NOTICE: issuing SET citus.enable_ddl_propagation TO 'off' +DETAIL: on server postgres@localhost:57638 connectionId: xxxxxxx +NOTICE: issuing SET citus.enable_ddl_propagation TO 'off' +DETAIL: on server postgres@localhost:57637 connectionId: xxxxxxx NOTICE: issuing SELECT citus_internal_add_partition_metadata ('fix_idx_names.p2'::regclass, 'h', 'dist_col', 1370000, 's') DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx NOTICE: issuing SELECT citus_internal_add_partition_metadata ('fix_idx_names.p2'::regclass, 'h', 'dist_col', 1370000, 's') diff --git a/src/test/regress/expected/multi_sequence_default.out b/src/test/regress/expected/multi_sequence_default.out index e00310b0e..764c9f938 100644 --- a/src/test/regress/expected/multi_sequence_default.out +++ b/src/test/regress/expected/multi_sequence_default.out @@ -130,72 +130,6 @@ ALTER SEQUENCE seq_0 AS bigint; ERROR: Altering a distributed sequence is currently not supported. ALTER SEQUENCE seq_0_local_table AS bigint; ERROR: Altering a distributed sequence is currently not supported. --- we can change other things like increment --- if metadata is not synced to workers -BEGIN; -SELECT stop_metadata_sync_to_node('localhost', :worker_1_port); -NOTICE: dropping metadata on the node (localhost,57637) - stop_metadata_sync_to_node ---------------------------------------------------------------------- - -(1 row) - -SELECT stop_metadata_sync_to_node('localhost', :worker_2_port); -NOTICE: dropping metadata on the node (localhost,57638) - stop_metadata_sync_to_node ---------------------------------------------------------------------- - -(1 row) - -CREATE SEQUENCE seq_13; -CREATE SEQUENCE seq_13_local_table; -CREATE TABLE seq_test_13 (x int, y int); -CREATE TABLE seq_test_13_local_table (x int, y int); -SELECT create_distributed_table('seq_test_13','x'); - create_distributed_table ---------------------------------------------------------------------- - -(1 row) - -SELECT citus_add_local_table_to_metadata('seq_test_13_local_table'); - citus_add_local_table_to_metadata ---------------------------------------------------------------------- - -(1 row) - -ALTER TABLE seq_test_13 ADD COLUMN z int DEFAULT nextval('seq_13'); -ALTER TABLE seq_test_13_local_table ADD COLUMN z int DEFAULT nextval('seq_13_local_table'); -ALTER SEQUENCE seq_13 INCREMENT BY 2; -ALTER SEQUENCE seq_13_local_table INCREMENT BY 2; -\d seq_13 - Sequence "sequence_default.seq_13" - Type | Start | Minimum | Maximum | Increment | Cycles? | Cache ---------------------------------------------------------------------- - integer | 1 | 1 | 2147483647 | 2 | no | 1 - -\d seq_13_local_table - Sequence "sequence_default.seq_13_local_table" - Type | Start | Minimum | Maximum | Increment | Cycles? | Cache ---------------------------------------------------------------------- - integer | 1 | 1 | 2147483647 | 2 | no | 1 - --- check that we can add serial pseudo-type columns --- when metadata is not synced to workers -TRUNCATE seq_test_0; -ALTER TABLE seq_test_0 ADD COLUMN w00 smallserial; -ALTER TABLE seq_test_0 ADD COLUMN w01 serial2; -ALTER TABLE seq_test_0 ADD COLUMN w10 serial; -ALTER TABLE seq_test_0 ADD COLUMN w11 serial4; -ALTER TABLE seq_test_0 ADD COLUMN w20 bigserial; -ALTER TABLE seq_test_0 ADD COLUMN w21 serial8; -TRUNCATE seq_test_0_local_table; -ALTER TABLE seq_test_0_local_table ADD COLUMN w00 smallserial; -ALTER TABLE seq_test_0_local_table ADD COLUMN w01 serial2; -ALTER TABLE seq_test_0_local_table ADD COLUMN w10 serial; -ALTER TABLE seq_test_0_local_table ADD COLUMN w11 serial4; -ALTER TABLE seq_test_0_local_table ADD COLUMN w20 bigserial; -ALTER TABLE seq_test_0_local_table ADD COLUMN w21 serial8; -ROLLBACK; -- check alter column type precaution ALTER TABLE seq_test_0 ALTER COLUMN z TYPE bigint; ERROR: cannot execute ALTER COLUMN TYPE .. command because the column involves a default coming from a sequence @@ -216,12 +150,6 @@ SELECT create_distributed_table('seq_test_4','x'); CREATE SEQUENCE seq_4; ALTER TABLE seq_test_4 ADD COLUMN a bigint DEFAULT nextval('seq_4'); -SELECT start_metadata_sync_to_node('localhost', :worker_1_port); - start_metadata_sync_to_node ---------------------------------------------------------------------- - -(1 row) - DROP SEQUENCE seq_4 CASCADE; NOTICE: drop cascades to default value for column a of table seq_test_4 TRUNCATE seq_test_4; @@ -770,10 +698,10 @@ SELECT create_reference_table('seq_test_10'); INSERT INTO seq_test_10 VALUES (0); CREATE TABLE seq_test_11 (col0 int, col1 bigint DEFAULT nextval('seq_11'::text)); -- works but doesn't create seq_11 in the workers -SELECT start_metadata_sync_to_node('localhost', :worker_1_port); - start_metadata_sync_to_node +SELECT citus_activate_node('localhost', :worker_1_port); + citus_activate_node --------------------------------------------------------------------- - + 1 (1 row) -- works because there is no dependency created between seq_11 and seq_test_10 @@ -812,10 +740,10 @@ SELECT create_distributed_table('seq_test_12', 'col0'); (1 row) -SELECT start_metadata_sync_to_node('localhost', :worker_1_port); - start_metadata_sync_to_node +SELECT citus_activate_node('localhost', :worker_1_port); + citus_activate_node --------------------------------------------------------------------- - + 1 (1 row) INSERT INTO seq_test_12 VALUES ('hello0') RETURNING *; @@ -918,10 +846,10 @@ ERROR: nextval: reached maximum value of sequence "seq_14" (32767) \c - - - :master_port SET citus.shard_replication_factor TO 1; SET search_path = sequence_default, public; -SELECT start_metadata_sync_to_node('localhost', :worker_1_port); - start_metadata_sync_to_node +SELECT citus_activate_node('localhost', :worker_1_port); + citus_activate_node --------------------------------------------------------------------- - + 1 (1 row) SELECT undistribute_table('seq_test_12'); diff --git a/src/test/regress/expected/single_node.out b/src/test/regress/expected/single_node.out index 9715232a2..1cd9161a7 100644 --- a/src/test/regress/expected/single_node.out +++ b/src/test/regress/expected/single_node.out @@ -164,6 +164,15 @@ HINT: Use SELECT rebalance_table_shards(); to balance shards data between worke (1 row) COMMIT; +-- After adding and removing non-metadata synced worker node, shell table +-- can stay on the remote node. So we are deleting it manually. +-- TODO: Update the test once sync by default guc will be removed +SELECT run_command_on_workers($$DROP TABLE single_node.test$$); + run_command_on_workers +---------------------------------- + (localhost,57637,t,"DROP TABLE") +(1 row) + -- we don't need this node anymore SELECT 1 FROM master_remove_node('localhost', :worker_1_port); ?column? diff --git a/src/test/regress/input/multi_copy.source b/src/test/regress/input/multi_copy.source index 0b3067be6..b40cdd2d5 100644 --- a/src/test/regress/input/multi_copy.source +++ b/src/test/regress/input/multi_copy.source @@ -467,7 +467,6 @@ SELECT shardid, nodename, nodeport -- disable the first node SET client_min_messages TO ERROR; -DROP FOREIGN TABLE foreign_table_to_distribute; \set VERBOSITY terse SELECT master_disable_node('localhost', :worker_1_port); SELECT public.wait_until_metadata_sync(30000); diff --git a/src/test/regress/output/multi_copy.source b/src/test/regress/output/multi_copy.source index fb2a33d20..2207dcd6c 100644 --- a/src/test/regress/output/multi_copy.source +++ b/src/test/regress/output/multi_copy.source @@ -625,7 +625,6 @@ SELECT shardid, nodename, nodeport (6 rows) -- add the node back -SET client_min_messages TO ERROR; SELECT 1 FROM master_activate_node('localhost', :worker_1_port); ?column? --------------------------------------------------------------------- diff --git a/src/test/regress/sql/multi_cluster_management.sql b/src/test/regress/sql/multi_cluster_management.sql index ca571d954..ed26dcc2f 100644 --- a/src/test/regress/sql/multi_cluster_management.sql +++ b/src/test/regress/sql/multi_cluster_management.sql @@ -120,9 +120,8 @@ SELECT 1 FROM master_add_node('localhost', :worker_2_port + 1); SELECT 1 FROM master_add_secondary_node('localhost', :worker_2_port + 2, 'localhost', :worker_2_port); SELECT master_update_node(nodeid, 'localhost', :worker_2_port + 3) FROM pg_dist_node WHERE nodeport = :worker_2_port; --- show that non-admin role can not activate a node +-- try to manipulate node metadata via privileged user SET ROLE node_metadata_user; -SET citus.enable_object_propagation TO off; -- prevent master activate node to actually connect for this test SELECT 1 FROM master_add_node('localhost', :worker_2_port); BEGIN; SELECT 1 FROM master_add_inactive_node('localhost', :worker_2_port); diff --git a/src/test/regress/sql/multi_sequence_default.sql b/src/test/regress/sql/multi_sequence_default.sql index 23d772f20..3f8efc154 100644 --- a/src/test/regress/sql/multi_sequence_default.sql +++ b/src/test/regress/sql/multi_sequence_default.sql @@ -55,9 +55,6 @@ ALTER SEQUENCE seq_0_local_table AS bigint; ALTER TABLE seq_test_0 ALTER COLUMN z TYPE bigint; ALTER TABLE seq_test_0 ALTER COLUMN z TYPE smallint; --- TODO: Sequences stay there after rollback! --- TODO: Talk with Onder about adjusting sequence limit - ALTER TABLE seq_test_0_local_table ALTER COLUMN z TYPE bigint; ALTER TABLE seq_test_0_local_table ALTER COLUMN z TYPE smallint; diff --git a/src/test/regress/sql/single_node.sql b/src/test/regress/sql/single_node.sql index c21066424..1b62a44dd 100644 --- a/src/test/regress/sql/single_node.sql +++ b/src/test/regress/sql/single_node.sql @@ -87,6 +87,11 @@ BEGIN; SELECT 1 FROM master_add_node('localhost', :worker_1_port); COMMIT; +-- After adding and removing non-metadata synced worker node, shell table +-- can stay on the remote node. So we are deleting it manually. +-- TODO: Update the test once sync by default guc will be removed +SELECT run_command_on_workers($$DROP TABLE single_node.test$$); + -- we don't need this node anymore SELECT 1 FROM master_remove_node('localhost', :worker_1_port);