mirror of https://github.com/citusdata/citus.git
Prevent RF>1 on streaming replicated tables on `master_create_worker_shards`
Prior to this change, `master_create_worker_shards` command was not checking the replication model of the target table, thus allowing RF>1 with streaming replicated tables. With this change, `master_create_worker_shards` errors out on the case.pull/1195/head
parent
e6e5f63d9d
commit
e96e50746f
|
@ -97,6 +97,7 @@ CreateShardsWithRoundRobinPolicy(Oid distributedTableId, int32 shardCount,
|
||||||
uint64 hashTokenIncrement = 0;
|
uint64 hashTokenIncrement = 0;
|
||||||
List *existingShardList = NIL;
|
List *existingShardList = NIL;
|
||||||
int64 shardIndex = 0;
|
int64 shardIndex = 0;
|
||||||
|
DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(distributedTableId);
|
||||||
|
|
||||||
/* make sure table is hash partitioned */
|
/* make sure table is hash partitioned */
|
||||||
CheckHashPartitionedTable(distributedTableId);
|
CheckHashPartitionedTable(distributedTableId);
|
||||||
|
@ -138,6 +139,24 @@ CreateShardsWithRoundRobinPolicy(Oid distributedTableId, int32 shardCount,
|
||||||
errmsg("replication_factor must be positive")));
|
errmsg("replication_factor must be positive")));
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/* make sure that RF=1 if the table is streaming replicated */
|
||||||
|
if (cacheEntry->replicationModel == REPLICATION_MODEL_STREAMING &&
|
||||||
|
replicationFactor > 1)
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
|
||||||
|
errmsg("using replication factor %d with streaming "
|
||||||
|
"replicated tables are not supported",
|
||||||
|
replicationFactor),
|
||||||
|
errdetail("When master_create_distributed_table is called with "
|
||||||
|
"citus.replication_model streaming, then the table is "
|
||||||
|
"marked for streaming replication and the shard "
|
||||||
|
"replication factor of streaming replicated tables "
|
||||||
|
"must be 1."),
|
||||||
|
errhint("Use replication factor 1 or set "
|
||||||
|
"citus.replication_model to streaming and recreate the "
|
||||||
|
"table")));
|
||||||
|
}
|
||||||
|
|
||||||
/* calculate the split of the hash space */
|
/* calculate the split of the hash space */
|
||||||
hashTokenIncrement = HASH_TOKEN_COUNT / shardCount;
|
hashTokenIncrement = HASH_TOKEN_COUNT / shardCount;
|
||||||
|
|
||||||
|
|
|
@ -160,6 +160,26 @@ SELECT repmodel FROM pg_dist_partition WHERE logicalrelid='mx_table_test'::regcl
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
DROP TABLE mx_table_test;
|
DROP TABLE mx_table_test;
|
||||||
|
-- Show that master_create_distributed_table honors citus.replication_model GUC
|
||||||
|
CREATE TABLE s_table(a int);
|
||||||
|
SELECT master_create_distributed_table('s_table', 'a', 'hash');
|
||||||
|
master_create_distributed_table
|
||||||
|
---------------------------------
|
||||||
|
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
SELECT repmodel FROM pg_dist_partition WHERE logicalrelid='s_table'::regclass;
|
||||||
|
repmodel
|
||||||
|
----------
|
||||||
|
s
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- Show that master_create_worker_shards complains when RF>1 and replication model is streaming
|
||||||
|
SELECT master_create_worker_shards('s_table', 4, 2);
|
||||||
|
ERROR: using replication factor 2 with streaming replicated tables are not supported
|
||||||
|
DETAIL: When master_create_distributed_table is called with citus.replication_model streaming, then the table is marked for streaming replication and the shard replication factor of streaming replicated tables must be 1.
|
||||||
|
HINT: Use replication factor 1 or set citus.replication_model to streaming and recreate the table
|
||||||
|
DROP TABLE s_table;
|
||||||
RESET citus.replication_model;
|
RESET citus.replication_model;
|
||||||
-- Show that it is not possible to create an mx table with the old
|
-- Show that it is not possible to create an mx table with the old
|
||||||
-- master_create_distributed_table function
|
-- master_create_distributed_table function
|
||||||
|
|
|
@ -126,6 +126,16 @@ SELECT create_distributed_table('mx_table_test', 'col1');
|
||||||
SELECT repmodel FROM pg_dist_partition WHERE logicalrelid='mx_table_test'::regclass;
|
SELECT repmodel FROM pg_dist_partition WHERE logicalrelid='mx_table_test'::regclass;
|
||||||
DROP TABLE mx_table_test;
|
DROP TABLE mx_table_test;
|
||||||
|
|
||||||
|
-- Show that master_create_distributed_table honors citus.replication_model GUC
|
||||||
|
CREATE TABLE s_table(a int);
|
||||||
|
SELECT master_create_distributed_table('s_table', 'a', 'hash');
|
||||||
|
SELECT repmodel FROM pg_dist_partition WHERE logicalrelid='s_table'::regclass;
|
||||||
|
|
||||||
|
-- Show that master_create_worker_shards complains when RF>1 and replication model is streaming
|
||||||
|
SELECT master_create_worker_shards('s_table', 4, 2);
|
||||||
|
|
||||||
|
DROP TABLE s_table;
|
||||||
|
|
||||||
RESET citus.replication_model;
|
RESET citus.replication_model;
|
||||||
|
|
||||||
-- Show that it is not possible to create an mx table with the old
|
-- Show that it is not possible to create an mx table with the old
|
||||||
|
|
Loading…
Reference in New Issue