Merge pull request #5912 from citusdata/relax_disable_node

Adds "synchronous" option to citus_disable_node() UDF
pull/5955/head
Önder Kalacı 2022-05-18 17:34:41 +02:00 committed by GitHub
commit 66378d00da
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
16 changed files with 617 additions and 441 deletions

View File

@ -97,6 +97,7 @@ static char * SchemaOwnerName(Oid objectId);
static bool HasMetadataWorkers(void); static bool HasMetadataWorkers(void);
static void CreateShellTableOnWorkers(Oid relationId); static void CreateShellTableOnWorkers(Oid relationId);
static void CreateTableMetadataOnWorkers(Oid relationId); static void CreateTableMetadataOnWorkers(Oid relationId);
static NodeMetadataSyncResult SyncNodeMetadataToNodesOptional(void);
static bool ShouldSyncTableMetadataInternal(bool hashDistributed, static bool ShouldSyncTableMetadataInternal(bool hashDistributed,
bool citusTableWithNoDistKey); bool citusTableWithNoDistKey);
static bool SyncNodeMetadataSnapshotToNode(WorkerNode *workerNode, bool raiseOnError); static bool SyncNodeMetadataSnapshotToNode(WorkerNode *workerNode, bool raiseOnError);
@ -2237,16 +2238,16 @@ DetachPartitionCommandList(void)
/* /*
* SyncNodeMetadataToNodes tries recreating the metadata snapshot in the * SyncNodeMetadataToNodesOptional tries recreating the metadata
* metadata workers that are out of sync. Returns the result of * snapshot in the metadata workers that are out of sync.
* synchronization. * Returns the result of synchronization.
* *
* This function must be called within coordinated transaction * This function must be called within coordinated transaction
* since updates on the pg_dist_node metadata must be rollbacked if anything * since updates on the pg_dist_node metadata must be rollbacked if anything
* goes wrong. * goes wrong.
*/ */
static NodeMetadataSyncResult static NodeMetadataSyncResult
SyncNodeMetadataToNodes(void) SyncNodeMetadataToNodesOptional(void)
{ {
NodeMetadataSyncResult result = NODE_METADATA_SYNC_SUCCESS; NodeMetadataSyncResult result = NODE_METADATA_SYNC_SUCCESS;
if (!IsCoordinator()) if (!IsCoordinator())
@ -2306,6 +2307,46 @@ SyncNodeMetadataToNodes(void)
} }
/*
* SyncNodeMetadataToNodes recreates the node metadata snapshot in all the
* metadata workers.
*
* This function runs within a coordinated transaction since updates on
* the pg_dist_node metadata must be rollbacked if anything
* goes wrong.
*/
void
SyncNodeMetadataToNodes(void)
{
EnsureCoordinator();
/*
* Request a RowExclusiveLock so we don't run concurrently with other
* functions updating pg_dist_node, but allow concurrency with functions
* which are just reading from pg_dist_node.
*/
if (!ConditionalLockRelationOid(DistNodeRelationId(), RowExclusiveLock))
{
ereport(ERROR, (errmsg("cannot sync metadata because a concurrent "
"metadata syncing operation is in progress")));
}
List *workerList = ActivePrimaryNonCoordinatorNodeList(NoLock);
WorkerNode *workerNode = NULL;
foreach_ptr(workerNode, workerList)
{
if (workerNode->hasMetadata)
{
SetWorkerColumnLocalOnly(workerNode, Anum_pg_dist_node_metadatasynced,
BoolGetDatum(true));
bool raiseOnError = true;
SyncNodeMetadataSnapshotToNode(workerNode, raiseOnError);
}
}
}
/* /*
* SyncNodeMetadataToNodesMain is the main function for syncing node metadata to * SyncNodeMetadataToNodesMain is the main function for syncing node metadata to
* MX nodes. It retries until success and then exits. * MX nodes. It retries until success and then exits.
@ -2352,7 +2393,7 @@ SyncNodeMetadataToNodesMain(Datum main_arg)
{ {
UseCoordinatedTransaction(); UseCoordinatedTransaction();
NodeMetadataSyncResult result = SyncNodeMetadataToNodes(); NodeMetadataSyncResult result = SyncNodeMetadataToNodesOptional();
syncedAllNodes = (result == NODE_METADATA_SYNC_SUCCESS); syncedAllNodes = (result == NODE_METADATA_SYNC_SUCCESS);
/* we use LISTEN/NOTIFY to wait for metadata syncing in tests */ /* we use LISTEN/NOTIFY to wait for metadata syncing in tests */

View File

@ -110,6 +110,7 @@ static void SyncDistributedObjectsToNode(WorkerNode *workerNode);
static void UpdateLocalGroupIdOnNode(WorkerNode *workerNode); static void UpdateLocalGroupIdOnNode(WorkerNode *workerNode);
static void SyncPgDistTableMetadataToNode(WorkerNode *workerNode); static void SyncPgDistTableMetadataToNode(WorkerNode *workerNode);
static List * InterTableRelationshipCommandList(); static List * InterTableRelationshipCommandList();
static void BlockDistributedQueriesOnMetadataNodes(void);
static WorkerNode * TupleToWorkerNode(TupleDesc tupleDescriptor, HeapTuple heapTuple); static WorkerNode * TupleToWorkerNode(TupleDesc tupleDescriptor, HeapTuple heapTuple);
static List * PropagateNodeWideObjectsCommandList(); static List * PropagateNodeWideObjectsCommandList();
static WorkerNode * ModifiableWorkerNode(const char *nodeName, int32 nodePort); static WorkerNode * ModifiableWorkerNode(const char *nodeName, int32 nodePort);
@ -452,7 +453,7 @@ citus_disable_node(PG_FUNCTION_ARGS)
{ {
text *nodeNameText = PG_GETARG_TEXT_P(0); text *nodeNameText = PG_GETARG_TEXT_P(0);
int32 nodePort = PG_GETARG_INT32(1); int32 nodePort = PG_GETARG_INT32(1);
bool forceDisableNode = PG_GETARG_BOOL(2); bool synchronousDisableNode = PG_GETARG_BOOL(2);
char *nodeName = text_to_cstring(nodeNameText); char *nodeName = text_to_cstring(nodeNameText);
WorkerNode *workerNode = ModifiableWorkerNode(nodeName, nodePort); WorkerNode *workerNode = ModifiableWorkerNode(nodeName, nodePort);
@ -463,8 +464,10 @@ citus_disable_node(PG_FUNCTION_ARGS)
"isactive"); "isactive");
WorkerNode *firstWorkerNode = GetFirstPrimaryWorkerNode(); WorkerNode *firstWorkerNode = GetFirstPrimaryWorkerNode();
if (!forceDisableNode && firstWorkerNode && bool disablingFirstNode =
firstWorkerNode->nodeId == workerNode->nodeId) (firstWorkerNode && firstWorkerNode->nodeId == workerNode->nodeId);
if (disablingFirstNode && !synchronousDisableNode)
{ {
/* /*
* We sync metadata async and optionally in the background worker, * We sync metadata async and optionally in the background worker,
@ -478,16 +481,21 @@ citus_disable_node(PG_FUNCTION_ARGS)
* possibility of diverged shard placements for the same shard. * possibility of diverged shard placements for the same shard.
* *
* To prevent that, we currently do not allow disabling the first * To prevent that, we currently do not allow disabling the first
* worker node. * worker node unless it is explicitly opted synchronous.
*/ */
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE), ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("disabling the first worker node in the " errmsg("disabling the first worker node in the "
"metadata is not allowed"), "metadata is not allowed"),
errhint("You can force disabling node, but this operation " errhint("You can force disabling node, SELECT "
"might cause replicated shards to diverge: SELECT " "citus_disable_node('%s', %d, "
"citus_disable_node('%s', %d, force:=true);", "synchronous:=true);", workerNode->workerName,
workerNode->workerName, nodePort),
nodePort))); errdetail("Citus uses the first worker node in the "
"metadata for certain internal operations when "
"replicated tables are modified. Synchronous mode "
"ensures that all nodes have the same view of the "
"first worker node, which is used for certain "
"locking operations.")));
} }
/* /*
@ -506,24 +514,42 @@ citus_disable_node(PG_FUNCTION_ARGS)
* for any given shard. * for any given shard.
*/ */
ErrorIfNodeContainsNonRemovablePlacements(workerNode); ErrorIfNodeContainsNonRemovablePlacements(workerNode);
bool onlyConsiderActivePlacements = false;
if (NodeGroupHasShardPlacements(workerNode->groupId,
onlyConsiderActivePlacements))
{
ereport(NOTICE, (errmsg(
"Node %s:%d has active shard placements. Some queries "
"may fail after this operation. Use "
"SELECT citus_activate_node('%s', %d) to activate this "
"node back.",
workerNode->workerName, nodePort,
workerNode->workerName,
nodePort)));
}
} }
TransactionModifiedNodeMetadata = true; TransactionModifiedNodeMetadata = true;
if (synchronousDisableNode)
{
/*
* The user might pick between sync vs async options.
* - Pros for the sync option:
* (a) the changes become visible on the cluster immediately
* (b) even if the first worker node is disabled, there is no
* risk of divergence of the placements of replicated shards
* - Cons for the sync options:
* (a) Does not work within 2PC transaction (e.g., BEGIN;
* citus_disable_node(); PREPARE TRANSACTION ...);
* (b) If there are multiple node failures (e.g., one another node
* than the current node being disabled), the sync option would
* fail because it'd try to sync the metadata changes to a node
* that is not up and running.
*/
if (firstWorkerNode && firstWorkerNode->nodeId == workerNode->nodeId)
{
/*
* We cannot let any modification query on a replicated table to run
* concurrently with citus_disable_node() on the first worker node. If
* we let that, some worker nodes might calculate FirstWorkerNode()
* different than others. See LockShardListResourcesOnFirstWorker()
* for the details.
*/
BlockDistributedQueriesOnMetadataNodes();
}
SyncNodeMetadataToNodes();
}
else if (UnsetMetadataSyncedForAllWorkers())
{
/* /*
* We have not propagated the node metadata changes yet, make sure that all the * We have not propagated the node metadata changes yet, make sure that all the
* active nodes get the metadata updates. We defer this operation to the * active nodes get the metadata updates. We defer this operation to the
@ -536,8 +562,7 @@ citus_disable_node(PG_FUNCTION_ARGS)
* metadata at this point. Instead, we defer that to citus_activate_node() * metadata at this point. Instead, we defer that to citus_activate_node()
* where we expect all nodes up and running. * where we expect all nodes up and running.
*/ */
if (UnsetMetadataSyncedForAllWorkers())
{
TriggerNodeMetadataSyncOnCommit(); TriggerNodeMetadataSyncOnCommit();
} }
@ -545,6 +570,33 @@ citus_disable_node(PG_FUNCTION_ARGS)
} }
/*
* BlockDistributedQueriesOnMetadataNodes blocks all the modification queries on
* all nodes. Hence, should be used with caution.
*/
static void
BlockDistributedQueriesOnMetadataNodes(void)
{
/* first, block on the coordinator */
LockRelationOid(DistNodeRelationId(), ExclusiveLock);
/*
* Note that we might re-design this lock to be more granular than
* pg_dist_node, scoping only for modifications on the replicated
* tables. However, we currently do not have any such mechanism and
* given that citus_disable_node() runs instantly, it seems acceptable
* to block reads (or modifications on non-replicated tables) for
* a while.
*/
/* only superuser can disable node */
Assert(superuser());
SendCommandToWorkersWithMetadata(
"LOCK TABLE pg_catalog.pg_dist_node IN EXCLUSIVE MODE;");
}
/* /*
* master_disable_node is a wrapper function for old UDF name. * master_disable_node is a wrapper function for old UDF name.
*/ */

View File

@ -393,7 +393,7 @@ NodeNamePortCompare(const char *workerLhsName, const char *workerRhsName,
WorkerNode * WorkerNode *
GetFirstPrimaryWorkerNode(void) GetFirstPrimaryWorkerNode(void)
{ {
List *workerNodeList = ActivePrimaryNonCoordinatorNodeList(NoLock); List *workerNodeList = ActivePrimaryNonCoordinatorNodeList(RowShareLock);
WorkerNode *firstWorkerNode = NULL; WorkerNode *firstWorkerNode = NULL;
WorkerNode *workerNode = NULL; WorkerNode *workerNode = NULL;
foreach_ptr(workerNode, workerNodeList) foreach_ptr(workerNode, workerNodeList)

View File

@ -1,3 +1,4 @@
#include "udfs/citus_shards_on_worker/11.0-2.sql" #include "udfs/citus_shards_on_worker/11.0-2.sql"
#include "udfs/citus_shard_indexes_on_worker/11.0-2.sql" #include "udfs/citus_shard_indexes_on_worker/11.0-2.sql"
#include "udfs/citus_is_coordinator/11.0-2.sql" #include "udfs/citus_is_coordinator/11.0-2.sql"
#include "udfs/citus_disable_node/11.0-2.sql"

View File

@ -1,2 +1,4 @@
#include "../udfs/citus_shards_on_worker/11.0-1.sql" #include "../udfs/citus_shards_on_worker/11.0-1.sql"
#include "../udfs/citus_shard_indexes_on_worker/11.0-1.sql" #include "../udfs/citus_shard_indexes_on_worker/11.0-1.sql"
#include "../udfs/citus_disable_node/11.0-1.sql"

View File

@ -0,0 +1,9 @@
DROP FUNCTION pg_catalog.citus_disable_node(nodename text, nodeport integer, force bool);
CREATE FUNCTION pg_catalog.citus_disable_node(nodename text, nodeport integer, synchronous bool default false)
RETURNS void
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$citus_disable_node$$;
COMMENT ON FUNCTION pg_catalog.citus_disable_node(nodename text, nodeport integer, synchronous bool)
IS 'removes node from the cluster temporarily';
REVOKE ALL ON FUNCTION pg_catalog.citus_disable_node(text,int, bool) FROM PUBLIC;

View File

@ -1,9 +1,9 @@
DROP FUNCTION pg_catalog.citus_disable_node(nodename text, nodeport integer); DROP FUNCTION pg_catalog.citus_disable_node(nodename text, nodeport integer, force bool);
CREATE FUNCTION pg_catalog.citus_disable_node(nodename text, nodeport integer, force bool default false) CREATE FUNCTION pg_catalog.citus_disable_node(nodename text, nodeport integer, synchronous bool default false)
RETURNS void RETURNS void
LANGUAGE C STRICT LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$citus_disable_node$$; AS 'MODULE_PATHNAME', $$citus_disable_node$$;
COMMENT ON FUNCTION pg_catalog.citus_disable_node(nodename text, nodeport integer, force bool) COMMENT ON FUNCTION pg_catalog.citus_disable_node(nodename text, nodeport integer, synchronous bool)
IS 'removes node from the cluster temporarily'; IS 'removes node from the cluster temporarily';
REVOKE ALL ON FUNCTION pg_catalog.citus_disable_node(text,int, bool) FROM PUBLIC; REVOKE ALL ON FUNCTION pg_catalog.citus_disable_node(text,int, bool) FROM PUBLIC;

View File

@ -65,6 +65,7 @@ extern TableDDLCommand * TruncateTriggerCreateCommand(Oid relationId);
extern void CreateInterTableRelationshipOfRelationOnWorkers(Oid relationId); extern void CreateInterTableRelationshipOfRelationOnWorkers(Oid relationId);
extern List * InterTableRelationshipOfRelationCommandList(Oid relationId); extern List * InterTableRelationshipOfRelationCommandList(Oid relationId);
extern List * DetachPartitionCommandList(void); extern List * DetachPartitionCommandList(void);
extern void SyncNodeMetadataToNodes(void);
extern BackgroundWorkerHandle * SpawnSyncNodeMetadataToNodes(Oid database, Oid owner); extern BackgroundWorkerHandle * SpawnSyncNodeMetadataToNodes(Oid database, Oid owner);
extern void SyncNodeMetadataToNodesMain(Datum main_arg); extern void SyncNodeMetadataToNodesMain(Datum main_arg);
extern void SignalMetadataSyncDaemon(Oid database, int sig); extern void SignalMetadataSyncDaemon(Oid database, int sig);

View File

@ -54,7 +54,6 @@ ORDER BY placementid;
(2 rows) (2 rows)
SELECT citus_disable_node('localhost', :worker_2_proxy_port, true); SELECT citus_disable_node('localhost', :worker_2_proxy_port, true);
NOTICE: Node localhost:xxxxx has active shard placements. Some queries may fail after this operation. Use SELECT citus_activate_node('localhost', 9060) to activate this node back.
citus_disable_node citus_disable_node
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -25,10 +25,11 @@ step s1-add-second-worker:
step s2-copy-to-reference-table: step s2-copy-to-reference-table:
COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5'; COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5';
<waiting ...>
step s1-commit: step s1-commit:
COMMIT; COMMIT;
step s2-copy-to-reference-table: <... completed>
step s2-print-content: step s2-print-content:
SELECT SELECT
nodeport, success, result nodeport, success, result
@ -69,15 +70,16 @@ step s2-copy-to-reference-table:
step s1-add-second-worker: step s1-add-second-worker:
SELECT 1 FROM master_add_node('localhost', 57638); SELECT 1 FROM master_add_node('localhost', 57638);
<waiting ...>
step s2-commit:
COMMIT;
step s1-add-second-worker: <... completed>
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
step s2-commit:
COMMIT;
step s2-print-content: step s2-print-content:
SELECT SELECT
nodeport, success, result nodeport, success, result
@ -123,10 +125,11 @@ step s1-add-second-worker:
step s2-insert-to-reference-table: step s2-insert-to-reference-table:
INSERT INTO test_reference_table VALUES (6); INSERT INTO test_reference_table VALUES (6);
<waiting ...>
step s1-commit: step s1-commit:
COMMIT; COMMIT;
step s2-insert-to-reference-table: <... completed>
step s2-print-content: step s2-print-content:
SELECT SELECT
nodeport, success, result nodeport, success, result
@ -167,15 +170,16 @@ step s2-insert-to-reference-table:
step s1-add-second-worker: step s1-add-second-worker:
SELECT 1 FROM master_add_node('localhost', 57638); SELECT 1 FROM master_add_node('localhost', 57638);
<waiting ...>
step s2-commit:
COMMIT;
step s1-add-second-worker: <... completed>
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
step s2-commit:
COMMIT;
step s2-print-content: step s2-print-content:
SELECT SELECT
nodeport, success, result nodeport, success, result
@ -429,10 +433,11 @@ step s1-add-second-worker:
step s2-copy-to-reference-table: step s2-copy-to-reference-table:
COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5'; COPY test_reference_table FROM PROGRAM 'echo 1 && echo 2 && echo 3 && echo 4 && echo 5';
<waiting ...>
step s1-commit: step s1-commit:
COMMIT; COMMIT;
step s2-copy-to-reference-table: <... completed>
step s2-print-content: step s2-print-content:
SELECT SELECT
nodeport, success, result nodeport, success, result
@ -470,15 +475,16 @@ step s2-copy-to-reference-table:
step s1-add-second-worker: step s1-add-second-worker:
SELECT 1 FROM master_add_node('localhost', 57638); SELECT 1 FROM master_add_node('localhost', 57638);
<waiting ...>
step s2-commit:
COMMIT;
step s1-add-second-worker: <... completed>
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
step s2-commit:
COMMIT;
step s2-print-content: step s2-print-content:
SELECT SELECT
nodeport, success, result nodeport, success, result
@ -521,10 +527,11 @@ step s1-add-second-worker:
step s2-insert-to-reference-table: step s2-insert-to-reference-table:
INSERT INTO test_reference_table VALUES (6); INSERT INTO test_reference_table VALUES (6);
<waiting ...>
step s1-commit: step s1-commit:
COMMIT; COMMIT;
step s2-insert-to-reference-table: <... completed>
step s2-print-content: step s2-print-content:
SELECT SELECT
nodeport, success, result nodeport, success, result
@ -562,15 +569,16 @@ step s2-insert-to-reference-table:
step s1-add-second-worker: step s1-add-second-worker:
SELECT 1 FROM master_add_node('localhost', 57638); SELECT 1 FROM master_add_node('localhost', 57638);
<waiting ...>
step s2-commit:
COMMIT;
step s1-add-second-worker: <... completed>
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
step s2-commit:
COMMIT;
step s2-print-content: step s2-print-content:
SELECT SELECT
nodeport, success, result nodeport, success, result

View File

@ -409,15 +409,16 @@ step s1-insert-ref:
step s2-create-restore: step s2-create-restore:
SELECT 1 FROM citus_create_restore_point('citus-test'); SELECT 1 FROM citus_create_restore_point('citus-test');
<waiting ...>
step s1-commit:
COMMIT;
step s2-create-restore: <... completed>
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
step s1-commit:
COMMIT;
starting permutation: s1-begin s1-modify-multiple-ref s2-create-restore s1-commit starting permutation: s1-begin s1-modify-multiple-ref s2-create-restore s1-commit
create_reference_table create_reference_table
@ -434,15 +435,16 @@ step s1-modify-multiple-ref:
step s2-create-restore: step s2-create-restore:
SELECT 1 FROM citus_create_restore_point('citus-test'); SELECT 1 FROM citus_create_restore_point('citus-test');
<waiting ...>
step s1-commit:
COMMIT;
step s2-create-restore: <... completed>
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
step s1-commit:
COMMIT;
starting permutation: s1-begin s1-ddl-ref s2-create-restore s1-commit starting permutation: s1-begin s1-ddl-ref s2-create-restore s1-commit
create_reference_table create_reference_table
@ -485,15 +487,16 @@ step s1-copy-ref:
step s2-create-restore: step s2-create-restore:
SELECT 1 FROM citus_create_restore_point('citus-test'); SELECT 1 FROM citus_create_restore_point('citus-test');
<waiting ...>
step s1-commit:
COMMIT;
step s2-create-restore: <... completed>
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (1 row)
step s1-commit:
COMMIT;
starting permutation: s1-begin s1-drop-ref s2-create-restore s1-commit starting permutation: s1-begin s1-drop-ref s2-create-restore s1-commit
create_reference_table create_reference_table

View File

@ -129,6 +129,54 @@ SELECT 1 FROM citus_activate_node('localhost', :worker_2_port);
1 1
(1 row) (1 row)
-- disable node with sync/force options
SELECT citus_disable_node('localhost', :worker_1_port);
ERROR: disabling the first worker node in the metadata is not allowed
DETAIL: Citus uses the first worker node in the metadata for certain internal operations when replicated tables are modified. Synchronous mode ensures that all nodes have the same view of the first worker node, which is used for certain locking operations.
HINT: You can force disabling node, SELECT citus_disable_node('localhost', 57637, synchronous:=true);
SELECT citus_disable_node('localhost', :worker_1_port, synchronous:=true);
citus_disable_node
---------------------------------------------------------------------
(1 row)
SELECT run_command_on_workers($$SELECT array_agg(isactive ORDER BY nodeport) FROM pg_dist_node WHERE hasmetadata and noderole='primary'::noderole AND nodecluster='default'$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57638,t,"{f,t}")
(1 row)
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
?column?
---------------------------------------------------------------------
1
(1 row)
-- disable node with sync/force options
SELECT citus_disable_node('localhost', :worker_2_port, synchronous:=true);
citus_disable_node
---------------------------------------------------------------------
(1 row)
SELECT run_command_on_workers($$SELECT array_agg(isactive ORDER BY nodeport) FROM pg_dist_node WHERE hasmetadata and noderole='primary'::noderole AND nodecluster='default'$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"{t,f}")
(1 row)
SELECT 1 FROM citus_activate_node('localhost', :worker_2_port);
?column?
---------------------------------------------------------------------
1
(1 row)
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
?column?
---------------------------------------------------------------------
1
(1 row)
CREATE TABLE cluster_management_test (col_1 text, col_2 int); CREATE TABLE cluster_management_test (col_1 text, col_2 int);
SELECT create_distributed_table('cluster_management_test', 'col_1', 'hash'); SELECT create_distributed_table('cluster_management_test', 'col_1', 'hash');
create_distributed_table create_distributed_table

View File

@ -1780,7 +1780,8 @@ ERROR: localhost:xxxxx is a metadata node, but is out of sync
HINT: If the node is up, wait until metadata gets synced to it and try again. HINT: If the node is up, wait until metadata gets synced to it and try again.
SELECT citus_disable_node_and_wait('localhost', :worker_1_port); SELECT citus_disable_node_and_wait('localhost', :worker_1_port);
ERROR: disabling the first worker node in the metadata is not allowed ERROR: disabling the first worker node in the metadata is not allowed
HINT: You can force disabling node, but this operation might cause replicated shards to diverge: SELECT citus_disable_node('localhost', 57637, force:=true); DETAIL: Citus uses the first worker node in the metadata for certain internal operations when replicated tables are modified. Synchronous mode ensures that all nodes have the same view of the first worker node, which is used for certain locking operations.
HINT: You can force disabling node, SELECT citus_disable_node('localhost', 57637, synchronous:=true);
CONTEXT: SQL statement "SELECT pg_catalog.citus_disable_node(nodename, nodeport, force)" CONTEXT: SQL statement "SELECT pg_catalog.citus_disable_node(nodename, nodeport, force)"
PL/pgSQL function citus_disable_node_and_wait(text,integer,boolean) line XX at PERFORM PL/pgSQL function citus_disable_node_and_wait(text,integer,boolean) line XX at PERFORM
SELECT citus_disable_node_and_wait('localhost', :worker_2_port); SELECT citus_disable_node_and_wait('localhost', :worker_2_port);

View File

@ -642,7 +642,6 @@ SELECT verify_metadata('localhost', :worker_1_port),
-- Don't drop the reference table so it has shards on the nodes being disabled -- Don't drop the reference table so it has shards on the nodes being disabled
DROP TABLE dist_table_1, dist_table_2; DROP TABLE dist_table_1, dist_table_2;
SELECT pg_catalog.citus_disable_node('localhost', :worker_2_port); SELECT pg_catalog.citus_disable_node('localhost', :worker_2_port);
NOTICE: Node localhost:xxxxx has active shard placements. Some queries may fail after this operation. Use SELECT citus_activate_node('localhost', 57638) to activate this node back.
citus_disable_node citus_disable_node
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -20,7 +20,6 @@ INSERT INTO replicated SELECT i,i FROM generate_series(0,10)i;
INSERT INTO ref SELECT i,i FROM generate_series(0,10)i; INSERT INTO ref SELECT i,i FROM generate_series(0,10)i;
-- should be successfully disable node -- should be successfully disable node
SELECT citus_disable_node('localhost', :worker_2_port, true); SELECT citus_disable_node('localhost', :worker_2_port, true);
NOTICE: Node localhost:xxxxx has active shard placements. Some queries may fail after this operation. Use SELECT citus_activate_node('localhost', 57638) to activate this node back.
citus_disable_node citus_disable_node
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -56,6 +56,19 @@ ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 1390000;
SELECT 1 FROM citus_activate_node('localhost', :worker_2_port); SELECT 1 FROM citus_activate_node('localhost', :worker_2_port);
-- disable node with sync/force options
SELECT citus_disable_node('localhost', :worker_1_port);
SELECT citus_disable_node('localhost', :worker_1_port, synchronous:=true);
SELECT run_command_on_workers($$SELECT array_agg(isactive ORDER BY nodeport) FROM pg_dist_node WHERE hasmetadata and noderole='primary'::noderole AND nodecluster='default'$$);
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
-- disable node with sync/force options
SELECT citus_disable_node('localhost', :worker_2_port, synchronous:=true);
SELECT run_command_on_workers($$SELECT array_agg(isactive ORDER BY nodeport) FROM pg_dist_node WHERE hasmetadata and noderole='primary'::noderole AND nodecluster='default'$$);
SELECT 1 FROM citus_activate_node('localhost', :worker_2_port);
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
CREATE TABLE cluster_management_test (col_1 text, col_2 int); CREATE TABLE cluster_management_test (col_1 text, col_2 int);
SELECT create_distributed_table('cluster_management_test', 'col_1', 'hash'); SELECT create_distributed_table('cluster_management_test', 'col_1', 'hash');