mirror of https://github.com/citusdata/citus.git
Fix crash in shard rebalancer when no distributed tables exist (#5205)
The logging of the amount of ignored moves crashed when no distributed tables existed in a cluster. This also fixes in passing that the logging of ignored moves logs the correct number of ignored moves if there exist multiple colocation groups and all are rebalanced at the same time.pull/5220/head
parent
d50830d4cc
commit
481f8be084
|
@ -1806,16 +1806,21 @@ RebalancePlacementUpdates(List *workerNodeList, List *shardPlacementListList,
|
|||
hash_destroy(state->placementsHash);
|
||||
}
|
||||
|
||||
int64 ignoredMoves = 0;
|
||||
foreach_ptr(state, rebalanceStates)
|
||||
{
|
||||
ignoredMoves += state->ignoredMoves;
|
||||
}
|
||||
|
||||
if (state->ignoredMoves > 0)
|
||||
if (ignoredMoves > 0)
|
||||
{
|
||||
if (MaxRebalancerLoggedIgnoredMoves == -1 ||
|
||||
state->ignoredMoves <= MaxRebalancerLoggedIgnoredMoves)
|
||||
ignoredMoves <= MaxRebalancerLoggedIgnoredMoves)
|
||||
{
|
||||
ereport(NOTICE, (
|
||||
errmsg(
|
||||
"Ignored %ld moves, all of which are shown in notices above",
|
||||
state->ignoredMoves
|
||||
ignoredMoves
|
||||
),
|
||||
errhint(
|
||||
"If you do want these moves to happen, try changing improvement_threshold to a lower value than what it is now (%g).",
|
||||
|
@ -1827,7 +1832,7 @@ RebalancePlacementUpdates(List *workerNodeList, List *shardPlacementListList,
|
|||
ereport(NOTICE, (
|
||||
errmsg(
|
||||
"Ignored %ld moves, %d of which are shown in notices above",
|
||||
state->ignoredMoves,
|
||||
ignoredMoves,
|
||||
MaxRebalancerLoggedIgnoredMoves
|
||||
),
|
||||
errhint(
|
||||
|
|
|
@ -87,6 +87,22 @@ SET citus.replication_model to 'statement';
|
|||
NOTICE: Setting citus.replication_model has no effect. Please use citus.shard_replication_factor instead.
|
||||
DETAIL: Citus determines the replication model based on the replication factor and the replication models of the colocated shards. If a colocated table is present, the replication model is inherited. Otherwise 'streaming' replication is preferred if supported by the replication factor.
|
||||
ROLLBACK;
|
||||
-- check that the rebalancer works even if there are no distributed tables
|
||||
SELECT * FROM get_rebalance_table_shards_plan();
|
||||
table_name | shardid | shard_size | sourcename | sourceport | targetname | targetport
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
SELECT * FROM rebalance_table_shards();
|
||||
rebalance_table_shards
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
-- TODO: Figure out why this is necessary, rebalance_table_shards shouldn't
|
||||
-- insert stuff into pg_dist_colocation
|
||||
TRUNCATE pg_dist_colocation;
|
||||
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 1390000;
|
||||
SELECT * FROM citus_activate_node('localhost', :worker_2_port);
|
||||
WARNING: citus.enable_object_propagation is off, not creating distributed objects on worker
|
||||
DETAIL: distributed objects are only kept in sync when citus.enable_object_propagation is set to on. Newly activated nodes will not get these objects created
|
||||
|
|
|
@ -38,6 +38,15 @@ BEGIN;
|
|||
SET citus.replication_model to 'statement';
|
||||
ROLLBACK;
|
||||
|
||||
-- check that the rebalancer works even if there are no distributed tables
|
||||
SELECT * FROM get_rebalance_table_shards_plan();
|
||||
SELECT * FROM rebalance_table_shards();
|
||||
|
||||
-- TODO: Figure out why this is necessary, rebalance_table_shards shouldn't
|
||||
-- insert stuff into pg_dist_colocation
|
||||
TRUNCATE pg_dist_colocation;
|
||||
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 1390000;
|
||||
|
||||
SELECT * FROM citus_activate_node('localhost', :worker_2_port);
|
||||
CREATE TABLE cluster_management_test (col_1 text, col_2 int);
|
||||
SELECT create_distributed_table('cluster_management_test', 'col_1', 'hash');
|
||||
|
|
Loading…
Reference in New Issue