mirror of https://github.com/citusdata/citus.git
Add the shard rebalancer implementation
parent
d9a3e26f20
commit
011283122b
|
@ -407,6 +407,12 @@ workflows:
|
|||
image_tag: '11.9'
|
||||
make: check-worker
|
||||
requires: [build-11]
|
||||
- test-citus:
|
||||
name: 'test-11_check-operations'
|
||||
pg_major: 11
|
||||
image_tag: '11.9'
|
||||
make: check-operations
|
||||
requires: [build-11]
|
||||
- test-citus:
|
||||
name: 'test-11_check-follower-cluster'
|
||||
pg_major: 11
|
||||
|
@ -451,6 +457,12 @@ workflows:
|
|||
image_tag: '12.4'
|
||||
make: check-worker
|
||||
requires: [build-12]
|
||||
- test-citus:
|
||||
name: 'test-12_check-operations'
|
||||
pg_major: 12
|
||||
image_tag: '12.4'
|
||||
make: check-operations
|
||||
requires: [build-12]
|
||||
- test-citus:
|
||||
name: 'test-12_check-follower-cluster'
|
||||
pg_major: 12
|
||||
|
@ -507,6 +519,12 @@ workflows:
|
|||
image_tag: '13.0'
|
||||
make: check-worker
|
||||
requires: [build-13]
|
||||
- test-citus:
|
||||
name: 'test-13_check-operations'
|
||||
pg_major: 13
|
||||
image_tag: '13.0'
|
||||
make: check-operations
|
||||
requires: [build-13]
|
||||
- test-citus:
|
||||
name: 'test-13_check-follower-cluster'
|
||||
pg_major: 13
|
||||
|
|
|
@ -15,6 +15,8 @@
|
|||
#include "distributed/pg_version_constants.h"
|
||||
|
||||
#include "access/htup_details.h"
|
||||
#include "access/sysattr.h"
|
||||
#include "access/xact.h"
|
||||
#include "catalog/namespace.h"
|
||||
#include "catalog/pg_constraint.h"
|
||||
#if (PG_VERSION_NUM >= PG_VERSION_12)
|
||||
|
@ -23,13 +25,16 @@
|
|||
#include "catalog/pg_type.h"
|
||||
#include "distributed/colocation_utils.h"
|
||||
#include "distributed/commands.h"
|
||||
#include "distributed/coordinator_protocol.h"
|
||||
#include "distributed/listutils.h"
|
||||
#include "distributed/coordinator_protocol.h"
|
||||
#include "distributed/multi_join_order.h"
|
||||
#include "distributed/namespace_utils.h"
|
||||
#include "distributed/reference_table_utils.h"
|
||||
#include "distributed/version_compat.h"
|
||||
#include "utils/builtins.h"
|
||||
#include "utils/fmgroids.h"
|
||||
#include "utils/inval.h"
|
||||
#include "utils/lsyscache.h"
|
||||
#include "utils/rel.h"
|
||||
#include "utils/relcache.h"
|
||||
|
@ -69,6 +74,8 @@ static List * GetForeignKeyIdsForColumn(char *columnName, Oid relationId,
|
|||
static List * GetForeignConstraintCommandsInternal(Oid relationId, int flags);
|
||||
static Oid get_relation_constraint_oid_compat(HeapTuple heapTuple);
|
||||
static bool IsTableTypeIncluded(Oid relationId, int flags);
|
||||
static void UpdateConstraintIsValid(Oid constraintId, bool isValid);
|
||||
|
||||
|
||||
/*
|
||||
* ConstraintIsAForeignKeyToReferenceTable checks if the given constraint is a
|
||||
|
@ -1015,3 +1022,199 @@ IsTableTypeIncluded(Oid relationId, int flags)
|
|||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* GetForeignConstraintCommandsToReferenceTable takes in a shardInterval, and
|
||||
* returns the list of commands that are required to create the foreign
|
||||
* constraints for that shardInterval.
|
||||
*
|
||||
* The function does the following hack:
|
||||
* - Create the foreign constraints as INVALID on the shards
|
||||
* - Manually update pg_constraint to mark the same foreign
|
||||
* constraints as VALID
|
||||
*
|
||||
* We implement the above hack because we aim to skip the validation phase
|
||||
* of foreign keys to reference tables. The validation is pretty costly and
|
||||
* given that the source placements already valid, the validation in the
|
||||
* target nodes is useless.
|
||||
*
|
||||
* The function does not apply the same logic for the already invalid foreign
|
||||
* constraints.
|
||||
*/
|
||||
List *
|
||||
GetForeignConstraintCommandsToReferenceTable(ShardInterval *shardInterval)
|
||||
{
|
||||
ScanKeyData scanKey[1];
|
||||
int scanKeyCount = 1;
|
||||
uint64 shardId = shardInterval->shardId;
|
||||
Oid relationId = shardInterval->relationId;
|
||||
|
||||
List *commandList = NIL;
|
||||
|
||||
/*
|
||||
* Set search_path to NIL so that all objects outside of pg_catalog will be
|
||||
* schema-prefixed. pg_catalog will be added automatically when we call
|
||||
* PushOverrideSearchPath(), since we set addCatalog to true;
|
||||
*/
|
||||
OverrideSearchPath *overridePath = GetOverrideSearchPath(CurrentMemoryContext);
|
||||
overridePath->schemas = NIL;
|
||||
overridePath->addCatalog = true;
|
||||
PushOverrideSearchPath(overridePath);
|
||||
|
||||
/* open system catalog and scan all constraints that belong to this table */
|
||||
Relation pgConstraint = table_open(ConstraintRelationId, AccessShareLock);
|
||||
ScanKeyInit(&scanKey[0], Anum_pg_constraint_conrelid, BTEqualStrategyNumber, F_OIDEQ,
|
||||
relationId);
|
||||
|
||||
SysScanDesc scanDescriptor = systable_beginscan(pgConstraint,
|
||||
ConstraintRelidTypidNameIndexId,
|
||||
true, NULL, scanKeyCount, scanKey);
|
||||
|
||||
HeapTuple heapTuple = systable_getnext(scanDescriptor);
|
||||
while (HeapTupleIsValid(heapTuple))
|
||||
{
|
||||
Form_pg_constraint constraintForm = (Form_pg_constraint) GETSTRUCT(heapTuple);
|
||||
char *constraintDefinition = NULL;
|
||||
|
||||
|
||||
if (constraintForm->contype != CONSTRAINT_FOREIGN)
|
||||
{
|
||||
heapTuple = systable_getnext(scanDescriptor);
|
||||
continue;
|
||||
}
|
||||
|
||||
Oid referencedRelationId = constraintForm->confrelid;
|
||||
if (PartitionMethod(referencedRelationId) != DISTRIBUTE_BY_NONE)
|
||||
{
|
||||
heapTuple = systable_getnext(scanDescriptor);
|
||||
continue;
|
||||
}
|
||||
|
||||
Oid constraintId = get_relation_constraint_oid(relationId,
|
||||
constraintForm->conname.data,
|
||||
true);
|
||||
|
||||
int64 referencedShardId = GetFirstShardId(referencedRelationId);
|
||||
Oid referencedSchemaId = get_rel_namespace(referencedRelationId);
|
||||
char *referencedSchemaName = get_namespace_name(referencedSchemaId);
|
||||
char *escapedReferencedSchemaName = quote_literal_cstr(referencedSchemaName);
|
||||
|
||||
Oid schemaId = get_rel_namespace(relationId);
|
||||
char *schemaName = get_namespace_name(schemaId);
|
||||
char *escapedSchemaName = quote_literal_cstr(schemaName);
|
||||
|
||||
/*
|
||||
* We're first marking the constraint's valid field as invalid
|
||||
* and get the constraint definition. Later, we mark the constraint
|
||||
* as valid back with directly updating to pg_constraint.
|
||||
*/
|
||||
if (constraintForm->convalidated == true)
|
||||
{
|
||||
UpdateConstraintIsValid(constraintId, false);
|
||||
constraintDefinition = pg_get_constraintdef_command(constraintId);
|
||||
UpdateConstraintIsValid(constraintId, true);
|
||||
}
|
||||
else
|
||||
{
|
||||
/* if the constraint is not valid, simply do nothing special */
|
||||
constraintDefinition = pg_get_constraintdef_command(constraintId);
|
||||
}
|
||||
|
||||
StringInfo applyForeignConstraintCommand = makeStringInfo();
|
||||
appendStringInfo(applyForeignConstraintCommand,
|
||||
WORKER_APPLY_INTER_SHARD_DDL_COMMAND, shardId,
|
||||
escapedSchemaName, referencedShardId,
|
||||
escapedReferencedSchemaName,
|
||||
quote_literal_cstr(constraintDefinition));
|
||||
commandList = lappend(commandList, applyForeignConstraintCommand->data);
|
||||
|
||||
/* mark the constraint as valid again on the shard */
|
||||
if (constraintForm->convalidated == true)
|
||||
{
|
||||
StringInfo markConstraintValid = makeStringInfo();
|
||||
char *qualifiedReferencingShardName =
|
||||
ConstructQualifiedShardName(shardInterval);
|
||||
|
||||
char *shardConstraintName = pstrdup(constraintForm->conname.data);
|
||||
AppendShardIdToName(&shardConstraintName, shardId);
|
||||
|
||||
appendStringInfo(markConstraintValid,
|
||||
"UPDATE pg_constraint SET convalidated = true WHERE "
|
||||
"conrelid = %s::regclass AND conname = '%s'",
|
||||
quote_literal_cstr(qualifiedReferencingShardName),
|
||||
shardConstraintName);
|
||||
commandList = lappend(commandList, markConstraintValid->data);
|
||||
}
|
||||
|
||||
heapTuple = systable_getnext(scanDescriptor);
|
||||
}
|
||||
|
||||
/* clean up scan and close system catalog */
|
||||
systable_endscan(scanDescriptor);
|
||||
table_close(pgConstraint, AccessShareLock);
|
||||
|
||||
/* revert back to original search_path */
|
||||
PopOverrideSearchPath();
|
||||
|
||||
return commandList;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* UpdateConstraintIsValid is a utility function with sets the
|
||||
* pg_constraint.convalidated to the given isValid for the given
|
||||
* constraintId.
|
||||
*
|
||||
* This function should be called with caution because if used wrong
|
||||
* could lead to data inconsistencies.
|
||||
*/
|
||||
static void
|
||||
UpdateConstraintIsValid(Oid constraintId, bool isValid)
|
||||
{
|
||||
HeapTuple heapTuple = NULL;
|
||||
SysScanDesc scanDescriptor;
|
||||
ScanKeyData scankey[1];
|
||||
Relation pgConstraint = table_open(ConstraintRelationId, AccessShareLock);
|
||||
TupleDesc tupleDescriptor = RelationGetDescr(pgConstraint);
|
||||
Datum values[Natts_pg_constraint];
|
||||
bool isnull[Natts_pg_constraint];
|
||||
bool replace[Natts_pg_constraint];
|
||||
|
||||
ScanKeyInit(&scankey[0],
|
||||
#if PG_VERSION_NUM >= 120000
|
||||
Anum_pg_constraint_oid,
|
||||
#else
|
||||
ObjectIdAttributeNumber,
|
||||
#endif
|
||||
BTEqualStrategyNumber, F_OIDEQ,
|
||||
ObjectIdGetDatum(constraintId));
|
||||
|
||||
scanDescriptor = systable_beginscan(pgConstraint,
|
||||
ConstraintOidIndexId,
|
||||
true,
|
||||
NULL,
|
||||
1,
|
||||
scankey);
|
||||
heapTuple = systable_getnext(scanDescriptor);
|
||||
if (!HeapTupleIsValid(heapTuple))
|
||||
{
|
||||
elog(ERROR, "could not find tuple for constraint %u", constraintId);
|
||||
}
|
||||
|
||||
memset(replace, 0, sizeof(replace));
|
||||
|
||||
values[Anum_pg_constraint_convalidated - 1] = BoolGetDatum(isValid);
|
||||
isnull[Anum_pg_constraint_convalidated - 1] = false;
|
||||
replace[Anum_pg_constraint_convalidated - 1] = true;
|
||||
|
||||
heapTuple = heap_modify_tuple(heapTuple, tupleDescriptor, values, isnull, replace);
|
||||
|
||||
CatalogTupleUpdate(pgConstraint, &heapTuple->t_self, heapTuple);
|
||||
|
||||
CacheInvalidateHeapTuple(pgConstraint, heapTuple, NULL);
|
||||
CommandCounterIncrement();
|
||||
|
||||
systable_endscan(scanDescriptor);
|
||||
table_close(pgConstraint, NoLock);
|
||||
}
|
||||
|
|
|
@ -132,6 +132,7 @@ typedef struct MetadataCacheData
|
|||
bool extensionLoaded;
|
||||
Oid distShardRelationId;
|
||||
Oid distPlacementRelationId;
|
||||
Oid distRebalanceStrategyRelationId;
|
||||
Oid distNodeRelationId;
|
||||
Oid distNodeNodeIdIndexId;
|
||||
Oid distLocalGroupRelationId;
|
||||
|
@ -2061,6 +2062,17 @@ DistLocalGroupIdRelationId(void)
|
|||
}
|
||||
|
||||
|
||||
/* return oid of pg_dist_rebalance_strategy relation */
|
||||
Oid
|
||||
DistRebalanceStrategyRelationId(void)
|
||||
{
|
||||
CachedRelationLookup("pg_dist_rebalance_strategy",
|
||||
&MetadataCache.distRebalanceStrategyRelationId);
|
||||
|
||||
return MetadataCache.distRebalanceStrategyRelationId;
|
||||
}
|
||||
|
||||
|
||||
/* return the oid of citus namespace */
|
||||
Oid
|
||||
CitusCatalogNamespaceId(void)
|
||||
|
@ -3939,6 +3951,37 @@ LookupShardRelationFromCatalog(int64 shardId, bool missingOk)
|
|||
}
|
||||
|
||||
|
||||
/*
|
||||
* ShardExists returns whether the given shard ID exists in pg_dist_shard.
|
||||
*/
|
||||
bool
|
||||
ShardExists(int64 shardId)
|
||||
{
|
||||
ScanKeyData scanKey[1];
|
||||
int scanKeyCount = 1;
|
||||
Relation pgDistShard = table_open(DistShardRelationId(), AccessShareLock);
|
||||
bool shardExists = false;
|
||||
|
||||
ScanKeyInit(&scanKey[0], Anum_pg_dist_shard_shardid,
|
||||
BTEqualStrategyNumber, F_INT8EQ, Int64GetDatum(shardId));
|
||||
|
||||
SysScanDesc scanDescriptor = systable_beginscan(pgDistShard,
|
||||
DistShardShardidIndexId(), true,
|
||||
NULL, scanKeyCount, scanKey);
|
||||
|
||||
HeapTuple heapTuple = systable_getnext(scanDescriptor);
|
||||
if (HeapTupleIsValid(heapTuple))
|
||||
{
|
||||
shardExists = true;
|
||||
}
|
||||
|
||||
systable_endscan(scanDescriptor);
|
||||
table_close(pgDistShard, NoLock);
|
||||
|
||||
return shardExists;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* GetPartitionTypeInputInfo populates output parameters with the interval type
|
||||
* identifier and modifier for the specified partition key/method combination.
|
||||
|
|
|
@ -902,6 +902,46 @@ AllShardPlacementsOnNodeGroup(int32 groupId)
|
|||
}
|
||||
|
||||
|
||||
/*
|
||||
* AllShardPlacementsWithShardPlacementState finds shard placements with the given
|
||||
* shardState from system catalogs, converts these placements to their in-memory
|
||||
* representation, and returns the converted shard placements in a new list.
|
||||
*/
|
||||
List *
|
||||
AllShardPlacementsWithShardPlacementState(ShardState shardState)
|
||||
{
|
||||
List *shardPlacementList = NIL;
|
||||
ScanKeyData scanKey[1];
|
||||
int scanKeyCount = 1;
|
||||
|
||||
Relation pgPlacement = table_open(DistPlacementRelationId(), AccessShareLock);
|
||||
|
||||
ScanKeyInit(&scanKey[0], Anum_pg_dist_placement_shardstate,
|
||||
BTEqualStrategyNumber, F_INT4EQ, Int32GetDatum(shardState));
|
||||
|
||||
SysScanDesc scanDescriptor = systable_beginscan(pgPlacement, InvalidOid, false,
|
||||
NULL, scanKeyCount, scanKey);
|
||||
|
||||
HeapTuple heapTuple = systable_getnext(scanDescriptor);
|
||||
while (HeapTupleIsValid(heapTuple))
|
||||
{
|
||||
TupleDesc tupleDescriptor = RelationGetDescr(pgPlacement);
|
||||
|
||||
GroupShardPlacement *placement =
|
||||
TupleToGroupShardPlacement(tupleDescriptor, heapTuple);
|
||||
|
||||
shardPlacementList = lappend(shardPlacementList, placement);
|
||||
|
||||
heapTuple = systable_getnext(scanDescriptor);
|
||||
}
|
||||
|
||||
systable_endscan(scanDescriptor);
|
||||
table_close(pgPlacement, NoLock);
|
||||
|
||||
return shardPlacementList;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* TupleToGroupShardPlacement takes in a heap tuple from pg_dist_placement,
|
||||
* and converts this tuple to in-memory struct. The function assumes the
|
||||
|
|
|
@ -11,18 +11,21 @@
|
|||
*/
|
||||
|
||||
#include "postgres.h"
|
||||
#include "c.h"
|
||||
#include "fmgr.h"
|
||||
#include "miscadmin.h"
|
||||
|
||||
#include <string.h>
|
||||
|
||||
#include "access/htup_details.h"
|
||||
#include "catalog/pg_class.h"
|
||||
#include "catalog/pg_enum.h"
|
||||
#include "distributed/citus_ruleutils.h"
|
||||
#include "distributed/colocation_utils.h"
|
||||
#include "distributed/commands.h"
|
||||
#include "distributed/connection_management.h"
|
||||
#include "distributed/distributed_planner.h"
|
||||
#include "distributed/listutils.h"
|
||||
#include "distributed/shard_cleaner.h"
|
||||
#include "distributed/coordinator_protocol.h"
|
||||
#include "distributed/metadata_cache.h"
|
||||
#include "distributed/metadata_sync.h"
|
||||
|
@ -35,6 +38,7 @@
|
|||
#include "distributed/worker_transaction.h"
|
||||
#include "lib/stringinfo.h"
|
||||
#include "nodes/pg_list.h"
|
||||
#include "storage/lmgr.h"
|
||||
#include "storage/lock.h"
|
||||
#include "storage/lmgr.h"
|
||||
#include "utils/builtins.h"
|
||||
|
@ -42,6 +46,8 @@
|
|||
#include "utils/errcodes.h"
|
||||
#include "utils/lsyscache.h"
|
||||
#include "utils/palloc.h"
|
||||
#include "utils/rel.h"
|
||||
#include "utils/syscache.h"
|
||||
|
||||
/* local function forward declarations */
|
||||
static void ErrorIfTableCannotBeReplicated(Oid relationId);
|
||||
|
@ -65,15 +71,27 @@ static void EnsureShardCanBeCopied(int64 shardId, const char *sourceNodeName,
|
|||
int32 sourceNodePort, const char *targetNodeName,
|
||||
int32 targetNodePort);
|
||||
static List * RecreateTableDDLCommandList(Oid relationId);
|
||||
static List * WorkerApplyShardDDLCommandList(List *ddlCommandList, int64 shardId);
|
||||
static void EnsureTableListOwner(List *tableIdList);
|
||||
static void EnsureTableListSuitableForReplication(List *tableIdList);
|
||||
|
||||
static void DropColocatedShardPlacement(ShardInterval *shardInterval, char *nodeName,
|
||||
int32 nodePort);
|
||||
static void MarkForDropColocatedShardPlacement(ShardInterval *shardInterval,
|
||||
char *nodeName, int32 nodePort);
|
||||
static void UpdateColocatedShardPlacementMetadataOnWorkers(int64 shardId,
|
||||
char *sourceNodeName,
|
||||
int32 sourceNodePort,
|
||||
char *targetNodeName,
|
||||
int32 targetNodePort);
|
||||
|
||||
/* declarations for dynamic loading */
|
||||
PG_FUNCTION_INFO_V1(master_copy_shard_placement);
|
||||
PG_FUNCTION_INFO_V1(master_move_shard_placement);
|
||||
|
||||
|
||||
bool DeferShardDeleteOnMove = false;
|
||||
|
||||
|
||||
/*
|
||||
* master_copy_shard_placement implements a user-facing UDF to repair data from
|
||||
* a healthy (source) node to an inactive (target) node. To accomplish this it
|
||||
|
@ -98,16 +116,15 @@ master_copy_shard_placement(PG_FUNCTION_ARGS)
|
|||
char *sourceNodeName = text_to_cstring(sourceNodeNameText);
|
||||
char *targetNodeName = text_to_cstring(targetNodeNameText);
|
||||
|
||||
CheckCitusVersion(ERROR);
|
||||
EnsureCoordinator();
|
||||
CheckCitusVersion(ERROR);
|
||||
|
||||
char shardReplicationMode = LookupShardTransferMode(shardReplicationModeOid);
|
||||
if (shardReplicationMode == TRANSFER_MODE_FORCE_LOGICAL)
|
||||
{
|
||||
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
||||
errmsg("using logical replication in "
|
||||
"master_copy_shard_placement() requires Citus "
|
||||
"Enterprise")));
|
||||
errmsg("the force_logical transfer mode is currently "
|
||||
"unsupported")));
|
||||
}
|
||||
|
||||
ShardInterval *shardInterval = LoadShardInterval(shardId);
|
||||
|
@ -131,19 +148,155 @@ master_copy_shard_placement(PG_FUNCTION_ARGS)
|
|||
|
||||
/*
|
||||
* master_move_shard_placement moves given shard (and its co-located shards) from one
|
||||
* node to the other node.
|
||||
* node to the other node. To accomplish this it entirely recreates the table structure
|
||||
* before copying all data.
|
||||
*
|
||||
* After that, there are two different paths. First one is blocking shard move in the
|
||||
* sense that during shard move all modifications are paused to the shard. The second
|
||||
* one relies on logical replication meaning that the writes blocked only for a very
|
||||
* short duration almost only when the metadata is actually being updated. This option
|
||||
* is currently only available in Citus Enterprise.
|
||||
*
|
||||
* After successful move operation, shards in the source node gets deleted. If the move
|
||||
* fails at any point, this function throws an error, leaving the cluster without doing
|
||||
* any changes in source node or target node.
|
||||
*/
|
||||
Datum
|
||||
master_move_shard_placement(PG_FUNCTION_ARGS)
|
||||
{
|
||||
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
||||
errmsg("master_move_shard_placement() is only supported on "
|
||||
"Citus Enterprise")));
|
||||
int64 shardId = PG_GETARG_INT64(0);
|
||||
char *sourceNodeName = text_to_cstring(PG_GETARG_TEXT_P(1));
|
||||
int32 sourceNodePort = PG_GETARG_INT32(2);
|
||||
char *targetNodeName = text_to_cstring(PG_GETARG_TEXT_P(3));
|
||||
int32 targetNodePort = PG_GETARG_INT32(4);
|
||||
Oid shardReplicationModeOid = PG_GETARG_OID(5);
|
||||
|
||||
|
||||
ListCell *colocatedTableCell = NULL;
|
||||
ListCell *colocatedShardCell = NULL;
|
||||
|
||||
|
||||
CheckCitusVersion(ERROR);
|
||||
EnsureCoordinator();
|
||||
|
||||
Oid relationId = RelationIdForShard(shardId);
|
||||
ErrorIfMoveCitusLocalTable(relationId);
|
||||
|
||||
ShardInterval *shardInterval = LoadShardInterval(shardId);
|
||||
Oid distributedTableId = shardInterval->relationId;
|
||||
|
||||
List *colocatedTableList = ColocatedTableList(distributedTableId);
|
||||
List *colocatedShardList = ColocatedShardIntervalList(shardInterval);
|
||||
|
||||
foreach(colocatedTableCell, colocatedTableList)
|
||||
{
|
||||
Oid colocatedTableId = lfirst_oid(colocatedTableCell);
|
||||
char relationKind = '\0';
|
||||
|
||||
/* check that user has owner rights in all co-located tables */
|
||||
EnsureTableOwner(colocatedTableId);
|
||||
|
||||
/*
|
||||
* Block concurrent DDL / TRUNCATE commands on the relation. Similarly,
|
||||
* block concurrent master_move_shard_placement() on any shard of
|
||||
* the same relation. This is OK for now since we're executing shard
|
||||
* moves sequentially anyway.
|
||||
*/
|
||||
LockRelationOid(colocatedTableId, ShareUpdateExclusiveLock);
|
||||
|
||||
relationKind = get_rel_relkind(colocatedTableId);
|
||||
if (relationKind == RELKIND_FOREIGN_TABLE)
|
||||
{
|
||||
char *relationName = get_rel_name(colocatedTableId);
|
||||
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
||||
errmsg("cannot repair shard"),
|
||||
errdetail("Table %s is a foreign table. Repairing "
|
||||
"shards backed by foreign tables is "
|
||||
"not supported.", relationName)));
|
||||
}
|
||||
}
|
||||
|
||||
/* we sort colocatedShardList so that lock operations will not cause any deadlocks */
|
||||
colocatedShardList = SortList(colocatedShardList, CompareShardIntervalsById);
|
||||
foreach(colocatedShardCell, colocatedShardList)
|
||||
{
|
||||
ShardInterval *colocatedShard = (ShardInterval *) lfirst(colocatedShardCell);
|
||||
uint64 colocatedShardId = colocatedShard->shardId;
|
||||
|
||||
EnsureShardCanBeCopied(colocatedShardId, sourceNodeName, sourceNodePort,
|
||||
targetNodeName, targetNodePort);
|
||||
}
|
||||
|
||||
char shardReplicationMode = LookupShardTransferMode(shardReplicationModeOid);
|
||||
if (shardReplicationMode == TRANSFER_MODE_FORCE_LOGICAL)
|
||||
{
|
||||
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
||||
errmsg("the force_logical transfer mode is currently "
|
||||
"unsupported")));
|
||||
}
|
||||
|
||||
BlockWritesToShardList(colocatedShardList);
|
||||
|
||||
/*
|
||||
* CopyColocatedShardPlacement function copies given shard with its co-located
|
||||
* shards.
|
||||
*/
|
||||
CopyShardTables(colocatedShardList, sourceNodeName, sourceNodePort, targetNodeName,
|
||||
targetNodePort);
|
||||
|
||||
ShardInterval *colocatedShard = NULL;
|
||||
foreach_ptr(colocatedShard, colocatedShardList)
|
||||
{
|
||||
uint64 colocatedShardId = colocatedShard->shardId;
|
||||
uint32 groupId = GroupForNode(targetNodeName, targetNodePort);
|
||||
uint64 placementId = GetNextPlacementId();
|
||||
|
||||
InsertShardPlacementRow(colocatedShardId, placementId,
|
||||
SHARD_STATE_ACTIVE, ShardLength(colocatedShardId),
|
||||
groupId);
|
||||
}
|
||||
|
||||
/* since this is move operation, we remove shards from source node after copy */
|
||||
if (DeferShardDeleteOnMove)
|
||||
{
|
||||
MarkForDropColocatedShardPlacement(shardInterval, sourceNodeName, sourceNodePort);
|
||||
}
|
||||
else
|
||||
{
|
||||
DropColocatedShardPlacement(shardInterval, sourceNodeName, sourceNodePort);
|
||||
}
|
||||
|
||||
UpdateColocatedShardPlacementMetadataOnWorkers(shardId, sourceNodeName,
|
||||
sourceNodePort, targetNodeName,
|
||||
targetNodePort);
|
||||
|
||||
PG_RETURN_VOID();
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* BlockWritesToShardList blocks writes to all shards in the given shard
|
||||
* ErrorIfMoveCitusLocalTable is a helper function for rebalance_table_shards
|
||||
* and master_move_shard_placement udf's to error out if relation with relationId
|
||||
* is a citus local table.
|
||||
*/
|
||||
void
|
||||
ErrorIfMoveCitusLocalTable(Oid relationId)
|
||||
{
|
||||
if (!IsCitusTableType(relationId, CITUS_LOCAL_TABLE))
|
||||
{
|
||||
return;
|
||||
}
|
||||
|
||||
char *qualifiedRelationName = generate_qualified_relation_name(relationId);
|
||||
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
||||
errmsg("table %s is a citus local table, moving shard of "
|
||||
"a citus local table is currently not supported",
|
||||
qualifiedRelationName)));
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* BlockWritesToColocatedShardList blocks writes to all shards in the given shard
|
||||
* list. The function assumes that all the shards in the list are colocated.
|
||||
*/
|
||||
void
|
||||
|
@ -185,8 +338,11 @@ BlockWritesToShardList(List *shardList)
|
|||
|
||||
/*
|
||||
* ErrorIfTableCannotBeReplicated function errors out if the given table is not suitable
|
||||
* for its shard being replicated. Shard replications is not allowed only for MX tables,
|
||||
* since RF=1 is a must MX tables.
|
||||
* for its shard being replicated. There are 2 cases in which shard replication is not
|
||||
* allowed:
|
||||
*
|
||||
* 1) MX tables, since RF=1 is a must MX tables
|
||||
* 2) Reference tables, since the shard should already exist in all workers
|
||||
*/
|
||||
static void
|
||||
ErrorIfTableCannotBeReplicated(Oid relationId)
|
||||
|
@ -336,6 +492,7 @@ RepairShardPlacement(int64 shardId, const char *sourceNodeName, int32 sourceNode
|
|||
/* we generate necessary commands to recreate the shard in target node */
|
||||
List *ddlCommandList =
|
||||
CopyShardCommandList(shardInterval, sourceNodeName, sourceNodePort, includeData);
|
||||
|
||||
List *foreignConstraintCommandList = CopyShardForeignConstraintCommandList(
|
||||
shardInterval);
|
||||
ddlCommandList = list_concat(ddlCommandList, foreignConstraintCommandList);
|
||||
|
@ -502,15 +659,7 @@ EnsureTableListSuitableForReplication(List *tableIdList)
|
|||
IsCitusTableType(tableId, DISTRIBUTED_TABLE))
|
||||
{
|
||||
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
||||
errmsg("cannot create foreign key constraint"),
|
||||
errdetail("This shard has foreign constraints on it. "
|
||||
"Citus currently supports "
|
||||
"foreign key constraints only for "
|
||||
"\"citus.shard_replication_factor = 1\"."),
|
||||
errhint("Please change \"citus.shard_replication_factor to "
|
||||
"1\". To learn more about using foreign keys with "
|
||||
"other replication factors, please contact us at "
|
||||
"https://citusdata.com/about/contact_us.")));
|
||||
errmsg("cannot replicate shards with foreign keys")));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -553,21 +702,12 @@ CopyShardTables(List *shardIntervalList, char *sourceNodeName, int32 sourceNodeP
|
|||
*
|
||||
* Iterate through the colocated shards and create the foreign constraints and
|
||||
* attach child tables to their parents in a partitioning hierarchy.
|
||||
*
|
||||
* Note: After implementing foreign constraints from distributed to reference
|
||||
* tables, we have decided to not create foreign constraints from hash
|
||||
* distributed to reference tables at this stage for nonblocking rebalancer.
|
||||
* We just create the co-located ones here. We add the foreign constraints
|
||||
* from hash distributed to reference tables after being completely done with
|
||||
* the copy procedure inside LogicallyReplicateShards. The reason is that,
|
||||
* the reference tables have placements in both source and target workers and
|
||||
* the copied shard would get updated twice because of a cascading DML coming
|
||||
* from both of the placements.
|
||||
*/
|
||||
foreach_ptr(shardInterval, shardIntervalList)
|
||||
{
|
||||
List *shardForeignConstraintCommandList = NIL;
|
||||
List *referenceTableForeignConstraintList = NIL;
|
||||
|
||||
char *tableOwner = TableOwner(shardInterval->relationId);
|
||||
|
||||
CopyShardForeignConstraintCommandListGrouped(shardInterval,
|
||||
|
@ -988,12 +1128,109 @@ RecreateTableDDLCommandList(Oid relationId)
|
|||
}
|
||||
|
||||
|
||||
/*
|
||||
* DropColocatedShardPlacement deletes the shard placement metadata for the given shard
|
||||
* placement from the pg_dist_placement, and then it drops the shard table
|
||||
* from the given node. The function does this for all colocated placements.
|
||||
*/
|
||||
static void
|
||||
DropColocatedShardPlacement(ShardInterval *shardInterval, char *nodeName, int32 nodePort)
|
||||
{
|
||||
List *colocatedShardList = ColocatedShardIntervalList(shardInterval);
|
||||
ListCell *colocatedShardCell = NULL;
|
||||
|
||||
foreach(colocatedShardCell, colocatedShardList)
|
||||
{
|
||||
ShardInterval *colocatedShard = (ShardInterval *) lfirst(colocatedShardCell);
|
||||
char *qualifiedTableName = ConstructQualifiedShardName(colocatedShard);
|
||||
StringInfo dropQuery = makeStringInfo();
|
||||
uint64 shardId = colocatedShard->shardId;
|
||||
List *shardPlacementList = ShardPlacementList(shardId);
|
||||
ShardPlacement *placement =
|
||||
SearchShardPlacementInListOrError(shardPlacementList, nodeName, nodePort);
|
||||
|
||||
appendStringInfo(dropQuery, DROP_REGULAR_TABLE_COMMAND, qualifiedTableName);
|
||||
|
||||
DeleteShardPlacementRow(placement->placementId);
|
||||
SendCommandToWorker(nodeName, nodePort, dropQuery->data);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* MarkForDropColocatedShardPlacement marks the shard placement metadata for the given
|
||||
* shard placement to be deleted in pg_dist_placement. The function does this for all
|
||||
* colocated placements.
|
||||
*/
|
||||
static void
|
||||
MarkForDropColocatedShardPlacement(ShardInterval *shardInterval, char *nodeName, int32
|
||||
nodePort)
|
||||
{
|
||||
List *colocatedShardList = ColocatedShardIntervalList(shardInterval);
|
||||
ListCell *colocatedShardCell = NULL;
|
||||
|
||||
foreach(colocatedShardCell, colocatedShardList)
|
||||
{
|
||||
ShardInterval *colocatedShard = (ShardInterval *) lfirst(colocatedShardCell);
|
||||
uint64 shardId = colocatedShard->shardId;
|
||||
List *shardPlacementList = ShardPlacementList(shardId);
|
||||
ShardPlacement *placement =
|
||||
SearchShardPlacementInListOrError(shardPlacementList, nodeName, nodePort);
|
||||
|
||||
UpdateShardPlacementState(placement->placementId, SHARD_STATE_TO_DELETE);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* UpdateColocatedShardPlacementMetadataOnWorkers updates the metadata about the
|
||||
* placements of the given shard and its colocated shards by changing the nodename and
|
||||
* nodeport of the shards from the source nodename/port to target nodename/port.
|
||||
*
|
||||
* Note that the function does nothing if the given shard belongs to a non-mx table.
|
||||
*/
|
||||
static void
|
||||
UpdateColocatedShardPlacementMetadataOnWorkers(int64 shardId,
|
||||
char *sourceNodeName, int32 sourceNodePort,
|
||||
char *targetNodeName, int32 targetNodePort)
|
||||
{
|
||||
ShardInterval *shardInterval = LoadShardInterval(shardId);
|
||||
ListCell *colocatedShardCell = NULL;
|
||||
bool shouldSyncMetadata = ShouldSyncTableMetadata(shardInterval->relationId);
|
||||
|
||||
if (!shouldSyncMetadata)
|
||||
{
|
||||
return;
|
||||
}
|
||||
|
||||
List *colocatedShardList = ColocatedShardIntervalList(shardInterval);
|
||||
|
||||
/* iterate through the colocated shards and copy each */
|
||||
foreach(colocatedShardCell, colocatedShardList)
|
||||
{
|
||||
ShardInterval *colocatedShard = (ShardInterval *) lfirst(colocatedShardCell);
|
||||
StringInfo updateCommand = makeStringInfo();
|
||||
|
||||
appendStringInfo(updateCommand, "UPDATE pg_dist_shard_placement "
|
||||
"SET nodename=%s, nodeport=%d WHERE "
|
||||
"shardid=%lu AND nodename=%s AND nodeport=%d",
|
||||
quote_literal_cstr(targetNodeName),
|
||||
targetNodePort,
|
||||
colocatedShard->shardId,
|
||||
quote_literal_cstr(sourceNodeName),
|
||||
sourceNodePort);
|
||||
|
||||
SendCommandToWorkersWithMetadata(updateCommand->data);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* WorkerApplyShardDDLCommandList wraps all DDL commands in ddlCommandList
|
||||
* in a call to worker_apply_shard_ddl_command to apply the DDL command to
|
||||
* the shard specified by shardId.
|
||||
*/
|
||||
static List *
|
||||
List *
|
||||
WorkerApplyShardDDLCommandList(List *ddlCommandList, int64 shardId)
|
||||
{
|
||||
List *applyDDLCommandList = NIL;
|
||||
|
|
|
@ -0,0 +1,144 @@
|
|||
/*-------------------------------------------------------------------------
|
||||
*
|
||||
* shard_cleaner.c
|
||||
* This implements the background process that cleans shards that are
|
||||
* left around. Shards that are left around are marked as state 4
|
||||
* (SHARD_STATE_TO_DELETE) in pg_dist_placement.
|
||||
*
|
||||
* Copyright (c), Citus Data, Inc.
|
||||
*
|
||||
*-------------------------------------------------------------------------
|
||||
*/
|
||||
|
||||
#include "postgres.h"
|
||||
|
||||
|
||||
#include "distributed/coordinator_protocol.h"
|
||||
#include "distributed/metadata_cache.h"
|
||||
#include "distributed/shard_cleaner.h"
|
||||
#include "distributed/worker_transaction.h"
|
||||
|
||||
|
||||
/* declarations for dynamic loading */
|
||||
PG_FUNCTION_INFO_V1(master_defer_delete_shards);
|
||||
|
||||
|
||||
static int DropMarkedShards(void);
|
||||
|
||||
|
||||
/*
|
||||
* master_defer_delete_shards implements a user-facing UDF to deleter orphaned shards that
|
||||
* are still haning around in the system. These shards are orphaned by previous actions
|
||||
* that were not directly able to delete the placements eg. shard moving or dropping of a
|
||||
* distributed table while one of the data nodes was not online.
|
||||
*
|
||||
* This function iterates through placements where shardstate is SHARD_STATE_TO_DELETE
|
||||
* (shardstate = 4), drops the corresponding tables from the node and removes the
|
||||
* placement information from the catalog.
|
||||
*
|
||||
* The function takes no arguments and runs cluster wide
|
||||
*/
|
||||
Datum
|
||||
master_defer_delete_shards(PG_FUNCTION_ARGS)
|
||||
{
|
||||
CheckCitusVersion(ERROR);
|
||||
EnsureCoordinator();
|
||||
|
||||
int droppedShardCount = DropMarkedShards();
|
||||
|
||||
PG_RETURN_INT32(droppedShardCount);
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* TryDropMarkedShards is a wrapper around DropMarkedShards that catches
|
||||
* any errors to make it safe to use in the maintenance daemon.
|
||||
*/
|
||||
int
|
||||
TryDropMarkedShards(void)
|
||||
{
|
||||
int droppedShardCount = 0;
|
||||
MemoryContext savedContext = CurrentMemoryContext;
|
||||
|
||||
PG_TRY();
|
||||
{
|
||||
droppedShardCount = DropMarkedShards();
|
||||
}
|
||||
PG_CATCH();
|
||||
{
|
||||
MemoryContextSwitchTo(savedContext);
|
||||
ErrorData *edata = CopyErrorData();
|
||||
FlushErrorState();
|
||||
|
||||
/* rethrow as WARNING */
|
||||
edata->elevel = WARNING;
|
||||
ThrowErrorData(edata);
|
||||
}
|
||||
PG_END_TRY();
|
||||
|
||||
return droppedShardCount;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* DropMarkedShards removes shards that were marked SHARD_STATE_TO_DELETE before.
|
||||
*
|
||||
* It does so by taking an exclusive lock on the shard and its colocated
|
||||
* placements before removing. If the lock cannot be obtained it skips the
|
||||
* group and continues with others. The group that has been skipped will be
|
||||
* removed at a later time when there are no locks held anymore on those
|
||||
* placements.
|
||||
*/
|
||||
static int
|
||||
DropMarkedShards(void)
|
||||
{
|
||||
int removedShardCount = 0;
|
||||
ListCell *shardPlacementCell = NULL;
|
||||
|
||||
if (!IsCoordinator())
|
||||
{
|
||||
return removedShardCount;
|
||||
}
|
||||
|
||||
List *shardPlacementList = AllShardPlacementsWithShardPlacementState(
|
||||
SHARD_STATE_TO_DELETE);
|
||||
foreach(shardPlacementCell, shardPlacementList)
|
||||
{
|
||||
GroupShardPlacement *placement = (GroupShardPlacement *) lfirst(
|
||||
shardPlacementCell);
|
||||
|
||||
if (!PrimaryNodeForGroup(placement->groupId, NULL) ||
|
||||
!ShardExists(placement->shardId))
|
||||
{
|
||||
continue;
|
||||
}
|
||||
|
||||
ShardPlacement *shardPlacement = LoadShardPlacement(placement->shardId,
|
||||
placement->placementId);
|
||||
ShardInterval *shardInterval = LoadShardInterval(shardPlacement->shardId);
|
||||
|
||||
ereport(LOG, (errmsg("dropping shard placement " INT64_FORMAT " of shard "
|
||||
INT64_FORMAT " on %s:%d after it was moved away",
|
||||
shardPlacement->placementId, shardPlacement->shardId,
|
||||
shardPlacement->nodeName, shardPlacement->nodePort)));
|
||||
|
||||
/* prepare sql query to execute to drop the shard */
|
||||
StringInfo dropQuery = makeStringInfo();
|
||||
char *qualifiedTableName = ConstructQualifiedShardName(shardInterval);
|
||||
appendStringInfo(dropQuery, DROP_REGULAR_TABLE_COMMAND, qualifiedTableName);
|
||||
|
||||
List *dropCommandList = list_make2("SET LOCAL lock_timeout TO '1s'",
|
||||
dropQuery->data);
|
||||
|
||||
/* remove the shard from the node and the placement information */
|
||||
SendCommandListToWorkerInSingleTransaction(shardPlacement->nodeName,
|
||||
shardPlacement->nodePort,
|
||||
NULL, dropCommandList);
|
||||
|
||||
DeleteShardPlacementRow(placement->placementId);
|
||||
|
||||
removedShardCount++;
|
||||
}
|
||||
|
||||
return removedShardCount;
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -60,6 +60,7 @@
|
|||
#include "distributed/reference_table_utils.h"
|
||||
#include "distributed/relation_access_tracking.h"
|
||||
#include "distributed/run_from_same_connection.h"
|
||||
#include "distributed/shard_cleaner.h"
|
||||
#include "distributed/shared_connection_stats.h"
|
||||
#include "distributed/query_pushdown_planning.h"
|
||||
#include "distributed/time_constants.h"
|
||||
|
@ -890,6 +891,38 @@ RegisterCitusConfigVariables(void)
|
|||
GUC_UNIT_MS | GUC_NO_SHOW_ALL,
|
||||
NULL, NULL, NULL);
|
||||
|
||||
DefineCustomBoolVariable(
|
||||
"citus.defer_drop_after_shard_move",
|
||||
gettext_noop("When enabled a shard move will mark old shards for deletion"),
|
||||
gettext_noop("The deletion of a shard can sometimes run into a conflict with a "
|
||||
"long running transactions on a the shard during the drop phase of "
|
||||
"the shard move. This causes some moves to be rolled back after "
|
||||
"resources have been spend on moving the shard. To prevent "
|
||||
"conflicts this feature lets you skip the actual deletion till a "
|
||||
"later point in time. When used one should set "
|
||||
"citus.defer_shard_delete_interval to make sure defered deletions "
|
||||
"will be executed"),
|
||||
&DeferShardDeleteOnMove,
|
||||
false,
|
||||
PGC_USERSET,
|
||||
0,
|
||||
NULL, NULL, NULL);
|
||||
|
||||
DefineCustomIntVariable(
|
||||
"citus.defer_shard_delete_interval",
|
||||
gettext_noop("Sets the time to wait between background deletion for shards."),
|
||||
gettext_noop("Shards that are marked for deferred deletion need to be deleted in "
|
||||
"the background at a later time. This is done at a regular interval "
|
||||
"configured here. The deletion is executed optimistically, it tries "
|
||||
"to take a lock on a shard to clean, if the lock can't be acquired "
|
||||
"the background worker moves on. When set to -1 this background "
|
||||
"process is skipped."),
|
||||
&DeferShardDeleteInterval,
|
||||
-1, -1, 7 * 24 * 3600 * 1000,
|
||||
PGC_SIGHUP,
|
||||
GUC_UNIT_MS,
|
||||
NULL, NULL, NULL);
|
||||
|
||||
DefineCustomBoolVariable(
|
||||
"citus.select_opens_transaction_block",
|
||||
gettext_noop("Open transaction blocks for SELECT commands"),
|
||||
|
|
|
@ -5,7 +5,7 @@
|
|||
* This file contains UDFs for getting foreign constraint relationship between
|
||||
* distributed tables.
|
||||
*
|
||||
* Copyright (c) Citus Data, Inc.
|
||||
* Copyright (c), Citus Data, Inc.
|
||||
*
|
||||
*-------------------------------------------------------------------------
|
||||
*/
|
||||
|
@ -15,6 +15,7 @@
|
|||
#include "funcapi.h"
|
||||
|
||||
#include "distributed/foreign_key_relationship.h"
|
||||
#include "distributed/coordinator_protocol.h"
|
||||
#include "distributed/listutils.h"
|
||||
#include "distributed/metadata_cache.h"
|
||||
#include "distributed/tuplestore.h"
|
||||
|
|
|
@ -0,0 +1,87 @@
|
|||
/*-------------------------------------------------------------------------
|
||||
*
|
||||
* foreign_key_relationship_query.c
|
||||
*
|
||||
* This file contains UDFs for getting foreign constraint relationship between
|
||||
* distributed tables.
|
||||
*
|
||||
* Copyright (c) 2018, Citus Data, Inc.
|
||||
*
|
||||
*-------------------------------------------------------------------------
|
||||
*/
|
||||
|
||||
#include "postgres.h"
|
||||
#include "fmgr.h"
|
||||
#include "funcapi.h"
|
||||
|
||||
#include "distributed/coordinator_protocol.h"
|
||||
#include "distributed/listutils.h"
|
||||
#include "distributed/metadata_cache.h"
|
||||
#include "utils/builtins.h"
|
||||
|
||||
|
||||
/* these functions are only exported in the regression tests */
|
||||
PG_FUNCTION_INFO_V1(get_foreign_key_to_reference_table_commands);
|
||||
|
||||
/*
|
||||
* get_foreign_key_to_reference_table_commands returns the list of commands
|
||||
* for creating foreign keys to reference tables.
|
||||
*/
|
||||
Datum
|
||||
get_foreign_key_to_reference_table_commands(PG_FUNCTION_ARGS)
|
||||
{
|
||||
FuncCallContext *functionContext = NULL;
|
||||
ListCell *commandsCell = NULL;
|
||||
|
||||
CheckCitusVersion(ERROR);
|
||||
|
||||
/* for the first we call this UDF, we need to populate the result to return set */
|
||||
if (SRF_IS_FIRSTCALL())
|
||||
{
|
||||
Oid relationId = PG_GETARG_OID(0);
|
||||
|
||||
/* create a function context for cross-call persistence */
|
||||
functionContext = SRF_FIRSTCALL_INIT();
|
||||
|
||||
/* switch to memory context appropriate for multiple function calls */
|
||||
MemoryContext oldContext = MemoryContextSwitchTo(
|
||||
functionContext->multi_call_memory_ctx);
|
||||
|
||||
CitusTableCacheEntry *cacheEntry = GetCitusTableCacheEntry(relationId);
|
||||
ShardInterval *firstShardInterval = cacheEntry->sortedShardIntervalArray[0];
|
||||
ListCellAndListWrapper *wrapper = palloc0(sizeof(ListCellAndListWrapper));
|
||||
List *commandsList =
|
||||
GetForeignConstraintCommandsToReferenceTable(firstShardInterval);
|
||||
|
||||
commandsCell = list_head(commandsList);
|
||||
wrapper->list = commandsList;
|
||||
wrapper->listCell = commandsCell;
|
||||
functionContext->user_fctx = wrapper;
|
||||
MemoryContextSwitchTo(oldContext);
|
||||
}
|
||||
|
||||
/*
|
||||
* On every call to this function, we get the current position in the
|
||||
* statement list. We then iterate to the next position in the list and
|
||||
* return the current statement, if we have not yet reached the end of
|
||||
* list.
|
||||
*/
|
||||
functionContext = SRF_PERCALL_SETUP();
|
||||
|
||||
ListCellAndListWrapper *wrapper =
|
||||
(ListCellAndListWrapper *) functionContext->user_fctx;
|
||||
|
||||
if (wrapper->listCell != NULL)
|
||||
{
|
||||
char *command = (char *) lfirst(wrapper->listCell);
|
||||
text *commandText = cstring_to_text(command);
|
||||
|
||||
wrapper->listCell = lnext_compat(wrapper->list, wrapper->listCell);
|
||||
|
||||
SRF_RETURN_NEXT(functionContext, PointerGetDatum(commandText));
|
||||
}
|
||||
else
|
||||
{
|
||||
SRF_RETURN_DONE(functionContext);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,628 @@
|
|||
/*-------------------------------------------------------------------------
|
||||
*
|
||||
* test/shard_rebalancer.c
|
||||
*
|
||||
* This file contains functions used for unit testing the planning part of the
|
||||
* shard rebalancer.
|
||||
*
|
||||
* Copyright (c) 2014-2019, Citus Data, Inc.
|
||||
*
|
||||
*-------------------------------------------------------------------------
|
||||
*/
|
||||
|
||||
#include "postgres.h"
|
||||
#include "libpq-fe.h"
|
||||
|
||||
#include "safe_lib.h"
|
||||
|
||||
#include "catalog/pg_type.h"
|
||||
#include "distributed/citus_safe_lib.h"
|
||||
#include "distributed/citus_ruleutils.h"
|
||||
#include "distributed/connection_management.h"
|
||||
#include "distributed/listutils.h"
|
||||
#include "distributed/multi_physical_planner.h"
|
||||
#include "distributed/shard_rebalancer.h"
|
||||
#include "funcapi.h"
|
||||
#include "miscadmin.h"
|
||||
#include "utils/builtins.h"
|
||||
#include "utils/int8.h"
|
||||
#include "utils/json.h"
|
||||
#include "utils/lsyscache.h"
|
||||
#include "utils/memutils.h"
|
||||
|
||||
/* static declarations for json conversion */
|
||||
static List * JsonArrayToShardPlacementTestInfoList(
|
||||
ArrayType *shardPlacementJsonArrayObject);
|
||||
static List * JsonArrayToWorkerTestInfoList(ArrayType *workerNodeJsonArrayObject);
|
||||
static bool JsonFieldValueBool(Datum jsonDocument, const char *key);
|
||||
static uint32 JsonFieldValueUInt32(Datum jsonDocument, const char *key);
|
||||
static uint64 JsonFieldValueUInt64(Datum jsonDocument, const char *key);
|
||||
static char * JsonFieldValueString(Datum jsonDocument, const char *key);
|
||||
static ArrayType * PlacementUpdateListToJsonArray(List *placementUpdateList);
|
||||
static bool ShardAllowedOnNode(uint64 shardId, WorkerNode *workerNode, void *context);
|
||||
static float NodeCapacity(WorkerNode *workerNode, void *context);
|
||||
static ShardCost GetShardCost(uint64 shardId, void *context);
|
||||
|
||||
|
||||
PG_FUNCTION_INFO_V1(shard_placement_rebalance_array);
|
||||
PG_FUNCTION_INFO_V1(shard_placement_replication_array);
|
||||
PG_FUNCTION_INFO_V1(worker_node_responsive);
|
||||
|
||||
typedef struct ShardPlacementTestInfo
|
||||
{
|
||||
ShardPlacement *placement;
|
||||
uint64 cost;
|
||||
bool nextColocationGroup;
|
||||
} ShardPlacementTestInfo;
|
||||
|
||||
typedef struct WorkerTestInfo
|
||||
{
|
||||
WorkerNode *node;
|
||||
List *disallowedShardIds;
|
||||
float capacity;
|
||||
} WorkerTestInfo;
|
||||
|
||||
typedef struct RebalancePlanContext
|
||||
{
|
||||
List *workerTestInfoList;
|
||||
List *shardPlacementTestInfoList;
|
||||
} RebalancePlacementContext;
|
||||
|
||||
|
||||
/*
|
||||
* shard_placement_rebalance_array returns a list of operations which can make a
|
||||
* cluster consisting of given shard placements and worker nodes balanced with
|
||||
* respect to the given threshold. Threshold is a value between 0 and 1 which
|
||||
* determines the evenness in shard distribution. When threshold is 0, then all
|
||||
* nodes should have equal number of shards. As threshold increases, cluster's
|
||||
* evenness requirements decrease, and we can rebalance the cluster using less
|
||||
* operations.
|
||||
*/
|
||||
Datum
|
||||
shard_placement_rebalance_array(PG_FUNCTION_ARGS)
|
||||
{
|
||||
ArrayType *workerNodeJsonArray = PG_GETARG_ARRAYTYPE_P(0);
|
||||
ArrayType *shardPlacementJsonArray = PG_GETARG_ARRAYTYPE_P(1);
|
||||
float threshold = PG_GETARG_FLOAT4(2);
|
||||
int32 maxShardMoves = PG_GETARG_INT32(3);
|
||||
bool drainOnly = PG_GETARG_BOOL(4);
|
||||
|
||||
List *workerNodeList = NIL;
|
||||
List *shardPlacementListList = NIL;
|
||||
List *shardPlacementList = NIL;
|
||||
WorkerTestInfo *workerTestInfo = NULL;
|
||||
ShardPlacementTestInfo *shardPlacementTestInfo = NULL;
|
||||
RebalancePlanFunctions rebalancePlanFunctions = {
|
||||
.shardAllowedOnNode = ShardAllowedOnNode,
|
||||
.nodeCapacity = NodeCapacity,
|
||||
.shardCost = GetShardCost,
|
||||
};
|
||||
RebalancePlacementContext context = {
|
||||
.workerTestInfoList = NULL,
|
||||
};
|
||||
|
||||
context.workerTestInfoList = JsonArrayToWorkerTestInfoList(workerNodeJsonArray);
|
||||
context.shardPlacementTestInfoList = JsonArrayToShardPlacementTestInfoList(
|
||||
shardPlacementJsonArray);
|
||||
|
||||
/* we don't need original arrays any more, so we free them to save memory */
|
||||
pfree(workerNodeJsonArray);
|
||||
pfree(shardPlacementJsonArray);
|
||||
|
||||
/* map workerTestInfoList to a list of its WorkerNodes */
|
||||
foreach_ptr(workerTestInfo, context.workerTestInfoList)
|
||||
{
|
||||
workerNodeList = lappend(workerNodeList, workerTestInfo->node);
|
||||
}
|
||||
|
||||
/* map shardPlacementTestInfoList to a list of list of its ShardPlacements */
|
||||
foreach_ptr(shardPlacementTestInfo, context.shardPlacementTestInfoList)
|
||||
{
|
||||
if (shardPlacementTestInfo->nextColocationGroup)
|
||||
{
|
||||
shardPlacementList = SortList(shardPlacementList, CompareShardPlacements);
|
||||
shardPlacementListList = lappend(shardPlacementListList, shardPlacementList);
|
||||
shardPlacementList = NIL;
|
||||
}
|
||||
shardPlacementList = lappend(shardPlacementList,
|
||||
shardPlacementTestInfo->placement);
|
||||
}
|
||||
shardPlacementList = SortList(shardPlacementList, CompareShardPlacements);
|
||||
shardPlacementListList = lappend(shardPlacementListList, shardPlacementList);
|
||||
|
||||
rebalancePlanFunctions.context = &context;
|
||||
|
||||
/* sort the lists to make the function more deterministic */
|
||||
workerNodeList = SortList(workerNodeList, CompareWorkerNodes);
|
||||
|
||||
List *placementUpdateList = RebalancePlacementUpdates(workerNodeList,
|
||||
shardPlacementListList,
|
||||
threshold,
|
||||
maxShardMoves,
|
||||
drainOnly,
|
||||
&rebalancePlanFunctions);
|
||||
ArrayType *placementUpdateJsonArray = PlacementUpdateListToJsonArray(
|
||||
placementUpdateList);
|
||||
|
||||
PG_RETURN_ARRAYTYPE_P(placementUpdateJsonArray);
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* ShardAllowedOnNode is the function that checks if shard is allowed to be on
|
||||
* a worker when running the shard rebalancer unit tests.
|
||||
*/
|
||||
static bool
|
||||
ShardAllowedOnNode(uint64 shardId, WorkerNode *workerNode, void *voidContext)
|
||||
{
|
||||
RebalancePlacementContext *context = voidContext;
|
||||
WorkerTestInfo *workerTestInfo = NULL;
|
||||
uint64 *disallowedShardIdPtr = NULL;
|
||||
foreach_ptr(workerTestInfo, context->workerTestInfoList)
|
||||
{
|
||||
if (workerTestInfo->node == workerNode)
|
||||
{
|
||||
break;
|
||||
}
|
||||
}
|
||||
Assert(workerTestInfo != NULL);
|
||||
|
||||
foreach_ptr(disallowedShardIdPtr, workerTestInfo->disallowedShardIds)
|
||||
{
|
||||
if (shardId == *disallowedShardIdPtr)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* NodeCapacity is the function that gets the capacity of a worker when running
|
||||
* the shard rebalancer unit tests.
|
||||
*/
|
||||
static float
|
||||
NodeCapacity(WorkerNode *workerNode, void *voidContext)
|
||||
{
|
||||
RebalancePlacementContext *context = voidContext;
|
||||
WorkerTestInfo *workerTestInfo = NULL;
|
||||
foreach_ptr(workerTestInfo, context->workerTestInfoList)
|
||||
{
|
||||
if (workerTestInfo->node == workerNode)
|
||||
{
|
||||
break;
|
||||
}
|
||||
}
|
||||
Assert(workerTestInfo != NULL);
|
||||
return workerTestInfo->capacity;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* GetShardCost is the function that gets the ShardCost of a shard when running
|
||||
* the shard rebalancer unit tests.
|
||||
*/
|
||||
static ShardCost
|
||||
GetShardCost(uint64 shardId, void *voidContext)
|
||||
{
|
||||
RebalancePlacementContext *context = voidContext;
|
||||
ShardCost shardCost;
|
||||
memset_struct_0(shardCost);
|
||||
shardCost.shardId = shardId;
|
||||
|
||||
ShardPlacementTestInfo *shardPlacementTestInfo = NULL;
|
||||
foreach_ptr(shardPlacementTestInfo, context->shardPlacementTestInfoList)
|
||||
{
|
||||
if (shardPlacementTestInfo->placement->shardId == shardId)
|
||||
{
|
||||
break;
|
||||
}
|
||||
}
|
||||
Assert(shardPlacementTestInfo != NULL);
|
||||
shardCost.cost = shardPlacementTestInfo->cost;
|
||||
return shardCost;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* shard_placement_replication_array returns a list of operations which will
|
||||
* replicate under-replicated shards in a cluster consisting of given shard
|
||||
* placements and worker nodes. A shard is under-replicated if it has less
|
||||
* active placements than the given shard replication factor.
|
||||
*/
|
||||
Datum
|
||||
shard_placement_replication_array(PG_FUNCTION_ARGS)
|
||||
{
|
||||
ArrayType *workerNodeJsonArray = PG_GETARG_ARRAYTYPE_P(0);
|
||||
ArrayType *shardPlacementJsonArray = PG_GETARG_ARRAYTYPE_P(1);
|
||||
uint32 shardReplicationFactor = PG_GETARG_INT32(2);
|
||||
|
||||
List *workerNodeList = NIL;
|
||||
List *shardPlacementList = NIL;
|
||||
WorkerTestInfo *workerTestInfo = NULL;
|
||||
ShardPlacementTestInfo *shardPlacementTestInfo = NULL;
|
||||
|
||||
/* validate shard replication factor */
|
||||
if (shardReplicationFactor < SHARD_REPLICATION_FACTOR_MINIMUM ||
|
||||
shardReplicationFactor > SHARD_REPLICATION_FACTOR_MAXIMUM)
|
||||
{
|
||||
ereport(ERROR, (errmsg("invalid shard replication factor"),
|
||||
errhint("Shard replication factor must be an integer "
|
||||
"between %d and %d", SHARD_REPLICATION_FACTOR_MINIMUM,
|
||||
SHARD_REPLICATION_FACTOR_MAXIMUM)));
|
||||
}
|
||||
|
||||
List *workerTestInfoList = JsonArrayToWorkerTestInfoList(workerNodeJsonArray);
|
||||
List *shardPlacementTestInfoList = JsonArrayToShardPlacementTestInfoList(
|
||||
shardPlacementJsonArray);
|
||||
|
||||
/* we don't need original arrays any more, so we free them to save memory */
|
||||
pfree(workerNodeJsonArray);
|
||||
pfree(shardPlacementJsonArray);
|
||||
|
||||
foreach_ptr(workerTestInfo, workerTestInfoList)
|
||||
{
|
||||
workerNodeList = lappend(workerNodeList, workerTestInfo->node);
|
||||
}
|
||||
|
||||
foreach_ptr(shardPlacementTestInfo, shardPlacementTestInfoList)
|
||||
{
|
||||
shardPlacementList = lappend(shardPlacementList,
|
||||
shardPlacementTestInfo->placement);
|
||||
}
|
||||
|
||||
/* sort the lists to make the function more deterministic */
|
||||
workerNodeList = SortList(workerNodeList, CompareWorkerNodes);
|
||||
shardPlacementList = SortList(shardPlacementList, CompareShardPlacements);
|
||||
|
||||
List *placementUpdateList = ReplicationPlacementUpdates(workerNodeList,
|
||||
shardPlacementList,
|
||||
shardReplicationFactor);
|
||||
ArrayType *placementUpdateJsonArray = PlacementUpdateListToJsonArray(
|
||||
placementUpdateList);
|
||||
|
||||
PG_RETURN_ARRAYTYPE_P(placementUpdateJsonArray);
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* JsonArrayToShardPlacementTestInfoList converts the given shard placement json array
|
||||
* to a list of ShardPlacement structs.
|
||||
*/
|
||||
static List *
|
||||
JsonArrayToShardPlacementTestInfoList(ArrayType *shardPlacementJsonArrayObject)
|
||||
{
|
||||
List *shardPlacementTestInfoList = NIL;
|
||||
Datum *shardPlacementJsonArray = NULL;
|
||||
int placementCount = 0;
|
||||
|
||||
/*
|
||||
* Memory is not automatically freed when we call UDFs using DirectFunctionCall.
|
||||
* We call these functions in functionCallContext, so we can free the memory
|
||||
* once they return.
|
||||
*/
|
||||
MemoryContext functionCallContext = AllocSetContextCreate(CurrentMemoryContext,
|
||||
"Function Call Context",
|
||||
ALLOCSET_DEFAULT_MINSIZE,
|
||||
ALLOCSET_DEFAULT_INITSIZE,
|
||||
ALLOCSET_DEFAULT_MAXSIZE);
|
||||
|
||||
deconstruct_array(shardPlacementJsonArrayObject, JSONOID, -1, false, 'i',
|
||||
&shardPlacementJsonArray, NULL, &placementCount);
|
||||
|
||||
for (int placementIndex = 0; placementIndex < placementCount; placementIndex++)
|
||||
{
|
||||
Datum placementJson = shardPlacementJsonArray[placementIndex];
|
||||
ShardPlacementTestInfo *placementTestInfo = palloc0(
|
||||
sizeof(ShardPlacementTestInfo));
|
||||
|
||||
MemoryContext oldContext = MemoryContextSwitchTo(functionCallContext);
|
||||
|
||||
uint64 shardId = JsonFieldValueUInt64(placementJson, FIELD_NAME_SHARD_ID);
|
||||
uint64 shardLength = JsonFieldValueUInt64(placementJson, FIELD_NAME_SHARD_LENGTH);
|
||||
int shardState = JsonFieldValueUInt32(placementJson, FIELD_NAME_SHARD_STATE);
|
||||
char *nodeName = JsonFieldValueString(placementJson, FIELD_NAME_NODE_NAME);
|
||||
int nodePort = JsonFieldValueUInt32(placementJson, FIELD_NAME_NODE_PORT);
|
||||
uint64 placementId = JsonFieldValueUInt64(placementJson, FIELD_NAME_PLACEMENT_ID);
|
||||
|
||||
MemoryContextSwitchTo(oldContext);
|
||||
|
||||
placementTestInfo->placement = palloc0(sizeof(ShardPlacement));
|
||||
placementTestInfo->placement->shardId = shardId;
|
||||
placementTestInfo->placement->shardLength = shardLength;
|
||||
placementTestInfo->placement->shardState = shardState;
|
||||
placementTestInfo->placement->nodeName = pstrdup(nodeName);
|
||||
placementTestInfo->placement->nodePort = nodePort;
|
||||
placementTestInfo->placement->placementId = placementId;
|
||||
|
||||
/*
|
||||
* We have copied whatever we needed from the UDF calls, so we can free
|
||||
* the memory allocated by them.
|
||||
*/
|
||||
MemoryContextReset(functionCallContext);
|
||||
|
||||
|
||||
shardPlacementTestInfoList = lappend(shardPlacementTestInfoList,
|
||||
placementTestInfo);
|
||||
|
||||
PG_TRY();
|
||||
{
|
||||
placementTestInfo->cost = JsonFieldValueUInt64(placementJson,
|
||||
"cost");
|
||||
}
|
||||
PG_CATCH();
|
||||
{
|
||||
/* Ignore errors about not being able to find the key in that case cost is 1 */
|
||||
FlushErrorState();
|
||||
MemoryContextSwitchTo(oldContext);
|
||||
placementTestInfo->cost = 1;
|
||||
}
|
||||
PG_END_TRY();
|
||||
|
||||
PG_TRY();
|
||||
{
|
||||
placementTestInfo->nextColocationGroup = JsonFieldValueBool(
|
||||
placementJson, "next_colocation");
|
||||
}
|
||||
PG_CATCH();
|
||||
{
|
||||
/* Ignore errors about not being able to find the key in that case cost is 1 */
|
||||
FlushErrorState();
|
||||
MemoryContextSwitchTo(oldContext);
|
||||
}
|
||||
PG_END_TRY();
|
||||
}
|
||||
|
||||
pfree(shardPlacementJsonArray);
|
||||
|
||||
return shardPlacementTestInfoList;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* JsonArrayToWorkerNodeList converts the given worker node json array to a list
|
||||
* of WorkerNode structs.
|
||||
*/
|
||||
static List *
|
||||
JsonArrayToWorkerTestInfoList(ArrayType *workerNodeJsonArrayObject)
|
||||
{
|
||||
List *workerTestInfoList = NIL;
|
||||
Datum *workerNodeJsonArray = NULL;
|
||||
int workerNodeCount = 0;
|
||||
|
||||
deconstruct_array(workerNodeJsonArrayObject, JSONOID, -1, false, 'i',
|
||||
&workerNodeJsonArray, NULL, &workerNodeCount);
|
||||
|
||||
for (int workerNodeIndex = 0; workerNodeIndex < workerNodeCount; workerNodeIndex++)
|
||||
{
|
||||
Datum workerNodeJson = workerNodeJsonArray[workerNodeIndex];
|
||||
char *workerName = JsonFieldValueString(workerNodeJson, FIELD_NAME_WORKER_NAME);
|
||||
uint32 workerPort = JsonFieldValueUInt32(workerNodeJson,
|
||||
FIELD_NAME_WORKER_PORT);
|
||||
List *disallowedShardIdList = NIL;
|
||||
char *disallowedShardsString = NULL;
|
||||
MemoryContext savedContext = CurrentMemoryContext;
|
||||
|
||||
|
||||
WorkerTestInfo *workerTestInfo = palloc0(sizeof(WorkerTestInfo));
|
||||
WorkerNode *workerNode = palloc0(sizeof(WorkerNode));
|
||||
strncpy_s(workerNode->workerName, sizeof(workerNode->workerName), workerName,
|
||||
WORKER_LENGTH);
|
||||
workerNode->nodeId = workerNodeIndex;
|
||||
workerNode->workerPort = workerPort;
|
||||
workerNode->shouldHaveShards = true;
|
||||
workerNode->nodeRole = PrimaryNodeRoleId();
|
||||
workerTestInfo->node = workerNode;
|
||||
|
||||
PG_TRY();
|
||||
{
|
||||
workerTestInfo->capacity = JsonFieldValueUInt64(workerNodeJson,
|
||||
"capacity");
|
||||
}
|
||||
PG_CATCH();
|
||||
{
|
||||
/* Ignore errors about not being able to find the key in that case capacity is 1 */
|
||||
FlushErrorState();
|
||||
MemoryContextSwitchTo(savedContext);
|
||||
workerTestInfo->capacity = 1;
|
||||
}
|
||||
PG_END_TRY();
|
||||
|
||||
|
||||
workerTestInfoList = lappend(workerTestInfoList, workerTestInfo);
|
||||
PG_TRY();
|
||||
{
|
||||
disallowedShardsString = JsonFieldValueString(workerNodeJson,
|
||||
"disallowed_shards");
|
||||
}
|
||||
PG_CATCH();
|
||||
{
|
||||
/* Ignore errors about not being able to find the key in that case all shards are allowed */
|
||||
FlushErrorState();
|
||||
MemoryContextSwitchTo(savedContext);
|
||||
disallowedShardsString = NULL;
|
||||
}
|
||||
PG_END_TRY();
|
||||
|
||||
if (disallowedShardsString == NULL)
|
||||
{
|
||||
continue;
|
||||
}
|
||||
|
||||
char *strtokPosition = NULL;
|
||||
char *shardString = strtok_r(disallowedShardsString, ",", &strtokPosition);
|
||||
while (shardString != NULL)
|
||||
{
|
||||
uint64 *shardInt = palloc0(sizeof(uint64));
|
||||
*shardInt = SafeStringToUint64(shardString);
|
||||
disallowedShardIdList = lappend(disallowedShardIdList, shardInt);
|
||||
shardString = strtok_r(NULL, ",", &strtokPosition);
|
||||
}
|
||||
workerTestInfo->disallowedShardIds = disallowedShardIdList;
|
||||
}
|
||||
|
||||
return workerTestInfoList;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* JsonFieldValueBool gets the value of the given key in the given json
|
||||
* document and returns it as a boolean.
|
||||
*/
|
||||
static bool
|
||||
JsonFieldValueBool(Datum jsonDocument, const char *key)
|
||||
{
|
||||
char *valueString = JsonFieldValueString(jsonDocument, key);
|
||||
Datum valueBoolDatum = DirectFunctionCall1(boolin, CStringGetDatum(valueString));
|
||||
|
||||
return DatumGetBool(valueBoolDatum);
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* JsonFieldValueUInt32 gets the value of the given key in the given json
|
||||
* document and returns it as an unsigned 32-bit integer.
|
||||
*/
|
||||
static uint32
|
||||
JsonFieldValueUInt32(Datum jsonDocument, const char *key)
|
||||
{
|
||||
char *valueString = JsonFieldValueString(jsonDocument, key);
|
||||
Datum valueInt4Datum = DirectFunctionCall1(int4in, CStringGetDatum(valueString));
|
||||
|
||||
uint32 valueUInt32 = DatumGetInt32(valueInt4Datum);
|
||||
return valueUInt32;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* JsonFieldValueUInt64 gets the value of the given key in the given json
|
||||
* document and returns it as an unsigned 64-bit integer.
|
||||
*/
|
||||
static uint64
|
||||
JsonFieldValueUInt64(Datum jsonDocument, const char *key)
|
||||
{
|
||||
char *valueString = JsonFieldValueString(jsonDocument, key);
|
||||
Datum valueInt8Datum = DirectFunctionCall1(int8in, CStringGetDatum(valueString));
|
||||
|
||||
uint64 valueUInt64 = DatumGetInt64(valueInt8Datum);
|
||||
return valueUInt64;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* JsonFieldValueString gets the value of the given key in the given json
|
||||
* document and returns it as a string.
|
||||
*/
|
||||
static char *
|
||||
JsonFieldValueString(Datum jsonDocument, const char *key)
|
||||
{
|
||||
Datum valueTextDatum = 0;
|
||||
bool valueFetched = false;
|
||||
Datum keyDatum = PointerGetDatum(cstring_to_text(key));
|
||||
|
||||
/*
|
||||
* json_object_field_text can return NULL, but DirectFunctionalCall2 raises
|
||||
* cryptic errors when the function returns NULL. We catch this error and
|
||||
* raise a more meaningful error.
|
||||
*/
|
||||
PG_TRY();
|
||||
{
|
||||
valueTextDatum = DirectFunctionCall2(json_object_field_text,
|
||||
jsonDocument, keyDatum);
|
||||
valueFetched = true;
|
||||
}
|
||||
PG_CATCH();
|
||||
{
|
||||
FlushErrorState();
|
||||
valueFetched = false;
|
||||
}
|
||||
PG_END_TRY();
|
||||
|
||||
if (!valueFetched)
|
||||
{
|
||||
ereport(ERROR, (errmsg("could not get value for '%s'", key)));
|
||||
}
|
||||
|
||||
char *valueString = text_to_cstring(DatumGetTextP(valueTextDatum));
|
||||
return valueString;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* PlacementUpdateListToJsonArray converts the given list of placement update
|
||||
* data to a json array.
|
||||
*/
|
||||
static ArrayType *
|
||||
PlacementUpdateListToJsonArray(List *placementUpdateList)
|
||||
{
|
||||
ListCell *placementUpdateCell = NULL;
|
||||
int placementUpdateIndex = 0;
|
||||
|
||||
int placementUpdateCount = list_length(placementUpdateList);
|
||||
Datum *placementUpdateJsonArray = palloc0(placementUpdateCount * sizeof(Datum));
|
||||
|
||||
foreach(placementUpdateCell, placementUpdateList)
|
||||
{
|
||||
PlacementUpdateEvent *placementUpdateEvent = lfirst(placementUpdateCell);
|
||||
WorkerNode *sourceNode = placementUpdateEvent->sourceNode;
|
||||
WorkerNode *targetNode = placementUpdateEvent->targetNode;
|
||||
|
||||
StringInfo escapedSourceName = makeStringInfo();
|
||||
escape_json(escapedSourceName, sourceNode->workerName);
|
||||
|
||||
StringInfo escapedTargetName = makeStringInfo();
|
||||
escape_json(escapedTargetName, targetNode->workerName);
|
||||
|
||||
StringInfo placementUpdateJsonString = makeStringInfo();
|
||||
appendStringInfo(placementUpdateJsonString, PLACEMENT_UPDATE_JSON_FORMAT,
|
||||
placementUpdateEvent->updateType, placementUpdateEvent->shardId,
|
||||
escapedSourceName->data, sourceNode->workerPort,
|
||||
escapedTargetName->data, targetNode->workerPort);
|
||||
|
||||
Datum placementUpdateStringDatum = CStringGetDatum(
|
||||
placementUpdateJsonString->data);
|
||||
Datum placementUpdateJsonDatum = DirectFunctionCall1(json_in,
|
||||
placementUpdateStringDatum);
|
||||
|
||||
placementUpdateJsonArray[placementUpdateIndex] = placementUpdateJsonDatum;
|
||||
placementUpdateIndex++;
|
||||
}
|
||||
|
||||
ArrayType *placementUpdateObject = construct_array(placementUpdateJsonArray,
|
||||
placementUpdateCount, JSONOID,
|
||||
-1, false, 'i');
|
||||
|
||||
return placementUpdateObject;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* worker_node_responsive returns true if the given worker node is responsive.
|
||||
* Otherwise, it returns false.
|
||||
*/
|
||||
Datum
|
||||
worker_node_responsive(PG_FUNCTION_ARGS)
|
||||
{
|
||||
text *workerNameText = PG_GETARG_TEXT_PP(0);
|
||||
uint32 workerPort = PG_GETARG_INT32(1);
|
||||
int connectionFlag = FORCE_NEW_CONNECTION;
|
||||
|
||||
bool workerNodeResponsive = false;
|
||||
const char *workerName = text_to_cstring(workerNameText);
|
||||
|
||||
MultiConnection *connection = GetNodeConnection(connectionFlag, workerName,
|
||||
workerPort);
|
||||
|
||||
if (connection != NULL && connection->pgConn != NULL)
|
||||
{
|
||||
if (PQstatus(connection->pgConn) == CONNECTION_OK)
|
||||
{
|
||||
workerNodeResponsive = true;
|
||||
}
|
||||
|
||||
CloseConnection(connection);
|
||||
}
|
||||
|
||||
PG_RETURN_BOOL(workerNodeResponsive);
|
||||
}
|
|
@ -32,3 +32,18 @@ hash_delete_all(HTAB *htab)
|
|||
Assert(found);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* foreach_htab_cleanup cleans up the hash iteration state after the iteration
|
||||
* is done. This is only needed when break statements are present in the
|
||||
* foreach block.
|
||||
*/
|
||||
void
|
||||
foreach_htab_cleanup(void *var, HASH_SEQ_STATUS *status)
|
||||
{
|
||||
if ((var) != NULL)
|
||||
{
|
||||
hash_seq_term(status);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -38,6 +38,7 @@
|
|||
#include "distributed/coordinator_protocol.h"
|
||||
#include "distributed/metadata_cache.h"
|
||||
#include "distributed/metadata_sync.h"
|
||||
#include "distributed/shard_cleaner.h"
|
||||
#include "distributed/statistics_collection.h"
|
||||
#include "distributed/transaction_recovery.h"
|
||||
#include "distributed/version_compat.h"
|
||||
|
@ -92,6 +93,7 @@ typedef struct MaintenanceDaemonDBData
|
|||
/* config variable for distributed deadlock detection timeout */
|
||||
double DistributedDeadlockDetectionTimeoutFactor = 2.0;
|
||||
int Recover2PCInterval = 60000;
|
||||
int DeferShardDeleteInterval = 60000;
|
||||
|
||||
/* config variables for metadata sync timeout */
|
||||
int MetadataSyncInterval = 60000;
|
||||
|
@ -289,6 +291,7 @@ CitusMaintenanceDaemonMain(Datum main_arg)
|
|||
bool retryStatsCollection USED_WITH_LIBCURL_ONLY = false;
|
||||
ErrorContextCallback errorCallback;
|
||||
TimestampTz lastRecoveryTime = 0;
|
||||
TimestampTz lastShardCleanTime = 0;
|
||||
TimestampTz nextMetadataSyncTime = 0;
|
||||
|
||||
/*
|
||||
|
@ -586,6 +589,45 @@ CitusMaintenanceDaemonMain(Datum main_arg)
|
|||
timeout = Min(timeout, deadlockTimeout);
|
||||
}
|
||||
|
||||
if (!RecoveryInProgress() && DeferShardDeleteInterval > 0 &&
|
||||
TimestampDifferenceExceeds(lastShardCleanTime, GetCurrentTimestamp(),
|
||||
DeferShardDeleteInterval))
|
||||
{
|
||||
int numberOfDroppedShards = 0;
|
||||
|
||||
InvalidateMetadataSystemCache();
|
||||
StartTransactionCommand();
|
||||
|
||||
if (!LockCitusExtension())
|
||||
{
|
||||
ereport(DEBUG1, (errmsg(
|
||||
"could not lock the citus extension, skipping shard cleaning")));
|
||||
}
|
||||
else if (CheckCitusVersion(DEBUG1) && CitusHasBeenLoaded())
|
||||
{
|
||||
/*
|
||||
* Record last shard clean time at start to ensure we run once per
|
||||
* DeferShardDeleteInterval.
|
||||
*/
|
||||
lastShardCleanTime = GetCurrentTimestamp();
|
||||
|
||||
numberOfDroppedShards = TryDropMarkedShards();
|
||||
}
|
||||
|
||||
CommitTransactionCommand();
|
||||
|
||||
if (numberOfDroppedShards > 0)
|
||||
{
|
||||
ereport(LOG, (errmsg("maintenance daemon dropped %d distributed "
|
||||
"shards previously marked to be removed",
|
||||
numberOfDroppedShards)));
|
||||
}
|
||||
|
||||
/* make sure we don't wait too long */
|
||||
timeout = Min(timeout, DeferShardDeleteInterval);
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* Wait until timeout, or until somebody wakes us up. Also cast the timeout to
|
||||
* integer where we've calculated it using double for not losing the precision.
|
||||
|
|
|
@ -253,6 +253,11 @@ extern ShardPlacement * SearchShardPlacementInList(List *shardPlacementList,
|
|||
extern ShardPlacement * SearchShardPlacementInListOrError(List *shardPlacementList,
|
||||
const char *nodeName,
|
||||
uint32 nodePort);
|
||||
extern void ErrorIfMoveCitusLocalTable(Oid relationId);
|
||||
extern char LookupShardTransferMode(Oid shardReplicationModeOid);
|
||||
extern void BlockWritesToShardList(List *shardList);
|
||||
extern List * WorkerApplyShardDDLCommandList(List *ddlCommandList, int64 shardId);
|
||||
extern List * GetForeignConstraintCommandsToReferenceTable(ShardInterval *shardInterval);
|
||||
|
||||
|
||||
#endif /* COORDINATOR_PROTOCOL_H */
|
||||
|
|
|
@ -48,4 +48,6 @@ extern void hash_delete_all(HTAB *htab);
|
|||
(var) != NULL; \
|
||||
(var) = hash_seq_search(status))
|
||||
|
||||
extern void foreach_htab_cleanup(void *var, HASH_SEQ_STATUS *status);
|
||||
|
||||
#endif
|
||||
|
|
|
@ -159,6 +159,7 @@ extern int32 GetLocalGroupId(void);
|
|||
extern void CitusTableCacheFlushInvalidatedEntries(void);
|
||||
extern Oid LookupShardRelationFromCatalog(int64 shardId, bool missing_ok);
|
||||
extern List * ShardPlacementList(uint64 shardId);
|
||||
extern bool ShardExists(int64 shardId);
|
||||
extern void CitusInvalidateRelcacheByRelid(Oid relationId);
|
||||
extern void CitusInvalidateRelcacheByShardId(int64 shardId);
|
||||
extern void InvalidateForeignKeyGraph(void);
|
||||
|
@ -210,6 +211,7 @@ extern Oid DistPartitionRelationId(void);
|
|||
extern Oid DistShardRelationId(void);
|
||||
extern Oid DistPlacementRelationId(void);
|
||||
extern Oid DistNodeRelationId(void);
|
||||
extern Oid DistRebalanceStrategyRelationId(void);
|
||||
extern Oid DistLocalGroupIdRelationId(void);
|
||||
extern Oid DistObjectRelationId(void);
|
||||
extern Oid DistEnabledCustomAggregatesId(void);
|
||||
|
|
|
@ -110,6 +110,7 @@ extern List * ActiveShardPlacementList(uint64 shardId);
|
|||
extern ShardPlacement * ActiveShardPlacement(uint64 shardId, bool missingOk);
|
||||
extern List * BuildShardPlacementList(ShardInterval *shardInterval);
|
||||
extern List * AllShardPlacementsOnNodeGroup(int32 groupId);
|
||||
extern List * AllShardPlacementsWithShardPlacementState(ShardState shardState);
|
||||
extern List * GroupShardPlacementsForTableOnGroup(Oid relationId, int32 groupId);
|
||||
extern StringInfo GenerateSizeQueryOnMultiplePlacements(List *shardIntervalList,
|
||||
char *sizeQuery);
|
||||
|
|
|
@ -0,0 +1,20 @@
|
|||
/*-------------------------------------------------------------------------
|
||||
*
|
||||
* shard_cleaner.h
|
||||
* Type and function declarations used in background shard cleaning
|
||||
*
|
||||
* Copyright (c) 2018, Citus Data, Inc.
|
||||
*
|
||||
*-------------------------------------------------------------------------
|
||||
*/
|
||||
|
||||
#ifndef CITUS_SHARD_CLEANER_H
|
||||
#define CITUS_SHARD_CLEANER_H
|
||||
|
||||
/* GUC to configure deferred shard deletion */
|
||||
extern int DeferShardDeleteInterval;
|
||||
extern bool DeferShardDeleteOnMove;
|
||||
|
||||
extern int TryDropMarkedShards(void);
|
||||
|
||||
#endif /*CITUS_SHARD_CLEANER_H */
|
|
@ -0,0 +1,159 @@
|
|||
/*-------------------------------------------------------------------------
|
||||
*
|
||||
* shard_rebalancer.h
|
||||
*
|
||||
* Type and function declarations for the shard rebalancer tool.
|
||||
*
|
||||
* Copyright (c), Citus Data, Inc.
|
||||
*
|
||||
*-------------------------------------------------------------------------
|
||||
*/
|
||||
|
||||
#ifndef SHARD_REBALANCER_H
|
||||
#define SHARD_REBALANCER_H
|
||||
|
||||
#include "postgres.h"
|
||||
|
||||
#include "fmgr.h"
|
||||
#include "nodes/pg_list.h"
|
||||
#include "distributed/coordinator_protocol.h"
|
||||
#include "distributed/worker_manager.h"
|
||||
|
||||
|
||||
/* Limits for function parameters */
|
||||
#define SHARD_REPLICATION_FACTOR_MINIMUM 1
|
||||
#define SHARD_REPLICATION_FACTOR_MAXIMUM 100
|
||||
|
||||
/* Definitions for metadata update commands */
|
||||
#define INSERT_SHARD_PLACEMENT_COMMAND "INSERT INTO pg_dist_shard_placement VALUES(" \
|
||||
UINT64_FORMAT ", %d, " UINT64_FORMAT ", '%s', %d)"
|
||||
#define DELETE_SHARD_PLACEMENT_COMMAND "DELETE FROM pg_dist_shard_placement WHERE " \
|
||||
"shardid=" UINT64_FORMAT \
|
||||
" AND nodename='%s' AND nodeport=%d"
|
||||
|
||||
/*
|
||||
* Definitions for shard placement json field names. These names should match
|
||||
* the column names in pg_dist_shard_placement.
|
||||
*/
|
||||
#define FIELD_NAME_SHARD_ID "shardid"
|
||||
#define FIELD_NAME_SHARD_LENGTH "shardlength"
|
||||
#define FIELD_NAME_SHARD_STATE "shardstate"
|
||||
#define FIELD_NAME_NODE_NAME "nodename"
|
||||
#define FIELD_NAME_NODE_PORT "nodeport"
|
||||
#define FIELD_NAME_PLACEMENT_ID "placementid"
|
||||
|
||||
/*
|
||||
* Definitions for worker node json field names. These names should match the
|
||||
* column names in master_get_active_worker_nodes().
|
||||
*/
|
||||
#define FIELD_NAME_WORKER_NAME "node_name"
|
||||
#define FIELD_NAME_WORKER_PORT "node_port"
|
||||
|
||||
/* Definitions for placement update json field names */
|
||||
#define FIELD_NAME_UPDATE_TYPE "updatetype"
|
||||
#define FIELD_NAME_SOURCE_NAME "sourcename"
|
||||
#define FIELD_NAME_SOURCE_PORT "sourceport"
|
||||
#define FIELD_NAME_TARGET_NAME "targetname"
|
||||
#define FIELD_NAME_TARGET_PORT "targetport"
|
||||
|
||||
/* *INDENT-OFF* */
|
||||
/* Definition for format of placement update json document */
|
||||
#define PLACEMENT_UPDATE_JSON_FORMAT \
|
||||
"{"\
|
||||
"\"" FIELD_NAME_UPDATE_TYPE "\":%d,"\
|
||||
"\"" FIELD_NAME_SHARD_ID "\":" UINT64_FORMAT ","\
|
||||
"\"" FIELD_NAME_SOURCE_NAME "\":%s,"\
|
||||
"\"" FIELD_NAME_SOURCE_PORT "\":%d,"\
|
||||
"\"" FIELD_NAME_TARGET_NAME "\":%s,"\
|
||||
"\"" FIELD_NAME_TARGET_PORT "\":%d"\
|
||||
"}"
|
||||
|
||||
/* *INDENT-ON* */
|
||||
|
||||
#define REBALANCE_ACTIVITY_MAGIC_NUMBER 1337
|
||||
#define REBALANCE_PROGRESS_ERROR -1
|
||||
#define REBALANCE_PROGRESS_WAITING 0
|
||||
#define REBALANCE_PROGRESS_MOVING 1
|
||||
#define REBALANCE_PROGRESS_MOVED 2
|
||||
|
||||
/* Enumeration that defines different placement update types */
|
||||
typedef enum
|
||||
{
|
||||
PLACEMENT_UPDATE_INVALID_FIRST = 0,
|
||||
PLACEMENT_UPDATE_MOVE = 1,
|
||||
PLACEMENT_UPDATE_COPY = 2
|
||||
} PlacementUpdateType;
|
||||
|
||||
|
||||
/*
|
||||
* PlacementUpdateEvent represents a logical unit of work that copies or
|
||||
* moves a shard placement.
|
||||
*/
|
||||
typedef struct PlacementUpdateEvent
|
||||
{
|
||||
PlacementUpdateType updateType;
|
||||
uint64 shardId;
|
||||
WorkerNode *sourceNode;
|
||||
WorkerNode *targetNode;
|
||||
} PlacementUpdateEvent;
|
||||
|
||||
|
||||
typedef struct PlacementUpdateEventProgress
|
||||
{
|
||||
uint64 shardId;
|
||||
char sourceName[255];
|
||||
int sourcePort;
|
||||
char targetName[255];
|
||||
int targetPort;
|
||||
uint64 shardSize;
|
||||
uint64 progress;
|
||||
} PlacementUpdateEventProgress;
|
||||
|
||||
typedef struct NodeFillState
|
||||
{
|
||||
WorkerNode *node;
|
||||
float4 capacity;
|
||||
float4 totalCost;
|
||||
float4 utilization;
|
||||
List *shardCostListDesc;
|
||||
} NodeFillState;
|
||||
|
||||
typedef struct ShardCost
|
||||
{
|
||||
uint64 shardId;
|
||||
float4 cost;
|
||||
} ShardCost;
|
||||
|
||||
typedef struct DisallowedPlacement
|
||||
{
|
||||
ShardCost *shardCost;
|
||||
NodeFillState *fillState;
|
||||
} DisallowedPlacement;
|
||||
|
||||
typedef struct RebalancePlanFunctions
|
||||
{
|
||||
bool (*shardAllowedOnNode)(uint64 shardId, WorkerNode *workerNode, void *context);
|
||||
float4 (*nodeCapacity)(WorkerNode *workerNode, void *context);
|
||||
ShardCost (*shardCost)(uint64 shardId, void *context);
|
||||
void *context;
|
||||
} RebalancePlanFunctions;
|
||||
|
||||
/* External function declarations */
|
||||
extern Datum shard_placement_rebalance_array(PG_FUNCTION_ARGS);
|
||||
extern Datum shard_placement_replication_array(PG_FUNCTION_ARGS);
|
||||
extern Datum worker_node_responsive(PG_FUNCTION_ARGS);
|
||||
extern Datum update_shard_placement(PG_FUNCTION_ARGS);
|
||||
extern Datum init_rebalance_monitor(PG_FUNCTION_ARGS);
|
||||
extern Datum finalize_rebalance_monitor(PG_FUNCTION_ARGS);
|
||||
extern Datum get_rebalance_progress(PG_FUNCTION_ARGS);
|
||||
|
||||
extern List * RebalancePlacementUpdates(List *workerNodeList, List *shardPlacementList,
|
||||
double threshold,
|
||||
int32 maxShardMoves,
|
||||
bool drainOnly,
|
||||
RebalancePlanFunctions *rebalancePlanFunctions);
|
||||
extern List * ReplicationPlacementUpdates(List *workerNodeList, List *shardPlacementList,
|
||||
int shardReplicationFactor);
|
||||
|
||||
|
||||
#endif /* SHARD_REBALANCER_H */
|
|
@ -42,7 +42,7 @@ output_files := $(patsubst $(citus_abs_srcdir)/output/%.source,expected/%.out, $
|
|||
# intermediate, for muscle memory backward compatibility.
|
||||
check: check-full
|
||||
# check-full triggers all tests that ought to be run routinely
|
||||
check-full: check-multi check-multi-mx check-worker check-follower-cluster check-failure
|
||||
check-full: check-multi check-multi-mx check-worker check-operations check-follower-cluster check-failure
|
||||
|
||||
|
||||
ISOLATION_DEPDIR=.deps/isolation
|
||||
|
@ -161,6 +161,10 @@ check-follower-cluster: all
|
|||
$(pg_regress_multi_check) --load-extension=citus --follower-cluster \
|
||||
-- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/multi_follower_schedule $(EXTRA_TESTS)
|
||||
|
||||
check-operations: all
|
||||
$(pg_regress_multi_check) --load-extension=citus \
|
||||
-- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/operations_schedule $(EXTRA_TESTS)
|
||||
|
||||
check-columnar:
|
||||
$(pg_regress_multi_check) --load-extension=citus \
|
||||
-- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/columnar_am_schedule $(EXTRA_TESTS)
|
||||
|
|
|
@ -0,0 +1,193 @@
|
|||
--
|
||||
-- FOREIGN_KEY_TO_REFERENCE_SHARD_REBALANCE
|
||||
--
|
||||
SET citus.next_shard_id TO 15000000;
|
||||
CREATE SCHEMA fkey_to_reference_shard_rebalance;
|
||||
SET search_path to fkey_to_reference_shard_rebalance;
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
SET citus.shard_count to 8;
|
||||
CREATE TYPE foreign_details AS (name text, relid text, refd_relid text);
|
||||
CREATE VIEW table_fkeys_in_workers AS
|
||||
SELECT
|
||||
(json_populate_record(NULL::foreign_details,
|
||||
json_array_elements_text((run_command_on_workers( $$
|
||||
SELECT
|
||||
COALESCE(json_agg(row_to_json(d)), '[]'::json)
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
distinct name,
|
||||
relid::regclass::text,
|
||||
refd_relid::regclass::text
|
||||
FROM
|
||||
table_fkey_cols
|
||||
)
|
||||
d $$ )).RESULT::json )::json )).* ;
|
||||
-- check if master_move_shard_placement with logical replication creates the
|
||||
-- foreign constraints properly after moving the shard
|
||||
CREATE TABLE referenced_table(test_column int, test_column2 int UNIQUE, PRIMARY KEY(test_column));
|
||||
CREATE TABLE referencing_table(id int PRIMARY KEY, ref_id int, FOREIGN KEY (id) REFERENCES referenced_table(test_column) ON DELETE CASCADE);
|
||||
CREATE TABLE referencing_table2(id int, ref_id int, FOREIGN KEY (ref_id) REFERENCES referenced_table(test_column2) ON DELETE CASCADE, FOREIGN KEY (id) REFERENCES referencing_table(id) ON DELETE CASCADE);
|
||||
SELECT create_reference_table('referenced_table');
|
||||
create_reference_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT create_distributed_table('referencing_table', 'id');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT create_distributed_table('referencing_table2', 'id');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
INSERT INTO referenced_table SELECT i,i FROM generate_series (0, 100) i;
|
||||
INSERT INTO referencing_table SELECT i,i FROM generate_series (0, 100) i;
|
||||
INSERT INTO referencing_table2 SELECT i,i FROM generate_series (0, 100) i;
|
||||
SELECT master_move_shard_placement(15000009, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
|
||||
master_move_shard_placement
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT count(*) FROM referencing_table2;
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
101
|
||||
(1 row)
|
||||
|
||||
SELECT * FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_to_reference_shard_rebalance.%' AND refd_relid LIKE 'fkey_to_reference_shard_rebalance.%' ORDER BY 1,2,3;
|
||||
name | relid | refd_relid
|
||||
---------------------------------------------------------------------
|
||||
referencing_table2_id_fkey_15000009 | fkey_to_reference_shard_rebalance.referencing_table2_15000009 | fkey_to_reference_shard_rebalance.referencing_table_15000001
|
||||
referencing_table2_id_fkey_15000010 | fkey_to_reference_shard_rebalance.referencing_table2_15000010 | fkey_to_reference_shard_rebalance.referencing_table_15000002
|
||||
referencing_table2_id_fkey_15000011 | fkey_to_reference_shard_rebalance.referencing_table2_15000011 | fkey_to_reference_shard_rebalance.referencing_table_15000003
|
||||
referencing_table2_id_fkey_15000012 | fkey_to_reference_shard_rebalance.referencing_table2_15000012 | fkey_to_reference_shard_rebalance.referencing_table_15000004
|
||||
referencing_table2_id_fkey_15000013 | fkey_to_reference_shard_rebalance.referencing_table2_15000013 | fkey_to_reference_shard_rebalance.referencing_table_15000005
|
||||
referencing_table2_id_fkey_15000014 | fkey_to_reference_shard_rebalance.referencing_table2_15000014 | fkey_to_reference_shard_rebalance.referencing_table_15000006
|
||||
referencing_table2_id_fkey_15000015 | fkey_to_reference_shard_rebalance.referencing_table2_15000015 | fkey_to_reference_shard_rebalance.referencing_table_15000007
|
||||
referencing_table2_id_fkey_15000016 | fkey_to_reference_shard_rebalance.referencing_table2_15000016 | fkey_to_reference_shard_rebalance.referencing_table_15000008
|
||||
referencing_table2_ref_id_fkey_15000009 | fkey_to_reference_shard_rebalance.referencing_table2_15000009 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table2_ref_id_fkey_15000010 | fkey_to_reference_shard_rebalance.referencing_table2_15000010 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table2_ref_id_fkey_15000011 | fkey_to_reference_shard_rebalance.referencing_table2_15000011 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table2_ref_id_fkey_15000012 | fkey_to_reference_shard_rebalance.referencing_table2_15000012 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table2_ref_id_fkey_15000013 | fkey_to_reference_shard_rebalance.referencing_table2_15000013 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table2_ref_id_fkey_15000014 | fkey_to_reference_shard_rebalance.referencing_table2_15000014 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table2_ref_id_fkey_15000015 | fkey_to_reference_shard_rebalance.referencing_table2_15000015 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table2_ref_id_fkey_15000016 | fkey_to_reference_shard_rebalance.referencing_table2_15000016 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table_id_fkey_15000001 | fkey_to_reference_shard_rebalance.referencing_table_15000001 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table_id_fkey_15000002 | fkey_to_reference_shard_rebalance.referencing_table_15000002 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table_id_fkey_15000003 | fkey_to_reference_shard_rebalance.referencing_table_15000003 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table_id_fkey_15000004 | fkey_to_reference_shard_rebalance.referencing_table_15000004 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table_id_fkey_15000005 | fkey_to_reference_shard_rebalance.referencing_table_15000005 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table_id_fkey_15000006 | fkey_to_reference_shard_rebalance.referencing_table_15000006 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table_id_fkey_15000007 | fkey_to_reference_shard_rebalance.referencing_table_15000007 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table_id_fkey_15000008 | fkey_to_reference_shard_rebalance.referencing_table_15000008 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
(24 rows)
|
||||
|
||||
SELECT master_move_shard_placement(15000009, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'block_writes');
|
||||
master_move_shard_placement
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT count(*) FROM referencing_table2;
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
101
|
||||
(1 row)
|
||||
|
||||
SELECT * FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_to_reference_shard_rebalance.%' AND refd_relid LIKE 'fkey_to_reference_shard_rebalance.%' ORDER BY 1,2,3;
|
||||
name | relid | refd_relid
|
||||
---------------------------------------------------------------------
|
||||
referencing_table2_id_fkey_15000009 | fkey_to_reference_shard_rebalance.referencing_table2_15000009 | fkey_to_reference_shard_rebalance.referencing_table_15000001
|
||||
referencing_table2_id_fkey_15000010 | fkey_to_reference_shard_rebalance.referencing_table2_15000010 | fkey_to_reference_shard_rebalance.referencing_table_15000002
|
||||
referencing_table2_id_fkey_15000011 | fkey_to_reference_shard_rebalance.referencing_table2_15000011 | fkey_to_reference_shard_rebalance.referencing_table_15000003
|
||||
referencing_table2_id_fkey_15000012 | fkey_to_reference_shard_rebalance.referencing_table2_15000012 | fkey_to_reference_shard_rebalance.referencing_table_15000004
|
||||
referencing_table2_id_fkey_15000013 | fkey_to_reference_shard_rebalance.referencing_table2_15000013 | fkey_to_reference_shard_rebalance.referencing_table_15000005
|
||||
referencing_table2_id_fkey_15000014 | fkey_to_reference_shard_rebalance.referencing_table2_15000014 | fkey_to_reference_shard_rebalance.referencing_table_15000006
|
||||
referencing_table2_id_fkey_15000015 | fkey_to_reference_shard_rebalance.referencing_table2_15000015 | fkey_to_reference_shard_rebalance.referencing_table_15000007
|
||||
referencing_table2_id_fkey_15000016 | fkey_to_reference_shard_rebalance.referencing_table2_15000016 | fkey_to_reference_shard_rebalance.referencing_table_15000008
|
||||
referencing_table2_ref_id_fkey_15000009 | fkey_to_reference_shard_rebalance.referencing_table2_15000009 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table2_ref_id_fkey_15000010 | fkey_to_reference_shard_rebalance.referencing_table2_15000010 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table2_ref_id_fkey_15000011 | fkey_to_reference_shard_rebalance.referencing_table2_15000011 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table2_ref_id_fkey_15000012 | fkey_to_reference_shard_rebalance.referencing_table2_15000012 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table2_ref_id_fkey_15000013 | fkey_to_reference_shard_rebalance.referencing_table2_15000013 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table2_ref_id_fkey_15000014 | fkey_to_reference_shard_rebalance.referencing_table2_15000014 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table2_ref_id_fkey_15000015 | fkey_to_reference_shard_rebalance.referencing_table2_15000015 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table2_ref_id_fkey_15000016 | fkey_to_reference_shard_rebalance.referencing_table2_15000016 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table_id_fkey_15000001 | fkey_to_reference_shard_rebalance.referencing_table_15000001 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table_id_fkey_15000002 | fkey_to_reference_shard_rebalance.referencing_table_15000002 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table_id_fkey_15000003 | fkey_to_reference_shard_rebalance.referencing_table_15000003 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table_id_fkey_15000004 | fkey_to_reference_shard_rebalance.referencing_table_15000004 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table_id_fkey_15000005 | fkey_to_reference_shard_rebalance.referencing_table_15000005 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table_id_fkey_15000006 | fkey_to_reference_shard_rebalance.referencing_table_15000006 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table_id_fkey_15000007 | fkey_to_reference_shard_rebalance.referencing_table_15000007 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
referencing_table_id_fkey_15000008 | fkey_to_reference_shard_rebalance.referencing_table_15000008 | fkey_to_reference_shard_rebalance.referenced_table_15000000
|
||||
(24 rows)
|
||||
|
||||
-- create a function to show the
|
||||
CREATE FUNCTION get_foreign_key_to_reference_table_commands(Oid)
|
||||
RETURNS SETOF text
|
||||
LANGUAGE C STABLE STRICT
|
||||
AS 'citus', $$get_foreign_key_to_reference_table_commands$$;
|
||||
CREATE TABLE reference_table_commands (id int UNIQUE);
|
||||
CREATE TABLE referenceing_dist_table (id int, col1 int, col2 int, col3 int);
|
||||
SELECT create_reference_table('reference_table_commands');
|
||||
create_reference_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT create_distributed_table('referenceing_dist_table', 'id');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
ALTER TABLE referenceing_dist_table ADD CONSTRAINT c1 FOREIGN KEY (col1) REFERENCES reference_table_commands(id) ON UPDATE CASCADE;
|
||||
ALTER TABLE referenceing_dist_table ADD CONSTRAINT c2 FOREIGN KEY (col2) REFERENCES reference_table_commands(id) ON UPDATE CASCADE NOT VALID;
|
||||
ALTER TABLE referenceing_dist_table ADD CONSTRAINT very_very_very_very_very_very_very_very_very_very_very_very_very_long FOREIGN KEY (col3) REFERENCES reference_table_commands(id) ON UPDATE CASCADE;
|
||||
NOTICE: identifier "very_very_very_very_very_very_very_very_very_very_very_very_very_long" will be truncated to "very_very_very_very_very_very_very_very_very_very_very_very_ver"
|
||||
SELECT * FROM get_foreign_key_to_reference_table_commands('referenceing_dist_table'::regclass);
|
||||
get_foreign_key_to_reference_table_commands
|
||||
---------------------------------------------------------------------
|
||||
SELECT worker_apply_inter_shard_ddl_command (15000018, 'fkey_to_reference_shard_rebalance', 15000017, 'fkey_to_reference_shard_rebalance', 'ALTER TABLE fkey_to_reference_shard_rebalance.referenceing_dist_table ADD CONSTRAINT c1 FOREIGN KEY (col1) REFERENCES fkey_to_reference_shard_rebalance.reference_table_commands(id) ON UPDATE CASCADE NOT VALID')
|
||||
UPDATE pg_constraint SET convalidated = true WHERE conrelid = 'fkey_to_reference_shard_rebalance.referenceing_dist_table_15000018'::regclass AND conname = 'c1_15000018'
|
||||
SELECT worker_apply_inter_shard_ddl_command (15000018, 'fkey_to_reference_shard_rebalance', 15000017, 'fkey_to_reference_shard_rebalance', 'ALTER TABLE fkey_to_reference_shard_rebalance.referenceing_dist_table ADD CONSTRAINT c2 FOREIGN KEY (col2) REFERENCES fkey_to_reference_shard_rebalance.reference_table_commands(id) ON UPDATE CASCADE NOT VALID')
|
||||
SELECT worker_apply_inter_shard_ddl_command (15000018, 'fkey_to_reference_shard_rebalance', 15000017, 'fkey_to_reference_shard_rebalance', 'ALTER TABLE fkey_to_reference_shard_rebalance.referenceing_dist_table ADD CONSTRAINT very_very_very_very_very_very_very_very_very_very_very_very_ver FOREIGN KEY (col3) REFERENCES fkey_to_reference_shard_rebalance.reference_table_commands(id) ON UPDATE CASCADE NOT VALID')
|
||||
UPDATE pg_constraint SET convalidated = true WHERE conrelid = 'fkey_to_reference_shard_rebalance.referenceing_dist_table_15000018'::regclass AND conname = 'very_very_very_very_very_very_very_very_very__754e8716_15000018'
|
||||
(5 rows)
|
||||
|
||||
-- and show that rebalancer works fine
|
||||
SELECT master_move_shard_placement(15000018, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
|
||||
master_move_shard_placement
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
\c - - - :worker_2_port
|
||||
SELECT conname, contype, convalidated FROM pg_constraint WHERE conrelid = 'fkey_to_reference_shard_rebalance.referenceing_dist_table_15000018'::regclass ORDER BY 1;
|
||||
conname | contype | convalidated
|
||||
---------------------------------------------------------------------
|
||||
c1_15000018 | f | t
|
||||
c2_15000018 | f | f
|
||||
very_very_very_very_very_very_very_very_very__754e8716_15000018 | f | t
|
||||
(3 rows)
|
||||
|
||||
\c - - - :master_port
|
||||
DROP SCHEMA fkey_to_reference_shard_rebalance CASCADE;
|
||||
NOTICE: drop cascades to 8 other objects
|
||||
DETAIL: drop cascades to type fkey_to_reference_shard_rebalance.foreign_details
|
||||
drop cascades to view fkey_to_reference_shard_rebalance.table_fkeys_in_workers
|
||||
drop cascades to table fkey_to_reference_shard_rebalance.referenced_table
|
||||
drop cascades to table fkey_to_reference_shard_rebalance.referencing_table
|
||||
drop cascades to table fkey_to_reference_shard_rebalance.referencing_table2
|
||||
drop cascades to function fkey_to_reference_shard_rebalance.get_foreign_key_to_reference_table_commands(oid)
|
||||
drop cascades to table fkey_to_reference_shard_rebalance.reference_table_commands
|
||||
drop cascades to table fkey_to_reference_shard_rebalance.referenceing_dist_table
|
|
@ -0,0 +1,305 @@
|
|||
Parsed test spec with 2 sessions
|
||||
|
||||
starting permutation: s1-begin s2-begin s2-insert s1-move-placement s2-end s1-end s1-select s1-get-shard-distribution
|
||||
step s1-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-insert:
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15), (172, 172);
|
||||
|
||||
step s1-move-placement:
|
||||
SELECT master_move_shard_placement(get_shard_id_for_distribution_column, 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes') FROM selected_shard;
|
||||
<waiting ...>
|
||||
step s2-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-move-placement: <... completed>
|
||||
master_move_shard_placement
|
||||
|
||||
|
||||
step s1-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-select:
|
||||
SELECT * FROM logical_replicate_placement order by y;
|
||||
|
||||
x y
|
||||
|
||||
15 15
|
||||
172 172
|
||||
step s1-get-shard-distribution:
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
|
||||
nodeport
|
||||
|
||||
57638
|
||||
|
||||
starting permutation: s1-begin s2-begin s2-upsert s1-move-placement s2-end s1-end s1-select s1-get-shard-distribution
|
||||
step s1-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-upsert:
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15), (172, 172);
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15), (172, 172) ON CONFLICT (x) DO UPDATE SET y = logical_replicate_placement.y + 1;
|
||||
|
||||
step s1-move-placement:
|
||||
SELECT master_move_shard_placement(get_shard_id_for_distribution_column, 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes') FROM selected_shard;
|
||||
<waiting ...>
|
||||
step s2-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-move-placement: <... completed>
|
||||
master_move_shard_placement
|
||||
|
||||
|
||||
step s1-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-select:
|
||||
SELECT * FROM logical_replicate_placement order by y;
|
||||
|
||||
x y
|
||||
|
||||
15 16
|
||||
172 173
|
||||
step s1-get-shard-distribution:
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
|
||||
nodeport
|
||||
|
||||
57638
|
||||
|
||||
starting permutation: s1-insert s1-begin s2-begin s2-update s1-move-placement s2-end s1-end s1-select s1-get-shard-distribution
|
||||
step s1-insert:
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15), (172, 172);
|
||||
|
||||
step s1-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-update:
|
||||
UPDATE logical_replicate_placement SET y = y + 1;
|
||||
|
||||
step s1-move-placement:
|
||||
SELECT master_move_shard_placement(get_shard_id_for_distribution_column, 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes') FROM selected_shard;
|
||||
<waiting ...>
|
||||
step s2-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-move-placement: <... completed>
|
||||
master_move_shard_placement
|
||||
|
||||
|
||||
step s1-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-select:
|
||||
SELECT * FROM logical_replicate_placement order by y;
|
||||
|
||||
x y
|
||||
|
||||
15 16
|
||||
172 173
|
||||
step s1-get-shard-distribution:
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
|
||||
nodeport
|
||||
|
||||
57638
|
||||
|
||||
starting permutation: s1-insert s1-begin s2-begin s2-delete s1-move-placement s2-end s1-end s1-select s1-get-shard-distribution
|
||||
step s1-insert:
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15), (172, 172);
|
||||
|
||||
step s1-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-delete:
|
||||
DELETE FROM logical_replicate_placement;
|
||||
|
||||
step s1-move-placement:
|
||||
SELECT master_move_shard_placement(get_shard_id_for_distribution_column, 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes') FROM selected_shard;
|
||||
<waiting ...>
|
||||
step s2-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-move-placement: <... completed>
|
||||
master_move_shard_placement
|
||||
|
||||
|
||||
step s1-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-select:
|
||||
SELECT * FROM logical_replicate_placement order by y;
|
||||
|
||||
x y
|
||||
|
||||
step s1-get-shard-distribution:
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
|
||||
nodeport
|
||||
|
||||
57638
|
||||
|
||||
starting permutation: s1-insert s1-begin s2-begin s2-select s1-move-placement s2-end s1-end s1-get-shard-distribution
|
||||
step s1-insert:
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15), (172, 172);
|
||||
|
||||
step s1-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-select:
|
||||
SELECT * FROM logical_replicate_placement ORDER BY y;
|
||||
|
||||
x y
|
||||
|
||||
15 15
|
||||
172 172
|
||||
step s1-move-placement:
|
||||
SELECT master_move_shard_placement(get_shard_id_for_distribution_column, 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes') FROM selected_shard;
|
||||
<waiting ...>
|
||||
step s2-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-move-placement: <... completed>
|
||||
master_move_shard_placement
|
||||
|
||||
|
||||
step s1-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-get-shard-distribution:
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
|
||||
nodeport
|
||||
|
||||
57638
|
||||
|
||||
starting permutation: s1-begin s2-begin s2-copy s1-move-placement s2-end s1-end s1-select s1-get-shard-distribution
|
||||
step s1-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-copy:
|
||||
COPY logical_replicate_placement FROM PROGRAM 'echo "1,1\n2,2\n3,3\n4,4\n5,5\n15,30"' WITH CSV;
|
||||
|
||||
step s1-move-placement:
|
||||
SELECT master_move_shard_placement(get_shard_id_for_distribution_column, 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes') FROM selected_shard;
|
||||
<waiting ...>
|
||||
step s2-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-move-placement: <... completed>
|
||||
master_move_shard_placement
|
||||
|
||||
|
||||
step s1-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-select:
|
||||
SELECT * FROM logical_replicate_placement order by y;
|
||||
|
||||
x y
|
||||
|
||||
1 1
|
||||
2 2
|
||||
3 3
|
||||
4 4
|
||||
5 5
|
||||
15 30
|
||||
step s1-get-shard-distribution:
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
|
||||
nodeport
|
||||
|
||||
57638
|
||||
|
||||
starting permutation: s1-insert s1-begin s2-begin s2-truncate s1-move-placement s2-end s1-end s1-select s1-get-shard-distribution
|
||||
step s1-insert:
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15), (172, 172);
|
||||
|
||||
step s1-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-truncate:
|
||||
TRUNCATE logical_replicate_placement;
|
||||
|
||||
step s1-move-placement:
|
||||
SELECT master_move_shard_placement(get_shard_id_for_distribution_column, 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes') FROM selected_shard;
|
||||
<waiting ...>
|
||||
step s2-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-move-placement: <... completed>
|
||||
master_move_shard_placement
|
||||
|
||||
|
||||
step s1-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-select:
|
||||
SELECT * FROM logical_replicate_placement order by y;
|
||||
|
||||
x y
|
||||
|
||||
step s1-get-shard-distribution:
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
|
||||
nodeport
|
||||
|
||||
57638
|
||||
|
||||
starting permutation: s1-begin s2-begin s2-alter-table s1-move-placement s2-end s1-end s1-select s1-get-shard-distribution
|
||||
step s1-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-alter-table:
|
||||
ALTER TABLE logical_replicate_placement ADD COLUMN z INT;
|
||||
|
||||
step s1-move-placement:
|
||||
SELECT master_move_shard_placement(get_shard_id_for_distribution_column, 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes') FROM selected_shard;
|
||||
<waiting ...>
|
||||
step s2-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-move-placement: <... completed>
|
||||
master_move_shard_placement
|
||||
|
||||
|
||||
step s1-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-select:
|
||||
SELECT * FROM logical_replicate_placement order by y;
|
||||
|
||||
x y z
|
||||
|
||||
step s1-get-shard-distribution:
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
|
||||
nodeport
|
||||
|
||||
57638
|
|
@ -0,0 +1,245 @@
|
|||
Parsed test spec with 2 sessions
|
||||
|
||||
starting permutation: s1-begin s2-start-session-level-connection s2-begin-on-worker s2-insert s1-move-placement s2-commit-worker s1-commit s1-select s1-get-shard-distribution s2-stop-connection
|
||||
step s1-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-start-session-level-connection:
|
||||
SELECT start_session_level_connection_to_node('localhost', 57638);
|
||||
|
||||
start_session_level_connection_to_node
|
||||
|
||||
|
||||
step s2-begin-on-worker:
|
||||
SELECT run_commands_on_session_level_connection_to_node('BEGIN');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s2-insert:
|
||||
SELECT run_commands_on_session_level_connection_to_node('INSERT INTO logical_replicate_placement VALUES (15, 15), (172, 172)');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s1-move-placement:
|
||||
SELECT master_move_shard_placement(get_shard_id_for_distribution_column, 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes') FROM selected_shard;
|
||||
<waiting ...>
|
||||
step s2-commit-worker:
|
||||
SELECT run_commands_on_session_level_connection_to_node('COMMIT');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s1-move-placement: <... completed>
|
||||
master_move_shard_placement
|
||||
|
||||
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
step s1-select:
|
||||
SELECT * FROM logical_replicate_placement order by y;
|
||||
|
||||
x y
|
||||
|
||||
15 15
|
||||
172 172
|
||||
step s1-get-shard-distribution:
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
|
||||
nodeport
|
||||
|
||||
57638
|
||||
step s2-stop-connection:
|
||||
SELECT stop_session_level_connection_to_node();
|
||||
|
||||
stop_session_level_connection_to_node
|
||||
|
||||
|
||||
restore_isolation_tester_func
|
||||
|
||||
|
||||
|
||||
starting permutation: s1-insert s1-begin s2-start-session-level-connection s2-begin-on-worker s2-update s1-move-placement s2-commit-worker s1-commit s1-select s1-get-shard-distribution s2-stop-connection
|
||||
step s1-insert:
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15), (172, 172);
|
||||
|
||||
step s1-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-start-session-level-connection:
|
||||
SELECT start_session_level_connection_to_node('localhost', 57638);
|
||||
|
||||
start_session_level_connection_to_node
|
||||
|
||||
|
||||
step s2-begin-on-worker:
|
||||
SELECT run_commands_on_session_level_connection_to_node('BEGIN');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s2-update:
|
||||
SELECT run_commands_on_session_level_connection_to_node('UPDATE logical_replicate_placement SET y = y + 1');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s1-move-placement:
|
||||
SELECT master_move_shard_placement(get_shard_id_for_distribution_column, 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes') FROM selected_shard;
|
||||
<waiting ...>
|
||||
step s2-commit-worker:
|
||||
SELECT run_commands_on_session_level_connection_to_node('COMMIT');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s1-move-placement: <... completed>
|
||||
master_move_shard_placement
|
||||
|
||||
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
step s1-select:
|
||||
SELECT * FROM logical_replicate_placement order by y;
|
||||
|
||||
x y
|
||||
|
||||
15 16
|
||||
172 173
|
||||
step s1-get-shard-distribution:
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
|
||||
nodeport
|
||||
|
||||
57638
|
||||
step s2-stop-connection:
|
||||
SELECT stop_session_level_connection_to_node();
|
||||
|
||||
stop_session_level_connection_to_node
|
||||
|
||||
|
||||
restore_isolation_tester_func
|
||||
|
||||
|
||||
|
||||
starting permutation: s1-insert s1-begin s2-start-session-level-connection s2-begin-on-worker s2-delete s1-move-placement s2-commit-worker s1-commit s1-select s1-get-shard-distribution s2-stop-connection
|
||||
step s1-insert:
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15), (172, 172);
|
||||
|
||||
step s1-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-start-session-level-connection:
|
||||
SELECT start_session_level_connection_to_node('localhost', 57638);
|
||||
|
||||
start_session_level_connection_to_node
|
||||
|
||||
|
||||
step s2-begin-on-worker:
|
||||
SELECT run_commands_on_session_level_connection_to_node('BEGIN');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s2-delete:
|
||||
SELECT run_commands_on_session_level_connection_to_node('DELETE FROM logical_replicate_placement');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s1-move-placement:
|
||||
SELECT master_move_shard_placement(get_shard_id_for_distribution_column, 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes') FROM selected_shard;
|
||||
<waiting ...>
|
||||
step s2-commit-worker:
|
||||
SELECT run_commands_on_session_level_connection_to_node('COMMIT');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s1-move-placement: <... completed>
|
||||
master_move_shard_placement
|
||||
|
||||
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
step s1-select:
|
||||
SELECT * FROM logical_replicate_placement order by y;
|
||||
|
||||
x y
|
||||
|
||||
step s1-get-shard-distribution:
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
|
||||
nodeport
|
||||
|
||||
57638
|
||||
step s2-stop-connection:
|
||||
SELECT stop_session_level_connection_to_node();
|
||||
|
||||
stop_session_level_connection_to_node
|
||||
|
||||
|
||||
restore_isolation_tester_func
|
||||
|
||||
|
||||
|
||||
starting permutation: s1-insert s1-begin s2-start-session-level-connection s2-begin-on-worker s2-select s1-move-placement s2-commit-worker s1-commit s1-get-shard-distribution s2-stop-connection
|
||||
step s1-insert:
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15), (172, 172);
|
||||
|
||||
step s1-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-start-session-level-connection:
|
||||
SELECT start_session_level_connection_to_node('localhost', 57638);
|
||||
|
||||
start_session_level_connection_to_node
|
||||
|
||||
|
||||
step s2-begin-on-worker:
|
||||
SELECT run_commands_on_session_level_connection_to_node('BEGIN');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s2-select:
|
||||
SELECT run_commands_on_session_level_connection_to_node('SELECT * FROM logical_replicate_placement ORDER BY y');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s1-move-placement:
|
||||
SELECT master_move_shard_placement(get_shard_id_for_distribution_column, 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes') FROM selected_shard;
|
||||
<waiting ...>
|
||||
step s2-commit-worker:
|
||||
SELECT run_commands_on_session_level_connection_to_node('COMMIT');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s1-move-placement: <... completed>
|
||||
master_move_shard_placement
|
||||
|
||||
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
step s1-get-shard-distribution:
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
|
||||
nodeport
|
||||
|
||||
57638
|
||||
step s2-stop-connection:
|
||||
SELECT stop_session_level_connection_to_node();
|
||||
|
||||
stop_session_level_connection_to_node
|
||||
|
||||
|
||||
restore_isolation_tester_func
|
||||
|
||||
|
|
@ -0,0 +1,223 @@
|
|||
Parsed test spec with 2 sessions
|
||||
|
||||
starting permutation: s1-begin s2-begin s2-insert s1-move-placement s2-end s1-end s1-select s1-get-shard-distribution
|
||||
step s1-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-insert:
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15);
|
||||
|
||||
step s1-move-placement:
|
||||
SELECT master_move_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes');
|
||||
<waiting ...>
|
||||
step s2-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-move-placement: <... completed>
|
||||
master_move_shard_placement
|
||||
|
||||
|
||||
step s1-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-select:
|
||||
SELECT * FROM logical_replicate_placement order by y;
|
||||
|
||||
x y
|
||||
|
||||
15 15
|
||||
step s1-get-shard-distribution:
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
|
||||
nodeport
|
||||
|
||||
57638
|
||||
|
||||
starting permutation: s1-begin s2-begin s2-upsert s1-move-placement s2-end s1-end s1-select s1-get-shard-distribution
|
||||
step s1-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-upsert:
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15);
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15) ON CONFLICT (x) DO UPDATE SET y = logical_replicate_placement.y + 1;
|
||||
|
||||
step s1-move-placement:
|
||||
SELECT master_move_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes');
|
||||
<waiting ...>
|
||||
step s2-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-move-placement: <... completed>
|
||||
master_move_shard_placement
|
||||
|
||||
|
||||
step s1-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-select:
|
||||
SELECT * FROM logical_replicate_placement order by y;
|
||||
|
||||
x y
|
||||
|
||||
15 16
|
||||
step s1-get-shard-distribution:
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
|
||||
nodeport
|
||||
|
||||
57638
|
||||
|
||||
starting permutation: s1-insert s1-begin s2-begin s2-update s1-move-placement s2-end s1-end s1-select s1-get-shard-distribution
|
||||
step s1-insert:
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15);
|
||||
|
||||
step s1-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-update:
|
||||
UPDATE logical_replicate_placement SET y = y + 1 WHERE x = 15;
|
||||
|
||||
step s1-move-placement:
|
||||
SELECT master_move_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes');
|
||||
<waiting ...>
|
||||
step s2-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-move-placement: <... completed>
|
||||
master_move_shard_placement
|
||||
|
||||
|
||||
step s1-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-select:
|
||||
SELECT * FROM logical_replicate_placement order by y;
|
||||
|
||||
x y
|
||||
|
||||
15 16
|
||||
step s1-get-shard-distribution:
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
|
||||
nodeport
|
||||
|
||||
57638
|
||||
|
||||
starting permutation: s1-insert s1-begin s2-begin s2-delete s1-move-placement s2-end s1-end s1-select s1-get-shard-distribution
|
||||
step s1-insert:
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15);
|
||||
|
||||
step s1-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-delete:
|
||||
DELETE FROM logical_replicate_placement WHERE x = 15;
|
||||
|
||||
step s1-move-placement:
|
||||
SELECT master_move_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes');
|
||||
<waiting ...>
|
||||
step s2-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-move-placement: <... completed>
|
||||
master_move_shard_placement
|
||||
|
||||
|
||||
step s1-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-select:
|
||||
SELECT * FROM logical_replicate_placement order by y;
|
||||
|
||||
x y
|
||||
|
||||
step s1-get-shard-distribution:
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
|
||||
nodeport
|
||||
|
||||
57638
|
||||
|
||||
starting permutation: s1-insert s1-begin s2-begin s2-select s1-move-placement s2-end s1-end s1-get-shard-distribution
|
||||
step s1-insert:
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15);
|
||||
|
||||
step s1-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-select:
|
||||
SELECT * FROM logical_replicate_placement ORDER BY y;
|
||||
|
||||
x y
|
||||
|
||||
15 15
|
||||
step s1-move-placement:
|
||||
SELECT master_move_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes');
|
||||
<waiting ...>
|
||||
step s2-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-move-placement: <... completed>
|
||||
master_move_shard_placement
|
||||
|
||||
|
||||
step s1-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-get-shard-distribution:
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
|
||||
nodeport
|
||||
|
||||
57638
|
||||
|
||||
starting permutation: s1-insert s1-begin s2-begin s2-select-for-update s1-move-placement s2-end s1-end s1-get-shard-distribution
|
||||
step s1-insert:
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15);
|
||||
|
||||
step s1-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-select-for-update:
|
||||
SELECT * FROM logical_replicate_placement WHERE x=15 FOR UPDATE;
|
||||
|
||||
x y
|
||||
|
||||
15 15
|
||||
step s1-move-placement:
|
||||
SELECT master_move_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes');
|
||||
<waiting ...>
|
||||
step s2-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-move-placement: <... completed>
|
||||
master_move_shard_placement
|
||||
|
||||
|
||||
step s1-end:
|
||||
COMMIT;
|
||||
|
||||
step s1-get-shard-distribution:
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
|
||||
nodeport
|
||||
|
||||
57638
|
|
@ -0,0 +1,300 @@
|
|||
Parsed test spec with 2 sessions
|
||||
|
||||
starting permutation: s1-begin s2-start-session-level-connection s2-begin-on-worker s2-insert s1-move-placement s2-commit-worker s1-commit s1-select s1-get-shard-distribution s2-stop-connection
|
||||
step s1-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-start-session-level-connection:
|
||||
SELECT start_session_level_connection_to_node('localhost', 57638);
|
||||
|
||||
start_session_level_connection_to_node
|
||||
|
||||
|
||||
step s2-begin-on-worker:
|
||||
SELECT run_commands_on_session_level_connection_to_node('BEGIN');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s2-insert:
|
||||
SELECT run_commands_on_session_level_connection_to_node('INSERT INTO logical_replicate_placement VALUES (15, 15)');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s1-move-placement:
|
||||
SELECT master_move_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes');
|
||||
<waiting ...>
|
||||
step s2-commit-worker:
|
||||
SELECT run_commands_on_session_level_connection_to_node('COMMIT');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s1-move-placement: <... completed>
|
||||
master_move_shard_placement
|
||||
|
||||
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
step s1-select:
|
||||
SELECT * FROM logical_replicate_placement order by y;
|
||||
|
||||
x y
|
||||
|
||||
15 15
|
||||
step s1-get-shard-distribution:
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
|
||||
nodeport
|
||||
|
||||
57638
|
||||
step s2-stop-connection:
|
||||
SELECT stop_session_level_connection_to_node();
|
||||
|
||||
stop_session_level_connection_to_node
|
||||
|
||||
|
||||
restore_isolation_tester_func
|
||||
|
||||
|
||||
|
||||
starting permutation: s1-insert s1-begin s2-start-session-level-connection s2-begin-on-worker s2-update s1-move-placement s2-commit-worker s1-commit s1-select s1-get-shard-distribution s2-stop-connection
|
||||
step s1-insert:
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15);
|
||||
|
||||
step s1-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-start-session-level-connection:
|
||||
SELECT start_session_level_connection_to_node('localhost', 57638);
|
||||
|
||||
start_session_level_connection_to_node
|
||||
|
||||
|
||||
step s2-begin-on-worker:
|
||||
SELECT run_commands_on_session_level_connection_to_node('BEGIN');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s2-update:
|
||||
SELECT run_commands_on_session_level_connection_to_node('UPDATE logical_replicate_placement SET y = y + 1 WHERE x = 15');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s1-move-placement:
|
||||
SELECT master_move_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes');
|
||||
<waiting ...>
|
||||
step s2-commit-worker:
|
||||
SELECT run_commands_on_session_level_connection_to_node('COMMIT');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s1-move-placement: <... completed>
|
||||
master_move_shard_placement
|
||||
|
||||
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
step s1-select:
|
||||
SELECT * FROM logical_replicate_placement order by y;
|
||||
|
||||
x y
|
||||
|
||||
15 16
|
||||
step s1-get-shard-distribution:
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
|
||||
nodeport
|
||||
|
||||
57638
|
||||
step s2-stop-connection:
|
||||
SELECT stop_session_level_connection_to_node();
|
||||
|
||||
stop_session_level_connection_to_node
|
||||
|
||||
|
||||
restore_isolation_tester_func
|
||||
|
||||
|
||||
|
||||
starting permutation: s1-insert s1-begin s2-start-session-level-connection s2-begin-on-worker s2-delete s1-move-placement s2-commit-worker s1-commit s1-select s1-get-shard-distribution s2-stop-connection
|
||||
step s1-insert:
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15);
|
||||
|
||||
step s1-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-start-session-level-connection:
|
||||
SELECT start_session_level_connection_to_node('localhost', 57638);
|
||||
|
||||
start_session_level_connection_to_node
|
||||
|
||||
|
||||
step s2-begin-on-worker:
|
||||
SELECT run_commands_on_session_level_connection_to_node('BEGIN');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s2-delete:
|
||||
SELECT run_commands_on_session_level_connection_to_node('DELETE FROM logical_replicate_placement WHERE x = 15');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s1-move-placement:
|
||||
SELECT master_move_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes');
|
||||
<waiting ...>
|
||||
step s2-commit-worker:
|
||||
SELECT run_commands_on_session_level_connection_to_node('COMMIT');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s1-move-placement: <... completed>
|
||||
master_move_shard_placement
|
||||
|
||||
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
step s1-select:
|
||||
SELECT * FROM logical_replicate_placement order by y;
|
||||
|
||||
x y
|
||||
|
||||
step s1-get-shard-distribution:
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
|
||||
nodeport
|
||||
|
||||
57638
|
||||
step s2-stop-connection:
|
||||
SELECT stop_session_level_connection_to_node();
|
||||
|
||||
stop_session_level_connection_to_node
|
||||
|
||||
|
||||
restore_isolation_tester_func
|
||||
|
||||
|
||||
|
||||
starting permutation: s1-insert s1-begin s2-start-session-level-connection s2-begin-on-worker s2-select s1-move-placement s2-commit-worker s1-commit s1-get-shard-distribution s2-stop-connection
|
||||
step s1-insert:
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15);
|
||||
|
||||
step s1-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-start-session-level-connection:
|
||||
SELECT start_session_level_connection_to_node('localhost', 57638);
|
||||
|
||||
start_session_level_connection_to_node
|
||||
|
||||
|
||||
step s2-begin-on-worker:
|
||||
SELECT run_commands_on_session_level_connection_to_node('BEGIN');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s2-select:
|
||||
SELECT run_commands_on_session_level_connection_to_node('SELECT * FROM logical_replicate_placement ORDER BY y');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s1-move-placement:
|
||||
SELECT master_move_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes');
|
||||
<waiting ...>
|
||||
step s2-commit-worker:
|
||||
SELECT run_commands_on_session_level_connection_to_node('COMMIT');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s1-move-placement: <... completed>
|
||||
master_move_shard_placement
|
||||
|
||||
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
step s1-get-shard-distribution:
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
|
||||
nodeport
|
||||
|
||||
57638
|
||||
step s2-stop-connection:
|
||||
SELECT stop_session_level_connection_to_node();
|
||||
|
||||
stop_session_level_connection_to_node
|
||||
|
||||
|
||||
restore_isolation_tester_func
|
||||
|
||||
|
||||
|
||||
starting permutation: s1-insert s1-begin s2-start-session-level-connection s2-begin-on-worker s2-select-for-update s1-move-placement s2-commit-worker s1-commit s1-get-shard-distribution s2-stop-connection
|
||||
step s1-insert:
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15);
|
||||
|
||||
step s1-begin:
|
||||
BEGIN;
|
||||
|
||||
step s2-start-session-level-connection:
|
||||
SELECT start_session_level_connection_to_node('localhost', 57638);
|
||||
|
||||
start_session_level_connection_to_node
|
||||
|
||||
|
||||
step s2-begin-on-worker:
|
||||
SELECT run_commands_on_session_level_connection_to_node('BEGIN');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s2-select-for-update:
|
||||
SELECT run_commands_on_session_level_connection_to_node('SELECT * FROM logical_replicate_placement WHERE x=15 FOR UPDATE');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s1-move-placement:
|
||||
SELECT master_move_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes');
|
||||
<waiting ...>
|
||||
step s2-commit-worker:
|
||||
SELECT run_commands_on_session_level_connection_to_node('COMMIT');
|
||||
|
||||
run_commands_on_session_level_connection_to_node
|
||||
|
||||
|
||||
step s1-move-placement: <... completed>
|
||||
master_move_shard_placement
|
||||
|
||||
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
step s1-get-shard-distribution:
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
|
||||
nodeport
|
||||
|
||||
57638
|
||||
step s2-stop-connection:
|
||||
SELECT stop_session_level_connection_to_node();
|
||||
|
||||
stop_session_level_connection_to_node
|
||||
|
||||
|
||||
restore_isolation_tester_func
|
||||
|
||||
|
|
@ -0,0 +1,449 @@
|
|||
Parsed test spec with 2 sessions
|
||||
|
||||
starting permutation: s1-rebalance-nc s2-rebalance-nc s1-commit
|
||||
create_distributed_table
|
||||
|
||||
|
||||
step s1-rebalance-nc:
|
||||
BEGIN;
|
||||
select rebalance_table_shards('non_colocated');
|
||||
|
||||
rebalance_table_shards
|
||||
|
||||
|
||||
step s2-rebalance-nc:
|
||||
select rebalance_table_shards('non_colocated');
|
||||
|
||||
ERROR: could not acquire the lock required to rebalance public.non_colocated
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
master_set_node_property
|
||||
|
||||
|
||||
|
||||
starting permutation: s1-rebalance-nc s2-replicate-nc s1-commit
|
||||
create_distributed_table
|
||||
|
||||
|
||||
step s1-rebalance-nc:
|
||||
BEGIN;
|
||||
select rebalance_table_shards('non_colocated');
|
||||
|
||||
rebalance_table_shards
|
||||
|
||||
|
||||
step s2-replicate-nc:
|
||||
select replicate_table_shards('non_colocated');
|
||||
|
||||
ERROR: could not acquire the lock required to replicate public.non_colocated
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
master_set_node_property
|
||||
|
||||
|
||||
|
||||
starting permutation: s1-replicate-nc s2-rebalance-nc s1-commit
|
||||
create_distributed_table
|
||||
|
||||
|
||||
step s1-replicate-nc:
|
||||
BEGIN;
|
||||
select replicate_table_shards('non_colocated');
|
||||
|
||||
replicate_table_shards
|
||||
|
||||
|
||||
step s2-rebalance-nc:
|
||||
select rebalance_table_shards('non_colocated');
|
||||
|
||||
ERROR: could not acquire the lock required to rebalance public.non_colocated
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
master_set_node_property
|
||||
|
||||
|
||||
|
||||
starting permutation: s1-replicate-nc s2-replicate-nc s1-commit
|
||||
create_distributed_table
|
||||
|
||||
|
||||
step s1-replicate-nc:
|
||||
BEGIN;
|
||||
select replicate_table_shards('non_colocated');
|
||||
|
||||
replicate_table_shards
|
||||
|
||||
|
||||
step s2-replicate-nc:
|
||||
select replicate_table_shards('non_colocated');
|
||||
|
||||
ERROR: could not acquire the lock required to replicate public.non_colocated
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
master_set_node_property
|
||||
|
||||
|
||||
|
||||
starting permutation: s1-rebalance-c1 s2-rebalance-c2 s1-commit
|
||||
create_distributed_table
|
||||
|
||||
|
||||
step s1-rebalance-c1:
|
||||
BEGIN;
|
||||
select rebalance_table_shards('colocated1');
|
||||
|
||||
rebalance_table_shards
|
||||
|
||||
|
||||
step s2-rebalance-c2:
|
||||
select rebalance_table_shards('colocated2');
|
||||
|
||||
ERROR: could not acquire the lock required to rebalance public.colocated2
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
master_set_node_property
|
||||
|
||||
|
||||
|
||||
starting permutation: s1-rebalance-c1 s2-replicate-c2 s1-commit
|
||||
create_distributed_table
|
||||
|
||||
|
||||
step s1-rebalance-c1:
|
||||
BEGIN;
|
||||
select rebalance_table_shards('colocated1');
|
||||
|
||||
rebalance_table_shards
|
||||
|
||||
|
||||
step s2-replicate-c2:
|
||||
select replicate_table_shards('colocated2');
|
||||
|
||||
ERROR: could not acquire the lock required to replicate public.colocated2
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
master_set_node_property
|
||||
|
||||
|
||||
|
||||
starting permutation: s1-replicate-c1 s2-rebalance-c2 s1-commit
|
||||
create_distributed_table
|
||||
|
||||
|
||||
step s1-replicate-c1:
|
||||
BEGIN;
|
||||
select replicate_table_shards('colocated1');
|
||||
|
||||
replicate_table_shards
|
||||
|
||||
|
||||
step s2-rebalance-c2:
|
||||
select rebalance_table_shards('colocated2');
|
||||
|
||||
ERROR: could not acquire the lock required to rebalance public.colocated2
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
master_set_node_property
|
||||
|
||||
|
||||
|
||||
starting permutation: s1-replicate-c1 s2-replicate-c2 s1-commit
|
||||
create_distributed_table
|
||||
|
||||
|
||||
step s1-replicate-c1:
|
||||
BEGIN;
|
||||
select replicate_table_shards('colocated1');
|
||||
|
||||
replicate_table_shards
|
||||
|
||||
|
||||
step s2-replicate-c2:
|
||||
select replicate_table_shards('colocated2');
|
||||
|
||||
ERROR: could not acquire the lock required to replicate public.colocated2
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
master_set_node_property
|
||||
|
||||
|
||||
|
||||
starting permutation: s1-rebalance-c1 s2-rebalance-nc s1-commit
|
||||
create_distributed_table
|
||||
|
||||
|
||||
step s1-rebalance-c1:
|
||||
BEGIN;
|
||||
select rebalance_table_shards('colocated1');
|
||||
|
||||
rebalance_table_shards
|
||||
|
||||
|
||||
step s2-rebalance-nc:
|
||||
select rebalance_table_shards('non_colocated');
|
||||
|
||||
rebalance_table_shards
|
||||
|
||||
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
master_set_node_property
|
||||
|
||||
|
||||
|
||||
starting permutation: s1-rebalance-c1 s2-replicate-nc s1-commit
|
||||
create_distributed_table
|
||||
|
||||
|
||||
step s1-rebalance-c1:
|
||||
BEGIN;
|
||||
select rebalance_table_shards('colocated1');
|
||||
|
||||
rebalance_table_shards
|
||||
|
||||
|
||||
step s2-replicate-nc:
|
||||
select replicate_table_shards('non_colocated');
|
||||
|
||||
replicate_table_shards
|
||||
|
||||
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
master_set_node_property
|
||||
|
||||
|
||||
|
||||
starting permutation: s1-replicate-c1 s2-rebalance-nc s1-commit
|
||||
create_distributed_table
|
||||
|
||||
|
||||
step s1-replicate-c1:
|
||||
BEGIN;
|
||||
select replicate_table_shards('colocated1');
|
||||
|
||||
replicate_table_shards
|
||||
|
||||
|
||||
step s2-rebalance-nc:
|
||||
select rebalance_table_shards('non_colocated');
|
||||
|
||||
rebalance_table_shards
|
||||
|
||||
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
master_set_node_property
|
||||
|
||||
|
||||
|
||||
starting permutation: s1-replicate-c1 s2-replicate-nc s1-commit
|
||||
create_distributed_table
|
||||
|
||||
|
||||
step s1-replicate-c1:
|
||||
BEGIN;
|
||||
select replicate_table_shards('colocated1');
|
||||
|
||||
replicate_table_shards
|
||||
|
||||
|
||||
step s2-replicate-nc:
|
||||
select replicate_table_shards('non_colocated');
|
||||
|
||||
replicate_table_shards
|
||||
|
||||
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
master_set_node_property
|
||||
|
||||
|
||||
|
||||
starting permutation: s1-rebalance-c1 s2-rebalance-all s1-commit
|
||||
create_distributed_table
|
||||
|
||||
|
||||
step s1-rebalance-c1:
|
||||
BEGIN;
|
||||
select rebalance_table_shards('colocated1');
|
||||
|
||||
rebalance_table_shards
|
||||
|
||||
|
||||
step s2-rebalance-all:
|
||||
select rebalance_table_shards();
|
||||
|
||||
ERROR: could not acquire the lock required to rebalance public.distributed_transaction_id_table
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
master_set_node_property
|
||||
|
||||
|
||||
|
||||
starting permutation: s1-replicate-c1 s2-rebalance-all s1-commit
|
||||
create_distributed_table
|
||||
|
||||
|
||||
step s1-replicate-c1:
|
||||
BEGIN;
|
||||
select replicate_table_shards('colocated1');
|
||||
|
||||
replicate_table_shards
|
||||
|
||||
|
||||
step s2-rebalance-all:
|
||||
select rebalance_table_shards();
|
||||
|
||||
ERROR: could not acquire the lock required to rebalance public.distributed_transaction_id_table
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
master_set_node_property
|
||||
|
||||
|
||||
|
||||
starting permutation: s1-rebalance-nc s2-rebalance-all s1-commit
|
||||
create_distributed_table
|
||||
|
||||
|
||||
step s1-rebalance-nc:
|
||||
BEGIN;
|
||||
select rebalance_table_shards('non_colocated');
|
||||
|
||||
rebalance_table_shards
|
||||
|
||||
|
||||
step s2-rebalance-all:
|
||||
select rebalance_table_shards();
|
||||
|
||||
ERROR: could not acquire the lock required to rebalance public.non_colocated
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
master_set_node_property
|
||||
|
||||
|
||||
|
||||
starting permutation: s1-replicate-nc s2-rebalance-all s1-commit
|
||||
create_distributed_table
|
||||
|
||||
|
||||
step s1-replicate-nc:
|
||||
BEGIN;
|
||||
select replicate_table_shards('non_colocated');
|
||||
|
||||
replicate_table_shards
|
||||
|
||||
|
||||
step s2-rebalance-all:
|
||||
select rebalance_table_shards();
|
||||
|
||||
ERROR: could not acquire the lock required to rebalance public.non_colocated
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
master_set_node_property
|
||||
|
||||
|
||||
|
||||
starting permutation: s1-rebalance-c1 s2-drain s1-commit
|
||||
create_distributed_table
|
||||
|
||||
|
||||
step s1-rebalance-c1:
|
||||
BEGIN;
|
||||
select rebalance_table_shards('colocated1');
|
||||
|
||||
rebalance_table_shards
|
||||
|
||||
|
||||
step s2-drain:
|
||||
select master_drain_node('localhost', 57638);
|
||||
|
||||
ERROR: could not acquire the lock required to move public.distributed_transaction_id_table
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
master_set_node_property
|
||||
|
||||
|
||||
|
||||
starting permutation: s1-replicate-c1 s2-drain s1-commit
|
||||
create_distributed_table
|
||||
|
||||
|
||||
step s1-replicate-c1:
|
||||
BEGIN;
|
||||
select replicate_table_shards('colocated1');
|
||||
|
||||
replicate_table_shards
|
||||
|
||||
|
||||
step s2-drain:
|
||||
select master_drain_node('localhost', 57638);
|
||||
|
||||
ERROR: could not acquire the lock required to move public.distributed_transaction_id_table
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
master_set_node_property
|
||||
|
||||
|
||||
|
||||
starting permutation: s1-rebalance-nc s2-drain s1-commit
|
||||
create_distributed_table
|
||||
|
||||
|
||||
step s1-rebalance-nc:
|
||||
BEGIN;
|
||||
select rebalance_table_shards('non_colocated');
|
||||
|
||||
rebalance_table_shards
|
||||
|
||||
|
||||
step s2-drain:
|
||||
select master_drain_node('localhost', 57638);
|
||||
|
||||
ERROR: could not acquire the lock required to move public.non_colocated
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
master_set_node_property
|
||||
|
||||
|
||||
|
||||
starting permutation: s1-replicate-nc s2-drain s1-commit
|
||||
create_distributed_table
|
||||
|
||||
|
||||
step s1-replicate-nc:
|
||||
BEGIN;
|
||||
select replicate_table_shards('non_colocated');
|
||||
|
||||
replicate_table_shards
|
||||
|
||||
|
||||
step s2-drain:
|
||||
select master_drain_node('localhost', 57638);
|
||||
|
||||
ERROR: could not acquire the lock required to move public.non_colocated
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
master_set_node_property
|
||||
|
||||
|
|
@ -45,7 +45,8 @@ SELECT master_copy_shard_placement(
|
|||
get_shard_id_for_distribution_column('data', 'key-1'),
|
||||
'localhost', :worker_1_port,
|
||||
'localhost', :worker_2_port,
|
||||
do_repair := false);
|
||||
do_repair := false,
|
||||
transfer_mode := 'block_writes');
|
||||
ERROR: could not find placement matching "localhost:xxxxx"
|
||||
HINT: Confirm the placement still exists and try again.
|
||||
-- verify we error out if source and destination are the same
|
||||
|
@ -53,14 +54,16 @@ SELECT master_copy_shard_placement(
|
|||
get_shard_id_for_distribution_column('data', 'key-1'),
|
||||
'localhost', :worker_2_port,
|
||||
'localhost', :worker_2_port,
|
||||
do_repair := false);
|
||||
do_repair := false,
|
||||
transfer_mode := 'block_writes');
|
||||
ERROR: shard xxxxx already exists in the target node
|
||||
-- verify we error out if target already contains a healthy placement
|
||||
SELECT master_copy_shard_placement(
|
||||
(SELECT shardid FROM pg_dist_shard WHERE logicalrelid='ref_table'::regclass::oid),
|
||||
'localhost', :worker_1_port,
|
||||
'localhost', :worker_2_port,
|
||||
do_repair := false);
|
||||
do_repair := false,
|
||||
transfer_mode := 'block_writes');
|
||||
ERROR: shard xxxxx already exists in the target node
|
||||
-- verify we error out if table has foreign key constraints
|
||||
INSERT INTO ref_table SELECT 1, value FROM data;
|
||||
|
@ -70,16 +73,15 @@ SELECT master_copy_shard_placement(
|
|||
'localhost', :worker_2_port,
|
||||
'localhost', :worker_1_port,
|
||||
do_repair := false);
|
||||
ERROR: cannot create foreign key constraint
|
||||
DETAIL: This shard has foreign constraints on it. Citus currently supports foreign key constraints only for "citus.shard_replication_factor = 1".
|
||||
HINT: Please change "citus.shard_replication_factor to 1". To learn more about using foreign keys with other replication factors, please contact us at https://citusdata.com/about/contact_us.
|
||||
ERROR: cannot replicate shards with foreign keys
|
||||
ALTER TABLE data DROP CONSTRAINT distfk;
|
||||
-- replicate shard that contains key-1
|
||||
SELECT master_copy_shard_placement(
|
||||
get_shard_id_for_distribution_column('data', 'key-1'),
|
||||
'localhost', :worker_2_port,
|
||||
'localhost', :worker_1_port,
|
||||
do_repair := false);
|
||||
do_repair := false,
|
||||
transfer_mode := 'block_writes');
|
||||
master_copy_shard_placement
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -123,7 +125,8 @@ SELECT master_copy_shard_placement(
|
|||
get_shard_id_for_distribution_column('mx_table', '1'),
|
||||
'localhost', :worker_1_port,
|
||||
'localhost', :worker_2_port,
|
||||
do_repair := false);
|
||||
do_repair := false,
|
||||
transfer_mode := 'block_writes');
|
||||
ERROR: Table 'mx_table' is streaming replicated. Shards of streaming replicated tables cannot be copied
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
stop_metadata_sync_to_node
|
||||
|
|
|
@ -0,0 +1,639 @@
|
|||
--
|
||||
-- MULTI_COLOCATED_SHARD_REBALANCE
|
||||
--
|
||||
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 13000000;
|
||||
SET citus.shard_count TO 6;
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
-- create distributed tables
|
||||
CREATE TABLE table1_group1 ( id int PRIMARY KEY);
|
||||
SELECT create_distributed_table('table1_group1', 'id', 'hash');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
CREATE TABLE table2_group1 ( id int );
|
||||
SELECT create_distributed_table('table2_group1', 'id', 'hash');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SET citus.shard_count TO 8;
|
||||
CREATE TABLE table5_groupX ( id int );
|
||||
SELECT create_distributed_table('table5_groupX', 'id', 'hash');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
CREATE TABLE table6_append ( id int );
|
||||
SELECT master_create_distributed_table('table6_append', 'id', 'append');
|
||||
master_create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT master_create_empty_shard('table6_append');
|
||||
master_create_empty_shard
|
||||
---------------------------------------------------------------------
|
||||
13000020
|
||||
(1 row)
|
||||
|
||||
SELECT master_create_empty_shard('table6_append');
|
||||
master_create_empty_shard
|
||||
---------------------------------------------------------------------
|
||||
13000021
|
||||
(1 row)
|
||||
|
||||
-- Mark tables as non-mx tables, in order to be able to test master_copy_shard_placement
|
||||
UPDATE pg_dist_partition SET repmodel='c' WHERE logicalrelid IN
|
||||
('table1_group1'::regclass, 'table2_group1'::regclass, 'table5_groupX'::regclass);
|
||||
-- test copy
|
||||
-- test copying colocated shards
|
||||
-- status before shard copy
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
colocationid = (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'table1_group1'::regclass)
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
shardid | logicalrelid | nodeport
|
||||
---------------------------------------------------------------------
|
||||
13000000 | table1_group1 | 57637
|
||||
13000001 | table1_group1 | 57638
|
||||
13000002 | table1_group1 | 57637
|
||||
13000003 | table1_group1 | 57638
|
||||
13000004 | table1_group1 | 57637
|
||||
13000005 | table1_group1 | 57638
|
||||
13000006 | table2_group1 | 57637
|
||||
13000007 | table2_group1 | 57638
|
||||
13000008 | table2_group1 | 57637
|
||||
13000009 | table2_group1 | 57638
|
||||
13000010 | table2_group1 | 57637
|
||||
13000011 | table2_group1 | 57638
|
||||
(12 rows)
|
||||
|
||||
-- copy colocated shards
|
||||
SELECT master_copy_shard_placement(13000000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, false);
|
||||
master_copy_shard_placement
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
-- status after shard copy
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
colocationid = (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'table1_group1'::regclass)
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
shardid | logicalrelid | nodeport
|
||||
---------------------------------------------------------------------
|
||||
13000000 | table1_group1 | 57637
|
||||
13000000 | table1_group1 | 57638
|
||||
13000001 | table1_group1 | 57638
|
||||
13000002 | table1_group1 | 57637
|
||||
13000003 | table1_group1 | 57638
|
||||
13000004 | table1_group1 | 57637
|
||||
13000005 | table1_group1 | 57638
|
||||
13000006 | table2_group1 | 57637
|
||||
13000006 | table2_group1 | 57638
|
||||
13000007 | table2_group1 | 57638
|
||||
13000008 | table2_group1 | 57637
|
||||
13000009 | table2_group1 | 57638
|
||||
13000010 | table2_group1 | 57637
|
||||
13000011 | table2_group1 | 57638
|
||||
(14 rows)
|
||||
|
||||
-- also connect worker to verify we successfully copied given shard (and other colocated shards)
|
||||
\c - - - :worker_2_port
|
||||
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.table1_group1_13000000'::regclass;
|
||||
Column | Type | Modifiers
|
||||
---------------------------------------------------------------------
|
||||
id | integer | not null
|
||||
(1 row)
|
||||
|
||||
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.table2_group1_13000006'::regclass;
|
||||
Column | Type | Modifiers
|
||||
---------------------------------------------------------------------
|
||||
id | integer |
|
||||
(1 row)
|
||||
|
||||
\c - - - :master_port
|
||||
-- copy colocated shards again to see error message
|
||||
SELECT master_copy_shard_placement(13000000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, false, 'force_logical');
|
||||
ERROR: the force_logical transfer mode is currently unsupported
|
||||
-- test copying NOT colocated shard
|
||||
-- status before shard copy
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
p.logicalrelid = 'table5_groupX'::regclass
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
shardid | logicalrelid | nodeport
|
||||
---------------------------------------------------------------------
|
||||
13000012 | table5_groupx | 57637
|
||||
13000013 | table5_groupx | 57638
|
||||
13000014 | table5_groupx | 57637
|
||||
13000015 | table5_groupx | 57638
|
||||
13000016 | table5_groupx | 57637
|
||||
13000017 | table5_groupx | 57638
|
||||
13000018 | table5_groupx | 57637
|
||||
13000019 | table5_groupx | 57638
|
||||
(8 rows)
|
||||
|
||||
-- copy NOT colocated shard
|
||||
SELECT master_copy_shard_placement(13000012, 'localhost', :worker_1_port, 'localhost', :worker_2_port, false);
|
||||
master_copy_shard_placement
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
-- status after shard copy
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
p.logicalrelid = 'table5_groupX'::regclass
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
shardid | logicalrelid | nodeport
|
||||
---------------------------------------------------------------------
|
||||
13000012 | table5_groupx | 57637
|
||||
13000012 | table5_groupx | 57638
|
||||
13000013 | table5_groupx | 57638
|
||||
13000014 | table5_groupx | 57637
|
||||
13000015 | table5_groupx | 57638
|
||||
13000016 | table5_groupx | 57637
|
||||
13000017 | table5_groupx | 57638
|
||||
13000018 | table5_groupx | 57637
|
||||
13000019 | table5_groupx | 57638
|
||||
(9 rows)
|
||||
|
||||
-- test copying shard in append distributed table
|
||||
-- status before shard copy
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
p.logicalrelid = 'table6_append'::regclass
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
shardid | logicalrelid | nodeport
|
||||
---------------------------------------------------------------------
|
||||
13000020 | table6_append | 57638
|
||||
13000021 | table6_append | 57637
|
||||
(2 rows)
|
||||
|
||||
-- copy shard in append distributed table
|
||||
SELECT master_copy_shard_placement(13000020, 'localhost', :worker_2_port, 'localhost', :worker_1_port, false, 'force_logical');
|
||||
ERROR: the force_logical transfer mode is currently unsupported
|
||||
-- status after shard copy
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
p.logicalrelid = 'table6_append'::regclass
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
shardid | logicalrelid | nodeport
|
||||
---------------------------------------------------------------------
|
||||
13000020 | table6_append | 57638
|
||||
13000021 | table6_append | 57637
|
||||
(2 rows)
|
||||
|
||||
-- test move
|
||||
-- test moving colocated shards
|
||||
-- status before shard move
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
colocationid = (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'table1_group1'::regclass)
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
shardid | logicalrelid | nodeport
|
||||
---------------------------------------------------------------------
|
||||
13000000 | table1_group1 | 57637
|
||||
13000000 | table1_group1 | 57638
|
||||
13000001 | table1_group1 | 57638
|
||||
13000002 | table1_group1 | 57637
|
||||
13000003 | table1_group1 | 57638
|
||||
13000004 | table1_group1 | 57637
|
||||
13000005 | table1_group1 | 57638
|
||||
13000006 | table2_group1 | 57637
|
||||
13000006 | table2_group1 | 57638
|
||||
13000007 | table2_group1 | 57638
|
||||
13000008 | table2_group1 | 57637
|
||||
13000009 | table2_group1 | 57638
|
||||
13000010 | table2_group1 | 57637
|
||||
13000011 | table2_group1 | 57638
|
||||
(14 rows)
|
||||
|
||||
-- try force_logical
|
||||
SELECT master_move_shard_placement(13000001, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'force_logical');
|
||||
ERROR: the force_logical transfer mode is currently unsupported
|
||||
-- move colocated shards
|
||||
SELECT master_move_shard_placement(13000001, 'localhost', :worker_2_port, 'localhost', :worker_1_port);
|
||||
master_move_shard_placement
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
-- status after shard move
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
colocationid = (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'table1_group1'::regclass)
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
shardid | logicalrelid | nodeport
|
||||
---------------------------------------------------------------------
|
||||
13000000 | table1_group1 | 57637
|
||||
13000000 | table1_group1 | 57638
|
||||
13000001 | table1_group1 | 57637
|
||||
13000002 | table1_group1 | 57637
|
||||
13000003 | table1_group1 | 57638
|
||||
13000004 | table1_group1 | 57637
|
||||
13000005 | table1_group1 | 57638
|
||||
13000006 | table2_group1 | 57637
|
||||
13000006 | table2_group1 | 57638
|
||||
13000007 | table2_group1 | 57637
|
||||
13000008 | table2_group1 | 57637
|
||||
13000009 | table2_group1 | 57638
|
||||
13000010 | table2_group1 | 57637
|
||||
13000011 | table2_group1 | 57638
|
||||
(14 rows)
|
||||
|
||||
-- also connect worker to verify we successfully moved given shard (and other colocated shards)
|
||||
\c - - - :worker_1_port
|
||||
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.table1_group1_13000001'::regclass;
|
||||
Column | Type | Modifiers
|
||||
---------------------------------------------------------------------
|
||||
id | integer | not null
|
||||
(1 row)
|
||||
|
||||
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.table2_group1_13000007'::regclass;
|
||||
Column | Type | Modifiers
|
||||
---------------------------------------------------------------------
|
||||
id | integer |
|
||||
(1 row)
|
||||
|
||||
\c - - - :master_port
|
||||
-- test moving NOT colocated shard
|
||||
-- status before shard move
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
p.logicalrelid = 'table5_groupX'::regclass
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
shardid | logicalrelid | nodeport
|
||||
---------------------------------------------------------------------
|
||||
13000012 | table5_groupx | 57637
|
||||
13000012 | table5_groupx | 57638
|
||||
13000013 | table5_groupx | 57638
|
||||
13000014 | table5_groupx | 57637
|
||||
13000015 | table5_groupx | 57638
|
||||
13000016 | table5_groupx | 57637
|
||||
13000017 | table5_groupx | 57638
|
||||
13000018 | table5_groupx | 57637
|
||||
13000019 | table5_groupx | 57638
|
||||
(9 rows)
|
||||
|
||||
-- move NOT colocated shard
|
||||
SELECT master_move_shard_placement(13000013, 'localhost', :worker_2_port, 'localhost', :worker_1_port);
|
||||
master_move_shard_placement
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
-- status after shard move
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
p.logicalrelid = 'table5_groupX'::regclass
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
shardid | logicalrelid | nodeport
|
||||
---------------------------------------------------------------------
|
||||
13000012 | table5_groupx | 57637
|
||||
13000012 | table5_groupx | 57638
|
||||
13000013 | table5_groupx | 57637
|
||||
13000014 | table5_groupx | 57637
|
||||
13000015 | table5_groupx | 57638
|
||||
13000016 | table5_groupx | 57637
|
||||
13000017 | table5_groupx | 57638
|
||||
13000018 | table5_groupx | 57637
|
||||
13000019 | table5_groupx | 57638
|
||||
(9 rows)
|
||||
|
||||
-- test moving shard in append distributed table
|
||||
-- status before shard move
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
p.logicalrelid = 'table6_append'::regclass
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
shardid | logicalrelid | nodeport
|
||||
---------------------------------------------------------------------
|
||||
13000020 | table6_append | 57638
|
||||
13000021 | table6_append | 57637
|
||||
(2 rows)
|
||||
|
||||
-- move shard in append distributed table
|
||||
SELECT master_move_shard_placement(13000021, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
|
||||
master_move_shard_placement
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
-- status after shard move
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
p.logicalrelid = 'table6_append'::regclass
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
shardid | logicalrelid | nodeport
|
||||
---------------------------------------------------------------------
|
||||
13000020 | table6_append | 57638
|
||||
13000021 | table6_append | 57638
|
||||
(2 rows)
|
||||
|
||||
-- try to move shard from wrong node
|
||||
SELECT master_move_shard_placement(13000021, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
|
||||
ERROR: could not find placement matching "localhost:xxxxx"
|
||||
HINT: Confirm the placement still exists and try again.
|
||||
-- test shard move with foreign constraints
|
||||
DROP TABLE IF EXISTS table1_group1, table2_group1;
|
||||
SET citus.shard_count TO 6;
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
-- create distributed tables
|
||||
CREATE TABLE table1_group1 ( id int PRIMARY KEY);
|
||||
SELECT create_distributed_table('table1_group1', 'id', 'hash');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
CREATE TABLE table2_group1 ( id int, table1_id int, FOREIGN KEY(table1_id) REFERENCES table1_group1(id));
|
||||
SELECT create_distributed_table('table2_group1', 'table1_id', 'hash');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
-- Mark the tables as non-mx tables
|
||||
UPDATE pg_dist_partition SET repmodel='c' WHERE logicalrelid IN
|
||||
('table1_group1'::regclass, 'table2_group1'::regclass);
|
||||
-- status before shard rebalance
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
colocationid = (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'table1_group1'::regclass)
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
shardid | logicalrelid | nodeport
|
||||
---------------------------------------------------------------------
|
||||
13000022 | table1_group1 | 57637
|
||||
13000023 | table1_group1 | 57638
|
||||
13000024 | table1_group1 | 57637
|
||||
13000025 | table1_group1 | 57638
|
||||
13000026 | table1_group1 | 57637
|
||||
13000027 | table1_group1 | 57638
|
||||
13000028 | table2_group1 | 57637
|
||||
13000029 | table2_group1 | 57638
|
||||
13000030 | table2_group1 | 57637
|
||||
13000031 | table2_group1 | 57638
|
||||
13000032 | table2_group1 | 57637
|
||||
13000033 | table2_group1 | 57638
|
||||
(12 rows)
|
||||
|
||||
SELECT master_move_shard_placement(13000022, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'block_writes');
|
||||
master_move_shard_placement
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
-- status after shard rebalance
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
colocationid = (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'table1_group1'::regclass)
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
shardid | logicalrelid | nodeport
|
||||
---------------------------------------------------------------------
|
||||
13000022 | table1_group1 | 57638
|
||||
13000023 | table1_group1 | 57638
|
||||
13000024 | table1_group1 | 57637
|
||||
13000025 | table1_group1 | 57638
|
||||
13000026 | table1_group1 | 57637
|
||||
13000027 | table1_group1 | 57638
|
||||
13000028 | table2_group1 | 57638
|
||||
13000029 | table2_group1 | 57638
|
||||
13000030 | table2_group1 | 57637
|
||||
13000031 | table2_group1 | 57638
|
||||
13000032 | table2_group1 | 57637
|
||||
13000033 | table2_group1 | 57638
|
||||
(12 rows)
|
||||
|
||||
-- also connect worker to verify we successfully moved given shard (and other colocated shards)
|
||||
\c - - - :worker_2_port
|
||||
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.table1_group1_13000022'::regclass;
|
||||
Column | Type | Modifiers
|
||||
---------------------------------------------------------------------
|
||||
id | integer | not null
|
||||
(1 row)
|
||||
|
||||
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.table2_group1_13000028'::regclass;
|
||||
Column | Type | Modifiers
|
||||
---------------------------------------------------------------------
|
||||
id | integer |
|
||||
table1_id | integer |
|
||||
(2 rows)
|
||||
|
||||
-- make sure that we've created the foreign keys
|
||||
SELECT "Constraint", "Definition" FROM table_fkeys
|
||||
WHERE "Constraint" LIKE 'table2_group%' OR "Constraint" LIKE 'table1_group%';
|
||||
Constraint | Definition
|
||||
---------------------------------------------------------------------
|
||||
table2_group1_table1_id_fkey_13000028 | FOREIGN KEY (table1_id) REFERENCES table1_group1_13000022(id)
|
||||
table2_group1_table1_id_fkey_13000029 | FOREIGN KEY (table1_id) REFERENCES table1_group1_13000023(id)
|
||||
table2_group1_table1_id_fkey_13000031 | FOREIGN KEY (table1_id) REFERENCES table1_group1_13000025(id)
|
||||
table2_group1_table1_id_fkey_13000033 | FOREIGN KEY (table1_id) REFERENCES table1_group1_13000027(id)
|
||||
(4 rows)
|
||||
|
||||
\c - - - :master_port
|
||||
-- test shard copy with foreign constraints
|
||||
-- we expect it to error out because we do not support foreign constraints with replication factor > 1
|
||||
SELECT master_copy_shard_placement(13000022, 'localhost', :worker_2_port, 'localhost', :worker_1_port, false);
|
||||
ERROR: cannot replicate shards with foreign keys
|
||||
-- lets also test that master_move_shard_placement doesn't break serials
|
||||
CREATE TABLE serial_move_test (key int, other_val serial);
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
SELECT create_distributed_table('serial_move_test', 'key');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
-- key 15 goes to shard xxxxx
|
||||
INSERT INTO serial_move_test (key) VALUES (15) RETURNING *;
|
||||
key | other_val
|
||||
---------------------------------------------------------------------
|
||||
15 | 1
|
||||
(1 row)
|
||||
|
||||
INSERT INTO serial_move_test (key) VALUES (15) RETURNING *;
|
||||
key | other_val
|
||||
---------------------------------------------------------------------
|
||||
15 | 2
|
||||
(1 row)
|
||||
|
||||
-- confirm the shard id
|
||||
SELECT * FROM run_command_on_placements('serial_move_test', 'SELECT DISTINCT key FROM %s WHERE key = 15') WHERE result = '15' AND shardid = 13000034;
|
||||
nodename | nodeport | shardid | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | 13000034 | t | 15
|
||||
(1 row)
|
||||
|
||||
SELECT master_move_shard_placement(13000034, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
|
||||
master_move_shard_placement
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
-- confirm the successfull move
|
||||
SELECT * FROM run_command_on_placements('serial_move_test', 'SELECT DISTINCT key FROM %s WHERE key = 15') WHERE result = '15' AND shardid = 13000034;
|
||||
nodename | nodeport | shardid | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57638 | 13000034 | t | 15
|
||||
(1 row)
|
||||
|
||||
-- finally show that serials work fine afterwards
|
||||
INSERT INTO serial_move_test (key) VALUES (15) RETURNING *;
|
||||
key | other_val
|
||||
---------------------------------------------------------------------
|
||||
15 | 3
|
||||
(1 row)
|
||||
|
||||
INSERT INTO serial_move_test (key) VALUES (15) RETURNING *;
|
||||
key | other_val
|
||||
---------------------------------------------------------------------
|
||||
15 | 4
|
||||
(1 row)
|
||||
|
||||
-- we should be able to move shard placements of partitioend tables
|
||||
CREATE SCHEMA move_partitions;
|
||||
CREATE TABLE move_partitions.events (
|
||||
id serial,
|
||||
t timestamptz default now(),
|
||||
payload text
|
||||
)
|
||||
PARTITION BY RANGE(t);
|
||||
SET citus.shard_count TO 6;
|
||||
SELECT create_distributed_table('move_partitions.events', 'id', colocate_with := 'none');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
CREATE TABLE move_partitions.events_1 PARTITION OF move_partitions.events
|
||||
FOR VALUES FROM ('2015-01-01') TO ('2016-01-01');
|
||||
INSERT INTO move_partitions.events (t, payload)
|
||||
SELECT '2015-01-01'::date + (interval '1 day' * s), s FROM generate_series(1, 100) s;
|
||||
SELECT count(*) FROM move_partitions.events;
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
100
|
||||
(1 row)
|
||||
|
||||
-- try to move automatically
|
||||
SELECT master_move_shard_placement(shardid, 'localhost', :worker_2_port, 'localhost', :worker_1_port)
|
||||
FROM pg_dist_shard JOIN pg_dist_shard_placement USING (shardid)
|
||||
WHERE logicalrelid = 'move_partitions.events'::regclass AND nodeport = :worker_2_port
|
||||
ORDER BY shardid LIMIT 1;
|
||||
master_move_shard_placement
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT count(*) FROM move_partitions.events;
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
100
|
||||
(1 row)
|
||||
|
||||
-- add a primary key to the partition
|
||||
ALTER TABLE move_partitions.events_1 ADD CONSTRAINT e_1_pk PRIMARY KEY (id);
|
||||
-- should be able to move automatically now
|
||||
SELECT master_move_shard_placement(shardid, 'localhost', :worker_2_port, 'localhost', :worker_1_port)
|
||||
FROM pg_dist_shard JOIN pg_dist_shard_placement USING (shardid)
|
||||
WHERE logicalrelid = 'move_partitions.events'::regclass AND nodeport = :worker_2_port
|
||||
ORDER BY shardid LIMIT 1;
|
||||
master_move_shard_placement
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT count(*) FROM move_partitions.events;
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
100
|
||||
(1 row)
|
||||
|
||||
-- should also be able to move with block writes
|
||||
SELECT master_move_shard_placement(shardid, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'block_writes')
|
||||
FROM pg_dist_shard JOIN pg_dist_shard_placement USING (shardid)
|
||||
WHERE logicalrelid = 'move_partitions.events'::regclass AND nodeport = :worker_2_port
|
||||
ORDER BY shardid LIMIT 1;
|
||||
master_move_shard_placement
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
SELECT count(*) FROM move_partitions.events;
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
100
|
||||
(1 row)
|
||||
|
||||
-- should have moved all shards to node 1 (2*6 = 12)
|
||||
SELECT count(*)
|
||||
FROM pg_dist_shard JOIN pg_dist_shard_placement USING (shardid)
|
||||
WHERE logicalrelid::text LIKE 'move_partitions.events%' AND nodeport = :worker_1_port;
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
12
|
||||
(1 row)
|
||||
|
||||
DROP TABLE move_partitions.events;
|
|
@ -0,0 +1,235 @@
|
|||
--
|
||||
-- MULTI_MOVE_MX
|
||||
--
|
||||
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1550000;
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_port);
|
||||
start_metadata_sync_to_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
-- Create mx test tables
|
||||
SET citus.shard_count TO 4;
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
SET citus.replication_model TO 'streaming';
|
||||
CREATE TABLE mx_table_1 (a int);
|
||||
SELECT create_distributed_table('mx_table_1', 'a');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
CREATE TABLE mx_table_2 (a int);
|
||||
SELECT create_distributed_table('mx_table_2', 'a');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
CREATE TABLE mx_table_3 (a text);
|
||||
SELECT create_distributed_table('mx_table_3', 'a');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
-- Check that the first two tables are colocated
|
||||
SELECT
|
||||
logicalrelid, repmodel
|
||||
FROM
|
||||
pg_dist_partition
|
||||
WHERE
|
||||
logicalrelid = 'mx_table_1'::regclass
|
||||
OR logicalrelid = 'mx_table_2'::regclass
|
||||
OR logicalrelid = 'mx_table_3'::regclass
|
||||
ORDER BY
|
||||
logicalrelid;
|
||||
logicalrelid | repmodel
|
||||
---------------------------------------------------------------------
|
||||
mx_table_1 | s
|
||||
mx_table_2 | s
|
||||
mx_table_3 | s
|
||||
(3 rows)
|
||||
|
||||
-- Check the list of shards
|
||||
SELECT
|
||||
logicalrelid, shardid, nodename, nodeport
|
||||
FROM
|
||||
pg_dist_shard NATURAL JOIN pg_dist_shard_placement
|
||||
WHERE
|
||||
logicalrelid = 'mx_table_1'::regclass
|
||||
OR logicalrelid = 'mx_table_2'::regclass
|
||||
OR logicalrelid = 'mx_table_3'::regclass
|
||||
ORDER BY
|
||||
logicalrelid, shardid;
|
||||
logicalrelid | shardid | nodename | nodeport
|
||||
---------------------------------------------------------------------
|
||||
mx_table_1 | 1550000 | localhost | 57637
|
||||
mx_table_1 | 1550001 | localhost | 57638
|
||||
mx_table_1 | 1550002 | localhost | 57637
|
||||
mx_table_1 | 1550003 | localhost | 57638
|
||||
mx_table_2 | 1550004 | localhost | 57637
|
||||
mx_table_2 | 1550005 | localhost | 57638
|
||||
mx_table_2 | 1550006 | localhost | 57637
|
||||
mx_table_2 | 1550007 | localhost | 57638
|
||||
mx_table_3 | 1550008 | localhost | 57637
|
||||
mx_table_3 | 1550009 | localhost | 57638
|
||||
mx_table_3 | 1550010 | localhost | 57637
|
||||
mx_table_3 | 1550011 | localhost | 57638
|
||||
(12 rows)
|
||||
|
||||
-- Check the data on the worker
|
||||
\c - - - :worker_2_port
|
||||
SELECT
|
||||
logicalrelid, shardid, nodename, nodeport
|
||||
FROM
|
||||
pg_dist_shard NATURAL JOIN pg_dist_shard_placement
|
||||
WHERE
|
||||
logicalrelid = 'mx_table_1'::regclass
|
||||
OR logicalrelid = 'mx_table_2'::regclass
|
||||
OR logicalrelid = 'mx_table_3'::regclass
|
||||
ORDER BY
|
||||
logicalrelid, shardid;
|
||||
logicalrelid | shardid | nodename | nodeport
|
||||
---------------------------------------------------------------------
|
||||
mx_table_1 | 1550000 | localhost | 57637
|
||||
mx_table_1 | 1550001 | localhost | 57638
|
||||
mx_table_1 | 1550002 | localhost | 57637
|
||||
mx_table_1 | 1550003 | localhost | 57638
|
||||
mx_table_2 | 1550004 | localhost | 57637
|
||||
mx_table_2 | 1550005 | localhost | 57638
|
||||
mx_table_2 | 1550006 | localhost | 57637
|
||||
mx_table_2 | 1550007 | localhost | 57638
|
||||
mx_table_3 | 1550008 | localhost | 57637
|
||||
mx_table_3 | 1550009 | localhost | 57638
|
||||
mx_table_3 | 1550010 | localhost | 57637
|
||||
mx_table_3 | 1550011 | localhost | 57638
|
||||
(12 rows)
|
||||
|
||||
\c - - - :master_port
|
||||
-- Check that master_copy_shard_placement cannot be run with MX tables
|
||||
SELECT
|
||||
master_copy_shard_placement(shardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port, false, 'force_logical')
|
||||
FROM
|
||||
pg_dist_shard NATURAL JOIN pg_dist_shard_placement
|
||||
WHERE
|
||||
logicalrelid = 'mx_table_1'::regclass
|
||||
AND nodeport = :worker_1_port
|
||||
ORDER BY
|
||||
shardid
|
||||
LIMIT 1;
|
||||
ERROR: the force_logical transfer mode is currently unsupported
|
||||
-- Move a shard from worker 1 to worker 2
|
||||
SELECT
|
||||
master_move_shard_placement(shardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port)
|
||||
FROM
|
||||
pg_dist_shard NATURAL JOIN pg_dist_shard_placement
|
||||
WHERE
|
||||
logicalrelid = 'mx_table_1'::regclass
|
||||
AND nodeport = :worker_1_port
|
||||
ORDER BY
|
||||
shardid
|
||||
LIMIT 1;
|
||||
master_move_shard_placement
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
-- Check that the shard and its colocated shard is moved, but not the other shards
|
||||
SELECT
|
||||
logicalrelid, shardid, nodename, nodeport
|
||||
FROM
|
||||
pg_dist_shard NATURAL JOIN pg_dist_shard_placement
|
||||
WHERE
|
||||
logicalrelid = 'mx_table_1'::regclass
|
||||
OR logicalrelid = 'mx_table_2'::regclass
|
||||
OR logicalrelid = 'mx_table_3'::regclass
|
||||
ORDER BY
|
||||
logicalrelid, shardid;
|
||||
logicalrelid | shardid | nodename | nodeport
|
||||
---------------------------------------------------------------------
|
||||
mx_table_1 | 1550000 | localhost | 57638
|
||||
mx_table_1 | 1550001 | localhost | 57638
|
||||
mx_table_1 | 1550002 | localhost | 57637
|
||||
mx_table_1 | 1550003 | localhost | 57638
|
||||
mx_table_2 | 1550004 | localhost | 57638
|
||||
mx_table_2 | 1550005 | localhost | 57638
|
||||
mx_table_2 | 1550006 | localhost | 57637
|
||||
mx_table_2 | 1550007 | localhost | 57638
|
||||
mx_table_3 | 1550008 | localhost | 57637
|
||||
mx_table_3 | 1550009 | localhost | 57638
|
||||
mx_table_3 | 1550010 | localhost | 57637
|
||||
mx_table_3 | 1550011 | localhost | 57638
|
||||
(12 rows)
|
||||
|
||||
-- Check that the changes are made in the worker as well
|
||||
\c - - - :worker_2_port
|
||||
SELECT
|
||||
logicalrelid, shardid, nodename, nodeport
|
||||
FROM
|
||||
pg_dist_shard NATURAL JOIN pg_dist_shard_placement
|
||||
WHERE
|
||||
logicalrelid = 'mx_table_1'::regclass
|
||||
OR logicalrelid = 'mx_table_2'::regclass
|
||||
OR logicalrelid = 'mx_table_3'::regclass
|
||||
ORDER BY
|
||||
logicalrelid, shardid;
|
||||
logicalrelid | shardid | nodename | nodeport
|
||||
---------------------------------------------------------------------
|
||||
mx_table_1 | 1550000 | localhost | 57638
|
||||
mx_table_1 | 1550001 | localhost | 57638
|
||||
mx_table_1 | 1550002 | localhost | 57637
|
||||
mx_table_1 | 1550003 | localhost | 57638
|
||||
mx_table_2 | 1550004 | localhost | 57638
|
||||
mx_table_2 | 1550005 | localhost | 57638
|
||||
mx_table_2 | 1550006 | localhost | 57637
|
||||
mx_table_2 | 1550007 | localhost | 57638
|
||||
mx_table_3 | 1550008 | localhost | 57637
|
||||
mx_table_3 | 1550009 | localhost | 57638
|
||||
mx_table_3 | 1550010 | localhost | 57637
|
||||
mx_table_3 | 1550011 | localhost | 57638
|
||||
(12 rows)
|
||||
|
||||
-- Check that the UDFs cannot be called from the workers
|
||||
SELECT
|
||||
master_copy_shard_placement(shardid, 'localhost', :worker_2_port, 'localhost', :worker_1_port, false, 'force_logical')
|
||||
FROM
|
||||
pg_dist_shard NATURAL JOIN pg_dist_shard_placement
|
||||
WHERE
|
||||
logicalrelid = 'mx_table_1'::regclass
|
||||
AND nodeport = :worker_2_port
|
||||
ORDER BY
|
||||
shardid
|
||||
LIMIT 1 OFFSET 1;
|
||||
ERROR: operation is not allowed on this node
|
||||
HINT: Connect to the coordinator and run it again.
|
||||
SELECT
|
||||
master_move_shard_placement(shardid, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'force_logical')
|
||||
FROM
|
||||
pg_dist_shard NATURAL JOIN pg_dist_shard_placement
|
||||
WHERE
|
||||
logicalrelid = 'mx_table_1'::regclass
|
||||
AND nodeport = :worker_2_port
|
||||
ORDER BY
|
||||
shardid
|
||||
LIMIT 1 OFFSET 1;
|
||||
ERROR: operation is not allowed on this node
|
||||
HINT: Connect to the coordinator and run it again.
|
||||
-- Cleanup
|
||||
\c - - - :master_port
|
||||
DROP TABLE mx_table_1;
|
||||
DROP TABLE mx_table_2;
|
||||
DROP TABLE mx_table_3;
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_2_port);
|
||||
stop_metadata_sync_to_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
\c - - - :worker_2_port
|
||||
DELETE FROM pg_dist_node;
|
||||
DELETE FROM pg_dist_partition;
|
||||
DELETE FROM pg_dist_shard;
|
||||
DELETE FROM pg_dist_shard_placement;
|
||||
\c - - - :master_port
|
||||
RESET citus.replication_model;
|
|
@ -1,3 +1,9 @@
|
|||
CREATE OR REPLACE FUNCTION master_defer_delete_shards()
|
||||
RETURNS int
|
||||
LANGUAGE C STRICT
|
||||
AS 'citus', $$master_defer_delete_shards$$;
|
||||
COMMENT ON FUNCTION master_defer_delete_shards()
|
||||
IS 'remove orphaned shards';
|
||||
CREATE OR REPLACE FUNCTION wait_until_metadata_sync(timeout INTEGER DEFAULT 15000)
|
||||
RETURNS void
|
||||
LANGUAGE C STRICT
|
||||
|
|
|
@ -25,7 +25,3 @@ ERROR: cannot write to pg_dist_poolinfo
|
|||
DETAIL: Citus Community Edition does not support the use of pooler options.
|
||||
HINT: To learn more about using advanced pooling schemes with Citus, please contact us at https://citusdata.com/about/contact_us
|
||||
ROLLBACK;
|
||||
INSERT INTO pg_dist_rebalance_strategy VALUES ('should fail', false, 'citus_shard_cost_1', 'citus_node_capacity_1', 'citus_shard_allowed_on_node_true', 0, 0);
|
||||
ERROR: cannot write to pg_dist_rebalance_strategy
|
||||
DETAIL: Citus Community Edition does not support the use of custom rebalance strategies.
|
||||
HINT: To learn more about using advanced rebalancing schemes with Citus, please contact us at https://citusdata.com/about/contact_us
|
||||
|
|
|
@ -0,0 +1,111 @@
|
|||
--
|
||||
-- SHARD_MOVE_DEFERRED_DELETE
|
||||
--
|
||||
SET citus.next_shard_id TO 20000000;
|
||||
SET citus.shard_count TO 6;
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
SET citus.defer_drop_after_shard_move TO on;
|
||||
CREATE SCHEMA shard_move_deferred_delete;
|
||||
SET search_path TO shard_move_deferred_delete;
|
||||
CREATE TABLE t1 ( id int PRIMARY KEY);
|
||||
SELECT create_distributed_table('t1', 'id');
|
||||
create_distributed_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
-- by counting how ofter we see the specific shard on all workers we can verify is the shard is there
|
||||
SELECT run_command_on_workers($cmd$
|
||||
SELECT count(*) FROM pg_class WHERE relname = 't1_20000000';
|
||||
$cmd$);
|
||||
run_command_on_workers
|
||||
---------------------------------------------------------------------
|
||||
(localhost,57637,t,1)
|
||||
(localhost,57638,t,0)
|
||||
(2 rows)
|
||||
|
||||
-- move shard
|
||||
SELECT master_move_shard_placement(20000000, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
|
||||
master_move_shard_placement
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
-- we expect the shard to be on both workers now
|
||||
SELECT run_command_on_workers($cmd$
|
||||
SELECT count(*) FROM pg_class WHERE relname = 't1_20000000';
|
||||
$cmd$);
|
||||
run_command_on_workers
|
||||
---------------------------------------------------------------------
|
||||
(localhost,57637,t,1)
|
||||
(localhost,57638,t,1)
|
||||
(2 rows)
|
||||
|
||||
-- execute delayed removal
|
||||
SELECT public.master_defer_delete_shards();
|
||||
master_defer_delete_shards
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
-- we expect the shard to be on only the second worker
|
||||
SELECT run_command_on_workers($cmd$
|
||||
SELECT count(*) FROM pg_class WHERE relname = 't1_20000000';
|
||||
$cmd$);
|
||||
run_command_on_workers
|
||||
---------------------------------------------------------------------
|
||||
(localhost,57637,t,0)
|
||||
(localhost,57638,t,1)
|
||||
(2 rows)
|
||||
|
||||
SELECT master_move_shard_placement(20000000, 'localhost', :worker_2_port, 'localhost', :worker_1_port);
|
||||
master_move_shard_placement
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
-- we expect the shard to be on both workers now
|
||||
SELECT run_command_on_workers($cmd$
|
||||
SELECT count(*) FROM pg_class WHERE relname = 't1_20000000';
|
||||
$cmd$);
|
||||
run_command_on_workers
|
||||
---------------------------------------------------------------------
|
||||
(localhost,57637,t,1)
|
||||
(localhost,57638,t,1)
|
||||
(2 rows)
|
||||
|
||||
-- enable auto delete
|
||||
ALTER SYSTEM SET citus.defer_shard_delete_interval TO 10;
|
||||
SELECT pg_reload_conf();
|
||||
pg_reload_conf
|
||||
---------------------------------------------------------------------
|
||||
t
|
||||
(1 row)
|
||||
|
||||
-- Sleep 1 second to give Valgrind enough time to clear transactions
|
||||
SELECT pg_sleep(1);
|
||||
pg_sleep
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
-- we expect the shard to be on only the first worker
|
||||
SELECT run_command_on_workers($cmd$
|
||||
SELECT count(*) FROM pg_class WHERE relname = 't1_20000000';
|
||||
$cmd$);
|
||||
run_command_on_workers
|
||||
---------------------------------------------------------------------
|
||||
(localhost,57637,t,1)
|
||||
(localhost,57638,t,0)
|
||||
(2 rows)
|
||||
|
||||
-- reset test suite
|
||||
ALTER SYSTEM SET citus.defer_shard_delete_interval TO -1;
|
||||
SELECT pg_reload_conf();
|
||||
pg_reload_conf
|
||||
---------------------------------------------------------------------
|
||||
t
|
||||
(1 row)
|
||||
|
||||
DROP SCHEMA shard_move_deferred_delete CASCADE;
|
||||
NOTICE: drop cascades to table t1
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,502 @@
|
|||
CREATE OR REPLACE FUNCTION shard_placement_rebalance_array(
|
||||
worker_node_list json[],
|
||||
shard_placement_list json[],
|
||||
threshold float4 DEFAULT 0,
|
||||
max_shard_moves int DEFAULT 1000000,
|
||||
drain_only bool DEFAULT false
|
||||
)
|
||||
RETURNS json[]
|
||||
AS 'citus'
|
||||
LANGUAGE C STRICT VOLATILE;
|
||||
-- Check that even with threshold=0.0 shard_placement_rebalance_array returns
|
||||
-- something when there's no completely balanced solution.
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432}',
|
||||
'{"node_name": "hostname2", "node_port": 5432}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}']::json[]
|
||||
));
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
{"updatetype":1,"shardid":1,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname2","targetport":5432}
|
||||
(1 row)
|
||||
|
||||
-- Check that a node can be drained in a balanced cluster
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432, "disallowed_shards": "1,2,3,4"}',
|
||||
'{"node_name": "hostname2", "node_port": 5432}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}'
|
||||
]::json[]
|
||||
));
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
{"updatetype":1,"shardid":1,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname2","targetport":5432}
|
||||
{"updatetype":1,"shardid":2,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname2","targetport":5432}
|
||||
(2 rows)
|
||||
|
||||
-- Check that an already drained node won't be filled again after a second
|
||||
-- rebalance
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432, "disallowed_shards": "1,2,3,4"}',
|
||||
'{"node_name": "hostname2", "node_port": 5432}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}'
|
||||
]::json[]
|
||||
));
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
-- Check that even when shards are already balanced, but shard xxxxx is on a node
|
||||
-- where it is not allowed it will be moved and there will be rebalancing later
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432, "disallowed_shards": "1,2,3,5,6"}',
|
||||
'{"node_name": "hostname2", "node_port": 5432, "disallowed_shards": "4"}',
|
||||
'{"node_name": "hostname3", "node_port": 5432, "disallowed_shards": "4"}'
|
||||
]::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":5, "shardid":5, "shardstate":1, "shardlength":1, "nodename":"hostname3", "nodeport":5432}',
|
||||
'{"placementid":6, "shardid":6, "shardstate":1, "shardlength":1, "nodename":"hostname3", "nodeport":5432}'
|
||||
]::json[]
|
||||
));
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
{"updatetype":1,"shardid":1,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname2","targetport":5432}
|
||||
{"updatetype":1,"shardid":2,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname3","targetport":5432}
|
||||
{"updatetype":1,"shardid":4,"sourcename":"hostname2","sourceport":5432,"targetname":"hostname1","targetport":5432}
|
||||
(3 rows)
|
||||
|
||||
-- Check that even when shards are already balanced, disallowed shards will be
|
||||
-- moved away from hostname1 and the only shard that is allowed there will be
|
||||
-- moved there
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432, "disallowed_shards": "1,2,3,5,6"}',
|
||||
'{"node_name": "hostname2", "node_port": 5432}',
|
||||
'{"node_name": "hostname3", "node_port": 5432}'
|
||||
]::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":5, "shardid":5, "shardstate":1, "shardlength":1, "nodename":"hostname3", "nodeport":5432}',
|
||||
'{"placementid":6, "shardid":6, "shardstate":1, "shardlength":1, "nodename":"hostname3", "nodeport":5432}'
|
||||
]::json[]
|
||||
));
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
{"updatetype":1,"shardid":1,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname2","targetport":5432}
|
||||
{"updatetype":1,"shardid":2,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname3","targetport":5432}
|
||||
{"updatetype":1,"shardid":4,"sourcename":"hostname2","sourceport":5432,"targetname":"hostname1","targetport":5432}
|
||||
(3 rows)
|
||||
|
||||
-- Check that an error is returned when a shard is not allowed anywhere
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432, "disallowed_shards": "2,4"}',
|
||||
'{"node_name": "hostname2", "node_port": 5432, "disallowed_shards": "1,4"}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}'
|
||||
]::json[]
|
||||
));
|
||||
WARNING: Not allowed to move shard xxxxx anywhere from hostname2:5432
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
{"updatetype":1,"shardid":2,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname2","targetport":5432}
|
||||
{"updatetype":1,"shardid":3,"sourcename":"hostname2","sourceport":5432,"targetname":"hostname1","targetport":5432}
|
||||
(2 rows)
|
||||
|
||||
-- Check that cost is taken into account when rebalancing
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432}',
|
||||
'{"node_name": "hostname2", "node_port": 5432}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432, "cost": 3}']::json[]
|
||||
));
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
{"updatetype":1,"shardid":4,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname2","targetport":5432}
|
||||
(1 row)
|
||||
|
||||
-- Check that cost is taken into account when rebalancing disallowed placements
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432, "disallowed_shards": "1,2,3,4"}',
|
||||
'{"node_name": "hostname2", "node_port": 5432}',
|
||||
'{"node_name": "hostname3", "node_port": 5432}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432, "cost": 3}']::json[]
|
||||
));
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
{"updatetype":1,"shardid":4,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname2","targetport":5432}
|
||||
{"updatetype":1,"shardid":1,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname3","targetport":5432}
|
||||
{"updatetype":1,"shardid":2,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname3","targetport":5432}
|
||||
{"updatetype":1,"shardid":3,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname3","targetport":5432}
|
||||
(4 rows)
|
||||
|
||||
-- Check that node capacacity is taken into account.
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432}',
|
||||
'{"node_name": "hostname2", "node_port": 5432, "capacity": 3}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}']::json[]
|
||||
));
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
{"updatetype":1,"shardid":1,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname2","targetport":5432}
|
||||
{"updatetype":1,"shardid":2,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname2","targetport":5432}
|
||||
{"updatetype":1,"shardid":3,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname2","targetport":5432}
|
||||
(3 rows)
|
||||
|
||||
-- Check that shards are not moved when target utilization stays the same and
|
||||
-- the source utilization goes below the original target utilization. hostname1
|
||||
-- has utilization of 1, after move hostname2 would have a utilization of 1 as
|
||||
-- well. hostname1 would have utilization of 1 while hostname2 has utilization
|
||||
-- of 2/3 now. Since load is spread more fairly with utilization 2/3 than 0 it
|
||||
-- should choose that distribution.
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432}',
|
||||
'{"node_name": "hostname2", "node_port": 5432, "capacity": 3}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}']::json[]
|
||||
));
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
-- Check that shards are moved even when target utilization stays the same, but
|
||||
-- source utilization goes below the original target utilization. hostname2
|
||||
-- has utilization of 1, after move hostname1 would have a utilization of 1 as
|
||||
-- well. hostname2 would have utilization of 2/3 while hostname1 now has
|
||||
-- utilization of 0 now. Since load is spread more fairly with utilization 2/3
|
||||
-- than 0 it should choose that distribution.
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432}',
|
||||
'{"node_name": "hostname2", "node_port": 5432, "capacity": 3}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}']::json[]
|
||||
));
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
{"updatetype":1,"shardid":1,"sourcename":"hostname2","sourceport":5432,"targetname":"hostname1","targetport":5432}
|
||||
(1 row)
|
||||
|
||||
-- Check that shards are moved even when target utilization stays the same, but
|
||||
-- source utilization goes below the original target utilization. hostname2
|
||||
-- has utilization of 2, after move hostname1 would have a utilization of 2 as
|
||||
-- well. hostname2 would have utilization of 1.5 while hostname1 now has
|
||||
-- utilization of 1. Since load is spread more fairly with utilization 1.5 than
|
||||
-- 1 it should choose that distribution.
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432}',
|
||||
'{"node_name": "hostname2", "node_port": 5432, "capacity": 2}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":5, "shardid":5, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}']::json[]
|
||||
));
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
{"updatetype":1,"shardid":2,"sourcename":"hostname2","sourceport":5432,"targetname":"hostname1","targetport":5432}
|
||||
(1 row)
|
||||
|
||||
-- Check that shards are moved even when target utilization stays the same, but
|
||||
-- source utilization goes below the original target utilization. hostname1
|
||||
-- has utilization of 2, after move hostname2 would have a utilization of 2 as
|
||||
-- well. hostname1 would have utilization of 1 while hostname2 now has
|
||||
-- utilization of 1.5. Since load is spread more fairly with utilization 1.5
|
||||
-- than 1 it should choose that distribution.
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432}',
|
||||
'{"node_name": "hostname2", "node_port": 5432, "capacity": 2}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":5, "shardid":5, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}']::json[]
|
||||
));
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
-- Check that all shards will be moved to 1 node if its capacity is big enough
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432}',
|
||||
'{"node_name": "hostname2", "node_port": 5432, "capacity": 4}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}']::json[]
|
||||
));
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
{"updatetype":1,"shardid":1,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname2","targetport":5432}
|
||||
{"updatetype":1,"shardid":2,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname2","targetport":5432}
|
||||
{"updatetype":1,"shardid":3,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname2","targetport":5432}
|
||||
(3 rows)
|
||||
|
||||
-- Check that shards will be moved to a smaller node node if utilization improves
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432}',
|
||||
'{"node_name": "hostname2", "node_port": 5432, "capacity": 3}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}']::json[]
|
||||
));
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
{"updatetype":1,"shardid":1,"sourcename":"hostname2","sourceport":5432,"targetname":"hostname1","targetport":5432}
|
||||
(1 row)
|
||||
|
||||
-- Check that node capacity works with different shard costs
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432}',
|
||||
'{"node_name": "hostname2", "node_port": 5432, "capacity": 3}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432, "cost": 3}']::json[]
|
||||
));
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
{"updatetype":1,"shardid":1,"sourcename":"hostname2","sourceport":5432,"targetname":"hostname1","targetport":5432}
|
||||
(1 row)
|
||||
|
||||
-- Check that node capacity works with different shard costs again
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432}',
|
||||
'{"node_name": "hostname2", "node_port": 5432, "capacity": 3}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432, "cost": 2}']::json[]
|
||||
));
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
{"updatetype":1,"shardid":3,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname2","targetport":5432}
|
||||
{"updatetype":1,"shardid":1,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname2","targetport":5432}
|
||||
(2 rows)
|
||||
|
||||
-- Check that max_shard_moves works and that we get a NOTICE that it is hit
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432}',
|
||||
'{"node_name": "hostname2", "node_port": 5432, "capacity": 3}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432, "cost": 2}']::json[],
|
||||
max_shard_moves := 1
|
||||
));
|
||||
NOTICE: Stopped searching before we were out of moves. Please rerun the rebalancer after it's finished for a more optimal placement.
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
{"updatetype":1,"shardid":3,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname2","targetport":5432}
|
||||
(1 row)
|
||||
|
||||
-- Check that node capacity works with different shard costs and disallowed_shards
|
||||
-- NOTE: these moves are not optimal, once we implement merging of updates this
|
||||
-- output should change.
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432}',
|
||||
'{"node_name": "hostname2", "node_port": 5432, "capacity": 5}',
|
||||
'{"node_name": "hostname3", "node_port": 5432, "disallowed_shards": "1,2"}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname3", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname3", "nodeport":5432, "cost": 2}']::json[]
|
||||
));
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
{"updatetype":1,"shardid":2,"sourcename":"hostname3","sourceport":5432,"targetname":"hostname2","targetport":5432}
|
||||
{"updatetype":1,"shardid":1,"sourcename":"hostname3","sourceport":5432,"targetname":"hostname1","targetport":5432}
|
||||
{"updatetype":1,"shardid":1,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname2","targetport":5432}
|
||||
(3 rows)
|
||||
|
||||
-- Check that draining + rebalancing nodes works
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432, "disallowed_shards": "1,2,3,4,5,6", "capacity": 0}',
|
||||
'{"node_name": "hostname2", "node_port": 5432}',
|
||||
'{"node_name": "hostname3", "node_port": 5432}'
|
||||
]::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":5, "shardid":5, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":6, "shardid":6, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}'
|
||||
]::json[]
|
||||
));
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
{"updatetype":1,"shardid":1,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname3","targetport":5432}
|
||||
{"updatetype":1,"shardid":2,"sourcename":"hostname2","sourceport":5432,"targetname":"hostname3","targetport":5432}
|
||||
{"updatetype":1,"shardid":3,"sourcename":"hostname2","sourceport":5432,"targetname":"hostname3","targetport":5432}
|
||||
(3 rows)
|
||||
|
||||
-- Check that draining nodes with drain only works
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432, "disallowed_shards": "1,2,3,4,5,6", "capacity": 0}',
|
||||
'{"node_name": "hostname2", "node_port": 5432}',
|
||||
'{"node_name": "hostname3", "node_port": 5432}'
|
||||
]::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":5, "shardid":5, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":6, "shardid":6, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}'
|
||||
]::json[],
|
||||
drain_only := true
|
||||
));
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
{"updatetype":1,"shardid":1,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname3","targetport":5432}
|
||||
(1 row)
|
||||
|
||||
-- Check that draining nodes has priority over max_shard_moves
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432, "disallowed_shards": "1,2,3,4,5,6", "capacity": 0}',
|
||||
'{"node_name": "hostname2", "node_port": 5432}',
|
||||
'{"node_name": "hostname3", "node_port": 5432}'
|
||||
]::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":5, "shardid":5, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":6, "shardid":6, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}'
|
||||
]::json[],
|
||||
max_shard_moves := 0
|
||||
));
|
||||
NOTICE: Stopped searching before we were out of moves. Please rerun the rebalancer after it's finished for a more optimal placement.
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
{"updatetype":1,"shardid":1,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname3","targetport":5432}
|
||||
(1 row)
|
||||
|
||||
-- Check that drained moves are counted towards shard moves and thus use up the
|
||||
-- limit when doing normal rebalance moves
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432, "disallowed_shards": "1,2,3,4,5,6", "capacity": 0}',
|
||||
'{"node_name": "hostname2", "node_port": 5432}',
|
||||
'{"node_name": "hostname3", "node_port": 5432}'
|
||||
]::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":5, "shardid":5, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":6, "shardid":6, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}'
|
||||
]::json[],
|
||||
max_shard_moves := 2
|
||||
));
|
||||
NOTICE: Stopped searching before we were out of moves. Please rerun the rebalancer after it's finished for a more optimal placement.
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
{"updatetype":1,"shardid":1,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname3","targetport":5432}
|
||||
{"updatetype":1,"shardid":2,"sourcename":"hostname2","sourceport":5432,"targetname":"hostname3","targetport":5432}
|
||||
(2 rows)
|
||||
|
||||
-- Check that draining for all colocation groups is done before rebalancing
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432, "disallowed_shards": "1,2,3,4,5,6,7,8,9,10,11,12", "capacity": 0}',
|
||||
'{"node_name": "hostname2", "node_port": 5432}',
|
||||
'{"node_name": "hostname3", "node_port": 5432}'
|
||||
]::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":5, "shardid":5, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":6, "shardid":6, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":7, "shardid":7, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432, "next_colocation": true}',
|
||||
'{"placementid":8, "shardid":8, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":9, "shardid":9, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":10, "shardid":10, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":11, "shardid":11, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":12, "shardid":12, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}'
|
||||
]::json[]
|
||||
));
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
{"updatetype":1,"shardid":1,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname3","targetport":5432}
|
||||
{"updatetype":1,"shardid":7,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname3","targetport":5432}
|
||||
{"updatetype":1,"shardid":2,"sourcename":"hostname2","sourceport":5432,"targetname":"hostname3","targetport":5432}
|
||||
{"updatetype":1,"shardid":3,"sourcename":"hostname2","sourceport":5432,"targetname":"hostname3","targetport":5432}
|
||||
{"updatetype":1,"shardid":8,"sourcename":"hostname2","sourceport":5432,"targetname":"hostname3","targetport":5432}
|
||||
{"updatetype":1,"shardid":9,"sourcename":"hostname2","sourceport":5432,"targetname":"hostname3","targetport":5432}
|
||||
(6 rows)
|
||||
|
||||
-- Check that max_shard_moves warning is only shown once even if more than one
|
||||
-- colocation group its placement updates are ignored because of it
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432, "disallowed_shards": "1,2,3,4,5,6,7,8,9,10,11,12", "capacity": 0}',
|
||||
'{"node_name": "hostname2", "node_port": 5432}',
|
||||
'{"node_name": "hostname3", "node_port": 5432}'
|
||||
]::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":5, "shardid":5, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":6, "shardid":6, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":7, "shardid":7, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432, "next_colocation": true}',
|
||||
'{"placementid":8, "shardid":8, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":9, "shardid":9, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":10, "shardid":10, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":11, "shardid":11, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":12, "shardid":12, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}'
|
||||
]::json[],
|
||||
max_shard_moves := 1
|
||||
));
|
||||
NOTICE: Stopped searching before we were out of moves. Please rerun the rebalancer after it's finished for a more optimal placement.
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
{"updatetype":1,"shardid":1,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname3","targetport":5432}
|
||||
{"updatetype":1,"shardid":7,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname3","targetport":5432}
|
||||
(2 rows)
|
||||
|
||||
-- Check that moves for different colocation groups are added together when
|
||||
-- taking into account max_shard_moves
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432, "disallowed_shards": "1,2,3,4,5,6,7,8,9,10,11,12", "capacity": 0}',
|
||||
'{"node_name": "hostname2", "node_port": 5432}',
|
||||
'{"node_name": "hostname3", "node_port": 5432}'
|
||||
]::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":5, "shardid":5, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":6, "shardid":6, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":7, "shardid":7, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432, "next_colocation": true}',
|
||||
'{"placementid":8, "shardid":8, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":9, "shardid":9, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":10, "shardid":10, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":11, "shardid":11, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":12, "shardid":12, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}'
|
||||
]::json[],
|
||||
max_shard_moves := 5
|
||||
));
|
||||
NOTICE: Stopped searching before we were out of moves. Please rerun the rebalancer after it's finished for a more optimal placement.
|
||||
unnest
|
||||
---------------------------------------------------------------------
|
||||
{"updatetype":1,"shardid":1,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname3","targetport":5432}
|
||||
{"updatetype":1,"shardid":7,"sourcename":"hostname1","sourceport":5432,"targetname":"hostname3","targetport":5432}
|
||||
{"updatetype":1,"shardid":2,"sourcename":"hostname2","sourceport":5432,"targetname":"hostname3","targetport":5432}
|
||||
{"updatetype":1,"shardid":3,"sourcename":"hostname2","sourceport":5432,"targetname":"hostname3","targetport":5432}
|
||||
{"updatetype":1,"shardid":8,"sourcename":"hostname2","sourceport":5432,"targetname":"hostname3","targetport":5432}
|
||||
(5 rows)
|
||||
|
|
@ -66,6 +66,13 @@ test: shared_connection_waits
|
|||
test: isolation_cancellation
|
||||
test: isolation_undistribute_table
|
||||
|
||||
# Rebalancer
|
||||
test: isolation_blocking_move_single_shard_commands
|
||||
test: isolation_blocking_move_multi_shard_commands
|
||||
test: isolation_blocking_move_single_shard_commands_on_mx
|
||||
test: isolation_blocking_move_multi_shard_commands_on_mx
|
||||
test: isolation_shard_rebalancer
|
||||
|
||||
# MX tests
|
||||
test: isolation_reference_on_mx
|
||||
test: isolation_ref2ref_foreign_keys_on_mx
|
||||
|
|
|
@ -0,0 +1,9 @@
|
|||
test: multi_cluster_management
|
||||
test: multi_test_helpers multi_test_helpers_superuser
|
||||
test: multi_test_catalog_views
|
||||
test: shard_rebalancer_unit
|
||||
test: shard_rebalancer
|
||||
test: foreign_key_to_reference_shard_rebalance
|
||||
test: multi_move_mx
|
||||
test: shard_move_deferred_delete
|
||||
test: multi_colocated_shard_rebalance
|
|
@ -0,0 +1,121 @@
|
|||
// we use 15 as partition key values through out the test
|
||||
// so setting the corresponding shard here is useful
|
||||
|
||||
setup
|
||||
{
|
||||
SELECT citus_internal.replace_isolation_tester_func();
|
||||
SELECT citus_internal.refresh_isolation_tester_prepared_statement();
|
||||
|
||||
SET citus.shard_count TO 8;
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
CREATE TABLE logical_replicate_placement (x int PRIMARY KEY, y int);
|
||||
SELECT create_distributed_table('logical_replicate_placement', 'x');
|
||||
|
||||
SELECT get_shard_id_for_distribution_column('logical_replicate_placement', 15) INTO selected_shard;
|
||||
|
||||
}
|
||||
|
||||
teardown
|
||||
{
|
||||
SELECT citus_internal.restore_isolation_tester_func();
|
||||
|
||||
DROP TABLE selected_shard;
|
||||
DROP TABLE logical_replicate_placement;
|
||||
}
|
||||
|
||||
|
||||
session "s1"
|
||||
|
||||
step "s1-begin"
|
||||
{
|
||||
BEGIN;
|
||||
}
|
||||
|
||||
step "s1-move-placement"
|
||||
{
|
||||
SELECT master_move_shard_placement(get_shard_id_for_distribution_column, 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes') FROM selected_shard;
|
||||
}
|
||||
|
||||
step "s1-end"
|
||||
{
|
||||
COMMIT;
|
||||
}
|
||||
|
||||
step "s1-select"
|
||||
{
|
||||
SELECT * FROM logical_replicate_placement order by y;
|
||||
}
|
||||
|
||||
step "s1-insert"
|
||||
{
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15), (172, 172);
|
||||
}
|
||||
|
||||
step "s1-get-shard-distribution"
|
||||
{
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
}
|
||||
|
||||
session "s2"
|
||||
|
||||
step "s2-begin"
|
||||
{
|
||||
BEGIN;
|
||||
}
|
||||
|
||||
step "s2-select"
|
||||
{
|
||||
SELECT * FROM logical_replicate_placement ORDER BY y;
|
||||
}
|
||||
|
||||
step "s2-insert"
|
||||
{
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15), (172, 172);
|
||||
}
|
||||
|
||||
step "s2-delete"
|
||||
{
|
||||
DELETE FROM logical_replicate_placement;
|
||||
}
|
||||
|
||||
step "s2-update"
|
||||
{
|
||||
UPDATE logical_replicate_placement SET y = y + 1;
|
||||
}
|
||||
|
||||
step "s2-upsert"
|
||||
{
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15), (172, 172);
|
||||
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15), (172, 172) ON CONFLICT (x) DO UPDATE SET y = logical_replicate_placement.y + 1;
|
||||
}
|
||||
|
||||
step "s2-copy"
|
||||
{
|
||||
COPY logical_replicate_placement FROM PROGRAM 'echo "1,1\n2,2\n3,3\n4,4\n5,5\n15,30"' WITH CSV;
|
||||
}
|
||||
|
||||
step "s2-truncate"
|
||||
{
|
||||
TRUNCATE logical_replicate_placement;
|
||||
}
|
||||
|
||||
step "s2-alter-table"
|
||||
{
|
||||
ALTER TABLE logical_replicate_placement ADD COLUMN z INT;
|
||||
}
|
||||
|
||||
step "s2-end"
|
||||
{
|
||||
COMMIT;
|
||||
}
|
||||
|
||||
permutation "s1-begin" "s2-begin" "s2-insert" "s1-move-placement" "s2-end" "s1-end" "s1-select" "s1-get-shard-distribution"
|
||||
permutation "s1-begin" "s2-begin" "s2-upsert" "s1-move-placement" "s2-end" "s1-end" "s1-select" "s1-get-shard-distribution"
|
||||
permutation "s1-insert" "s1-begin" "s2-begin" "s2-update" "s1-move-placement" "s2-end" "s1-end" "s1-select" "s1-get-shard-distribution"
|
||||
permutation "s1-insert" "s1-begin" "s2-begin" "s2-delete" "s1-move-placement" "s2-end" "s1-end" "s1-select" "s1-get-shard-distribution"
|
||||
permutation "s1-insert" "s1-begin" "s2-begin" "s2-select" "s1-move-placement" "s2-end" "s1-end" "s1-get-shard-distribution"
|
||||
permutation "s1-begin" "s2-begin" "s2-copy" "s1-move-placement" "s2-end" "s1-end" "s1-select" "s1-get-shard-distribution"
|
||||
permutation "s1-insert" "s1-begin" "s2-begin" "s2-truncate" "s1-move-placement" "s2-end" "s1-end" "s1-select" "s1-get-shard-distribution"
|
||||
permutation "s1-begin" "s2-begin" "s2-alter-table" "s1-move-placement" "s2-end" "s1-end" "s1-select" "s1-get-shard-distribution"
|
||||
|
|
@ -0,0 +1,132 @@
|
|||
// we use 15 as partition key values through out the test
|
||||
// so setting the corresponding shard here is useful
|
||||
|
||||
setup
|
||||
{
|
||||
CREATE OR REPLACE FUNCTION start_session_level_connection_to_node(text, integer)
|
||||
RETURNS void
|
||||
LANGUAGE C STRICT VOLATILE
|
||||
AS 'citus', $$start_session_level_connection_to_node$$;
|
||||
|
||||
CREATE OR REPLACE FUNCTION run_commands_on_session_level_connection_to_node(text)
|
||||
RETURNS void
|
||||
LANGUAGE C STRICT VOLATILE
|
||||
AS 'citus', $$run_commands_on_session_level_connection_to_node$$;
|
||||
|
||||
CREATE OR REPLACE FUNCTION stop_session_level_connection_to_node()
|
||||
RETURNS void
|
||||
LANGUAGE C STRICT VOLATILE
|
||||
AS 'citus', $$stop_session_level_connection_to_node$$;
|
||||
|
||||
SELECT citus_internal.replace_isolation_tester_func();
|
||||
SELECT citus_internal.refresh_isolation_tester_prepared_statement();
|
||||
|
||||
-- start_metadata_sync_to_node can not be run inside a transaction block
|
||||
-- following is a workaround to overcome that
|
||||
-- port numbers are hard coded at the moment
|
||||
SELECT master_run_on_worker(
|
||||
ARRAY['localhost']::text[],
|
||||
ARRAY[57636]::int[],
|
||||
ARRAY[format('SELECT start_metadata_sync_to_node(''%s'', %s)', nodename, nodeport)]::text[],
|
||||
false)
|
||||
FROM pg_dist_node;
|
||||
|
||||
SET citus.replication_model to streaming;
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
|
||||
SET citus.shard_count TO 8;
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
CREATE TABLE logical_replicate_placement (x int PRIMARY KEY, y int);
|
||||
SELECT create_distributed_table('logical_replicate_placement', 'x');
|
||||
|
||||
SELECT get_shard_id_for_distribution_column('logical_replicate_placement', 15) INTO selected_shard;
|
||||
}
|
||||
|
||||
teardown
|
||||
{
|
||||
DROP TABLE selected_shard;
|
||||
DROP TABLE logical_replicate_placement;
|
||||
|
||||
SELECT citus_internal.restore_isolation_tester_func();
|
||||
}
|
||||
|
||||
|
||||
session "s1"
|
||||
|
||||
step "s1-begin"
|
||||
{
|
||||
BEGIN;
|
||||
}
|
||||
|
||||
step "s1-move-placement"
|
||||
{
|
||||
SELECT master_move_shard_placement(get_shard_id_for_distribution_column, 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes') FROM selected_shard;
|
||||
}
|
||||
|
||||
step "s1-commit"
|
||||
{
|
||||
COMMIT;
|
||||
}
|
||||
|
||||
step "s1-select"
|
||||
{
|
||||
SELECT * FROM logical_replicate_placement order by y;
|
||||
}
|
||||
|
||||
step "s1-insert"
|
||||
{
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15), (172, 172);
|
||||
}
|
||||
|
||||
step "s1-get-shard-distribution"
|
||||
{
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
}
|
||||
|
||||
session "s2"
|
||||
|
||||
step "s2-start-session-level-connection"
|
||||
{
|
||||
SELECT start_session_level_connection_to_node('localhost', 57638);
|
||||
}
|
||||
|
||||
step "s2-begin-on-worker"
|
||||
{
|
||||
SELECT run_commands_on_session_level_connection_to_node('BEGIN');
|
||||
}
|
||||
|
||||
step "s2-select"
|
||||
{
|
||||
SELECT run_commands_on_session_level_connection_to_node('SELECT * FROM logical_replicate_placement ORDER BY y');
|
||||
}
|
||||
|
||||
step "s2-insert"
|
||||
{
|
||||
SELECT run_commands_on_session_level_connection_to_node('INSERT INTO logical_replicate_placement VALUES (15, 15), (172, 172)');
|
||||
}
|
||||
|
||||
step "s2-delete"
|
||||
{
|
||||
SELECT run_commands_on_session_level_connection_to_node('DELETE FROM logical_replicate_placement');
|
||||
}
|
||||
|
||||
step "s2-update"
|
||||
{
|
||||
SELECT run_commands_on_session_level_connection_to_node('UPDATE logical_replicate_placement SET y = y + 1');
|
||||
}
|
||||
|
||||
step "s2-commit-worker"
|
||||
{
|
||||
SELECT run_commands_on_session_level_connection_to_node('COMMIT');
|
||||
}
|
||||
|
||||
step "s2-stop-connection"
|
||||
{
|
||||
SELECT stop_session_level_connection_to_node();
|
||||
}
|
||||
|
||||
permutation "s1-begin" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-insert" "s1-move-placement" "s2-commit-worker" "s1-commit" "s1-select" "s1-get-shard-distribution" "s2-stop-connection"
|
||||
permutation "s1-insert" "s1-begin" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update" "s1-move-placement" "s2-commit-worker" "s1-commit" "s1-select" "s1-get-shard-distribution" "s2-stop-connection"
|
||||
permutation "s1-insert" "s1-begin" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-delete" "s1-move-placement" "s2-commit-worker" "s1-commit" "s1-select" "s1-get-shard-distribution" "s2-stop-connection"
|
||||
permutation "s1-insert" "s1-begin" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-select" "s1-move-placement" "s2-commit-worker" "s1-commit" "s1-get-shard-distribution" "s2-stop-connection"
|
||||
|
|
@ -0,0 +1,107 @@
|
|||
// we use 15 as the partition key value through out the test
|
||||
// so setting the corresponding shard here is useful
|
||||
setup
|
||||
{
|
||||
SELECT citus_internal.replace_isolation_tester_func();
|
||||
SELECT citus_internal.refresh_isolation_tester_prepared_statement();
|
||||
|
||||
SET citus.shard_count TO 8;
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
CREATE TABLE logical_replicate_placement (x int PRIMARY KEY, y int);
|
||||
SELECT create_distributed_table('logical_replicate_placement', 'x');
|
||||
|
||||
SELECT get_shard_id_for_distribution_column('logical_replicate_placement', 15) INTO selected_shard;
|
||||
}
|
||||
|
||||
teardown
|
||||
{
|
||||
SELECT citus_internal.restore_isolation_tester_func();
|
||||
|
||||
DROP TABLE selected_shard;
|
||||
DROP TABLE logical_replicate_placement;
|
||||
}
|
||||
|
||||
|
||||
session "s1"
|
||||
|
||||
step "s1-begin"
|
||||
{
|
||||
BEGIN;
|
||||
}
|
||||
|
||||
step "s1-move-placement"
|
||||
{
|
||||
SELECT master_move_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes');
|
||||
}
|
||||
|
||||
step "s1-end"
|
||||
{
|
||||
COMMIT;
|
||||
}
|
||||
|
||||
step "s1-select"
|
||||
{
|
||||
SELECT * FROM logical_replicate_placement order by y;
|
||||
}
|
||||
|
||||
step "s1-insert"
|
||||
{
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15);
|
||||
}
|
||||
|
||||
step "s1-get-shard-distribution"
|
||||
{
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
}
|
||||
|
||||
session "s2"
|
||||
|
||||
step "s2-begin"
|
||||
{
|
||||
BEGIN;
|
||||
}
|
||||
|
||||
step "s2-select"
|
||||
{
|
||||
SELECT * FROM logical_replicate_placement ORDER BY y;
|
||||
}
|
||||
|
||||
step "s2-insert"
|
||||
{
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15);
|
||||
}
|
||||
|
||||
step "s2-select-for-update"
|
||||
{
|
||||
SELECT * FROM logical_replicate_placement WHERE x=15 FOR UPDATE;
|
||||
}
|
||||
|
||||
step "s2-delete"
|
||||
{
|
||||
DELETE FROM logical_replicate_placement WHERE x = 15;
|
||||
}
|
||||
|
||||
step "s2-update"
|
||||
{
|
||||
UPDATE logical_replicate_placement SET y = y + 1 WHERE x = 15;
|
||||
}
|
||||
|
||||
step "s2-upsert"
|
||||
{
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15);
|
||||
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15) ON CONFLICT (x) DO UPDATE SET y = logical_replicate_placement.y + 1;
|
||||
}
|
||||
|
||||
step "s2-end"
|
||||
{
|
||||
COMMIT;
|
||||
}
|
||||
|
||||
permutation "s1-begin" "s2-begin" "s2-insert" "s1-move-placement" "s2-end" "s1-end" "s1-select" "s1-get-shard-distribution"
|
||||
permutation "s1-begin" "s2-begin" "s2-upsert" "s1-move-placement" "s2-end" "s1-end" "s1-select" "s1-get-shard-distribution"
|
||||
permutation "s1-insert" "s1-begin" "s2-begin" "s2-update" "s1-move-placement" "s2-end" "s1-end" "s1-select" "s1-get-shard-distribution"
|
||||
permutation "s1-insert" "s1-begin" "s2-begin" "s2-delete" "s1-move-placement" "s2-end" "s1-end" "s1-select" "s1-get-shard-distribution"
|
||||
permutation "s1-insert" "s1-begin" "s2-begin" "s2-select" "s1-move-placement" "s2-end" "s1-end" "s1-get-shard-distribution"
|
||||
permutation "s1-insert" "s1-begin" "s2-begin" "s2-select-for-update" "s1-move-placement" "s2-end" "s1-end" "s1-get-shard-distribution"
|
||||
|
|
@ -0,0 +1,136 @@
|
|||
// 15 as the partition key value through out the test
|
||||
// so setting the corresponding shard here is useful
|
||||
setup
|
||||
{
|
||||
CREATE OR REPLACE FUNCTION start_session_level_connection_to_node(text, integer)
|
||||
RETURNS void
|
||||
LANGUAGE C STRICT VOLATILE
|
||||
AS 'citus', $$start_session_level_connection_to_node$$;
|
||||
|
||||
CREATE OR REPLACE FUNCTION run_commands_on_session_level_connection_to_node(text)
|
||||
RETURNS void
|
||||
LANGUAGE C STRICT VOLATILE
|
||||
AS 'citus', $$run_commands_on_session_level_connection_to_node$$;
|
||||
|
||||
CREATE OR REPLACE FUNCTION stop_session_level_connection_to_node()
|
||||
RETURNS void
|
||||
LANGUAGE C STRICT VOLATILE
|
||||
AS 'citus', $$stop_session_level_connection_to_node$$;
|
||||
|
||||
SELECT citus_internal.replace_isolation_tester_func();
|
||||
SELECT citus_internal.refresh_isolation_tester_prepared_statement();
|
||||
|
||||
-- start_metadata_sync_to_node can not be run inside a transaction block
|
||||
-- following is a workaround to overcome that
|
||||
-- port numbers are hard coded at the moment
|
||||
SELECT master_run_on_worker(
|
||||
ARRAY['localhost']::text[],
|
||||
ARRAY[57636]::int[],
|
||||
ARRAY[format('SELECT start_metadata_sync_to_node(''%s'', %s)', nodename, nodeport)]::text[],
|
||||
false)
|
||||
FROM pg_dist_node;
|
||||
|
||||
SET citus.replication_model to streaming;
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
|
||||
SET citus.shard_count TO 8;
|
||||
CREATE TABLE logical_replicate_placement (x int PRIMARY KEY, y int);
|
||||
SELECT create_distributed_table('logical_replicate_placement', 'x');
|
||||
|
||||
SELECT get_shard_id_for_distribution_column('logical_replicate_placement', 15) INTO selected_shard;
|
||||
}
|
||||
|
||||
teardown
|
||||
{
|
||||
DROP TABLE selected_shard;
|
||||
DROP TABLE logical_replicate_placement;
|
||||
|
||||
SELECT citus_internal.restore_isolation_tester_func();
|
||||
}
|
||||
|
||||
|
||||
session "s1"
|
||||
|
||||
step "s1-begin"
|
||||
{
|
||||
BEGIN;
|
||||
}
|
||||
|
||||
step "s1-move-placement"
|
||||
{
|
||||
SELECT master_move_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638, shard_transfer_mode:='block_writes');
|
||||
}
|
||||
|
||||
step "s1-commit"
|
||||
{
|
||||
COMMIT;
|
||||
}
|
||||
|
||||
step "s1-select"
|
||||
{
|
||||
SELECT * FROM logical_replicate_placement order by y;
|
||||
}
|
||||
|
||||
step "s1-insert"
|
||||
{
|
||||
INSERT INTO logical_replicate_placement VALUES (15, 15);
|
||||
}
|
||||
|
||||
step "s1-get-shard-distribution"
|
||||
{
|
||||
select nodeport from pg_dist_placement inner join pg_dist_node on(pg_dist_placement.groupid = pg_dist_node.groupid) where shardid in (SELECT * FROM selected_shard) order by nodeport;
|
||||
}
|
||||
|
||||
session "s2"
|
||||
|
||||
step "s2-start-session-level-connection"
|
||||
{
|
||||
SELECT start_session_level_connection_to_node('localhost', 57638);
|
||||
}
|
||||
|
||||
step "s2-begin-on-worker"
|
||||
{
|
||||
SELECT run_commands_on_session_level_connection_to_node('BEGIN');
|
||||
}
|
||||
|
||||
step "s2-select"
|
||||
{
|
||||
SELECT run_commands_on_session_level_connection_to_node('SELECT * FROM logical_replicate_placement ORDER BY y');
|
||||
}
|
||||
|
||||
step "s2-insert"
|
||||
{
|
||||
SELECT run_commands_on_session_level_connection_to_node('INSERT INTO logical_replicate_placement VALUES (15, 15)');
|
||||
}
|
||||
|
||||
step "s2-select-for-update"
|
||||
{
|
||||
SELECT run_commands_on_session_level_connection_to_node('SELECT * FROM logical_replicate_placement WHERE x=15 FOR UPDATE');
|
||||
}
|
||||
|
||||
step "s2-delete"
|
||||
{
|
||||
SELECT run_commands_on_session_level_connection_to_node('DELETE FROM logical_replicate_placement WHERE x = 15');
|
||||
}
|
||||
|
||||
step "s2-update"
|
||||
{
|
||||
SELECT run_commands_on_session_level_connection_to_node('UPDATE logical_replicate_placement SET y = y + 1 WHERE x = 15');
|
||||
}
|
||||
|
||||
step "s2-commit-worker"
|
||||
{
|
||||
SELECT run_commands_on_session_level_connection_to_node('COMMIT');
|
||||
}
|
||||
|
||||
step "s2-stop-connection"
|
||||
{
|
||||
SELECT stop_session_level_connection_to_node();
|
||||
}
|
||||
|
||||
permutation "s1-begin" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-insert" "s1-move-placement" "s2-commit-worker" "s1-commit" "s1-select" "s1-get-shard-distribution" "s2-stop-connection"
|
||||
permutation "s1-insert" "s1-begin" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update" "s1-move-placement" "s2-commit-worker" "s1-commit" "s1-select" "s1-get-shard-distribution" "s2-stop-connection"
|
||||
permutation "s1-insert" "s1-begin" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-delete" "s1-move-placement" "s2-commit-worker" "s1-commit" "s1-select" "s1-get-shard-distribution" "s2-stop-connection"
|
||||
permutation "s1-insert" "s1-begin" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-select" "s1-move-placement" "s2-commit-worker" "s1-commit" "s1-get-shard-distribution" "s2-stop-connection"
|
||||
permutation "s1-insert" "s1-begin" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-select-for-update" "s1-move-placement" "s2-commit-worker" "s1-commit" "s1-get-shard-distribution" "s2-stop-connection"
|
||||
|
|
@ -0,0 +1,114 @@
|
|||
setup
|
||||
{
|
||||
SELECT 1 FROM master_add_node('localhost', 57637);
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
CREATE TABLE colocated1 (test_id integer NOT NULL, data text);
|
||||
SELECT create_distributed_table('colocated1', 'test_id', 'hash');
|
||||
CREATE TABLE colocated2 (test_id integer NOT NULL, data text);
|
||||
SELECT create_distributed_table('colocated2', 'test_id', 'hash');
|
||||
CREATE TABLE non_colocated (test_id integer NOT NULL, data text);
|
||||
SELECT create_distributed_table('non_colocated', 'test_id', 'hash', 'none');
|
||||
}
|
||||
|
||||
teardown
|
||||
{
|
||||
DROP TABLE non_colocated;
|
||||
DROP TABLE colocated2;
|
||||
DROP TABLE colocated1;
|
||||
SELECT master_set_node_property('localhost', 57638, 'shouldhaveshards', true);
|
||||
}
|
||||
|
||||
session "s1"
|
||||
|
||||
step "s1-rebalance-c1"
|
||||
{
|
||||
BEGIN;
|
||||
select rebalance_table_shards('colocated1');
|
||||
}
|
||||
|
||||
step "s1-replicate-c1"
|
||||
{
|
||||
BEGIN;
|
||||
select replicate_table_shards('colocated1');
|
||||
}
|
||||
|
||||
step "s1-rebalance-nc"
|
||||
{
|
||||
BEGIN;
|
||||
select rebalance_table_shards('non_colocated');
|
||||
}
|
||||
|
||||
step "s1-replicate-nc"
|
||||
{
|
||||
BEGIN;
|
||||
select replicate_table_shards('non_colocated');
|
||||
}
|
||||
|
||||
step "s1-commit"
|
||||
{
|
||||
COMMIT;
|
||||
}
|
||||
|
||||
session "s2"
|
||||
|
||||
|
||||
step "s2-rebalance-c2"
|
||||
{
|
||||
select rebalance_table_shards('colocated2');
|
||||
}
|
||||
|
||||
step "s2-replicate-c2"
|
||||
{
|
||||
select replicate_table_shards('colocated2');
|
||||
}
|
||||
|
||||
step "s2-rebalance-nc"
|
||||
{
|
||||
select rebalance_table_shards('non_colocated');
|
||||
}
|
||||
|
||||
step "s2-replicate-nc"
|
||||
{
|
||||
select replicate_table_shards('non_colocated');
|
||||
}
|
||||
|
||||
step "s2-rebalance-all"
|
||||
{
|
||||
select rebalance_table_shards();
|
||||
}
|
||||
|
||||
step "s2-drain"
|
||||
{
|
||||
select master_drain_node('localhost', 57638);
|
||||
}
|
||||
|
||||
|
||||
// disallowed because it's the same table
|
||||
permutation "s1-rebalance-nc" "s2-rebalance-nc" "s1-commit"
|
||||
permutation "s1-rebalance-nc" "s2-replicate-nc" "s1-commit"
|
||||
permutation "s1-replicate-nc" "s2-rebalance-nc" "s1-commit"
|
||||
permutation "s1-replicate-nc" "s2-replicate-nc" "s1-commit"
|
||||
|
||||
// disallowed because it's the same colocation group
|
||||
permutation "s1-rebalance-c1" "s2-rebalance-c2" "s1-commit"
|
||||
permutation "s1-rebalance-c1" "s2-replicate-c2" "s1-commit"
|
||||
permutation "s1-replicate-c1" "s2-rebalance-c2" "s1-commit"
|
||||
permutation "s1-replicate-c1" "s2-replicate-c2" "s1-commit"
|
||||
|
||||
// allowed because it's a different colocation group
|
||||
permutation "s1-rebalance-c1" "s2-rebalance-nc" "s1-commit"
|
||||
permutation "s1-rebalance-c1" "s2-replicate-nc" "s1-commit"
|
||||
permutation "s1-replicate-c1" "s2-rebalance-nc" "s1-commit"
|
||||
permutation "s1-replicate-c1" "s2-replicate-nc" "s1-commit"
|
||||
|
||||
// disallowed because we because colocated1 is part of all
|
||||
permutation "s1-rebalance-c1" "s2-rebalance-all" "s1-commit"
|
||||
permutation "s1-replicate-c1" "s2-rebalance-all" "s1-commit"
|
||||
permutation "s1-rebalance-nc" "s2-rebalance-all" "s1-commit"
|
||||
permutation "s1-replicate-nc" "s2-rebalance-all" "s1-commit"
|
||||
|
||||
// disallowed because we because draining is rebalancing
|
||||
permutation "s1-rebalance-c1" "s2-drain" "s1-commit"
|
||||
permutation "s1-replicate-c1" "s2-drain" "s1-commit"
|
||||
permutation "s1-rebalance-nc" "s2-drain" "s1-commit"
|
||||
permutation "s1-replicate-nc" "s2-drain" "s1-commit"
|
|
@ -0,0 +1,79 @@
|
|||
--
|
||||
-- FOREIGN_KEY_TO_REFERENCE_SHARD_REBALANCE
|
||||
--
|
||||
|
||||
SET citus.next_shard_id TO 15000000;
|
||||
CREATE SCHEMA fkey_to_reference_shard_rebalance;
|
||||
SET search_path to fkey_to_reference_shard_rebalance;
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
SET citus.shard_count to 8;
|
||||
|
||||
CREATE TYPE foreign_details AS (name text, relid text, refd_relid text);
|
||||
|
||||
CREATE VIEW table_fkeys_in_workers AS
|
||||
SELECT
|
||||
(json_populate_record(NULL::foreign_details,
|
||||
json_array_elements_text((run_command_on_workers( $$
|
||||
SELECT
|
||||
COALESCE(json_agg(row_to_json(d)), '[]'::json)
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
distinct name,
|
||||
relid::regclass::text,
|
||||
refd_relid::regclass::text
|
||||
FROM
|
||||
table_fkey_cols
|
||||
)
|
||||
d $$ )).RESULT::json )::json )).* ;
|
||||
|
||||
-- check if master_move_shard_placement with logical replication creates the
|
||||
-- foreign constraints properly after moving the shard
|
||||
CREATE TABLE referenced_table(test_column int, test_column2 int UNIQUE, PRIMARY KEY(test_column));
|
||||
CREATE TABLE referencing_table(id int PRIMARY KEY, ref_id int, FOREIGN KEY (id) REFERENCES referenced_table(test_column) ON DELETE CASCADE);
|
||||
CREATE TABLE referencing_table2(id int, ref_id int, FOREIGN KEY (ref_id) REFERENCES referenced_table(test_column2) ON DELETE CASCADE, FOREIGN KEY (id) REFERENCES referencing_table(id) ON DELETE CASCADE);
|
||||
SELECT create_reference_table('referenced_table');
|
||||
SELECT create_distributed_table('referencing_table', 'id');
|
||||
SELECT create_distributed_table('referencing_table2', 'id');
|
||||
|
||||
INSERT INTO referenced_table SELECT i,i FROM generate_series (0, 100) i;
|
||||
INSERT INTO referencing_table SELECT i,i FROM generate_series (0, 100) i;
|
||||
INSERT INTO referencing_table2 SELECT i,i FROM generate_series (0, 100) i;
|
||||
|
||||
SELECT master_move_shard_placement(15000009, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
|
||||
|
||||
SELECT count(*) FROM referencing_table2;
|
||||
|
||||
SELECT * FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_to_reference_shard_rebalance.%' AND refd_relid LIKE 'fkey_to_reference_shard_rebalance.%' ORDER BY 1,2,3;
|
||||
|
||||
SELECT master_move_shard_placement(15000009, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'block_writes');
|
||||
|
||||
SELECT count(*) FROM referencing_table2;
|
||||
|
||||
SELECT * FROM table_fkeys_in_workers WHERE relid LIKE 'fkey_to_reference_shard_rebalance.%' AND refd_relid LIKE 'fkey_to_reference_shard_rebalance.%' ORDER BY 1,2,3;
|
||||
|
||||
-- create a function to show the
|
||||
CREATE FUNCTION get_foreign_key_to_reference_table_commands(Oid)
|
||||
RETURNS SETOF text
|
||||
LANGUAGE C STABLE STRICT
|
||||
AS 'citus', $$get_foreign_key_to_reference_table_commands$$;
|
||||
|
||||
CREATE TABLE reference_table_commands (id int UNIQUE);
|
||||
CREATE TABLE referenceing_dist_table (id int, col1 int, col2 int, col3 int);
|
||||
SELECT create_reference_table('reference_table_commands');
|
||||
SELECT create_distributed_table('referenceing_dist_table', 'id');
|
||||
ALTER TABLE referenceing_dist_table ADD CONSTRAINT c1 FOREIGN KEY (col1) REFERENCES reference_table_commands(id) ON UPDATE CASCADE;
|
||||
ALTER TABLE referenceing_dist_table ADD CONSTRAINT c2 FOREIGN KEY (col2) REFERENCES reference_table_commands(id) ON UPDATE CASCADE NOT VALID;
|
||||
ALTER TABLE referenceing_dist_table ADD CONSTRAINT very_very_very_very_very_very_very_very_very_very_very_very_very_long FOREIGN KEY (col3) REFERENCES reference_table_commands(id) ON UPDATE CASCADE;
|
||||
SELECT * FROM get_foreign_key_to_reference_table_commands('referenceing_dist_table'::regclass);
|
||||
|
||||
-- and show that rebalancer works fine
|
||||
SELECT master_move_shard_placement(15000018, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
|
||||
|
||||
\c - - - :worker_2_port
|
||||
|
||||
SELECT conname, contype, convalidated FROM pg_constraint WHERE conrelid = 'fkey_to_reference_shard_rebalance.referenceing_dist_table_15000018'::regclass ORDER BY 1;
|
||||
|
||||
\c - - - :master_port
|
||||
|
||||
DROP SCHEMA fkey_to_reference_shard_rebalance CASCADE;
|
|
@ -36,21 +36,24 @@ SELECT master_copy_shard_placement(
|
|||
get_shard_id_for_distribution_column('data', 'key-1'),
|
||||
'localhost', :worker_1_port,
|
||||
'localhost', :worker_2_port,
|
||||
do_repair := false);
|
||||
do_repair := false,
|
||||
transfer_mode := 'block_writes');
|
||||
|
||||
-- verify we error out if source and destination are the same
|
||||
SELECT master_copy_shard_placement(
|
||||
get_shard_id_for_distribution_column('data', 'key-1'),
|
||||
'localhost', :worker_2_port,
|
||||
'localhost', :worker_2_port,
|
||||
do_repair := false);
|
||||
do_repair := false,
|
||||
transfer_mode := 'block_writes');
|
||||
|
||||
-- verify we error out if target already contains a healthy placement
|
||||
SELECT master_copy_shard_placement(
|
||||
(SELECT shardid FROM pg_dist_shard WHERE logicalrelid='ref_table'::regclass::oid),
|
||||
'localhost', :worker_1_port,
|
||||
'localhost', :worker_2_port,
|
||||
do_repair := false);
|
||||
do_repair := false,
|
||||
transfer_mode := 'block_writes');
|
||||
|
||||
-- verify we error out if table has foreign key constraints
|
||||
INSERT INTO ref_table SELECT 1, value FROM data;
|
||||
|
@ -69,7 +72,8 @@ SELECT master_copy_shard_placement(
|
|||
get_shard_id_for_distribution_column('data', 'key-1'),
|
||||
'localhost', :worker_2_port,
|
||||
'localhost', :worker_1_port,
|
||||
do_repair := false);
|
||||
do_repair := false,
|
||||
transfer_mode := 'block_writes');
|
||||
|
||||
-- forcefully mark the old replica as inactive
|
||||
UPDATE pg_dist_shard_placement SET shardstate = 3
|
||||
|
@ -95,7 +99,8 @@ SELECT master_copy_shard_placement(
|
|||
get_shard_id_for_distribution_column('mx_table', '1'),
|
||||
'localhost', :worker_1_port,
|
||||
'localhost', :worker_2_port,
|
||||
do_repair := false);
|
||||
do_repair := false,
|
||||
transfer_mode := 'block_writes');
|
||||
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
|
||||
|
|
|
@ -0,0 +1,336 @@
|
|||
--
|
||||
-- MULTI_COLOCATED_SHARD_REBALANCE
|
||||
--
|
||||
|
||||
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 13000000;
|
||||
|
||||
SET citus.shard_count TO 6;
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
|
||||
-- create distributed tables
|
||||
CREATE TABLE table1_group1 ( id int PRIMARY KEY);
|
||||
SELECT create_distributed_table('table1_group1', 'id', 'hash');
|
||||
|
||||
CREATE TABLE table2_group1 ( id int );
|
||||
SELECT create_distributed_table('table2_group1', 'id', 'hash');
|
||||
|
||||
SET citus.shard_count TO 8;
|
||||
CREATE TABLE table5_groupX ( id int );
|
||||
SELECT create_distributed_table('table5_groupX', 'id', 'hash');
|
||||
|
||||
CREATE TABLE table6_append ( id int );
|
||||
SELECT master_create_distributed_table('table6_append', 'id', 'append');
|
||||
SELECT master_create_empty_shard('table6_append');
|
||||
SELECT master_create_empty_shard('table6_append');
|
||||
|
||||
-- Mark tables as non-mx tables, in order to be able to test master_copy_shard_placement
|
||||
UPDATE pg_dist_partition SET repmodel='c' WHERE logicalrelid IN
|
||||
('table1_group1'::regclass, 'table2_group1'::regclass, 'table5_groupX'::regclass);
|
||||
|
||||
-- test copy
|
||||
|
||||
-- test copying colocated shards
|
||||
-- status before shard copy
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
colocationid = (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'table1_group1'::regclass)
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
|
||||
-- copy colocated shards
|
||||
SELECT master_copy_shard_placement(13000000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, false);
|
||||
|
||||
-- status after shard copy
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
colocationid = (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'table1_group1'::regclass)
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
|
||||
-- also connect worker to verify we successfully copied given shard (and other colocated shards)
|
||||
\c - - - :worker_2_port
|
||||
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.table1_group1_13000000'::regclass;
|
||||
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.table2_group1_13000006'::regclass;
|
||||
\c - - - :master_port
|
||||
|
||||
-- copy colocated shards again to see error message
|
||||
SELECT master_copy_shard_placement(13000000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, false, 'force_logical');
|
||||
|
||||
|
||||
-- test copying NOT colocated shard
|
||||
-- status before shard copy
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
p.logicalrelid = 'table5_groupX'::regclass
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
|
||||
-- copy NOT colocated shard
|
||||
SELECT master_copy_shard_placement(13000012, 'localhost', :worker_1_port, 'localhost', :worker_2_port, false);
|
||||
|
||||
-- status after shard copy
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
p.logicalrelid = 'table5_groupX'::regclass
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
|
||||
|
||||
-- test copying shard in append distributed table
|
||||
-- status before shard copy
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
p.logicalrelid = 'table6_append'::regclass
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
|
||||
-- copy shard in append distributed table
|
||||
SELECT master_copy_shard_placement(13000020, 'localhost', :worker_2_port, 'localhost', :worker_1_port, false, 'force_logical');
|
||||
|
||||
-- status after shard copy
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
p.logicalrelid = 'table6_append'::regclass
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
|
||||
|
||||
-- test move
|
||||
|
||||
-- test moving colocated shards
|
||||
-- status before shard move
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
colocationid = (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'table1_group1'::regclass)
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
|
||||
-- try force_logical
|
||||
SELECT master_move_shard_placement(13000001, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'force_logical');
|
||||
|
||||
-- move colocated shards
|
||||
SELECT master_move_shard_placement(13000001, 'localhost', :worker_2_port, 'localhost', :worker_1_port);
|
||||
|
||||
-- status after shard move
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
colocationid = (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'table1_group1'::regclass)
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
|
||||
-- also connect worker to verify we successfully moved given shard (and other colocated shards)
|
||||
\c - - - :worker_1_port
|
||||
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.table1_group1_13000001'::regclass;
|
||||
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.table2_group1_13000007'::regclass;
|
||||
\c - - - :master_port
|
||||
|
||||
|
||||
-- test moving NOT colocated shard
|
||||
-- status before shard move
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
p.logicalrelid = 'table5_groupX'::regclass
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
|
||||
-- move NOT colocated shard
|
||||
SELECT master_move_shard_placement(13000013, 'localhost', :worker_2_port, 'localhost', :worker_1_port);
|
||||
|
||||
-- status after shard move
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
p.logicalrelid = 'table5_groupX'::regclass
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
|
||||
|
||||
-- test moving shard in append distributed table
|
||||
-- status before shard move
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
p.logicalrelid = 'table6_append'::regclass
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
|
||||
-- move shard in append distributed table
|
||||
SELECT master_move_shard_placement(13000021, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
|
||||
|
||||
-- status after shard move
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
p.logicalrelid = 'table6_append'::regclass
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
|
||||
|
||||
-- try to move shard from wrong node
|
||||
SELECT master_move_shard_placement(13000021, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
|
||||
|
||||
|
||||
-- test shard move with foreign constraints
|
||||
DROP TABLE IF EXISTS table1_group1, table2_group1;
|
||||
|
||||
SET citus.shard_count TO 6;
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
|
||||
-- create distributed tables
|
||||
CREATE TABLE table1_group1 ( id int PRIMARY KEY);
|
||||
SELECT create_distributed_table('table1_group1', 'id', 'hash');
|
||||
|
||||
CREATE TABLE table2_group1 ( id int, table1_id int, FOREIGN KEY(table1_id) REFERENCES table1_group1(id));
|
||||
SELECT create_distributed_table('table2_group1', 'table1_id', 'hash');
|
||||
|
||||
-- Mark the tables as non-mx tables
|
||||
UPDATE pg_dist_partition SET repmodel='c' WHERE logicalrelid IN
|
||||
('table1_group1'::regclass, 'table2_group1'::regclass);
|
||||
|
||||
-- status before shard rebalance
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
colocationid = (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'table1_group1'::regclass)
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
|
||||
SELECT master_move_shard_placement(13000022, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'block_writes');
|
||||
|
||||
-- status after shard rebalance
|
||||
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
|
||||
FROM
|
||||
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
|
||||
WHERE
|
||||
p.logicalrelid = s.logicalrelid AND
|
||||
s.shardid = sp.shardid AND
|
||||
colocationid = (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'table1_group1'::regclass)
|
||||
ORDER BY s.shardid, sp.nodeport;
|
||||
|
||||
-- also connect worker to verify we successfully moved given shard (and other colocated shards)
|
||||
\c - - - :worker_2_port
|
||||
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.table1_group1_13000022'::regclass;
|
||||
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.table2_group1_13000028'::regclass;
|
||||
|
||||
-- make sure that we've created the foreign keys
|
||||
SELECT "Constraint", "Definition" FROM table_fkeys
|
||||
WHERE "Constraint" LIKE 'table2_group%' OR "Constraint" LIKE 'table1_group%';
|
||||
|
||||
\c - - - :master_port
|
||||
|
||||
|
||||
-- test shard copy with foreign constraints
|
||||
-- we expect it to error out because we do not support foreign constraints with replication factor > 1
|
||||
SELECT master_copy_shard_placement(13000022, 'localhost', :worker_2_port, 'localhost', :worker_1_port, false);
|
||||
|
||||
|
||||
-- lets also test that master_move_shard_placement doesn't break serials
|
||||
CREATE TABLE serial_move_test (key int, other_val serial);
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
|
||||
SELECT create_distributed_table('serial_move_test', 'key');
|
||||
|
||||
-- key 15 goes to shard 13000035
|
||||
INSERT INTO serial_move_test (key) VALUES (15) RETURNING *;
|
||||
INSERT INTO serial_move_test (key) VALUES (15) RETURNING *;
|
||||
|
||||
-- confirm the shard id
|
||||
SELECT * FROM run_command_on_placements('serial_move_test', 'SELECT DISTINCT key FROM %s WHERE key = 15') WHERE result = '15' AND shardid = 13000034;
|
||||
|
||||
SELECT master_move_shard_placement(13000034, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
|
||||
|
||||
-- confirm the successfull move
|
||||
SELECT * FROM run_command_on_placements('serial_move_test', 'SELECT DISTINCT key FROM %s WHERE key = 15') WHERE result = '15' AND shardid = 13000034;
|
||||
|
||||
-- finally show that serials work fine afterwards
|
||||
INSERT INTO serial_move_test (key) VALUES (15) RETURNING *;
|
||||
INSERT INTO serial_move_test (key) VALUES (15) RETURNING *;
|
||||
|
||||
-- we should be able to move shard placements of partitioend tables
|
||||
CREATE SCHEMA move_partitions;
|
||||
CREATE TABLE move_partitions.events (
|
||||
id serial,
|
||||
t timestamptz default now(),
|
||||
payload text
|
||||
)
|
||||
PARTITION BY RANGE(t);
|
||||
|
||||
SET citus.shard_count TO 6;
|
||||
SELECT create_distributed_table('move_partitions.events', 'id', colocate_with := 'none');
|
||||
|
||||
CREATE TABLE move_partitions.events_1 PARTITION OF move_partitions.events
|
||||
FOR VALUES FROM ('2015-01-01') TO ('2016-01-01');
|
||||
|
||||
INSERT INTO move_partitions.events (t, payload)
|
||||
SELECT '2015-01-01'::date + (interval '1 day' * s), s FROM generate_series(1, 100) s;
|
||||
|
||||
SELECT count(*) FROM move_partitions.events;
|
||||
|
||||
-- try to move automatically
|
||||
SELECT master_move_shard_placement(shardid, 'localhost', :worker_2_port, 'localhost', :worker_1_port)
|
||||
FROM pg_dist_shard JOIN pg_dist_shard_placement USING (shardid)
|
||||
WHERE logicalrelid = 'move_partitions.events'::regclass AND nodeport = :worker_2_port
|
||||
ORDER BY shardid LIMIT 1;
|
||||
|
||||
SELECT count(*) FROM move_partitions.events;
|
||||
|
||||
-- add a primary key to the partition
|
||||
ALTER TABLE move_partitions.events_1 ADD CONSTRAINT e_1_pk PRIMARY KEY (id);
|
||||
|
||||
-- should be able to move automatically now
|
||||
SELECT master_move_shard_placement(shardid, 'localhost', :worker_2_port, 'localhost', :worker_1_port)
|
||||
FROM pg_dist_shard JOIN pg_dist_shard_placement USING (shardid)
|
||||
WHERE logicalrelid = 'move_partitions.events'::regclass AND nodeport = :worker_2_port
|
||||
ORDER BY shardid LIMIT 1;
|
||||
|
||||
SELECT count(*) FROM move_partitions.events;
|
||||
|
||||
-- should also be able to move with block writes
|
||||
SELECT master_move_shard_placement(shardid, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'block_writes')
|
||||
FROM pg_dist_shard JOIN pg_dist_shard_placement USING (shardid)
|
||||
WHERE logicalrelid = 'move_partitions.events'::regclass AND nodeport = :worker_2_port
|
||||
ORDER BY shardid LIMIT 1;
|
||||
|
||||
SELECT count(*) FROM move_partitions.events;
|
||||
|
||||
-- should have moved all shards to node 1 (2*6 = 12)
|
||||
SELECT count(*)
|
||||
FROM pg_dist_shard JOIN pg_dist_shard_placement USING (shardid)
|
||||
WHERE logicalrelid::text LIKE 'move_partitions.events%' AND nodeport = :worker_1_port;
|
||||
|
||||
DROP TABLE move_partitions.events;
|
|
@ -0,0 +1,144 @@
|
|||
--
|
||||
-- MULTI_MOVE_MX
|
||||
--
|
||||
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1550000;
|
||||
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_2_port);
|
||||
|
||||
-- Create mx test tables
|
||||
SET citus.shard_count TO 4;
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
SET citus.replication_model TO 'streaming';
|
||||
|
||||
CREATE TABLE mx_table_1 (a int);
|
||||
SELECT create_distributed_table('mx_table_1', 'a');
|
||||
|
||||
CREATE TABLE mx_table_2 (a int);
|
||||
SELECT create_distributed_table('mx_table_2', 'a');
|
||||
|
||||
CREATE TABLE mx_table_3 (a text);
|
||||
SELECT create_distributed_table('mx_table_3', 'a');
|
||||
|
||||
-- Check that the first two tables are colocated
|
||||
SELECT
|
||||
logicalrelid, repmodel
|
||||
FROM
|
||||
pg_dist_partition
|
||||
WHERE
|
||||
logicalrelid = 'mx_table_1'::regclass
|
||||
OR logicalrelid = 'mx_table_2'::regclass
|
||||
OR logicalrelid = 'mx_table_3'::regclass
|
||||
ORDER BY
|
||||
logicalrelid;
|
||||
|
||||
-- Check the list of shards
|
||||
SELECT
|
||||
logicalrelid, shardid, nodename, nodeport
|
||||
FROM
|
||||
pg_dist_shard NATURAL JOIN pg_dist_shard_placement
|
||||
WHERE
|
||||
logicalrelid = 'mx_table_1'::regclass
|
||||
OR logicalrelid = 'mx_table_2'::regclass
|
||||
OR logicalrelid = 'mx_table_3'::regclass
|
||||
ORDER BY
|
||||
logicalrelid, shardid;
|
||||
|
||||
-- Check the data on the worker
|
||||
\c - - - :worker_2_port
|
||||
SELECT
|
||||
logicalrelid, shardid, nodename, nodeport
|
||||
FROM
|
||||
pg_dist_shard NATURAL JOIN pg_dist_shard_placement
|
||||
WHERE
|
||||
logicalrelid = 'mx_table_1'::regclass
|
||||
OR logicalrelid = 'mx_table_2'::regclass
|
||||
OR logicalrelid = 'mx_table_3'::regclass
|
||||
ORDER BY
|
||||
logicalrelid, shardid;
|
||||
|
||||
\c - - - :master_port
|
||||
-- Check that master_copy_shard_placement cannot be run with MX tables
|
||||
SELECT
|
||||
master_copy_shard_placement(shardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port, false, 'force_logical')
|
||||
FROM
|
||||
pg_dist_shard NATURAL JOIN pg_dist_shard_placement
|
||||
WHERE
|
||||
logicalrelid = 'mx_table_1'::regclass
|
||||
AND nodeport = :worker_1_port
|
||||
ORDER BY
|
||||
shardid
|
||||
LIMIT 1;
|
||||
|
||||
-- Move a shard from worker 1 to worker 2
|
||||
SELECT
|
||||
master_move_shard_placement(shardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port)
|
||||
FROM
|
||||
pg_dist_shard NATURAL JOIN pg_dist_shard_placement
|
||||
WHERE
|
||||
logicalrelid = 'mx_table_1'::regclass
|
||||
AND nodeport = :worker_1_port
|
||||
ORDER BY
|
||||
shardid
|
||||
LIMIT 1;
|
||||
|
||||
-- Check that the shard and its colocated shard is moved, but not the other shards
|
||||
SELECT
|
||||
logicalrelid, shardid, nodename, nodeport
|
||||
FROM
|
||||
pg_dist_shard NATURAL JOIN pg_dist_shard_placement
|
||||
WHERE
|
||||
logicalrelid = 'mx_table_1'::regclass
|
||||
OR logicalrelid = 'mx_table_2'::regclass
|
||||
OR logicalrelid = 'mx_table_3'::regclass
|
||||
ORDER BY
|
||||
logicalrelid, shardid;
|
||||
|
||||
-- Check that the changes are made in the worker as well
|
||||
\c - - - :worker_2_port
|
||||
SELECT
|
||||
logicalrelid, shardid, nodename, nodeport
|
||||
FROM
|
||||
pg_dist_shard NATURAL JOIN pg_dist_shard_placement
|
||||
WHERE
|
||||
logicalrelid = 'mx_table_1'::regclass
|
||||
OR logicalrelid = 'mx_table_2'::regclass
|
||||
OR logicalrelid = 'mx_table_3'::regclass
|
||||
ORDER BY
|
||||
logicalrelid, shardid;
|
||||
|
||||
-- Check that the UDFs cannot be called from the workers
|
||||
SELECT
|
||||
master_copy_shard_placement(shardid, 'localhost', :worker_2_port, 'localhost', :worker_1_port, false, 'force_logical')
|
||||
FROM
|
||||
pg_dist_shard NATURAL JOIN pg_dist_shard_placement
|
||||
WHERE
|
||||
logicalrelid = 'mx_table_1'::regclass
|
||||
AND nodeport = :worker_2_port
|
||||
ORDER BY
|
||||
shardid
|
||||
LIMIT 1 OFFSET 1;
|
||||
|
||||
SELECT
|
||||
master_move_shard_placement(shardid, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'force_logical')
|
||||
FROM
|
||||
pg_dist_shard NATURAL JOIN pg_dist_shard_placement
|
||||
WHERE
|
||||
logicalrelid = 'mx_table_1'::regclass
|
||||
AND nodeport = :worker_2_port
|
||||
ORDER BY
|
||||
shardid
|
||||
LIMIT 1 OFFSET 1;
|
||||
|
||||
-- Cleanup
|
||||
\c - - - :master_port
|
||||
DROP TABLE mx_table_1;
|
||||
DROP TABLE mx_table_2;
|
||||
DROP TABLE mx_table_3;
|
||||
SELECT stop_metadata_sync_to_node('localhost', :worker_2_port);
|
||||
\c - - - :worker_2_port
|
||||
DELETE FROM pg_dist_node;
|
||||
DELETE FROM pg_dist_partition;
|
||||
DELETE FROM pg_dist_shard;
|
||||
DELETE FROM pg_dist_shard_placement;
|
||||
\c - - - :master_port
|
||||
RESET citus.replication_model;
|
|
@ -1,3 +1,10 @@
|
|||
CREATE OR REPLACE FUNCTION master_defer_delete_shards()
|
||||
RETURNS int
|
||||
LANGUAGE C STRICT
|
||||
AS 'citus', $$master_defer_delete_shards$$;
|
||||
COMMENT ON FUNCTION master_defer_delete_shards()
|
||||
IS 'remove orphaned shards';
|
||||
|
||||
CREATE OR REPLACE FUNCTION wait_until_metadata_sync(timeout INTEGER DEFAULT 15000)
|
||||
RETURNS void
|
||||
LANGUAGE C STRICT
|
||||
|
|
|
@ -21,4 +21,3 @@ BEGIN;
|
|||
INSERT INTO pg_dist_node VALUES (1234567890, 1234567890, 'localhost', 5432);
|
||||
INSERT INTO pg_dist_poolinfo VALUES (1234567890, 'port=1234');
|
||||
ROLLBACK;
|
||||
INSERT INTO pg_dist_rebalance_strategy VALUES ('should fail', false, 'citus_shard_cost_1', 'citus_node_capacity_1', 'citus_shard_allowed_on_node_true', 0, 0);
|
||||
|
|
|
@ -0,0 +1,61 @@
|
|||
--
|
||||
-- SHARD_MOVE_DEFERRED_DELETE
|
||||
--
|
||||
|
||||
SET citus.next_shard_id TO 20000000;
|
||||
|
||||
SET citus.shard_count TO 6;
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
SET citus.defer_drop_after_shard_move TO on;
|
||||
|
||||
CREATE SCHEMA shard_move_deferred_delete;
|
||||
SET search_path TO shard_move_deferred_delete;
|
||||
|
||||
CREATE TABLE t1 ( id int PRIMARY KEY);
|
||||
SELECT create_distributed_table('t1', 'id');
|
||||
|
||||
-- by counting how ofter we see the specific shard on all workers we can verify is the shard is there
|
||||
SELECT run_command_on_workers($cmd$
|
||||
SELECT count(*) FROM pg_class WHERE relname = 't1_20000000';
|
||||
$cmd$);
|
||||
|
||||
-- move shard
|
||||
SELECT master_move_shard_placement(20000000, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
|
||||
|
||||
-- we expect the shard to be on both workers now
|
||||
SELECT run_command_on_workers($cmd$
|
||||
SELECT count(*) FROM pg_class WHERE relname = 't1_20000000';
|
||||
$cmd$);
|
||||
|
||||
-- execute delayed removal
|
||||
SELECT public.master_defer_delete_shards();
|
||||
|
||||
-- we expect the shard to be on only the second worker
|
||||
SELECT run_command_on_workers($cmd$
|
||||
SELECT count(*) FROM pg_class WHERE relname = 't1_20000000';
|
||||
$cmd$);
|
||||
|
||||
SELECT master_move_shard_placement(20000000, 'localhost', :worker_2_port, 'localhost', :worker_1_port);
|
||||
|
||||
-- we expect the shard to be on both workers now
|
||||
SELECT run_command_on_workers($cmd$
|
||||
SELECT count(*) FROM pg_class WHERE relname = 't1_20000000';
|
||||
$cmd$);
|
||||
|
||||
-- enable auto delete
|
||||
ALTER SYSTEM SET citus.defer_shard_delete_interval TO 10;
|
||||
SELECT pg_reload_conf();
|
||||
|
||||
-- Sleep 1 second to give Valgrind enough time to clear transactions
|
||||
SELECT pg_sleep(1);
|
||||
|
||||
-- we expect the shard to be on only the first worker
|
||||
SELECT run_command_on_workers($cmd$
|
||||
SELECT count(*) FROM pg_class WHERE relname = 't1_20000000';
|
||||
$cmd$);
|
||||
|
||||
-- reset test suite
|
||||
ALTER SYSTEM SET citus.defer_shard_delete_interval TO -1;
|
||||
SELECT pg_reload_conf();
|
||||
|
||||
DROP SCHEMA shard_move_deferred_delete CASCADE;
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,383 @@
|
|||
CREATE OR REPLACE FUNCTION shard_placement_rebalance_array(
|
||||
worker_node_list json[],
|
||||
shard_placement_list json[],
|
||||
threshold float4 DEFAULT 0,
|
||||
max_shard_moves int DEFAULT 1000000,
|
||||
drain_only bool DEFAULT false
|
||||
)
|
||||
RETURNS json[]
|
||||
AS 'citus'
|
||||
LANGUAGE C STRICT VOLATILE;
|
||||
|
||||
-- Check that even with threshold=0.0 shard_placement_rebalance_array returns
|
||||
-- something when there's no completely balanced solution.
|
||||
|
||||
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432}',
|
||||
'{"node_name": "hostname2", "node_port": 5432}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}']::json[]
|
||||
));
|
||||
|
||||
-- Check that a node can be drained in a balanced cluster
|
||||
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432, "disallowed_shards": "1,2,3,4"}',
|
||||
'{"node_name": "hostname2", "node_port": 5432}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}'
|
||||
]::json[]
|
||||
));
|
||||
|
||||
-- Check that an already drained node won't be filled again after a second
|
||||
-- rebalance
|
||||
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432, "disallowed_shards": "1,2,3,4"}',
|
||||
'{"node_name": "hostname2", "node_port": 5432}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}'
|
||||
]::json[]
|
||||
));
|
||||
|
||||
|
||||
-- Check that even when shards are already balanced, but shard 4 is on a node
|
||||
-- where it is not allowed it will be moved and there will be rebalancing later
|
||||
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432, "disallowed_shards": "1,2,3,5,6"}',
|
||||
'{"node_name": "hostname2", "node_port": 5432, "disallowed_shards": "4"}',
|
||||
'{"node_name": "hostname3", "node_port": 5432, "disallowed_shards": "4"}'
|
||||
]::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":5, "shardid":5, "shardstate":1, "shardlength":1, "nodename":"hostname3", "nodeport":5432}',
|
||||
'{"placementid":6, "shardid":6, "shardstate":1, "shardlength":1, "nodename":"hostname3", "nodeport":5432}'
|
||||
]::json[]
|
||||
));
|
||||
|
||||
-- Check that even when shards are already balanced, disallowed shards will be
|
||||
-- moved away from hostname1 and the only shard that is allowed there will be
|
||||
-- moved there
|
||||
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432, "disallowed_shards": "1,2,3,5,6"}',
|
||||
'{"node_name": "hostname2", "node_port": 5432}',
|
||||
'{"node_name": "hostname3", "node_port": 5432}'
|
||||
]::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":5, "shardid":5, "shardstate":1, "shardlength":1, "nodename":"hostname3", "nodeport":5432}',
|
||||
'{"placementid":6, "shardid":6, "shardstate":1, "shardlength":1, "nodename":"hostname3", "nodeport":5432}'
|
||||
]::json[]
|
||||
));
|
||||
|
||||
-- Check that an error is returned when a shard is not allowed anywhere
|
||||
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432, "disallowed_shards": "2,4"}',
|
||||
'{"node_name": "hostname2", "node_port": 5432, "disallowed_shards": "1,4"}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}'
|
||||
]::json[]
|
||||
));
|
||||
|
||||
-- Check that cost is taken into account when rebalancing
|
||||
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432}',
|
||||
'{"node_name": "hostname2", "node_port": 5432}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432, "cost": 3}']::json[]
|
||||
));
|
||||
|
||||
|
||||
-- Check that cost is taken into account when rebalancing disallowed placements
|
||||
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432, "disallowed_shards": "1,2,3,4"}',
|
||||
'{"node_name": "hostname2", "node_port": 5432}',
|
||||
'{"node_name": "hostname3", "node_port": 5432}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432, "cost": 3}']::json[]
|
||||
));
|
||||
|
||||
|
||||
-- Check that node capacacity is taken into account.
|
||||
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432}',
|
||||
'{"node_name": "hostname2", "node_port": 5432, "capacity": 3}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}']::json[]
|
||||
));
|
||||
|
||||
-- Check that shards are not moved when target utilization stays the same and
|
||||
-- the source utilization goes below the original target utilization. hostname1
|
||||
-- has utilization of 1, after move hostname2 would have a utilization of 1 as
|
||||
-- well. hostname1 would have utilization of 1 while hostname2 has utilization
|
||||
-- of 2/3 now. Since load is spread more fairly with utilization 2/3 than 0 it
|
||||
-- should choose that distribution.
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432}',
|
||||
'{"node_name": "hostname2", "node_port": 5432, "capacity": 3}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}']::json[]
|
||||
));
|
||||
|
||||
|
||||
-- Check that shards are moved even when target utilization stays the same, but
|
||||
-- source utilization goes below the original target utilization. hostname2
|
||||
-- has utilization of 1, after move hostname1 would have a utilization of 1 as
|
||||
-- well. hostname2 would have utilization of 2/3 while hostname1 now has
|
||||
-- utilization of 0 now. Since load is spread more fairly with utilization 2/3
|
||||
-- than 0 it should choose that distribution.
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432}',
|
||||
'{"node_name": "hostname2", "node_port": 5432, "capacity": 3}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}']::json[]
|
||||
));
|
||||
|
||||
-- Check that shards are moved even when target utilization stays the same, but
|
||||
-- source utilization goes below the original target utilization. hostname2
|
||||
-- has utilization of 2, after move hostname1 would have a utilization of 2 as
|
||||
-- well. hostname2 would have utilization of 1.5 while hostname1 now has
|
||||
-- utilization of 1. Since load is spread more fairly with utilization 1.5 than
|
||||
-- 1 it should choose that distribution.
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432}',
|
||||
'{"node_name": "hostname2", "node_port": 5432, "capacity": 2}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":5, "shardid":5, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}']::json[]
|
||||
));
|
||||
|
||||
-- Check that shards are moved even when target utilization stays the same, but
|
||||
-- source utilization goes below the original target utilization. hostname1
|
||||
-- has utilization of 2, after move hostname2 would have a utilization of 2 as
|
||||
-- well. hostname1 would have utilization of 1 while hostname2 now has
|
||||
-- utilization of 1.5. Since load is spread more fairly with utilization 1.5
|
||||
-- than 1 it should choose that distribution.
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432}',
|
||||
'{"node_name": "hostname2", "node_port": 5432, "capacity": 2}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":5, "shardid":5, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}']::json[]
|
||||
));
|
||||
|
||||
|
||||
-- Check that all shards will be moved to 1 node if its capacity is big enough
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432}',
|
||||
'{"node_name": "hostname2", "node_port": 5432, "capacity": 4}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}']::json[]
|
||||
));
|
||||
|
||||
-- Check that shards will be moved to a smaller node node if utilization improves
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432}',
|
||||
'{"node_name": "hostname2", "node_port": 5432, "capacity": 3}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}']::json[]
|
||||
));
|
||||
|
||||
-- Check that node capacity works with different shard costs
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432}',
|
||||
'{"node_name": "hostname2", "node_port": 5432, "capacity": 3}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432, "cost": 3}']::json[]
|
||||
));
|
||||
|
||||
-- Check that node capacity works with different shard costs again
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432}',
|
||||
'{"node_name": "hostname2", "node_port": 5432, "capacity": 3}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432, "cost": 2}']::json[]
|
||||
));
|
||||
|
||||
-- Check that max_shard_moves works and that we get a NOTICE that it is hit
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432}',
|
||||
'{"node_name": "hostname2", "node_port": 5432, "capacity": 3}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432, "cost": 2}']::json[],
|
||||
max_shard_moves := 1
|
||||
));
|
||||
|
||||
|
||||
-- Check that node capacity works with different shard costs and disallowed_shards
|
||||
-- NOTE: these moves are not optimal, once we implement merging of updates this
|
||||
-- output should change.
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432}',
|
||||
'{"node_name": "hostname2", "node_port": 5432, "capacity": 5}',
|
||||
'{"node_name": "hostname3", "node_port": 5432, "disallowed_shards": "1,2"}']::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname3", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname3", "nodeport":5432, "cost": 2}']::json[]
|
||||
));
|
||||
|
||||
-- Check that draining + rebalancing nodes works
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432, "disallowed_shards": "1,2,3,4,5,6", "capacity": 0}',
|
||||
'{"node_name": "hostname2", "node_port": 5432}',
|
||||
'{"node_name": "hostname3", "node_port": 5432}'
|
||||
]::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":5, "shardid":5, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":6, "shardid":6, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}'
|
||||
]::json[]
|
||||
));
|
||||
|
||||
|
||||
-- Check that draining nodes with drain only works
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432, "disallowed_shards": "1,2,3,4,5,6", "capacity": 0}',
|
||||
'{"node_name": "hostname2", "node_port": 5432}',
|
||||
'{"node_name": "hostname3", "node_port": 5432}'
|
||||
]::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":5, "shardid":5, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":6, "shardid":6, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}'
|
||||
]::json[],
|
||||
drain_only := true
|
||||
));
|
||||
|
||||
-- Check that draining nodes has priority over max_shard_moves
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432, "disallowed_shards": "1,2,3,4,5,6", "capacity": 0}',
|
||||
'{"node_name": "hostname2", "node_port": 5432}',
|
||||
'{"node_name": "hostname3", "node_port": 5432}'
|
||||
]::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":5, "shardid":5, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":6, "shardid":6, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}'
|
||||
]::json[],
|
||||
max_shard_moves := 0
|
||||
));
|
||||
|
||||
-- Check that drained moves are counted towards shard moves and thus use up the
|
||||
-- limit when doing normal rebalance moves
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432, "disallowed_shards": "1,2,3,4,5,6", "capacity": 0}',
|
||||
'{"node_name": "hostname2", "node_port": 5432}',
|
||||
'{"node_name": "hostname3", "node_port": 5432}'
|
||||
]::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":5, "shardid":5, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":6, "shardid":6, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}'
|
||||
]::json[],
|
||||
max_shard_moves := 2
|
||||
));
|
||||
|
||||
-- Check that draining for all colocation groups is done before rebalancing
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432, "disallowed_shards": "1,2,3,4,5,6,7,8,9,10,11,12", "capacity": 0}',
|
||||
'{"node_name": "hostname2", "node_port": 5432}',
|
||||
'{"node_name": "hostname3", "node_port": 5432}'
|
||||
]::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":5, "shardid":5, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":6, "shardid":6, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":7, "shardid":7, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432, "next_colocation": true}',
|
||||
'{"placementid":8, "shardid":8, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":9, "shardid":9, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":10, "shardid":10, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":11, "shardid":11, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":12, "shardid":12, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}'
|
||||
]::json[]
|
||||
));
|
||||
|
||||
-- Check that max_shard_moves warning is only shown once even if more than one
|
||||
-- colocation group its placement updates are ignored because of it
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432, "disallowed_shards": "1,2,3,4,5,6,7,8,9,10,11,12", "capacity": 0}',
|
||||
'{"node_name": "hostname2", "node_port": 5432}',
|
||||
'{"node_name": "hostname3", "node_port": 5432}'
|
||||
]::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":5, "shardid":5, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":6, "shardid":6, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":7, "shardid":7, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432, "next_colocation": true}',
|
||||
'{"placementid":8, "shardid":8, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":9, "shardid":9, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":10, "shardid":10, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":11, "shardid":11, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":12, "shardid":12, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}'
|
||||
]::json[],
|
||||
max_shard_moves := 1
|
||||
));
|
||||
|
||||
-- Check that moves for different colocation groups are added together when
|
||||
-- taking into account max_shard_moves
|
||||
SELECT unnest(shard_placement_rebalance_array(
|
||||
ARRAY['{"node_name": "hostname1", "node_port": 5432, "disallowed_shards": "1,2,3,4,5,6,7,8,9,10,11,12", "capacity": 0}',
|
||||
'{"node_name": "hostname2", "node_port": 5432}',
|
||||
'{"node_name": "hostname3", "node_port": 5432}'
|
||||
]::json[],
|
||||
ARRAY['{"placementid":1, "shardid":1, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432}',
|
||||
'{"placementid":2, "shardid":2, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":3, "shardid":3, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":4, "shardid":4, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":5, "shardid":5, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":6, "shardid":6, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":7, "shardid":7, "shardstate":1, "shardlength":1, "nodename":"hostname1", "nodeport":5432, "next_colocation": true}',
|
||||
'{"placementid":8, "shardid":8, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":9, "shardid":9, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":10, "shardid":10, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":11, "shardid":11, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}',
|
||||
'{"placementid":12, "shardid":12, "shardstate":1, "shardlength":1, "nodename":"hostname2", "nodeport":5432}'
|
||||
]::json[],
|
||||
max_shard_moves := 5
|
||||
));
|
Loading…
Reference in New Issue