Drop SHARD_STATE_TO_DELETE (#6494)

DESCRIPTION: Drop `SHARD_STATE_TO_DELETE` and use the cleanup records
instead

Drops the shard state that is used to mark shards as orphaned. Now we
insert cleanup records into `pg_dist_cleanup` so "orphaned" shards will
be dropped either by maintenance daemon or internal cleanup calls. With
this PR, we make the "cleanup orphaned shards" functions to be no-op, as
they would not be needed anymore.

This PR includes some naming changes about placement functions. We don't
need functions that filter orphaned shards, as there will be no orphaned
shards anymore.

We will also be introducing a small script with this PR, for users with
orphaned shards. We'll basically delete the orphaned shard entries from
`pg_dist_placement` and insert cleanup records into `pg_dist_cleanup`
for each one of them, during Citus upgrade.

We also have a lot of flakiness fixes in this PR.

Co-authored-by: Jelte Fennema <github-tech@jeltef.nl>
pull/6537/head
Ahmet Gedemenli 2023-01-03 14:38:16 +03:00 committed by GitHub
parent f56904fe04
commit 235047670d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
63 changed files with 1077 additions and 1432 deletions

View File

@ -1173,14 +1173,13 @@ LookupNodeForGroup(int32 groupId)
/*
* ShardPlacementList returns the list of placements for the given shard from
* the cache. This list includes placements that are orphaned, because they
* their deletion is postponed to a later point (shardstate = 4).
* the cache.
*
* The returned list is deep copied from the cache and thus can be modified
* and pfree()d freely.
*/
List *
ShardPlacementListIncludingOrphanedPlacements(uint64 shardId)
ShardPlacementList(uint64 shardId)
{
List *placementList = NIL;

View File

@ -3645,7 +3645,7 @@ citus_internal_delete_shard_metadata(PG_FUNCTION_ARGS)
EnsureShardOwner(shardId, missingOk);
}
List *shardPlacementList = ShardPlacementListIncludingOrphanedPlacements(shardId);
List *shardPlacementList = ShardPlacementList(shardId);
ShardPlacement *shardPlacement = NULL;
foreach_ptr(shardPlacement, shardPlacementList)
{

View File

@ -1078,7 +1078,7 @@ TableShardReplicationFactor(Oid relationId)
{
uint64 shardId = shardInterval->shardId;
List *shardPlacementList = ShardPlacementListWithoutOrphanedPlacements(shardId);
List *shardPlacementList = ShardPlacementListSortedByWorker(shardId);
uint32 shardPlacementCount = list_length(shardPlacementList);
/*
@ -1480,8 +1480,7 @@ ActiveShardPlacementListOnGroup(uint64 shardId, int32 groupId)
List *
ActiveShardPlacementList(uint64 shardId)
{
List *shardPlacementList =
ShardPlacementListIncludingOrphanedPlacements(shardId);
List *shardPlacementList = ShardPlacementList(shardId);
List *activePlacementList = FilterShardPlacementList(shardPlacementList,
IsActiveShardPlacement);
@ -1491,30 +1490,14 @@ ActiveShardPlacementList(uint64 shardId)
/*
* IsShardPlacementNotOrphaned checks returns true if a shard placement is not orphaned
* Orphaned shards are shards marked to be deleted at a later point (shardstate = 4).
*/
static inline bool
IsShardPlacementNotOrphaned(ShardPlacement *shardPlacement)
{
return shardPlacement->shardState != SHARD_STATE_TO_DELETE;
}
/*
* ShardPlacementListWithoutOrphanedPlacements returns shard placements exluding
* the ones that are orphaned.
* ShardPlacementListSortedByWorker returns shard placements sorted by worker port.
*/
List *
ShardPlacementListWithoutOrphanedPlacements(uint64 shardId)
ShardPlacementListSortedByWorker(uint64 shardId)
{
List *shardPlacementList =
ShardPlacementListIncludingOrphanedPlacements(shardId);
List *shardPlacementList = ShardPlacementList(shardId);
List *activePlacementList = FilterShardPlacementList(shardPlacementList,
IsShardPlacementNotOrphaned);
return SortList(activePlacementList, CompareShardPlacementsByWorker);
return SortList(shardPlacementList, CompareShardPlacementsByWorker);
}
@ -1658,46 +1641,6 @@ 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
@ -2053,62 +1996,6 @@ DeleteShardPlacementRow(uint64 placementId)
}
/*
* UpdateShardPlacementState sets the shardState for the placement identified
* by placementId.
*/
void
UpdateShardPlacementState(uint64 placementId, char shardState)
{
ScanKeyData scanKey[1];
int scanKeyCount = 1;
bool indexOK = true;
Datum values[Natts_pg_dist_placement];
bool isnull[Natts_pg_dist_placement];
bool replace[Natts_pg_dist_placement];
bool colIsNull = false;
Relation pgDistPlacement = table_open(DistPlacementRelationId(), RowExclusiveLock);
TupleDesc tupleDescriptor = RelationGetDescr(pgDistPlacement);
ScanKeyInit(&scanKey[0], Anum_pg_dist_placement_placementid,
BTEqualStrategyNumber, F_INT8EQ, Int64GetDatum(placementId));
SysScanDesc scanDescriptor = systable_beginscan(pgDistPlacement,
DistPlacementPlacementidIndexId(),
indexOK,
NULL, scanKeyCount, scanKey);
HeapTuple heapTuple = systable_getnext(scanDescriptor);
if (!HeapTupleIsValid(heapTuple))
{
ereport(ERROR, (errmsg("could not find valid entry for shard placement "
UINT64_FORMAT,
placementId)));
}
memset(replace, 0, sizeof(replace));
values[Anum_pg_dist_placement_shardstate - 1] = CharGetDatum(shardState);
isnull[Anum_pg_dist_placement_shardstate - 1] = false;
replace[Anum_pg_dist_placement_shardstate - 1] = true;
heapTuple = heap_modify_tuple(heapTuple, tupleDescriptor, values, isnull, replace);
CatalogTupleUpdate(pgDistPlacement, &heapTuple->t_self, heapTuple);
uint64 shardId = DatumGetInt64(heap_getattr(heapTuple,
Anum_pg_dist_placement_shardid,
tupleDescriptor, &colIsNull));
Assert(!colIsNull);
CitusInvalidateRelcacheByShardId(shardId);
CommandCounterIncrement();
systable_endscan(scanDescriptor);
table_close(pgDistPlacement, NoLock);
}
/*
* UpdatePlacementGroupId sets the groupId for the placement identified
* by placementId.

View File

@ -120,7 +120,6 @@ static char * NodeMetadataSyncedUpdateCommand(uint32 nodeId, bool metadataSynced
static void ErrorIfCoordinatorMetadataSetFalse(WorkerNode *workerNode, Datum value,
char *field);
static WorkerNode * SetShouldHaveShards(WorkerNode *workerNode, bool shouldHaveShards);
static void RemoveOldShardPlacementForNodeGroup(int groupId);
static int FindCoordinatorNodeId(void);
static WorkerNode * FindNodeAnyClusterByNodeId(uint32 nodeId);
@ -1897,8 +1896,6 @@ RemoveNodeFromCluster(char *nodeName, int32 nodePort)
DeleteNodeRow(workerNode->workerName, nodePort);
RemoveOldShardPlacementForNodeGroup(workerNode->groupId);
/* make sure we don't have any lingering session lifespan connections */
CloseNodeConnectionsAfterTransaction(workerNode->workerName, nodePort);
@ -1970,29 +1967,6 @@ PlacementHasActivePlacementOnAnotherGroup(GroupShardPlacement *sourcePlacement)
}
/*
* RemoveOldShardPlacementForNodeGroup removes all old shard placements
* for the given node group from pg_dist_placement.
*/
static void
RemoveOldShardPlacementForNodeGroup(int groupId)
{
/*
* Prevent concurrent deferred drop
*/
LockPlacementCleanup();
List *shardPlacementsOnNode = AllShardPlacementsOnNodeGroup(groupId);
GroupShardPlacement *placement = NULL;
foreach_ptr(placement, shardPlacementsOnNode)
{
if (placement->shardState == SHARD_STATE_TO_DELETE)
{
DeleteShardPlacementRow(placement->placementId);
}
}
}
/* CountPrimariesWithMetadata returns the number of primary nodes which have metadata. */
uint32
CountPrimariesWithMetadata(void)

View File

@ -260,7 +260,7 @@ CreateColocatedShards(Oid targetRelationId, Oid sourceRelationId, bool
int32 shardMaxValue = DatumGetInt32(sourceShardInterval->maxValue);
text *shardMinValueText = IntegerToText(shardMinValue);
text *shardMaxValueText = IntegerToText(shardMaxValue);
List *sourceShardPlacementList = ShardPlacementListWithoutOrphanedPlacements(
List *sourceShardPlacementList = ShardPlacementListSortedByWorker(
sourceShardId);
InsertShardRow(targetRelationId, newShardId, targetShardStorageType,

View File

@ -44,6 +44,7 @@
#include "distributed/placement_connection.h"
#include "distributed/relay_utility.h"
#include "distributed/remote_commands.h"
#include "distributed/shard_cleaner.h"
#include "distributed/worker_protocol.h"
#include "distributed/worker_transaction.h"
#include "lib/stringinfo.h"
@ -372,8 +373,7 @@ DropTaskList(Oid relationId, char *schemaName, char *relationName,
task->dependentTaskList = NULL;
task->replicationModel = REPLICATION_MODEL_INVALID;
task->anchorShardId = shardId;
task->taskPlacementList =
ShardPlacementListIncludingOrphanedPlacements(shardId);
task->taskPlacementList = ShardPlacementList(shardId);
taskList = lappend(taskList, task);
}
@ -410,8 +410,6 @@ ExecuteDropShardPlacementCommandRemotely(ShardPlacement *shardPlacement,
if (PQstatus(connection->pgConn) != CONNECTION_OK)
{
uint64 placementId = shardPlacement->placementId;
char *workerName = shardPlacement->nodeName;
uint32 workerPort = shardPlacement->nodePort;
@ -427,7 +425,10 @@ ExecuteDropShardPlacementCommandRemotely(ShardPlacement *shardPlacement,
errdetail("Marking this shard placement for "
"deletion")));
UpdateShardPlacementState(placementId, SHARD_STATE_TO_DELETE);
InsertCleanupRecordInCurrentTransaction(CLEANUP_OBJECT_SHARD_PLACEMENT,
shardRelationName,
shardPlacement->groupId,
CLEANUP_DEFERRED_ON_SUCCESS);
return;
}

View File

@ -1,9 +1,8 @@
/*-------------------------------------------------------------------------
*
* 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.
* This implements the background process that cleans shards and resources
* that are left around.
*
* Copyright (c) 2018, Citus Data, Inc.
*
@ -72,10 +71,9 @@ OperationId CurrentOperationId = INVALID_OPERATION_ID;
/* declarations for dynamic loading */
PG_FUNCTION_INFO_V1(citus_cleanup_orphaned_shards);
PG_FUNCTION_INFO_V1(isolation_cleanup_orphaned_shards);
PG_FUNCTION_INFO_V1(citus_cleanup_orphaned_resources);
PG_FUNCTION_INFO_V1(isolation_cleanup_orphaned_resources);
static int DropOrphanedShardsForMove(bool waitForLocks);
static bool TryDropResourceByCleanupRecordOutsideTransaction(CleanupRecord *record,
char *nodeName,
int nodePort);
@ -92,7 +90,9 @@ static bool TryDropReplicationSlotOutsideTransaction(char *replicationSlotName,
char *nodeName,
int nodePort);
static bool TryDropUserOutsideTransaction(char *username, char *nodeName, int nodePort);
static bool TryLockRelationAndPlacementCleanup(Oid relationId, LOCKMODE lockmode);
static CleanupRecord * GetCleanupRecordByNameAndType(char *objectName,
CleanupObject type);
/* Functions for cleanup infrastructure */
static CleanupRecord * TupleToCleanupRecord(HeapTuple heapTuple,
@ -112,57 +112,14 @@ static int CompareCleanupRecordsByObjectType(const void *leftElement,
const void *rightElement);
/*
* citus_cleanup_orphaned_shards implements a user-facing UDF to delete
* 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. It cannot be run in a
* transaction, because holding the locks it takes for a long time is not good.
* While the locks are held, it is impossible for the background daemon to
* cleanup orphaned shards.
* citus_cleanup_orphaned_shards is noop.
* Use citus_cleanup_orphaned_resources instead.
*/
Datum
citus_cleanup_orphaned_shards(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
EnsureCoordinator();
PreventInTransactionBlock(true, "citus_cleanup_orphaned_shards");
bool waitForLocks = true;
int droppedShardCount = DropOrphanedShardsForMove(waitForLocks);
if (droppedShardCount > 0)
{
ereport(NOTICE, (errmsg("cleaned up %d orphaned shards", droppedShardCount)));
}
PG_RETURN_VOID();
}
/*
* isolation_cleanup_orphaned_shards implements a test UDF that's the same as
* citus_cleanup_orphaned_shards. The only difference is that this command can
* be run in transactions, this is to test
*/
Datum
isolation_cleanup_orphaned_shards(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
EnsureCoordinator();
bool waitForLocks = true;
int droppedShardCount = DropOrphanedShardsForMove(waitForLocks);
if (droppedShardCount > 0)
{
ereport(NOTICE, (errmsg("cleaned up %d orphaned shards", droppedShardCount)));
}
ereport(WARNING, (errmsg("citus_cleanup_orphaned_shards is deprecated. "
"Use citus_cleanup_orphaned_resources instead")));
PG_RETURN_VOID();
}
@ -195,10 +152,27 @@ citus_cleanup_orphaned_resources(PG_FUNCTION_ARGS)
}
/*
* isolation_cleanup_orphaned_resources implements a test UDF that's the same as
* citus_cleanup_orphaned_resources. The only difference is that this command can
* be run in transactions, this is needed to test this function in isolation tests
* since commands are automatically run in transactions there.
*/
Datum
isolation_cleanup_orphaned_resources(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
EnsureCoordinator();
DropOrphanedResourcesForCleanup();
PG_RETURN_VOID();
}
/*
* DropOrphanedResourcesInSeparateTransaction cleans up orphaned resources by
* connecting to localhost. This is done, so that the locks that
* DropOrphanedShardsForMove takes are only held for a short time.
* connecting to localhost.
*/
void
DropOrphanedResourcesInSeparateTransaction(void)
@ -207,21 +181,19 @@ DropOrphanedResourcesInSeparateTransaction(void)
MultiConnection *connection = GetNodeConnection(connectionFlag, LocalHostName,
PostPortNumber);
ExecuteCriticalRemoteCommand(connection, "CALL citus_cleanup_orphaned_resources()");
ExecuteCriticalRemoteCommand(connection, "CALL citus_cleanup_orphaned_shards()");
CloseConnection(connection);
}
/*
* TryDropOrphanedResources is a wrapper around DropOrphanedShardsForMove and
* DropOrphanedResourcesForCleanup that catches any errors to make it safe to
* use in the maintenance daemon.
* TryDropOrphanedResources is a wrapper around DropOrphanedResourcesForCleanup
* that catches any errors to make it safe to use in the maintenance daemon.
*
* If dropping any of the shards failed this function returns -1, otherwise it
* If dropping any of the resources failed this function returns -1, otherwise it
* returns the number of dropped resources.
*/
int
TryDropOrphanedResources(bool waitForLocks)
TryDropOrphanedResources()
{
int droppedResourceCount = 0;
MemoryContext savedContext = CurrentMemoryContext;
@ -234,8 +206,7 @@ TryDropOrphanedResources(bool waitForLocks)
PG_TRY();
{
droppedResourceCount = DropOrphanedShardsForMove(waitForLocks);
droppedResourceCount += DropOrphanedResourcesForCleanup();
droppedResourceCount = DropOrphanedResourcesForCleanup();
/*
* Releasing a subtransaction doesn't free its memory context, since the
@ -362,107 +333,6 @@ DropOrphanedResourcesForCleanup()
}
/*
* DropOrphanedShardsForMove removes shards that were marked SHARD_STATE_TO_DELETE before.
*
* It does so by trying to take 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.
*
* If waitForLocks is false, then if we cannot take a lock on pg_dist_placement
* we continue without waiting.
*
* Before doing any of this it will take an exclusive PlacementCleanup lock.
* This is to ensure that this function is not being run concurrently.
* Otherwise really bad race conditions are possible, such as removing all
* placements of a shard. waitForLocks indicates if this function should
* wait for this lock or not.
*
*/
static int
DropOrphanedShardsForMove(bool waitForLocks)
{
int removedShardCount = 0;
/*
* We should try to take the highest lock that we take
* later in this function for pg_dist_placement. We take RowExclusiveLock
* in DeleteShardPlacementRow.
*/
LOCKMODE lockmode = RowExclusiveLock;
if (!IsCoordinator())
{
return 0;
}
if (waitForLocks)
{
LockPlacementCleanup();
}
else
{
Oid distPlacementId = DistPlacementRelationId();
if (!TryLockRelationAndPlacementCleanup(distPlacementId, lockmode))
{
return 0;
}
}
int failedShardDropCount = 0;
List *shardPlacementList = AllShardPlacementsWithShardPlacementState(
SHARD_STATE_TO_DELETE);
GroupShardPlacement *placement = NULL;
foreach_ptr(placement, shardPlacementList)
{
if (!PrimaryNodeForGroup(placement->groupId, NULL) ||
!ShardExists(placement->shardId))
{
continue;
}
ShardPlacement *shardPlacement = LoadShardPlacement(placement->shardId,
placement->placementId);
ShardInterval *shardInterval = LoadShardInterval(placement->shardId);
char *qualifiedTableName = ConstructQualifiedShardName(shardInterval);
if (TryDropShardOutsideTransaction(qualifiedTableName,
shardPlacement->nodeName,
shardPlacement->nodePort))
{
ereport(LOG, (errmsg("deferred drop of orphaned shard %s on %s:%d "
"after a move completed",
qualifiedTableName,
shardPlacement->nodeName,
shardPlacement->nodePort)));
/* delete the actual placement */
DeleteShardPlacementRow(placement->placementId);
removedShardCount++;
}
else
{
/*
* We log failures at the end, since they occur repeatedly
* for a large number of objects.
*/
failedShardDropCount++;
}
}
if (failedShardDropCount > 0)
{
ereport(WARNING, (errmsg("failed to clean up %d orphaned shards out of %d",
failedShardDropCount, list_length(shardPlacementList))));
}
return removedShardCount;
}
/*
* RegisterOperationNeedingCleanup is be called by an operation to register
* for cleanup.
@ -691,30 +561,6 @@ DeleteCleanupRecordByRecordIdOutsideTransaction(uint64 recordId)
}
/*
* TryLockRelationAndPlacementCleanup tries to lock the given relation
* and the placement cleanup. If it cannot, it returns false.
*
*/
static bool
TryLockRelationAndPlacementCleanup(Oid relationId, LOCKMODE lockmode)
{
if (!ConditionalLockRelationOid(relationId, lockmode))
{
ereport(DEBUG1, (errmsg(
"could not acquire shard lock to cleanup placements")));
return false;
}
if (!TryLockPlacementCleanup())
{
ereport(DEBUG1, (errmsg("could not acquire lock to cleanup placements")));
return false;
}
return true;
}
/*
* TryDropResourceByCleanupRecordOutsideTransaction tries to drop the given resource
* and returns true on success.
@ -1035,6 +881,27 @@ TryDropUserOutsideTransaction(char *username,
}
/*
* ErrorIfCleanupRecordForShardExists errors out if a cleanup record for the given
* shard name exists.
*/
void
ErrorIfCleanupRecordForShardExists(char *shardName)
{
CleanupRecord *record =
GetCleanupRecordByNameAndType(shardName, CLEANUP_OBJECT_SHARD_PLACEMENT);
if (record == NULL)
{
return;
}
ereport(ERROR, (errmsg("shard move failed as the orphaned shard %s leftover "
"from the previous move could not be cleaned up",
record->objectName)));
}
/*
* GetNextOperationId allocates and returns a unique operationId for an operation
* requiring potential cleanup. This allocation occurs both in shared memory and
@ -1162,6 +1029,47 @@ ListCleanupRecordsForCurrentOperation(void)
}
/*
* GetCleanupRecordByNameAndType returns the cleanup record with given name and type,
* if any, returns NULL otherwise.
*/
static CleanupRecord *
GetCleanupRecordByNameAndType(char *objectName, CleanupObject type)
{
CleanupRecord *objectFound = NULL;
Relation pgDistCleanup = table_open(DistCleanupRelationId(), AccessShareLock);
TupleDesc tupleDescriptor = RelationGetDescr(pgDistCleanup);
ScanKeyData scanKey[1];
ScanKeyInit(&scanKey[0], Anum_pg_dist_cleanup_object_type, BTEqualStrategyNumber,
F_INT4EQ, Int32GetDatum(type));
int scanKeyCount = 1;
Oid scanIndexId = InvalidOid;
bool useIndex = false;
SysScanDesc scanDescriptor = systable_beginscan(pgDistCleanup, scanIndexId, useIndex,
NULL,
scanKeyCount, scanKey);
HeapTuple heapTuple = NULL;
while (HeapTupleIsValid(heapTuple = systable_getnext(scanDescriptor)))
{
CleanupRecord *record = TupleToCleanupRecord(heapTuple, tupleDescriptor);
if (strcmp(record->objectName, objectName) == 0)
{
objectFound = record;
break;
}
}
systable_endscan(scanDescriptor);
table_close(pgDistCleanup, NoLock);
return objectFound;
}
/*
* TupleToCleanupRecord converts a pg_dist_cleanup record tuple into a CleanupRecord struct.
*/

View File

@ -156,7 +156,6 @@ static uint64 GetNextShardIdForSplitChild(void);
static void AcquireNonblockingSplitLock(Oid relationId);
static List * GetWorkerNodesFromWorkerIds(List *nodeIdsForPlacementList);
static void DropShardListMetadata(List *shardIntervalList);
static void InsertDeferredDropCleanupRecordsForShards(List *shardIntervalList);
/* Customize error message strings based on operation type */
static const char *const SplitOperationName[] =
@ -223,7 +222,7 @@ ErrorIfCannotSplitShard(SplitOperation splitOperation, ShardInterval *sourceShar
uint64 shardId = shardInterval->shardId;
ListCell *shardPlacementCell = NULL;
List *shardPlacementList = ShardPlacementListWithoutOrphanedPlacements(shardId);
List *shardPlacementList = ShardPlacementListSortedByWorker(shardId);
foreach(shardPlacementCell, shardPlacementList)
{
ShardPlacement *placement = (ShardPlacement *) lfirst(shardPlacementCell);
@ -1363,45 +1362,6 @@ DropShardListMetadata(List *shardIntervalList)
}
/*
* Insert deferred cleanup records.
* The shards will be dropped by background cleaner later.
*/
static void
InsertDeferredDropCleanupRecordsForShards(List *shardIntervalList)
{
ListCell *shardIntervalCell = NULL;
foreach(shardIntervalCell, shardIntervalList)
{
ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell);
ListCell *shardPlacementCell = NULL;
uint64 oldShardId = shardInterval->shardId;
/* mark for deferred drop */
List *shardPlacementList = ActiveShardPlacementList(oldShardId);
foreach(shardPlacementCell, shardPlacementList)
{
ShardPlacement *placement = (ShardPlacement *) lfirst(shardPlacementCell);
/* get shard name */
char *qualifiedShardName = ConstructQualifiedShardName(shardInterval);
/* Log shard in pg_dist_cleanup.
* Parent shards are to be dropped only on sucess after split workflow is complete,
* so mark the policy as 'CLEANUP_DEFERRED_ON_SUCCESS'.
* We also log cleanup record in the current transaction. If the current transaction rolls back,
* we do not generate a record at all.
*/
InsertCleanupRecordInCurrentTransaction(CLEANUP_OBJECT_SHARD_PLACEMENT,
qualifiedShardName,
placement->groupId,
CLEANUP_DEFERRED_ON_SUCCESS);
}
}
}
/*
* AcquireNonblockingSplitLock does not allow concurrent nonblocking splits, because we share memory and
* replication slots.

View File

@ -102,9 +102,9 @@ static List * RecreateTableDDLCommandList(Oid relationId);
static void EnsureTableListOwner(List *tableIdList);
static void EnsureTableListSuitableForReplication(List *tableIdList);
static void MarkForDropColocatedShardPlacement(ShardInterval *shardInterval,
char *nodeName,
int32 nodePort);
static void DropShardPlacementsFromMetadata(List *shardList,
char *nodeName,
int32 nodePort);
static void UpdateColocatedShardPlacementMetadataOnWorkers(int64 shardId,
char *sourceNodeName,
int32 sourceNodePort,
@ -377,6 +377,20 @@ citus_move_shard_placement(PG_FUNCTION_ARGS)
PlacementMovedUsingLogicalReplicationInTX = true;
}
DropOrphanedResourcesInSeparateTransaction();
ShardInterval *colocatedShard = NULL;
foreach_ptr(colocatedShard, colocatedShardList)
{
/*
* This is to prevent any race condition possibility among the shard moves.
* We don't allow the move to happen if the shard we are going to move has an
* orphaned placement somewhere that is not cleanup up yet.
*/
char *qualifiedShardName = ConstructQualifiedShardName(colocatedShard);
ErrorIfCleanupRecordForShardExists(qualifiedShardName);
}
/*
* CopyColocatedShardPlacement function copies given shard with its co-located
* shards.
@ -384,7 +398,12 @@ citus_move_shard_placement(PG_FUNCTION_ARGS)
CopyShardTables(colocatedShardList, sourceNodeName, sourceNodePort, targetNodeName,
targetNodePort, useLogicalReplication, "citus_move_shard_placement");
ShardInterval *colocatedShard = NULL;
/* delete old shards metadata and mark the shards as to be deferred drop */
int32 sourceGroupId = GroupForNode(sourceNodeName, sourceNodePort);
InsertCleanupRecordsForShardPlacementsOnNode(colocatedShardList,
sourceGroupId);
colocatedShard = NULL;
foreach_ptr(colocatedShard, colocatedShardList)
{
uint64 colocatedShardId = colocatedShard->shardId;
@ -396,8 +415,11 @@ citus_move_shard_placement(PG_FUNCTION_ARGS)
groupId);
}
/* since this is move operation, we remove shards from source node after copy */
MarkForDropColocatedShardPlacement(shardInterval, sourceNodeName, sourceNodePort);
/*
* Since this is move operation, we remove the placements from the metadata
* for the source node after copy.
*/
DropShardPlacementsFromMetadata(colocatedShardList, sourceNodeName, sourceNodePort);
UpdateColocatedShardPlacementMetadataOnWorkers(shardId, sourceNodeName,
sourceNodePort, targetNodeName,
@ -414,6 +436,74 @@ citus_move_shard_placement(PG_FUNCTION_ARGS)
}
/*
* Insert deferred cleanup records.
* The shards will be dropped by background cleaner later.
*/
void
InsertDeferredDropCleanupRecordsForShards(List *shardIntervalList)
{
ListCell *shardIntervalCell = NULL;
foreach(shardIntervalCell, shardIntervalList)
{
ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell);
ListCell *shardPlacementCell = NULL;
uint64 oldShardId = shardInterval->shardId;
/* mark for deferred drop */
List *shardPlacementList = ActiveShardPlacementList(oldShardId);
foreach(shardPlacementCell, shardPlacementList)
{
ShardPlacement *placement = (ShardPlacement *) lfirst(shardPlacementCell);
/* get shard name */
char *qualifiedShardName = ConstructQualifiedShardName(shardInterval);
/* Log shard in pg_dist_cleanup.
* Parent shards are to be dropped only on sucess after split workflow is complete,
* so mark the policy as 'CLEANUP_DEFERRED_ON_SUCCESS'.
* We also log cleanup record in the current transaction. If the current transaction rolls back,
* we do not generate a record at all.
*/
InsertCleanupRecordInCurrentTransaction(CLEANUP_OBJECT_SHARD_PLACEMENT,
qualifiedShardName,
placement->groupId,
CLEANUP_DEFERRED_ON_SUCCESS);
}
}
}
/*
* InsertCleanupRecordsForShardPlacementsOnNode inserts deferred cleanup records.
* The shards will be dropped by background cleaner later.
* This function does this only for the placements on the given node.
*/
void
InsertCleanupRecordsForShardPlacementsOnNode(List *shardIntervalList,
int32 groupId)
{
ShardInterval *shardInterval = NULL;
foreach_ptr(shardInterval, shardIntervalList)
{
/* get shard name */
char *qualifiedShardName = ConstructQualifiedShardName(shardInterval);
/* Log shard in pg_dist_cleanup.
* Parent shards are to be dropped only on sucess after split workflow is complete,
* so mark the policy as 'CLEANUP_DEFERRED_ON_SUCCESS'.
* We also log cleanup record in the current transaction. If the current transaction rolls back,
* we do not generate a record at all.
*/
InsertCleanupRecordInCurrentTransaction(CLEANUP_OBJECT_SHARD_PLACEMENT,
qualifiedShardName,
groupId,
CLEANUP_DEFERRED_ON_SUCCESS);
}
}
/*
* IsShardListOnNode determines whether a co-located shard list has
* active placements on a given node.
@ -1059,6 +1149,8 @@ ReplicateColocatedShardPlacement(int64 shardId, char *sourceNodeName,
EnsureReferenceTablesExistOnAllNodesExtended(shardReplicationMode);
}
DropOrphanedResourcesInSeparateTransaction();
CopyShardTables(colocatedShardList, sourceNodeName, sourceNodePort,
targetNodeName, targetNodePort, useLogicalReplication,
"citus_copy_shard_placement");
@ -1160,8 +1252,6 @@ CopyShardTables(List *shardIntervalList, char *sourceNodeName, int32 sourceNodeP
return;
}
DropOrphanedResourcesInSeparateTransaction();
/* Start operation to prepare for generating cleanup records */
RegisterOperationNeedingCleanup();
@ -1498,7 +1588,7 @@ static void
EnsureShardCanBeCopied(int64 shardId, const char *sourceNodeName, int32 sourceNodePort,
const char *targetNodeName, int32 targetNodePort)
{
List *shardPlacementList = ShardPlacementListIncludingOrphanedPlacements(shardId);
List *shardPlacementList = ShardPlacementList(shardId);
ShardPlacement *sourcePlacement = SearchShardPlacementInListOrError(
shardPlacementList,
@ -1516,39 +1606,10 @@ EnsureShardCanBeCopied(int64 shardId, const char *sourceNodeName, int32 sourceNo
if (targetPlacement != NULL)
{
if (targetPlacement->shardState == SHARD_STATE_TO_DELETE)
{
/*
* Trigger deletion of orphaned resources and hope that this removes
* the shard.
*/
DropOrphanedResourcesInSeparateTransaction();
shardPlacementList = ShardPlacementListIncludingOrphanedPlacements(shardId);
targetPlacement = SearchShardPlacementInList(shardPlacementList,
targetNodeName,
targetNodePort);
/*
* If it still doesn't remove the shard, then we error.
*/
if (targetPlacement != NULL)
{
ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg(
"shard " INT64_FORMAT
" still exists on the target node as an orphaned shard",
shardId),
errdetail(
"The existing shard is orphaned, but could not be deleted because there are still active queries on it")));
}
}
else
{
ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg(
"shard " INT64_FORMAT " already exists in the target node",
shardId)));
}
ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg(
"shard " INT64_FORMAT " already exists in the target node",
shardId)));
}
}
@ -1839,27 +1900,22 @@ RecreateTableDDLCommandList(Oid relationId)
/*
* 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.
* DropShardPlacementsFromMetadata drops the shard placement metadata for
* the shard placements of given shard interval list from pg_dist_placement.
*/
static void
MarkForDropColocatedShardPlacement(ShardInterval *shardInterval, char *nodeName,
int32 nodePort)
DropShardPlacementsFromMetadata(List *shardList,
char *nodeName, int32 nodePort)
{
List *colocatedShardList = ColocatedShardIntervalList(shardInterval);
ListCell *colocatedShardCell = NULL;
foreach(colocatedShardCell, colocatedShardList)
ShardInterval *shardInverval = NULL;
foreach_ptr(shardInverval, shardList)
{
ShardInterval *colocatedShard = (ShardInterval *) lfirst(colocatedShardCell);
uint64 shardId = colocatedShard->shardId;
List *shardPlacementList =
ShardPlacementListIncludingOrphanedPlacements(shardId);
uint64 shardId = shardInverval->shardId;
List *shardPlacementList = ShardPlacementList(shardId);
ShardPlacement *placement =
SearchShardPlacementInListOrError(shardPlacementList, nodeName, nodePort);
UpdateShardPlacementState(placement->placementId, SHARD_STATE_TO_DELETE);
DeleteShardPlacementRow(placement->placementId);
}
}

View File

@ -13,3 +13,12 @@ DROP FUNCTION pg_catalog.worker_append_table_to_shard(text, text, text, integer)
#include "udfs/citus_task_wait/11.2-1.sql"
#include "udfs/citus_prepare_pg_upgrade/11.2-1.sql"
#include "udfs/citus_finish_pg_upgrade/11.2-1.sql"
-- drop orphaned shards after inserting records for them into pg_dist_cleanup
INSERT INTO pg_dist_cleanup
SELECT nextval('pg_dist_cleanup_recordid_seq'), 0, 1, shard_name(sh.logicalrelid, sh.shardid) AS object_name, plc.groupid AS node_group_id, 0
FROM pg_dist_placement plc
JOIN pg_dist_shard sh ON sh.shardid = plc.shardid
WHERE plc.shardstate = 4;
DELETE FROM pg_dist_placement WHERE shardstate = 4;

View File

@ -104,7 +104,28 @@ DROP FUNCTION pg_catalog.isolate_tenant_to_new_shard(table_name regclass, tenant
DROP FUNCTION pg_catalog.create_distributed_table_concurrently;
DROP FUNCTION pg_catalog.citus_internal_delete_partition_metadata(regclass);
DROP TABLE pg_catalog.pg_dist_cleanup;
-- Check if user has any cleanup records.
-- If not, DROP pg_dist_cleanup and continue safely.
-- Otherwise, raise an exception to stop the downgrade process.
DO $$
DECLARE
cleanup_record_count INTEGER;
BEGIN
SELECT COUNT(*) INTO cleanup_record_count FROM pg_dist_cleanup;
IF cleanup_record_count = 0 THEN
-- no cleanup records exist, can safely downgrade
DROP TABLE pg_catalog.pg_dist_cleanup;
ELSE
RAISE EXCEPTION 'pg_dist_cleanup is introduced in Citus 11.1'
USING HINT = 'To downgrade Citus to an older version, you should '
'first cleanup all the orphaned resources and make sure '
'pg_dist_cleanup is empty, by executing '
'CALL citus_cleanup_orphaned_resources();';
END IF;
END;
$$ LANGUAGE plpgsql;
DROP SEQUENCE pg_catalog.pg_dist_operationid_seq;
DROP SEQUENCE pg_catalog.pg_dist_cleanup_recordid_seq;
DROP PROCEDURE pg_catalog.citus_cleanup_orphaned_resources();

View File

@ -133,7 +133,7 @@ load_shard_placement_array(PG_FUNCTION_ARGS)
}
else
{
placementList = ShardPlacementListIncludingOrphanedPlacements(shardId);
placementList = ShardPlacementList(shardId);
}
placementList = SortList(placementList, CompareShardPlacementsByWorker);

View File

@ -80,8 +80,7 @@ typedef struct RebalancePlanContext
Datum
run_try_drop_marked_resources(PG_FUNCTION_ARGS)
{
bool waitForLocks = false;
TryDropOrphanedResources(waitForLocks);
TryDropOrphanedResources();
PG_RETURN_VOID();
}

View File

@ -356,9 +356,9 @@ ErrorIfShardPlacementsNotColocated(Oid leftRelationId, Oid rightRelationId)
leftRelationName, rightRelationName)));
}
List *leftPlacementList = ShardPlacementListWithoutOrphanedPlacements(
List *leftPlacementList = ShardPlacementListSortedByWorker(
leftShardId);
List *rightPlacementList = ShardPlacementListWithoutOrphanedPlacements(
List *rightPlacementList = ShardPlacementListSortedByWorker(
rightShardId);
if (list_length(leftPlacementList) != list_length(rightPlacementList))

View File

@ -645,8 +645,7 @@ CitusMaintenanceDaemonMain(Datum main_arg)
*/
lastShardCleanTime = GetCurrentTimestamp();
bool waitForLocks = false;
numberOfDroppedResources = TryDropOrphanedResources(waitForLocks);
numberOfDroppedResources = TryDropOrphanedResources();
}
CommitTransactionCommand();

View File

@ -475,7 +475,7 @@ SingleReplicatedTable(Oid relationId)
foreach_ptr(shardIdPointer, shardIntervalList)
{
uint64 shardId = *shardIdPointer;
shardPlacementList = ShardPlacementListWithoutOrphanedPlacements(shardId);
shardPlacementList = ShardPlacementListSortedByWorker(shardId);
if (list_length(shardPlacementList) != 1)
{

View File

@ -147,7 +147,7 @@ WorkerDropDistributedTable(Oid relationId)
{
uint64 shardId = *shardIdPointer;
List *shardPlacementList = ShardPlacementListIncludingOrphanedPlacements(shardId);
List *shardPlacementList = ShardPlacementList(shardId);
ShardPlacement *placement = NULL;
foreach_ptr(placement, shardPlacementList)
{

View File

@ -174,7 +174,7 @@ extern int32 GetLocalGroupId(void);
extern int32 GetLocalNodeId(void);
extern void CitusTableCacheFlushInvalidatedEntries(void);
extern Oid LookupShardRelationFromCatalog(int64 shardId, bool missing_ok);
extern List * ShardPlacementListIncludingOrphanedPlacements(uint64 shardId);
extern List * ShardPlacementList(uint64 shardId);
extern void CitusInvalidateRelcacheByRelid(Oid relationId);
extern void CitusInvalidateRelcacheByShardId(int64 shardId);
extern void InvalidateForeignKeyGraph(void);

View File

@ -296,12 +296,11 @@ extern List * FilterActiveShardPlacementListByNode(List *shardPlacementList,
WorkerNode *workerNode);
extern List * ActiveShardPlacementListOnGroup(uint64 shardId, int32 groupId);
extern List * ActiveShardPlacementList(uint64 shardId);
extern List * ShardPlacementListWithoutOrphanedPlacements(uint64 shardId);
extern List * ShardPlacementListSortedByWorker(uint64 shardId);
extern ShardPlacement * ActiveShardPlacement(uint64 shardId, bool missingOk);
extern WorkerNode * ActiveShardPlacementWorkerNode(uint64 shardId);
extern List * BuildShardPlacementList(int64 shardId);
extern List * AllShardPlacementsOnNodeGroup(int32 groupId);
extern List * AllShardPlacementsWithShardPlacementState(ShardState shardState);
extern List * GroupShardPlacementsForTableOnGroup(Oid relationId, int32 groupId);
extern StringInfo GenerateSizeQueryOnMultiplePlacements(List *shardIntervalList,
SizeQueryType sizeQueryType,
@ -325,7 +324,6 @@ extern void UpdateDistributionColumn(Oid relationId, char distributionMethod,
Var *distributionColumn, int colocationId);
extern void DeletePartitionRow(Oid distributedRelationId);
extern void DeleteShardRow(uint64 shardId);
extern void UpdateShardPlacementState(uint64 placementId, char shardState);
extern void UpdatePlacementGroupId(uint64 placementId, int groupId);
extern void DeleteShardPlacementRow(uint64 placementId);
extern void CreateDistributedTable(Oid relationId, char *distributionColumnName,

View File

@ -35,7 +35,6 @@ typedef enum
{
SHARD_STATE_INVALID_FIRST = 0,
SHARD_STATE_ACTIVE = 1,
SHARD_STATE_TO_DELETE = 4,
} ShardState;

View File

@ -23,8 +23,9 @@ extern bool CheckAvailableSpaceBeforeMove;
extern int NextOperationId;
extern int NextCleanupRecordId;
extern int TryDropOrphanedResources(bool waitForLocks);
extern int TryDropOrphanedResources(void);
extern void DropOrphanedResourcesInSeparateTransaction(void);
extern void ErrorIfCleanupRecordForShardExists(char *shardName);
/* Members for cleanup infrastructure */
typedef uint64 OperationId;

View File

@ -23,3 +23,6 @@ extern void UpdatePlacementUpdateStatusForShardIntervalList(List *shardIntervalL
char *sourceName,
int sourcePort,
PlacementUpdateStatus status);
extern void InsertDeferredDropCleanupRecordsForShards(List *shardIntervalList);
extern void InsertCleanupRecordsForShardPlacementsOnNode(List *shardIntervalList,
int32 groupId);

View File

@ -297,3 +297,6 @@ s/"Total Cost": [0-9].[0-9]+/"Total Cost": xxxx/g
# normalize gpids
s/(NOTICE: issuing SET LOCAL application_name TO 'citus_rebalancer gpid=)[0-9]+/\1xxxxx/g
# shard_rebalancer output, flaky improvement number
s/improvement of 0.1[0-9]* is lower/improvement of 0.1xxxxx is lower/g

View File

@ -23,7 +23,7 @@ SELECT create_distributed_table('test','x');
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT nodename, nodeport, COUNT(*)
FROM pg_dist_placement AS placement,
pg_dist_node AS node
@ -48,7 +48,7 @@ SELECT * from citus_drain_node('localhost', :worker_1_port, shard_transfer_mode
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT nodename, nodeport, COUNT(*)
FROM pg_dist_placement AS placement,
pg_dist_node AS node
@ -78,7 +78,7 @@ SELECT * FROM rebalance_table_shards(shard_transfer_mode :='force_logical');
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT nodename, nodeport, COUNT(*)
FROM pg_dist_placement AS placement,
pg_dist_node AS node
@ -103,7 +103,7 @@ SELECT * FROM rebalance_table_shards(shard_transfer_mode :='force_logical');
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT nodename, nodeport, COUNT(*)
FROM pg_dist_placement AS placement,
pg_dist_node AS node

View File

@ -218,8 +218,12 @@ SELECT citus_move_shard_placement(8981000, 'localhost', :worker_1_port, 'localho
(1 row)
-- END : Move one shard before we split it.
CALL pg_catalog.citus_cleanup_orphaned_shards();
NOTICE: cleaned up 3 orphaned shards
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
-- BEGIN : Set node id variables
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
@ -272,8 +276,12 @@ SELECT citus_move_shard_placement(8981007, 'localhost', :worker_1_port, 'localho
(1 row)
-- END : Move a shard post split.
CALL pg_catalog.citus_cleanup_orphaned_shards();
NOTICE: cleaned up 3 orphaned shards
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
-- BEGIN : Display current state.
SELECT shard.shardid, logicalrelid, shardminvalue, shardmaxvalue, nodename, nodeport
FROM pg_dist_shard AS shard
@ -636,4 +644,5 @@ DETAIL: drop cascades to table citus_split_test_schema.sensors
drop cascades to table citus_split_test_schema.reference_table
drop cascades to table citus_split_test_schema.colocated_dist_table
drop cascades to table citus_split_test_schema.table_with_index_rep_identity
DROP ROLE test_shard_split_role;
--END : Cleanup

View File

@ -214,8 +214,12 @@ SELECT citus_move_shard_placement(8981000, 'localhost', :worker_1_port, 'localho
(1 row)
-- END : Move one shard before we split it.
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 3 orphaned shards
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
-- BEGIN : Set node id variables
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
@ -268,8 +272,12 @@ SELECT citus_move_shard_placement(8981007, 'localhost', :worker_1_port, 'localho
(1 row)
-- END : Move a shard post split.
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 3 orphaned shards
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
-- BEGIN : Display current state.
SELECT shard.shardid, logicalrelid, shardminvalue, shardmaxvalue, nodename, nodeport
FROM pg_dist_shard AS shard

View File

@ -130,7 +130,12 @@ SELECT citus.mitmproxy('conn.onQuery(query="ADD CONSTRAINT").cancel(' || :pid ||
SELECT master_move_shard_placement(201, 'localhost', :worker_1_port, 'localhost', :worker_2_proxy_port, 'block_writes');
ERROR: canceling statement due to user request
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
-- Verify that the shard is not moved and the number of rows are still 100k
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
@ -166,8 +171,12 @@ SELECT master_move_shard_placement(201, 'localhost', :worker_1_port, 'localhost'
(1 row)
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 1 orphaned shards
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
SELECT * FROM shards_in_workers;
shardid | worker
---------------------------------------------------------------------

View File

@ -232,8 +232,6 @@ SELECT public.wait_for_resource_cleanup();
(1 row)
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 1 orphaned shards
-- failure on setting lock_timeout (right before dropping subscriptions & replication slots)
SELECT citus.mitmproxy('conn.onQuery(query="^SET LOCAL lock_timeout").kill()');
mitmproxy
@ -284,8 +282,6 @@ SELECT public.wait_for_resource_cleanup();
(1 row)
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 1 orphaned shards
-- cancellation on disabling subscription (right before dropping it)
SELECT citus.mitmproxy('conn.onQuery(query="^ALTER SUBSCRIPTION .* DISABLE").cancel(' || :pid || ')');
mitmproxy
@ -316,6 +312,7 @@ SELECT pg_reload_conf();
t
(1 row)
SET citus.next_operation_id TO 777;
-- failure on dropping subscription
SELECT citus.mitmproxy('conn.onQuery(query="^DROP SUBSCRIPTION").killall()');
mitmproxy
@ -338,14 +335,12 @@ SELECT citus.mitmproxy('conn.allow()');
-- first, manually drop the subscsription object. But the record for it will remain on pg_dist_cleanup
-- we expect the drop query will succeed on only one node
SELECT COUNT(*)
FROM run_command_on_workers(
$$DROP SUBSCRIPTION citus_shard_move_subscription_xxxxxxx_xxxxxxx$$)
WHERE success AND result = 'DROP SUBSCRIPTION';
count
SELECT 1 FROM run_command_on_workers('DROP SUBSCRIPTION citus_shard_move_subscription_xxxxxxx_xxxxxxx');
?column?
---------------------------------------------------------------------
1
(1 row)
1
1
(2 rows)
-- reset back
ALTER SYSTEM RESET citus.defer_shard_delete_interval;
@ -514,6 +509,19 @@ SELECT count(*) FROM t;
100000
(1 row)
-- cleanup leftovers
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
DROP SCHEMA move_shard CASCADE ;
NOTICE: drop cascades to 2 other objects
DETAIL: drop cascades to table t

View File

@ -61,8 +61,12 @@ SELECT count(*) FROM referencing_table2;
101
(1 row)
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 2 orphaned shards
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(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
---------------------------------------------------------------------
@ -110,8 +114,12 @@ SELECT count(*) FROM referencing_table2;
101
(1 row)
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 2 orphaned shards
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(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
---------------------------------------------------------------------

View File

@ -1,336 +0,0 @@
CREATE SCHEMA ignoring_orphaned_shards;
SET search_path TO ignoring_orphaned_shards;
-- Use a weird shard count that we don't use in any other tests
SET citus.shard_count TO 13;
SET citus.shard_replication_factor TO 1;
SET citus.next_shard_id TO 92448000;
CREATE TABLE ref(id int PRIMARY KEY);
SELECT * FROM create_reference_table('ref');
create_reference_table
---------------------------------------------------------------------
(1 row)
SET citus.next_shard_id TO 92448100;
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 92448100;
CREATE TABLE dist1(id int);
SELECT * FROM create_distributed_table('dist1', 'id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT logicalrelid FROM pg_dist_partition WHERE colocationid = 92448100 ORDER BY 1;
logicalrelid
---------------------------------------------------------------------
dist1
(1 row)
-- Move first shard, so that the first shard now has 2 placements. One that's
-- active and one that's orphaned.
SELECT citus_move_shard_placement(92448100, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'block_writes');
citus_move_shard_placement
---------------------------------------------------------------------
(1 row)
SELECT shardid, shardstate, nodeport FROM pg_dist_shard_placement WHERE shardid = 92448100 ORDER BY placementid;
shardid | shardstate | nodeport
---------------------------------------------------------------------
92448100 | 4 | 57637
92448100 | 1 | 57638
(2 rows)
-- Add a new table that should get colocated with dist1 automatically, but
-- should not get a shard for the orphaned placement.
SET citus.next_shard_id TO 92448200;
CREATE TABLE dist2(id int);
SELECT * FROM create_distributed_table('dist2', 'id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT logicalrelid FROM pg_dist_partition WHERE colocationid = 92448100 ORDER BY 1;
logicalrelid
---------------------------------------------------------------------
dist1
dist2
(2 rows)
SELECT shardid, shardstate, nodeport FROM pg_dist_shard_placement WHERE shardid = 92448200 ORDER BY placementid;
shardid | shardstate | nodeport
---------------------------------------------------------------------
92448200 | 1 | 57638
(1 row)
-- uncolocate it
SELECT update_distributed_table_colocation('dist2', 'none');
update_distributed_table_colocation
---------------------------------------------------------------------
(1 row)
SELECT logicalrelid FROM pg_dist_partition WHERE colocationid = 92448100 ORDER BY 1;
logicalrelid
---------------------------------------------------------------------
dist1
(1 row)
-- Make sure we can add it back to the colocation, even though it has a
-- different number of shard placements for the first shard.
SELECT update_distributed_table_colocation('dist2', 'dist1');
update_distributed_table_colocation
---------------------------------------------------------------------
(1 row)
SELECT logicalrelid FROM pg_dist_partition WHERE colocationid = 92448100 ORDER BY 1;
logicalrelid
---------------------------------------------------------------------
dist1
dist2
(2 rows)
-- Make sure that replication count check in FOR UPDATE ignores orphaned
-- shards.
SELECT * FROM dist1 WHERE id = 1 FOR UPDATE;
id
---------------------------------------------------------------------
(0 rows)
-- Make sure we don't send a query to the orphaned shard
BEGIN;
SET LOCAL citus.log_remote_commands TO ON;
INSERT INTO dist1 VALUES (1);
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing INSERT INTO ignoring_orphaned_shards.dist1_92448100 (id) VALUES (1)
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
ROLLBACK;
NOTICE: issuing ROLLBACK
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
-- Make sure we can create a foreign key on community edition, because
-- replication factor is 1
ALTER TABLE dist1
ADD CONSTRAINT dist1_ref_fk
FOREIGN KEY (id)
REFERENCES ref(id);
SET citus.shard_replication_factor TO 2;
SET citus.next_shard_id TO 92448300;
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 92448300;
CREATE TABLE rep1(id int);
SELECT * FROM create_distributed_table('rep1', 'id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- Add the coordinator, so we can have a replicated shard
SELECT 1 FROM citus_add_node('localhost', :master_port, 0);
NOTICE: localhost:xxxxx is the coordinator and already contains metadata, skipping syncing the metadata
?column?
---------------------------------------------------------------------
1
(1 row)
SELECT 1 FROM citus_set_node_property('localhost', :master_port, 'shouldhaveshards', true);
?column?
---------------------------------------------------------------------
1
(1 row)
SELECT logicalrelid FROM pg_dist_partition WHERE colocationid = 92448300 ORDER BY 1;
logicalrelid
---------------------------------------------------------------------
rep1
(1 row)
SELECT citus_move_shard_placement(92448300, 'localhost', :worker_1_port, 'localhost', :master_port, 'block_writes');
citus_move_shard_placement
---------------------------------------------------------------------
(1 row)
SELECT shardid, shardstate, nodeport FROM pg_dist_shard_placement WHERE shardid = 92448300 ORDER BY placementid;
shardid | shardstate | nodeport
---------------------------------------------------------------------
92448300 | 4 | 57637
92448300 | 1 | 57638
92448300 | 1 | 57636
(3 rows)
-- Add a new table that should get colocated with rep1 automatically, but
-- should not get a shard for the orphaned placement.
SET citus.next_shard_id TO 92448400;
CREATE TABLE rep2(id int);
SELECT * FROM create_distributed_table('rep2', 'id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT logicalrelid FROM pg_dist_partition WHERE colocationid = 92448300 ORDER BY 1;
logicalrelid
---------------------------------------------------------------------
rep1
rep2
(2 rows)
SELECT shardid, shardstate, nodeport FROM pg_dist_shard_placement WHERE shardid = 92448400 ORDER BY placementid;
shardid | shardstate | nodeport
---------------------------------------------------------------------
92448400 | 1 | 57636
92448400 | 1 | 57638
(2 rows)
-- uncolocate it
SELECT update_distributed_table_colocation('rep2', 'none');
update_distributed_table_colocation
---------------------------------------------------------------------
(1 row)
SELECT logicalrelid FROM pg_dist_partition WHERE colocationid = 92448300 ORDER BY 1;
logicalrelid
---------------------------------------------------------------------
rep1
(1 row)
-- Make sure we can add it back to the colocation, even though it has a
-- different number of shard placements for the first shard.
SELECT update_distributed_table_colocation('rep2', 'rep1');
update_distributed_table_colocation
---------------------------------------------------------------------
(1 row)
SELECT logicalrelid FROM pg_dist_partition WHERE colocationid = 92448300 ORDER BY 1;
logicalrelid
---------------------------------------------------------------------
rep1
rep2
(2 rows)
-- Make sure we don't send a query to the orphaned shard
BEGIN;
SET LOCAL citus.log_remote_commands TO ON;
SET LOCAL citus.log_local_commands TO ON;
INSERT INTO rep1 VALUES (1);
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing SELECT lock_shard_resources(3, ARRAY[92448300])
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing INSERT INTO ignoring_orphaned_shards.rep1_92448300 (id) VALUES (1)
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: executing the command locally: INSERT INTO ignoring_orphaned_shards.rep1_92448300 (id) VALUES (1)
ROLLBACK;
NOTICE: issuing ROLLBACK
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing ROLLBACK
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
-- Cause the orphaned shard to be local
SELECT 1 FROM citus_drain_node('localhost', :master_port, 'block_writes');
NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ...
?column?
---------------------------------------------------------------------
1
(1 row)
SELECT shardid, shardstate, nodeport FROM pg_dist_shard_placement WHERE shardid = 92448300 ORDER BY placementid;
shardid | shardstate | nodeport
---------------------------------------------------------------------
92448300 | 1 | 57638
92448300 | 4 | 57636
92448300 | 1 | 57637
(3 rows)
-- Make sure we don't send a query to the orphaned shard if it's local
BEGIN;
SET LOCAL citus.log_remote_commands TO ON;
SET LOCAL citus.log_local_commands TO ON;
INSERT INTO rep1 VALUES (1);
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing SELECT lock_shard_resources(3, ARRAY[92448300])
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing INSERT INTO ignoring_orphaned_shards.rep1_92448300 (id) VALUES (1)
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing INSERT INTO ignoring_orphaned_shards.rep1_92448300 (id) VALUES (1)
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
ROLLBACK;
NOTICE: issuing ROLLBACK
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing ROLLBACK
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
SET citus.shard_replication_factor TO 1;
SET citus.next_shard_id TO 92448500;
CREATE TABLE range1(id int);
SELECT create_distributed_table('range1', 'id', 'range');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CALL public.create_range_partitioned_shards('range1', '{0,3}','{2,5}');
-- Move shard placement and clean it up
SELECT citus_move_shard_placement(92448500, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'block_writes');
citus_move_shard_placement
---------------------------------------------------------------------
(1 row)
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 1 orphaned shards
SELECT shardid, shardstate, nodeport FROM pg_dist_shard_placement WHERE shardid = 92448300 ORDER BY placementid;
shardid | shardstate | nodeport
---------------------------------------------------------------------
92448300 | 1 | 57638
92448300 | 1 | 57637
(2 rows)
SET citus.next_shard_id TO 92448600;
CREATE TABLE range2(id int);
SELECT create_distributed_table('range2', 'id', 'range');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CALL public.create_range_partitioned_shards('range2', '{0,3}','{2,5}');
-- Mark tables co-located
UPDATE pg_dist_partition SET colocationid = 30001
WHERE logicalrelid = 'range1'::regclass OR logicalrelid = 'range2'::regclass;
-- Move shard placement and DON'T clean it up, now range1 and range2 are
-- colocated, but only range2 has an orphaned shard.
SELECT citus_move_shard_placement(92448600, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'block_writes');
citus_move_shard_placement
---------------------------------------------------------------------
(1 row)
SELECT shardid, shardstate, nodeport FROM pg_dist_shard_placement WHERE shardid = 92448600 ORDER BY placementid;
shardid | shardstate | nodeport
---------------------------------------------------------------------
92448600 | 4 | 57638
92448600 | 1 | 57637
(2 rows)
-- Make sure co-located join works
SELECT * FROM range1 JOIN range2 ON range1.id = range2.id;
id | id
---------------------------------------------------------------------
(0 rows)
-- Make sure we can create a foreign key on community edition, because
-- replication factor is 1
ALTER TABLE range1
ADD CONSTRAINT range1_ref_fk
FOREIGN KEY (id)
REFERENCES ref(id);
SET client_min_messages TO WARNING;
DROP SCHEMA ignoring_orphaned_shards CASCADE;

View File

@ -76,3 +76,67 @@ nodename |nodeport|count
localhost| 57638| 2
(1 row)
starting permutation: s2-print-placements s2-begin s2-select-from-table s1-move-placement s1-move-placement-back s1-wait s2-commit s2-print-placements
step s2-print-placements:
SELECT
nodename, nodeport, count(*)
FROM
pg_dist_shard_placement
WHERE
shardid IN (SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_move_table'::regclass)
AND
shardstate = 1
GROUP BY
nodename, nodeport;
nodename |nodeport|count
---------------------------------------------------------------------
localhost| 57637| 1
localhost| 57638| 1
(2 rows)
step s2-begin:
BEGIN;
step s2-select-from-table:
SELECT * FROM test_move_table WHERE x=5;
x|y
---------------------------------------------------------------------
(0 rows)
step s1-move-placement:
SELECT master_move_shard_placement((SELECT * FROM selected_shard_for_test_table), 'localhost', 57637, 'localhost', 57638, 'force_logical');
master_move_shard_placement
---------------------------------------------------------------------
(1 row)
step s1-move-placement-back:
SELECT master_move_shard_placement((SELECT * FROM selected_shard_for_test_table), 'localhost', 57638, 'localhost', 57637, 'force_logical');
<waiting ...>
step s1-move-placement-back: <... completed>
ERROR: shard move failed as the orphaned shard public.test_move_table_102011 leftover from the previous move could not be cleaned up
step s1-wait:
step s2-commit:
COMMIT;
step s2-print-placements:
SELECT
nodename, nodeport, count(*)
FROM
pg_dist_shard_placement
WHERE
shardid IN (SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_move_table'::regclass)
AND
shardstate = 1
GROUP BY
nodename, nodeport;
nodename |nodeport|count
---------------------------------------------------------------------
localhost| 57638| 2
(1 row)

View File

@ -12,19 +12,23 @@ master_move_shard_placement
(1 row)
s1: NOTICE: cleaned up 1 orphaned shards
step s1-drop-marked-shards:
SET client_min_messages to NOTICE;
CALL isolation_cleanup_orphaned_shards();
SET client_min_messages to ERROR;
CALL isolation_cleanup_orphaned_resources();
SELECT COUNT(*) FROM pg_dist_cleanup WHERE object_type = 1 AND object_name LIKE 'public.t1_%';
count
---------------------------------------------------------------------
0
(1 row)
step s2-drop-marked-shards:
SET client_min_messages to DEBUG1;
CALL isolation_cleanup_orphaned_shards();
<waiting ...>
step s1-commit:
CALL isolation_cleanup_orphaned_resources();
step s1-commit:
COMMIT;
step s2-drop-marked-shards: <... completed>
starting permutation: s1-begin s1-move-placement s2-drop-marked-shards s1-drop-marked-shards s1-commit
step s1-begin:
@ -40,12 +44,17 @@ master_move_shard_placement
step s2-drop-marked-shards:
SET client_min_messages to DEBUG1;
CALL isolation_cleanup_orphaned_shards();
CALL isolation_cleanup_orphaned_resources();
s1: NOTICE: cleaned up 1 orphaned shards
step s1-drop-marked-shards:
SET client_min_messages to NOTICE;
CALL isolation_cleanup_orphaned_shards();
SET client_min_messages to ERROR;
CALL isolation_cleanup_orphaned_resources();
SELECT COUNT(*) FROM pg_dist_cleanup WHERE object_type = 1 AND object_name LIKE 'public.t1_%';
count
---------------------------------------------------------------------
0
(1 row)
step s1-commit:
COMMIT;
@ -86,12 +95,16 @@ run_commands_on_session_level_connection_to_node
(1 row)
step s1-drop-marked-shards:
SET client_min_messages to NOTICE;
CALL isolation_cleanup_orphaned_shards();
SET client_min_messages to ERROR;
CALL isolation_cleanup_orphaned_resources();
SELECT COUNT(*) FROM pg_dist_cleanup WHERE object_type = 1 AND object_name LIKE 'public.t1_%';
<waiting ...>
s1: WARNING: canceling statement due to lock timeout
step s1-drop-marked-shards: <... completed>
s1: WARNING: failed to clean up 1 orphaned shards out of 1
count
---------------------------------------------------------------------
1
(1 row)
step s1-commit:
COMMIT;
@ -103,87 +116,3 @@ stop_session_level_connection_to_node
(1 row)
starting permutation: s1-begin s1-move-placement s2-start-session-level-connection s2-lock-table-on-worker s1-commit s1-begin s1-move-placement-back s1-commit s2-stop-connection
step s1-begin:
BEGIN;
step s1-move-placement:
SELECT master_move_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638);
master_move_shard_placement
---------------------------------------------------------------------
(1 row)
step s2-start-session-level-connection:
SELECT start_session_level_connection_to_node('localhost', 57637);
start_session_level_connection_to_node
---------------------------------------------------------------------
(1 row)
step s2-lock-table-on-worker:
SELECT run_commands_on_session_level_connection_to_node('BEGIN;');
SELECT run_commands_on_session_level_connection_to_node('LOCK TABLE t1_120000');
run_commands_on_session_level_connection_to_node
---------------------------------------------------------------------
(1 row)
run_commands_on_session_level_connection_to_node
---------------------------------------------------------------------
(1 row)
step s1-commit:
COMMIT;
step s1-begin:
BEGIN;
step s1-move-placement-back:
SET client_min_messages to NOTICE;
SHOW log_error_verbosity;
SELECT master_move_shard_placement((SELECT * FROM selected_shard), 'localhost', 57638, 'localhost', 57637);
<waiting ...>
step s1-move-placement-back: <... completed>
log_error_verbosity
---------------------------------------------------------------------
verbose
(1 row)
ERROR: shard xxxxx still exists on the target node as an orphaned shard
step s1-commit:
COMMIT;
step s2-stop-connection:
SELECT stop_session_level_connection_to_node();
stop_session_level_connection_to_node
---------------------------------------------------------------------
(1 row)
starting permutation: s1-begin s1-lock-pg-dist-placement s2-drop-old-shards s1-commit
step s1-begin:
BEGIN;
step s1-lock-pg-dist-placement:
LOCK TABLE pg_dist_placement IN SHARE ROW EXCLUSIVE MODE;
s2: DEBUG: could not acquire shard lock to cleanup placements
step s2-drop-old-shards:
SELECT run_try_drop_marked_resources();
run_try_drop_marked_resources
---------------------------------------------------------------------
(1 row)
step s1-commit:
COMMIT;

View File

@ -427,90 +427,6 @@ SELECT master_get_active_worker_nodes();
(localhost,57637)
(2 rows)
-- mark all placements in the candidate node as to be deleted
UPDATE pg_dist_placement SET shardstate=4 WHERE groupid=:worker_2_group;
SELECT run_command_on_workers('UPDATE pg_dist_placement SET shardstate=4 WHERE groupid=' || :'worker_2_group');
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"UPDATE 8")
(localhost,57638,t,"UPDATE 8")
(2 rows)
SELECT shardid, shardstate, nodename, nodeport FROM pg_dist_shard_placement WHERE nodeport=:worker_2_port;
shardid | shardstate | nodename | nodeport
---------------------------------------------------------------------
1220001 | 4 | localhost | 57638
1220003 | 4 | localhost | 57638
1220005 | 4 | localhost | 57638
1220007 | 4 | localhost | 57638
1220009 | 4 | localhost | 57638
1220011 | 4 | localhost | 57638
1220013 | 4 | localhost | 57638
1220015 | 4 | localhost | 57638
(8 rows)
CREATE TABLE cluster_management_test_colocated (col_1 text, col_2 int);
-- Check that we warn the user about colocated shards that will not get created for shards that do not have active placements
SELECT create_distributed_table('cluster_management_test_colocated', 'col_1', 'hash', colocate_with => 'cluster_management_test');
WARNING: could not find any shard placements for shardId 1220017
WARNING: could not find any shard placements for shardId 1220019
WARNING: could not find any shard placements for shardId 1220021
WARNING: could not find any shard placements for shardId 1220023
WARNING: could not find any shard placements for shardId 1220025
WARNING: could not find any shard placements for shardId 1220027
WARNING: could not find any shard placements for shardId 1220029
WARNING: could not find any shard placements for shardId 1220031
WARNING: could not find any shard placements for shardId 1220017
WARNING: could not find any shard placements for shardId 1220017
WARNING: could not find any shard placements for shardId 1220017
WARNING: could not find any shard placements for shardId 1220017
WARNING: could not find any shard placements for shardId 1220017
WARNING: could not find any shard placements for shardId 1220017
WARNING: could not find any shard placements for shardId 1220017
WARNING: could not find any shard placements for shardId 1220017
WARNING: could not find any shard placements for shardId 1220017
WARNING: could not find any shard placements for shardId 1220017
WARNING: could not find any shard placements for shardId 1220017
WARNING: could not find any shard placements for shardId 1220017
WARNING: could not find any shard placements for shardId 1220017
WARNING: could not find any shard placements for shardId 1220017
WARNING: could not find any shard placements for shardId 1220017
WARNING: could not find any shard placements for shardId 1220017
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- Check that colocated shards don't get created for shards that are to be deleted
SELECT logicalrelid, shardid, shardstate, nodename, nodeport FROM pg_dist_shard_placement NATURAL JOIN pg_dist_shard ORDER BY shardstate, shardid;
logicalrelid | shardid | shardstate | nodename | nodeport
---------------------------------------------------------------------
cluster_management_test | 1220000 | 1 | localhost | 57637
cluster_management_test | 1220002 | 1 | localhost | 57637
cluster_management_test | 1220004 | 1 | localhost | 57637
cluster_management_test | 1220006 | 1 | localhost | 57637
cluster_management_test | 1220008 | 1 | localhost | 57637
cluster_management_test | 1220010 | 1 | localhost | 57637
cluster_management_test | 1220012 | 1 | localhost | 57637
cluster_management_test | 1220014 | 1 | localhost | 57637
cluster_management_test_colocated | 1220016 | 1 | localhost | 57637
cluster_management_test_colocated | 1220018 | 1 | localhost | 57637
cluster_management_test_colocated | 1220020 | 1 | localhost | 57637
cluster_management_test_colocated | 1220022 | 1 | localhost | 57637
cluster_management_test_colocated | 1220024 | 1 | localhost | 57637
cluster_management_test_colocated | 1220026 | 1 | localhost | 57637
cluster_management_test_colocated | 1220028 | 1 | localhost | 57637
cluster_management_test_colocated | 1220030 | 1 | localhost | 57637
cluster_management_test | 1220001 | 4 | localhost | 57638
cluster_management_test | 1220003 | 4 | localhost | 57638
cluster_management_test | 1220005 | 4 | localhost | 57638
cluster_management_test | 1220007 | 4 | localhost | 57638
cluster_management_test | 1220009 | 4 | localhost | 57638
cluster_management_test | 1220011 | 4 | localhost | 57638
cluster_management_test | 1220013 | 4 | localhost | 57638
cluster_management_test | 1220015 | 4 | localhost | 57638
(24 rows)
-- clean-up
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
?column?
@ -526,9 +442,6 @@ SELECT run_command_on_workers('UPDATE pg_dist_placement SET shardstate=1 WHERE g
(localhost,57638,t,"UPDATE 8")
(2 rows)
SET client_min_messages TO ERROR;
DROP TABLE cluster_management_test_colocated;
RESET client_min_messages;
-- when there is no primary we should get a pretty error
UPDATE pg_dist_node SET noderole = 'secondary' WHERE nodeport=:worker_2_port;
SELECT * FROM cluster_management_test;

View File

@ -88,6 +88,9 @@ SELECT citus_copy_shard_placement(13000000, 'localhost', :worker_1_port, 'localh
(1 row)
-- error out if trying to move a shard that already has a placement on the target
SELECT citus_move_shard_placement(13000000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'force_logical');
ERROR: shard xxxxx already exists in the target node
-- status after shard copy
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
FROM
@ -564,8 +567,12 @@ SELECT master_move_shard_placement(13000034, 'localhost', :worker_1_port, 'local
(1 row)
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 1 orphaned shards
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(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
@ -612,7 +619,12 @@ SELECT master_move_shard_placement(13000038, 'localhost', :worker_2_port, 'local
WARNING: relation "public.logical_failure_test_13000038" does not exist
CONTEXT: while executing command on localhost:xxxxx
ERROR: cannot get the size because of a connection error
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
DROP TABLE logical_failure_test;
-- lets test the logical replication modes
CREATE TABLE test_with_pkey (key int PRIMARY KEY, value int NOT NULL);
@ -631,8 +643,12 @@ SELECT master_move_shard_placement(13000042, 'localhost', :worker_1_port, 'local
(1 row)
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 1 orphaned shards
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
-- should succeed since we still have a replica identity
ALTER TABLE test_with_pkey REPLICA IDENTITY FULL;
SELECT master_move_shard_placement(13000042, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'auto');
@ -641,8 +657,12 @@ SELECT master_move_shard_placement(13000042, 'localhost', :worker_2_port, 'local
(1 row)
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 1 orphaned shards
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
-- make sure we have the replica identity after the move
SELECT result FROM run_command_on_placements( 'test_with_pkey', 'SELECT relreplident FROM pg_class WHERE relname = ''%s''') WHERE shardid = 13000042;
result
@ -656,7 +676,12 @@ SELECT master_move_shard_placement(13000042, 'localhost', :worker_1_port, 'local
ERROR: cannot use logical replication to transfer shards of the relation test_with_pkey since it doesn't have a REPLICA IDENTITY or PRIMARY KEY
DETAIL: UPDATE and DELETE commands on the shard will error out during logical replication unless there is a REPLICA IDENTITY or PRIMARY KEY.
HINT: If you wish to continue without a replica identity set the shard_transfer_mode to 'force_logical' or 'block_writes'.
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
-- make sure we have the replica identity after the move
SELECT result FROM run_command_on_placements( 'test_with_pkey', 'SELECT relreplident FROM pg_class WHERE relname = ''%s''') WHERE shardid = 13000042;
result
@ -672,8 +697,12 @@ SELECT master_move_shard_placement(13000042, 'localhost', :worker_1_port, 'local
(1 row)
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 1 orphaned shards
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
-- make sure we have the replica identity after the move
SELECT result FROM run_command_on_placements( 'test_with_pkey', 'SELECT relreplident FROM pg_class WHERE relname = ''%s''') WHERE shardid = 13000042;
result
@ -690,8 +719,12 @@ SELECT master_move_shard_placement(13000042, 'localhost', :worker_2_port, 'local
(1 row)
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 1 orphaned shards
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
-- make sure we have the replica identity after the move
SELECT result FROM run_command_on_placements( 'test_with_pkey', 'SELECT relreplident FROM pg_class WHERE relname = ''%s''') WHERE shardid = 13000042;
result
@ -706,8 +739,12 @@ SELECT master_move_shard_placement(13000042, 'localhost', :worker_1_port, 'local
(1 row)
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 1 orphaned shards
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
-- make sure we have the replica identity after the move
SELECT result FROM run_command_on_placements( 'test_with_pkey', 'SELECT relreplident FROM pg_class WHERE relname = ''%s''') WHERE shardid = 13000042;
result
@ -726,10 +763,13 @@ DETAIL: from localhost:xxxxx
(1 row)
CALL citus_cleanup_orphaned_shards();
LOG: deferred drop of orphaned shard public.test_with_pkey_13000042 on localhost:xxxxx after a move completed
NOTICE: cleaned up 1 orphaned shards
SET client_min_messages TO DEFAULT;
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
-- we don't support multiple shard moves in a single transaction
SELECT
master_move_shard_placement(shardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port, shard_transfer_mode:='force_logical')
@ -738,7 +778,12 @@ FROM
shardid IN (SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_with_pkey'::regclass);
ERROR: moving multiple shard placements via logical replication in the same transaction is currently not supported
HINT: If you wish to move multiple shard placements in a single transaction set the shard_transfer_mode to 'block_writes'.
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
-- similar test with explicit transaction block
BEGIN;
SELECT master_move_shard_placement(13000042, 'localhost', :worker_1_port, 'localhost', :worker_2_port, shard_transfer_mode:='force_logical');
@ -751,7 +796,12 @@ BEGIN;
ERROR: moving multiple shard placements via logical replication in the same transaction is currently not supported
HINT: If you wish to move multiple shard placements in a single transaction set the shard_transfer_mode to 'block_writes'.
COMMIT;
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
-- we do support the same with block writes
SELECT
master_move_shard_placement(shardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port, shard_transfer_mode:='block_writes')
@ -764,8 +814,12 @@ FROM
(2 rows)
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 2 orphaned shards
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
-- we should be able to move shard placements after COMMIT/ABORT
BEGIN;
SELECT master_move_shard_placement(13000043, 'localhost', :worker_2_port, 'localhost', :worker_1_port, shard_transfer_mode:='force_logical');
@ -775,16 +829,24 @@ BEGIN;
(1 row)
COMMIT;
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 1 orphaned shards
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
SELECT master_move_shard_placement(13000045, 'localhost', :worker_2_port, 'localhost', :worker_1_port, shard_transfer_mode:='force_logical');
master_move_shard_placement
---------------------------------------------------------------------
(1 row)
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 1 orphaned shards
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
BEGIN;
SELECT master_move_shard_placement(13000043, 'localhost', :worker_1_port, 'localhost', :worker_2_port, shard_transfer_mode:='force_logical');
master_move_shard_placement
@ -799,8 +861,12 @@ SELECT master_move_shard_placement(13000045, 'localhost', :worker_1_port, 'local
(1 row)
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 1 orphaned shards
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
-- we should be able to move shard placements of partitioend tables
CREATE SCHEMA move_partitions;
CREATE TABLE move_partitions.events (
@ -835,7 +901,12 @@ ORDER BY shardid LIMIT 1;
ERROR: cannot use logical replication to transfer shards of the relation events_1 since it doesn't have a REPLICA IDENTITY or PRIMARY KEY
DETAIL: UPDATE and DELETE commands on the shard will error out during logical replication unless there is a REPLICA IDENTITY or PRIMARY KEY.
HINT: If you wish to continue without a replica identity set the shard_transfer_mode to 'force_logical' or 'block_writes'.
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
-- force logical replication
SELECT master_move_shard_placement(shardid, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'force_logical')
FROM pg_dist_shard JOIN pg_dist_shard_placement USING (shardid)
@ -846,8 +917,12 @@ ORDER BY shardid LIMIT 1;
(1 row)
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 2 orphaned shards
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
SELECT count(*) FROM move_partitions.events;
count
---------------------------------------------------------------------
@ -866,8 +941,12 @@ ORDER BY shardid LIMIT 1;
(1 row)
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 2 orphaned shards
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
SELECT count(*) FROM move_partitions.events;
count
---------------------------------------------------------------------
@ -884,8 +963,12 @@ ORDER BY shardid LIMIT 1;
(1 row)
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 2 orphaned shards
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
SELECT count(*) FROM move_partitions.events;
count
---------------------------------------------------------------------
@ -910,3 +993,4 @@ DROP TABLE table1_group1;
DROP TABLE table5_groupX;
DROP TABLE table6_append;
DROP TABLE serial_move_test;
DROP SCHEMA move_partitions CASCADE;

View File

@ -1189,7 +1189,44 @@ SELECT * FROM multi_extension.print_extension_changes();
-- Test downgrade to 11.1-1 from 11.2-1
ALTER EXTENSION citus UPDATE TO '11.2-1';
-- create a table with orphaned shards to see if orphaned shards will be dropped
-- and cleanup records will be created for them
SET citus.shard_replication_factor to 1;
CREATE TABLE table_with_orphaned_shards (a int);
SELECT create_distributed_table('table_with_orphaned_shards', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- show there are 4 placements
SELECT * FROM pg_dist_placement ORDER BY shardid;
placementid | shardid | shardstate | shardlength | groupid
---------------------------------------------------------------------
1 | 102008 | 1 | 0 | 0
2 | 102009 | 1 | 0 | 0
3 | 102010 | 1 | 0 | 0
4 | 102011 | 1 | 0 | 0
(4 rows)
-- mark two of them as orphaned
UPDATE pg_dist_placement SET shardstate = 4 WHERE shardid % 2 = 1;
ALTER EXTENSION citus UPDATE TO '11.1-1';
-- show placements and cleanup records
SELECT * FROM pg_dist_placement ORDER BY shardid;
placementid | shardid | shardstate | shardlength | groupid
---------------------------------------------------------------------
1 | 102008 | 1 | 0 | 0
2 | 102009 | 4 | 0 | 0
3 | 102010 | 1 | 0 | 0
4 | 102011 | 4 | 0 | 0
(4 rows)
SELECT * FROM pg_dist_cleanup;
record_id | operation_id | object_type | object_name | node_group_id | policy_type
---------------------------------------------------------------------
(0 rows)
-- Should be empty result since upgrade+downgrade should be a no-op
SELECT * FROM multi_extension.print_extension_changes();
previous_object | current_object
@ -1198,6 +1235,31 @@ SELECT * FROM multi_extension.print_extension_changes();
-- Snapshot of state at 11.2-1
ALTER EXTENSION citus UPDATE TO '11.2-1';
-- verify that the placements are deleted and cleanup records are created
SELECT * FROM pg_dist_placement ORDER BY shardid;
placementid | shardid | shardstate | shardlength | groupid
---------------------------------------------------------------------
1 | 102008 | 1 | 0 | 0
3 | 102010 | 1 | 0 | 0
(2 rows)
SELECT * FROM pg_dist_cleanup;
record_id | operation_id | object_type | object_name | node_group_id | policy_type
---------------------------------------------------------------------
1 | 0 | 1 | table_with_orphaned_shards_102009 | 0 | 0
2 | 0 | 1 | table_with_orphaned_shards_102011 | 0 | 0
(2 rows)
-- error out as cleanup records remain
ALTER EXTENSION citus UPDATE TO '11.0-4';
ERROR: pg_dist_cleanup is introduced in Citus 11.1
HINT: To downgrade Citus to an older version, you should first cleanup all the orphaned resources and make sure pg_dist_cleanup is empty, by executing CALL citus_cleanup_orphaned_resources();
CONTEXT: PL/pgSQL function inline_code_block line XX at RAISE
-- cleanup
SET client_min_messages TO ERROR;
CALL citus_cleanup_orphaned_resources();
DROP TABLE table_with_orphaned_shards;
RESET client_min_messages;
SELECT * FROM multi_extension.print_extension_changes();
previous_object | current_object
---------------------------------------------------------------------

View File

@ -156,7 +156,7 @@ END;
$$ LANGUAGE plpgsql;
-- Introduce a function that waits until all cleanup records are deleted, for testing purposes
CREATE OR REPLACE FUNCTION wait_for_resource_cleanup() RETURNS void
SET client_min_messages TO WARNING
SET client_min_messages TO ERROR
AS $$
DECLARE
record_count integer;

View File

@ -310,8 +310,12 @@ NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ...
(1 row)
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 1 orphaned shards
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
SELECT result AS column_compression FROM run_command_on_workers($$SELECT ARRAY(
SELECT attname || ' ' || attcompression::text FROM pg_attribute WHERE attrelid::regclass::text LIKE 'pg14.col\_compression%' AND attnum > 0 ORDER BY 1
)$$);

View File

@ -223,8 +223,12 @@ SELECT citus_move_shard_placement(8970000, 'localhost', :worker_1_port, 'localho
(1 row)
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 8 orphaned shards
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
\c - postgres - :worker_2_port
SET search_path TO "shard Move Fkeys Indexes", public, pg_catalog;
SELECT "Constraint", "Definition" FROM table_fkeys WHERE relid='sensors_8970000'::regclass ORDER BY 1,2;
@ -422,7 +426,22 @@ SELECT citus_move_shard_placement(8970000, 'localhost', :worker_1_port, 'localho
(1 row)
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
\c - postgres - :master_port
-- create a fake cleanup record
INSERT INTO pg_dist_cleanup
SELECT nextval('pg_dist_cleanup_recordid_seq'), 0, 1, shard_name(logicalrelid, shardid) AS object_name, -13 AS node_group_id, 0
FROM pg_dist_shard WHERE shardid = 8970000;
-- make sure we error out if there's a cleanup record for the shard to be moved
SELECT citus_move_shard_placement(8970000, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'force_logical');
ERROR: shard move failed as the orphaned shard "shard Move Fkeys Indexes".sensors_8970000 leftover from the previous move could not be cleaned up
-- delete the fake record
DELETE FROM pg_dist_cleanup WHERE node_group_id = -13;
-- stop and re-sync the metadata to make sure all works fine
SELECT stop_metadata_sync_to_node('localhost', :worker_1_port);
NOTICE: dropping metadata on the node (localhost,57637)
@ -459,3 +478,4 @@ drop cascades to table "shard Move Fkeys Indexes".colocated_partitioned_table
drop cascades to table "shard Move Fkeys Indexes".reference_table
drop cascades to table "shard Move Fkeys Indexes".index_backed_rep_identity
drop cascades to table "shard Move Fkeys Indexes".multiple_unique_keys
DROP ROLE mx_rebalancer_role_ent;

View File

@ -222,8 +222,12 @@ SELECT citus_move_shard_placement(8970000, 'localhost', :worker_1_port, 'localho
(1 row)
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 8 orphaned shards
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(1 row)
\c - postgres - :worker_2_port
SET search_path TO "blocking shard Move Fkeys Indexes", public, pg_catalog;
SELECT "Constraint", "Definition" FROM table_fkeys WHERE relid='sensors_8970000'::regclass ORDER BY 1,2;
@ -360,3 +364,4 @@ drop cascades to table "blocking shard Move Fkeys Indexes".colocated_dist_table
drop cascades to table "blocking shard Move Fkeys Indexes".colocated_partitioned_table
drop cascades to table "blocking shard Move Fkeys Indexes".reference_table
drop cascades to table "blocking shard Move Fkeys Indexes".index_backed_rep_identity
DROP ROLE mx_rebalancer_blocking_role_ent;

View File

@ -51,12 +51,15 @@ $cmd$);
-- Make sure this cannot be run in a transaction
BEGIN;
CALL citus_cleanup_orphaned_shards();
ERROR: citus_cleanup_orphaned_shards cannot run inside a transaction block
CALL citus_cleanup_orphaned_resources();
ERROR: citus_cleanup_orphaned_resources cannot run inside a transaction block
COMMIT;
-- execute delayed removal
-- citus_cleanup_orphaned_shards is deprecated
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 1 orphaned shards
WARNING: citus_cleanup_orphaned_shards is deprecated. Use citus_cleanup_orphaned_resources instead
-- execute delayed removal
CALL citus_cleanup_orphaned_resources();
NOTICE: cleaned up 1 orphaned resources
-- 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';
@ -98,6 +101,12 @@ SELECT pg_sleep(1);
(1 row)
SELECT public.wait_for_resource_cleanup();
wait_for_resource_cleanup
---------------------------------------------------------------------
(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';
@ -152,7 +161,7 @@ SELECT master_move_shard_placement(20000000, 'localhost', :worker_1_port, 'local
ROLLBACK;
-- see the cleanup record for the shard on the target node
-- https://github.com/citusdata/citus/issues/6580
select object_name, object_type from pg_dist_cleanup;
select object_name, object_type from pg_dist_cleanup where object_type = 1;
object_name | object_type
---------------------------------------------------------------------
shard_move_deferred_delete.t1_20000000 | 1

View File

@ -34,14 +34,14 @@ SELECT rebalance_table_shards('dist_table_test');
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT rebalance_table_shards();
rebalance_table_shards
---------------------------------------------------------------------
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
-- test that calling rebalance_table_shards without specifying relation
-- wouldn't move shard of the citus local table.
SET citus.next_shard_id TO 433100;
@ -59,7 +59,7 @@ SELECT rebalance_table_shards();
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
-- Check that rebalance_table_shards and get_rebalance_table_shards_plan fail
-- for any type of table, but distributed tables.
SELECT rebalance_table_shards('ref_table_test');
@ -110,7 +110,7 @@ SELECT pg_sleep(.1); -- wait to make sure the config has changed before running
SELECT master_drain_node('localhost', :master_port);
ERROR: connection to the remote node foobar:57636 failed with the following error: could not translate host name "foobar" to address: <system specific error>
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
ALTER SYSTEM RESET citus.local_hostname;
SELECT pg_reload_conf();
pg_reload_conf
@ -130,7 +130,7 @@ SELECT master_drain_node('localhost', :master_port);
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
-- show that citus local table shard is still on the coordinator
SELECT tablename FROM pg_catalog.pg_tables where tablename like 'citus_local_table_%';
tablename
@ -202,8 +202,6 @@ BEGIN;
SELECT replicate_table_shards('dist_table_test_2', max_shard_copies := 4, shard_transfer_mode:='block_writes');
NOTICE: issuing CALL citus_cleanup_orphaned_resources()
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing CALL citus_cleanup_orphaned_shards()
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: Copying shard xxxxx from localhost:xxxxx to localhost:xxxxx ...
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
@ -328,13 +326,13 @@ CREATE OR REPLACE FUNCTION shard_placement_rebalance_array(
RETURNS json[]
AS 'citus'
LANGUAGE C STRICT VOLATILE;
CREATE FUNCTION shard_placement_replication_array(worker_node_list json[],
shard_placement_list json[],
shard_replication_factor int)
CREATE OR REPLACE FUNCTION shard_placement_replication_array(worker_node_list json[],
shard_placement_list json[],
shard_replication_factor int)
RETURNS json[]
AS 'citus'
LANGUAGE C STRICT VOLATILE;
CREATE FUNCTION worker_node_responsive(worker_node_name text, worker_node_port int)
CREATE OR REPLACE FUNCTION worker_node_responsive(worker_node_name text, worker_node_port int)
RETURNS boolean
AS 'citus'
LANGUAGE C STRICT VOLATILE;
@ -633,7 +631,7 @@ GROUP BY logicalrelid::regclass, nodename, nodeport
ORDER BY logicalrelid::regclass, nodename, nodeport;
-- Create six shards with replication factor 1 and move them to the same
-- node to create an unbalanced cluster.
CREATE PROCEDURE create_unbalanced_shards(rel text)
CREATE OR REPLACE PROCEDURE create_unbalanced_shards(rel text)
LANGUAGE SQL
AS $$
SET citus.shard_replication_factor TO 1;
@ -646,7 +644,7 @@ AS $$
pg_dist_shard_placement src USING (shardid),
(SELECT nodename, nodeport FROM pg_dist_shard_placement ORDER BY nodeport DESC LIMIT 1) dst
WHERE src.nodeport < dst.nodeport AND s.logicalrelid = rel::regclass;
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
$$;
CALL create_unbalanced_shards('rebalance_test_table');
SET citus.shard_replication_factor TO 2;
@ -683,7 +681,7 @@ FROM (
WHERE logicalrelid = 'rebalance_test_table'::regclass
) T;
ERROR: connection to the remote node foobar:57636 failed with the following error: could not translate host name "foobar" to address: <system specific error>
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
ALTER SYSTEM RESET citus.local_hostname;
SELECT pg_reload_conf();
pg_reload_conf
@ -712,7 +710,7 @@ FROM (
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM table_placements_per_node;
nodeport | logicalrelid | count
---------------------------------------------------------------------
@ -760,7 +758,7 @@ SELECT rebalance_table_shards('rebalance_test_table',
ERROR: must be owner of table rebalance_test_table
CONTEXT: while executing command on localhost:xxxxx
RESET ROLE;
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
-- Confirm no moves took place at all during these errors
SELECT * FROM table_placements_per_node;
nodeport | logicalrelid | count
@ -769,7 +767,7 @@ SELECT * FROM table_placements_per_node;
57638 | rebalance_test_table | 5
(2 rows)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT rebalance_table_shards('rebalance_test_table',
threshold := 0, max_shard_moves := 1,
shard_transfer_mode:='block_writes');
@ -778,7 +776,7 @@ SELECT rebalance_table_shards('rebalance_test_table',
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM table_placements_per_node;
nodeport | logicalrelid | count
---------------------------------------------------------------------
@ -793,7 +791,7 @@ SELECT rebalance_table_shards('rebalance_test_table', threshold := 1, shard_tran
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM table_placements_per_node;
nodeport | logicalrelid | count
---------------------------------------------------------------------
@ -808,7 +806,7 @@ SELECT rebalance_table_shards('rebalance_test_table', threshold := 0, shard_tran
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM table_placements_per_node;
nodeport | logicalrelid | count
---------------------------------------------------------------------
@ -824,7 +822,7 @@ SELECT rebalance_table_shards('rebalance_test_table', threshold := 0, shard_tran
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM table_placements_per_node;
nodeport | logicalrelid | count
---------------------------------------------------------------------
@ -983,7 +981,7 @@ SELECT rebalance_table_shards('imbalanced_table', threshold:=0, shard_transfer_m
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
-- Confirm rebalance
-- Shard counts in each node after rebalance
SELECT * FROM public.table_placements_per_node;
@ -1004,6 +1002,7 @@ DROP TABLE test_schema_support.imbalanced_table;
DROP TABLE test_schema_support.imbalanced_table_local;
SET citus.shard_replication_factor TO 1;
SET citus.shard_count = 4;
ALTER SEQUENCE pg_catalog.pg_dist_placement_placementid_seq RESTART 136;
CREATE TABLE colocated_rebalance_test(id integer);
CREATE TABLE colocated_rebalance_test2(id integer);
SELECT create_distributed_table('colocated_rebalance_test', 'id');
@ -1020,7 +1019,7 @@ FROM pg_dist_shard_placement
WHERE nodeport = :worker_2_port;
ERROR: Moving shards to a non-existing node is not supported
HINT: Add the target node via SELECT citus_add_node('localhost', 10000);
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
-- Try to move shards to a node where shards are not allowed
SELECT * from master_set_node_property('localhost', :worker_1_port, 'shouldhaveshards', false);
master_set_node_property
@ -1064,7 +1063,7 @@ WHERE nodeport = :worker_2_port;
(2 rows)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT create_distributed_table('colocated_rebalance_test2', 'id');
create_distributed_table
---------------------------------------------------------------------
@ -1092,7 +1091,7 @@ SELECT * FROM rebalance_table_shards('colocated_rebalance_test', threshold := 0,
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
-- Confirm that nothing changed
SELECT * FROM public.table_placements_per_node;
nodeport | logicalrelid | count
@ -1134,7 +1133,7 @@ SELECT * FROM rebalance_table_shards('colocated_rebalance_test', threshold := 0,
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
-- Check that we can call this function without a crash
SELECT * FROM get_rebalance_progress();
sessionid | table_name | shardid | shard_size | sourcename | sourceport | targetname | targetport | progress | source_shard_size | target_shard_size | operation_type | source_lsn | target_lsn | status
@ -1151,18 +1150,18 @@ SELECT * FROM public.table_placements_per_node;
57638 | colocated_rebalance_test2 | 2
(4 rows)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
select * from pg_dist_placement ORDER BY placementid;
placementid | shardid | shardstate | shardlength | groupid
---------------------------------------------------------------------
146 | 123023 | 1 | 0 | 14
149 | 123024 | 1 | 0 | 14
152 | 123027 | 1 | 0 | 14
153 | 123028 | 1 | 0 | 14
154 | 123021 | 1 | 0 | 16
155 | 123025 | 1 | 0 | 16
156 | 123022 | 1 | 0 | 16
157 | 123026 | 1 | 0 | 16
138 | 123023 | 1 | 0 | 14
141 | 123024 | 1 | 0 | 14
144 | 123027 | 1 | 0 | 14
145 | 123028 | 1 | 0 | 14
146 | 123021 | 1 | 0 | 16
147 | 123025 | 1 | 0 | 16
148 | 123022 | 1 | 0 | 16
149 | 123026 | 1 | 0 | 16
(8 rows)
-- Move all shards to worker1 again
@ -1183,7 +1182,7 @@ SELECT * FROM public.table_placements_per_node;
57637 | colocated_rebalance_test2 | 4
(2 rows)
-- Explicitly don't run citus_cleanup_orphaned_shards, rebalance_table_shards
-- Explicitly don't run citus_cleanup_orphaned_resources, rebalance_table_shards
-- should do that for automatically.
SELECT * FROM rebalance_table_shards('colocated_rebalance_test', threshold := 0, shard_transfer_mode := 'block_writes');
rebalance_table_shards
@ -1242,7 +1241,7 @@ SELECT * FROM rebalance_table_shards('colocated_rebalance_test', threshold := 0,
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
nodeport | logicalrelid | count
---------------------------------------------------------------------
@ -1265,7 +1264,7 @@ SELECT * FROM rebalance_table_shards('non_colocated_rebalance_test', threshold :
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
nodeport | logicalrelid | count
---------------------------------------------------------------------
@ -1287,7 +1286,7 @@ SELECT * FROM rebalance_table_shards('colocated_rebalance_test', threshold := 0,
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
nodeport | logicalrelid | count
---------------------------------------------------------------------
@ -1304,7 +1303,7 @@ SELECT * FROM rebalance_table_shards('non_colocated_rebalance_test', threshold :
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
nodeport | logicalrelid | count
---------------------------------------------------------------------
@ -1324,12 +1323,14 @@ SELECT * from master_set_node_property('localhost', :worker_2_port, 'shouldhaves
(1 row)
SELECT * FROM get_rebalance_table_shards_plan(threshold := 0, drain_only := true);
-- we actually shouldn't need the ORDER BY clause as the output will be in execution order
-- but this one involves different colocation groups and which colocation group is first moved is not consistent
SELECT * FROM get_rebalance_table_shards_plan(threshold := 0, drain_only := true) ORDER BY shardid;
table_name | shardid | shard_size | sourcename | sourceport | targetname | targetport
---------------------------------------------------------------------
colocated_rebalance_test | 123021 | 0 | localhost | 57638 | localhost | 57637
colocated_rebalance_test2 | 123025 | 0 | localhost | 57638 | localhost | 57637
colocated_rebalance_test | 123022 | 0 | localhost | 57638 | localhost | 57637
colocated_rebalance_test2 | 123025 | 0 | localhost | 57638 | localhost | 57637
colocated_rebalance_test2 | 123026 | 0 | localhost | 57638 | localhost | 57637
non_colocated_rebalance_test | 123029 | 0 | localhost | 57638 | localhost | 57637
non_colocated_rebalance_test | 123030 | 0 | localhost | 57638 | localhost | 57637
@ -1341,7 +1342,7 @@ SELECT * FROM rebalance_table_shards(threshold := 0, shard_transfer_mode := 'blo
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
nodeport | logicalrelid | count
---------------------------------------------------------------------
@ -1363,7 +1364,7 @@ SELECT * FROM rebalance_table_shards(threshold := 0, shard_transfer_mode := 'blo
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
nodeport | logicalrelid | count
---------------------------------------------------------------------
@ -1383,12 +1384,14 @@ SELECT * from master_set_node_property('localhost', :worker_2_port, 'shouldhaves
(1 row)
SELECT * FROM get_rebalance_table_shards_plan(threshold := 0);
-- we actually shouldn't need the ORDER BY clause as the output will be in execution order
-- but this one involves different colocation groups and which colocation group is first moved is not consistent
SELECT * FROM get_rebalance_table_shards_plan(threshold := 0) ORDER BY shardid;
table_name | shardid | shard_size | sourcename | sourceport | targetname | targetport
---------------------------------------------------------------------
colocated_rebalance_test | 123021 | 0 | localhost | 57638 | localhost | 57637
colocated_rebalance_test2 | 123025 | 0 | localhost | 57638 | localhost | 57637
colocated_rebalance_test | 123022 | 0 | localhost | 57638 | localhost | 57637
colocated_rebalance_test2 | 123025 | 0 | localhost | 57638 | localhost | 57637
colocated_rebalance_test2 | 123026 | 0 | localhost | 57638 | localhost | 57637
non_colocated_rebalance_test | 123029 | 0 | localhost | 57638 | localhost | 57637
non_colocated_rebalance_test | 123030 | 0 | localhost | 57638 | localhost | 57637
@ -1400,7 +1403,7 @@ SELECT * FROM rebalance_table_shards(threshold := 0, shard_transfer_mode := 'blo
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
nodeport | logicalrelid | count
---------------------------------------------------------------------
@ -1422,7 +1425,7 @@ SELECT * FROM rebalance_table_shards(threshold := 0, shard_transfer_mode := 'blo
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
nodeport | logicalrelid | count
---------------------------------------------------------------------
@ -1448,7 +1451,7 @@ SELECT * from master_drain_node('localhost', :worker_2_port, shard_transfer_mode
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
select shouldhaveshards from pg_dist_node where nodeport = :worker_2_port;
shouldhaveshards
---------------------------------------------------------------------
@ -1476,7 +1479,7 @@ SELECT * FROM rebalance_table_shards(threshold := 0, shard_transfer_mode := 'blo
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
nodeport | logicalrelid | count
---------------------------------------------------------------------
@ -1489,6 +1492,7 @@ SELECT * FROM public.table_placements_per_node;
(6 rows)
-- Drop some tables for clear consistent error
DROP TABLE test_schema_support.non_colocated_rebalance_test;
DROP TABLE test_schema_support.colocated_rebalance_test2;
-- testing behaviour when a transfer fails when using master_drain_node
SELECT * from master_drain_node('localhost', :worker_2_port);
@ -1505,13 +1509,11 @@ select shouldhaveshards from pg_dist_node where nodeport = :worker_2_port;
-- Make sure no actual nodes are moved
SELECT * FROM public.table_placements_per_node;
nodeport | logicalrelid | count
nodeport | logicalrelid | count
---------------------------------------------------------------------
57637 | colocated_rebalance_test | 2
57638 | colocated_rebalance_test | 2
57637 | non_colocated_rebalance_test | 2
57638 | non_colocated_rebalance_test | 2
(4 rows)
57637 | colocated_rebalance_test | 2
57638 | colocated_rebalance_test | 2
(2 rows)
-- Make it a data node again
SELECT * from master_set_node_property('localhost', :worker_2_port, 'shouldhaveshards', true);
@ -1546,7 +1548,13 @@ INSERT INTO tab SELECT 6 from generate_series(1, 10000);
VACUUM FULL tab;
ANALYZE tab;
\c - - - :worker_1_port
SELECT table_schema, table_name, row_estimate, total_bytes
SELECT table_schema, table_name, row_estimate,
CASE
WHEN total_bytes BETWEEN 1900000 AND 2300000 THEN 2179072
WHEN total_bytes BETWEEN 900000 AND 1200000 THEN 1089536
WHEN total_bytes BETWEEN 300000 AND 440000 THEN 368640
ELSE total_bytes
END
FROM (
SELECT *, total_bytes-index_bytes-COALESCE(toast_bytes,0) AS table_bytes FROM (
SELECT c.oid,nspname AS table_schema, relname AS TABLE_NAME
@ -1568,7 +1576,13 @@ WHERE table_schema = 'public'
(2 rows)
\c - - - :worker_2_port
SELECT table_schema, table_name, row_estimate, total_bytes
SELECT table_schema, table_name, row_estimate,
CASE
WHEN total_bytes BETWEEN 1900000 AND 2300000 THEN 2179072
WHEN total_bytes BETWEEN 900000 AND 1200000 THEN 1089536
WHEN total_bytes BETWEEN 300000 AND 440000 THEN 368640
ELSE total_bytes
END
FROM (
SELECT *, total_bytes-index_bytes-COALESCE(toast_bytes,0) AS table_bytes FROM (
SELECT c.oid,nspname AS table_schema, relname AS TABLE_NAME
@ -1615,7 +1629,7 @@ SELECT * FROM rebalance_table_shards('tab', shard_transfer_mode:='block_writes')
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
nodeport | logicalrelid | count
---------------------------------------------------------------------
@ -1630,8 +1644,8 @@ NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ...
(1 row)
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 1 orphaned shards
CALL citus_cleanup_orphaned_resources();
NOTICE: cleaned up 1 orphaned resources
SELECT * FROM public.table_placements_per_node;
nodeport | logicalrelid | count
---------------------------------------------------------------------
@ -1647,7 +1661,7 @@ DETAIL: Using threshold of 0.01
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
nodeport | logicalrelid | count
---------------------------------------------------------------------
@ -1672,7 +1686,13 @@ INSERT INTO tab2 SELECT 6 from generate_series(1, 10000);
VACUUM FULL tab, tab2;
ANALYZE tab, tab2;
\c - - - :worker_1_port
SELECT table_schema, table_name, row_estimate, total_bytes
SELECT table_schema, table_name, row_estimate,
CASE
WHEN total_bytes BETWEEN 1900000 AND 2300000 THEN 2179072
WHEN total_bytes BETWEEN 900000 AND 1200000 THEN 1089536
WHEN total_bytes BETWEEN 300000 AND 440000 THEN 368640
ELSE total_bytes
END
FROM (
SELECT *, total_bytes-index_bytes-COALESCE(toast_bytes,0) AS table_bytes FROM (
SELECT c.oid,nspname AS table_schema, relname AS TABLE_NAME
@ -1694,7 +1714,13 @@ WHERE table_schema = 'public'
(2 rows)
\c - - - :worker_2_port
SELECT table_schema, table_name, row_estimate, total_bytes
SELECT table_schema, table_name, row_estimate,
CASE
WHEN total_bytes BETWEEN 1900000 AND 2300000 THEN 2179072
WHEN total_bytes BETWEEN 900000 AND 1200000 THEN 1089536
WHEN total_bytes BETWEEN 300000 AND 440000 THEN 368640
ELSE total_bytes
END
FROM (
SELECT *, total_bytes-index_bytes-COALESCE(toast_bytes,0) AS table_bytes FROM (
SELECT c.oid,nspname AS table_schema, relname AS TABLE_NAME
@ -1720,16 +1746,17 @@ WHERE table_schema = 'public'
(6 rows)
\c - - - :master_port
SELECT * FROM get_rebalance_table_shards_plan('tab', rebalance_strategy := 'by_disk_size');
-- disk sizes can be slightly different, so ORDER BY shardid gives us a consistent output
SELECT * FROM get_rebalance_table_shards_plan('tab', rebalance_strategy := 'by_disk_size') ORDER BY shardid;
NOTICE: Ignoring move of shard xxxxx from localhost:xxxxx to localhost:xxxxx, because the move only brings a small improvement relative to the shard its size
DETAIL: The balance improvement of 0.151125 is lower than the improvement_threshold of 0.5
DETAIL: The balance improvement of 0.1xxxxx is lower than the improvement_threshold of 0.5
NOTICE: Ignored 1 moves, all of which are shown in notices above
HINT: If you do want these moves to happen, try changing improvement_threshold to a lower value than what it is now (0.5).
table_name | shardid | shard_size | sourcename | sourceport | targetname | targetport
---------------------------------------------------------------------
tab | 123041 | 0 | localhost | 57638 | localhost | 57637
tab2 | 123051 | 0 | localhost | 57638 | localhost | 57637
tab | 123042 | 0 | localhost | 57638 | localhost | 57637
tab2 | 123051 | 0 | localhost | 57638 | localhost | 57637
tab2 | 123052 | 0 | localhost | 57638 | localhost | 57637
(4 rows)
@ -1745,7 +1772,7 @@ SELECT * FROM get_rebalance_table_shards_plan('tab', rebalance_strategy := 'by_d
SELECT * FROM rebalance_table_shards('tab', rebalance_strategy := 'by_disk_size', shard_transfer_mode:='block_writes');
NOTICE: Ignoring move of shard xxxxx from localhost:xxxxx to localhost:xxxxx, because the move only brings a small improvement relative to the shard its size
DETAIL: The balance improvement of 0.151125 is lower than the improvement_threshold of 0.5
DETAIL: The balance improvement of 0.1xxxxx is lower than the improvement_threshold of 0.5
NOTICE: Ignored 1 moves, all of which are shown in notices above
HINT: If you do want these moves to happen, try changing improvement_threshold to a lower value than what it is now (0.5).
NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ...
@ -1755,8 +1782,8 @@ NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ...
(1 row)
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 2 orphaned shards
CALL citus_cleanup_orphaned_resources();
NOTICE: cleaned up 2 orphaned resources
SELECT * FROM public.table_placements_per_node;
nodeport | logicalrelid | count
---------------------------------------------------------------------
@ -1769,7 +1796,13 @@ SELECT * FROM public.table_placements_per_node;
VACUUM FULL tab, tab2;
ANALYZE tab, tab2;
\c - - - :worker_1_port
SELECT table_schema, table_name, row_estimate, total_bytes
SELECT table_schema, table_name, row_estimate,
CASE
WHEN total_bytes BETWEEN 1900000 AND 2300000 THEN 2179072
WHEN total_bytes BETWEEN 900000 AND 1200000 THEN 1089536
WHEN total_bytes BETWEEN 300000 AND 440000 THEN 368640
ELSE total_bytes
END
FROM (
SELECT *, total_bytes-index_bytes-COALESCE(toast_bytes,0) AS table_bytes FROM (
SELECT c.oid,nspname AS table_schema, relname AS TABLE_NAME
@ -1795,7 +1828,13 @@ WHERE table_schema = 'public'
(6 rows)
\c - - - :worker_2_port
SELECT table_schema, table_name, row_estimate, total_bytes
SELECT table_schema, table_name, row_estimate,
CASE
WHEN total_bytes BETWEEN 1900000 AND 2300000 THEN 2179072
WHEN total_bytes BETWEEN 900000 AND 1200000 THEN 1089536
WHEN total_bytes BETWEEN 300000 AND 440000 THEN 368640
ELSE total_bytes
END
FROM (
SELECT *, total_bytes-index_bytes-COALESCE(toast_bytes,0) AS table_bytes FROM (
SELECT c.oid,nspname AS table_schema, relname AS TABLE_NAME
@ -1854,8 +1893,8 @@ NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ...
(1 row)
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 1 orphaned shards
CALL citus_cleanup_orphaned_resources();
NOTICE: cleaned up 1 orphaned resources
SELECT * FROM public.table_placements_per_node;
nodeport | logicalrelid | count
---------------------------------------------------------------------
@ -1879,14 +1918,14 @@ SELECT * FROM rebalance_table_shards('tab', shard_transfer_mode:='block_writes')
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
nodeport | logicalrelid | count
---------------------------------------------------------------------
57638 | tab | 4
(1 row)
CREATE FUNCTION only_worker_1(shardid bigint, nodeidarg int)
CREATE OR REPLACE FUNCTION only_worker_1(shardid bigint, nodeidarg int)
RETURNS boolean AS $$
SELECT
(CASE WHEN nodeport = 57637 THEN TRUE ELSE FALSE END)
@ -1929,8 +1968,8 @@ NOTICE: Moving shard xxxxx from localhost:xxxxx to localhost:xxxxx ...
(1 row)
CALL citus_cleanup_orphaned_shards();
NOTICE: cleaned up 1 orphaned shards
CALL citus_cleanup_orphaned_resources();
NOTICE: cleaned up 1 orphaned resources
SELECT * FROM public.table_placements_per_node;
nodeport | logicalrelid | count
---------------------------------------------------------------------
@ -1955,10 +1994,10 @@ SELECT * FROM get_rebalance_table_shards_plan('tab', rebalance_strategy := 'non_
ERROR: could not find rebalance strategy with name non_existing
SELECT * FROM rebalance_table_shards('tab', rebalance_strategy := 'non_existing');
ERROR: could not find rebalance strategy with name non_existing
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM master_drain_node('localhost', :worker_2_port, rebalance_strategy := 'non_existing');
ERROR: could not find rebalance strategy with name non_existing
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT citus_set_default_rebalance_strategy('non_existing');
ERROR: strategy with specified name does not exist
UPDATE pg_dist_rebalance_strategy SET default_strategy=false;
@ -1966,10 +2005,10 @@ SELECT * FROM get_rebalance_table_shards_plan('tab');
ERROR: no rebalance_strategy was provided, but there is also no default strategy set
SELECT * FROM rebalance_table_shards('tab');
ERROR: no rebalance_strategy was provided, but there is also no default strategy set
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM master_drain_node('localhost', :worker_2_port);
ERROR: no rebalance_strategy was provided, but there is also no default strategy set
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
UPDATE pg_dist_rebalance_strategy SET default_strategy=true WHERE name='by_shard_count';
CREATE OR REPLACE FUNCTION shard_cost_no_arguments()
RETURNS real AS $$ SELECT 1.0::real $$ LANGUAGE sql;
@ -2254,7 +2293,7 @@ SELECT rebalance_table_shards('rebalance_test_table', shard_transfer_mode:='bloc
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT count(*) FROM pg_dist_shard NATURAL JOIN pg_dist_shard_placement WHERE logicalrelid = 'ref_table'::regclass;
count
---------------------------------------------------------------------
@ -2325,7 +2364,7 @@ SELECT 1 from master_add_node('localhost', :worker_2_port);
-- since r2 has no replica identity we expect an error here
SELECT rebalance_table_shards();
ERROR: cannot use logical replication to transfer shards of the relation r2 since it doesn't have a REPLICA IDENTITY or PRIMARY KEY
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
DROP TABLE t1, r1, r2;
-- verify there are no distributed tables before we perform the following tests. Preceding
-- test suites should clean up their distributed tables.
@ -2356,6 +2395,7 @@ SELECT create_reference_table('r1');
(1 row)
ALTER SEQUENCE pg_dist_groupid_seq RESTART WITH 15;
SELECT 1 from master_add_node('localhost', :worker_2_port);
?column?
---------------------------------------------------------------------
@ -2380,7 +2420,7 @@ SELECT rebalance_table_shards();
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
-- verify the reference table is on all nodes after the rebalance
SELECT count(*)
FROM pg_dist_shard
@ -2480,7 +2520,7 @@ SELECT * FROM master_drain_node('localhost', :worker_2_port);
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
UPDATE pg_dist_node SET shouldhaveshards=true WHERE nodeport = :worker_2_port;
SELECT rebalance_table_shards();
rebalance_table_shards
@ -2488,7 +2528,7 @@ SELECT rebalance_table_shards();
(1 row)
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
DROP TABLE test_rebalance_with_index CASCADE;
-- Test rebalancer with disabled worker
SET citus.next_shard_id TO 433500;
@ -2519,10 +2559,10 @@ SELECT rebalance_table_shards('test_rebalance_with_disabled_worker');
(1 row)
SELECT citus_activate_node('localhost', :worker_2_port);
citus_activate_node
SELECT 1 FROM citus_activate_node('localhost', :worker_2_port);
?column?
---------------------------------------------------------------------
35
1
(1 row)
DROP TABLE test_rebalance_with_disabled_worker;
@ -2600,3 +2640,15 @@ DEBUG: Size Query: SELECT (SELECT SUM(worker_partitioned_relation_total_size(re
RESET client_min_messages;
DROP TABLE "events.Energy Added", colocated_t1, colocated_t2, colocated_t3;
RESET citus.shard_count;
DROP VIEW table_placements_per_node;
DELETE FROM pg_catalog.pg_dist_rebalance_strategy WHERE name='capacity_high_worker_2';
DELETE FROM pg_catalog.pg_dist_rebalance_strategy WHERE name='only_worker_1';
\c - - - :worker_1_port
SET citus.enable_ddl_propagation TO OFF;
REVOKE ALL ON SCHEMA public FROM testrole;
DROP USER testrole;
\c - - - :worker_2_port
SET citus.enable_ddl_propagation TO OFF;
REVOKE ALL ON SCHEMA public FROM testrole;
DROP USER testrole;
DROP TABLE test_rebalance_with_disabled_worker_433500, test_rebalance_with_disabled_worker_433501, test_rebalance_with_disabled_worker_433502, test_rebalance_with_disabled_worker_433503;

View File

@ -512,7 +512,7 @@ SELECT unnest(shard_placement_rebalance_array(
]::json[]
));
NOTICE: Ignoring move of shard xxxxx from b:5432 to a:5432, because the move only brings a small improvement relative to the shard its size
DETAIL: The balance improvement of 0.1 is lower than the improvement_threshold of 0.5
DETAIL: The balance improvement of 0.1xxxxx is lower than the improvement_threshold of 0.5
NOTICE: Ignored 1 moves, all of which are shown in notices above
HINT: If you do want these moves to happen, try changing improvement_threshold to a lower value than what it is now (0.5).
unnest

View File

@ -9,7 +9,6 @@ test: foreign_key_to_reference_shard_rebalance
test: multi_move_mx
test: shard_move_deferred_delete
test: multi_colocated_shard_rebalance
test: ignoring_orphaned_shards
test: cpu_priority
test: check_mx
test: citus_drain_node

View File

@ -28,6 +28,12 @@ setup
teardown
{
CREATE OR REPLACE PROCEDURE isolation_cleanup_orphaned_resources()
LANGUAGE C
AS 'citus', $$isolation_cleanup_orphaned_resources$$;
COMMENT ON PROCEDURE isolation_cleanup_orphaned_resources()
IS 'cleanup orphaned resources';
CALL isolation_cleanup_orphaned_resources();
DROP TABLE selected_shard;
DROP TABLE dist;
REVOKE ALL ON SCHEMA public FROM new_user;

View File

@ -4,6 +4,7 @@ setup
{
SET citus.shard_count TO 2;
SET citus.shard_replication_factor TO 1;
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 102011;
CREATE TABLE test_move_table (x int, y int);
SELECT create_distributed_table('test_move_table', 'x');
@ -14,6 +15,14 @@ teardown
{
DROP TABLE test_move_table;
DROP TABLE selected_shard_for_test_table;
CREATE OR REPLACE PROCEDURE isolation_cleanup_orphaned_resources()
LANGUAGE C
AS 'citus', $$isolation_cleanup_orphaned_resources$$;
COMMENT ON PROCEDURE isolation_cleanup_orphaned_resources()
IS 'cleanup orphaned shards';
RESET citus.enable_metadata_sync;
CALL isolation_cleanup_orphaned_resources();
}
session "s1"
@ -29,6 +38,13 @@ step "s1-move-placement"
SELECT master_move_shard_placement((SELECT * FROM selected_shard_for_test_table), 'localhost', 57637, 'localhost', 57638, 'force_logical');
}
step "s1-move-placement-back"
{
SELECT master_move_shard_placement((SELECT * FROM selected_shard_for_test_table), 'localhost', 57638, 'localhost', 57637, 'force_logical');
}
step "s1-wait" {}
session "s2"
step "s2-begin"
@ -36,6 +52,11 @@ step "s2-begin"
BEGIN;
}
step "s2-select-from-table"
{
SELECT * FROM test_move_table WHERE x=5;
}
step "s2-move-placement"
{
SELECT master_move_shard_placement((SELECT * FROM selected_shard_for_test_table), 'localhost', 57637, 'localhost', 57638, 'force_logical');
@ -68,3 +89,7 @@ permutation "s1-load-cache" "s2-begin" "s2-move-placement" "s1-move-placement" "
// the same test without the load caches
permutation "s2-begin" "s2-move-placement" "s1-move-placement" "s2-commit" "s2-print-placements"
// for some reason s1-move-placement-back is detected as being blocked,
// eventhough it can complete successfully.
permutation "s2-print-placements" "s2-begin" "s2-select-from-table" "s1-move-placement" "s1-move-placement-back"(*) "s1-wait" "s2-commit" "s2-print-placements"

View File

@ -1,5 +1,7 @@
// we use 15 as the partition key value through out the test
// so setting the corresponding shard here is useful
#include "isolation_mx_common.include.spec"
setup
{
SET citus.enable_metadata_sync TO off;
@ -23,13 +25,14 @@ setup
LANGUAGE C STRICT VOLATILE
AS 'citus', $$stop_session_level_connection_to_node$$;
CREATE OR REPLACE PROCEDURE isolation_cleanup_orphaned_shards()
CREATE OR REPLACE PROCEDURE isolation_cleanup_orphaned_resources()
LANGUAGE C
AS 'citus', $$isolation_cleanup_orphaned_shards$$;
COMMENT ON PROCEDURE isolation_cleanup_orphaned_shards()
AS 'citus', $$isolation_cleanup_orphaned_resources$$;
COMMENT ON PROCEDURE isolation_cleanup_orphaned_resources()
IS 'cleanup orphaned shards';
RESET citus.enable_metadata_sync;
CALL isolation_cleanup_orphaned_resources();
SET citus.next_shard_id to 120000;
SET citus.shard_count TO 8;
SET citus.shard_replication_factor TO 1;
@ -58,21 +61,11 @@ step "s1-move-placement"
SELECT master_move_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638);
}
step "s1-move-placement-back"
{
SET client_min_messages to NOTICE;
SHOW log_error_verbosity;
SELECT master_move_shard_placement((SELECT * FROM selected_shard), 'localhost', 57638, 'localhost', 57637);
}
step "s1-drop-marked-shards"
{
SET client_min_messages to NOTICE;
CALL isolation_cleanup_orphaned_shards();
}
step "s1-lock-pg-dist-placement" {
LOCK TABLE pg_dist_placement IN SHARE ROW EXCLUSIVE MODE;
SET client_min_messages to ERROR;
CALL isolation_cleanup_orphaned_resources();
SELECT COUNT(*) FROM pg_dist_cleanup WHERE object_type = 1 AND object_name LIKE 'public.t1_%';
}
step "s1-commit"
@ -82,10 +75,6 @@ step "s1-commit"
session "s2"
step "s2-drop-old-shards" {
SELECT run_try_drop_marked_resources();
}
step "s2-start-session-level-connection"
{
SELECT start_session_level_connection_to_node('localhost', 57637);
@ -105,14 +94,10 @@ step "s2-lock-table-on-worker"
step "s2-drop-marked-shards"
{
SET client_min_messages to DEBUG1;
CALL isolation_cleanup_orphaned_shards();
CALL isolation_cleanup_orphaned_resources();
}
permutation "s1-begin" "s1-move-placement" "s1-drop-marked-shards" "s2-drop-marked-shards" "s1-commit"
permutation "s1-begin" "s1-move-placement" "s2-drop-marked-shards" "s1-drop-marked-shards" "s1-commit"
permutation "s1-begin" "s1-move-placement" "s2-start-session-level-connection" "s2-lock-table-on-worker" "s1-drop-marked-shards" "s1-commit" "s2-stop-connection"
// make sure we give a clear error when we try to replace an orphaned shard that is still in use
permutation "s1-begin" "s1-move-placement" "s2-start-session-level-connection" "s2-lock-table-on-worker" "s1-commit" "s1-begin" "s1-move-placement-back" "s1-commit" "s2-stop-connection"
// make sure we error if we cannot get the lock on pg_dist_placement
permutation "s1-begin" "s1-lock-pg-dist-placement" "s2-drop-old-shards" "s1-commit"

View File

@ -4,6 +4,15 @@ setup
SET citus.shard_count TO 4;
SET citus.shard_replication_factor TO 1;
SELECT 1 FROM master_add_node('localhost', 57637);
CREATE OR REPLACE PROCEDURE isolation_cleanup_orphaned_resources()
LANGUAGE C
AS 'citus', $$isolation_cleanup_orphaned_resources$$;
COMMENT ON PROCEDURE isolation_cleanup_orphaned_resources()
IS 'cleanup orphaned shards';
RESET citus.enable_metadata_sync;
CALL isolation_cleanup_orphaned_resources();
SELECT master_set_node_property('localhost', 57638, 'shouldhaveshards', false);
CREATE TABLE colocated1 (test_id integer NOT NULL, data text);
SELECT create_distributed_table('colocated1', 'test_id', 'hash', 'none');

View File

@ -12,7 +12,7 @@ SELECT * FROM master_set_node_property('localhost', :master_port, 'shouldhavesha
CREATE TABLE test (x INT, y INT);
SELECT create_distributed_table('test','x');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT nodename, nodeport, COUNT(*)
FROM pg_dist_placement AS placement,
@ -23,7 +23,7 @@ SELECT nodename, nodeport, COUNT(*)
SELECT * FROM citus_set_node_property('localhost', :worker_2_port, 'shouldhaveshards', false);
SELECT * from citus_drain_node('localhost', :worker_1_port, shard_transfer_mode :='force_logical');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT nodename, nodeport, COUNT(*)
FROM pg_dist_placement AS placement,
@ -36,7 +36,7 @@ SELECT * FROM citus_set_node_property('localhost', :worker_2_port, 'shouldhavesh
SELECT * FROM rebalance_table_shards(shard_transfer_mode :='force_logical');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT nodename, nodeport, COUNT(*)
FROM pg_dist_placement AS placement,
@ -48,7 +48,7 @@ SELECT * FROM citus_set_node_property('localhost', :master_port, 'shouldhaveshar
SELECT * FROM rebalance_table_shards(shard_transfer_mode :='force_logical');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT nodename, nodeport, COUNT(*)
FROM pg_dist_placement AS placement,

View File

@ -133,7 +133,7 @@ SET citus.next_shard_id TO 8981007;
SELECT citus_move_shard_placement(8981000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, shard_transfer_mode:='force_logical');
-- END : Move one shard before we split it.
CALL pg_catalog.citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
-- BEGIN : Set node id variables
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
@ -168,7 +168,7 @@ SELECT public.wait_for_resource_cleanup();
SELECT citus_move_shard_placement(8981007, 'localhost', :worker_1_port, 'localhost', :worker_2_port, shard_transfer_mode:='block_writes');
-- END : Move a shard post split.
CALL pg_catalog.citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
-- BEGIN : Display current state.
SELECT shard.shardid, logicalrelid, shardminvalue, shardmaxvalue, nodename, nodeport
@ -316,4 +316,5 @@ SELECT run_command_on_workers($$SELECT count(*) FROM pg_subscription$$);
ALTER SYSTEM RESET citus.defer_shard_delete_interval;
SELECT pg_reload_conf();
DROP SCHEMA "citus_split_test_schema" CASCADE;
DROP ROLE test_shard_split_role;
--END : Cleanup

View File

@ -128,7 +128,7 @@ SET citus.next_shard_id TO 8981007;
SELECT citus_move_shard_placement(8981000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, shard_transfer_mode:='force_logical');
-- END : Move one shard before we split it.
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
-- BEGIN : Set node id variables
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
@ -163,7 +163,7 @@ SELECT public.wait_for_resource_cleanup();
SELECT citus_move_shard_placement(8981007, 'localhost', :worker_1_port, 'localhost', :worker_2_port, shard_transfer_mode:='block_writes');
-- END : Move a shard post split.
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
-- BEGIN : Display current state.
SELECT shard.shardid, logicalrelid, shardminvalue, shardmaxvalue, nodename, nodeport

View File

@ -73,7 +73,7 @@ SELECT master_move_shard_placement(201, 'localhost', :worker_1_port, 'localhost'
SELECT citus.mitmproxy('conn.onQuery(query="ADD CONSTRAINT").cancel(' || :pid || ')');
SELECT master_move_shard_placement(201, 'localhost', :worker_1_port, 'localhost', :worker_2_proxy_port, 'block_writes');
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
-- Verify that the shard is not moved and the number of rows are still 100k
SELECT citus.mitmproxy('conn.allow()');
@ -83,7 +83,7 @@ SELECT * FROM find_index_for_shard_201_in_workers;
-- Verify that shard can be moved after a temporary failure
SELECT master_move_shard_placement(201, 'localhost', :worker_1_port, 'localhost', :worker_2_proxy_port, 'block_writes');
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
SELECT * FROM shards_in_workers;
SELECT count(*) FROM t;
SELECT * FROM find_index_for_shard_201_in_workers;

View File

@ -103,7 +103,6 @@ SELECT master_move_shard_placement(101, 'localhost', :worker_2_proxy_port, 'loca
-- cleanup leftovers
SELECT citus.mitmproxy('conn.allow()');
SELECT public.wait_for_resource_cleanup();
CALL citus_cleanup_orphaned_shards();
-- failure on setting lock_timeout (right before dropping subscriptions & replication slots)
SELECT citus.mitmproxy('conn.onQuery(query="^SET LOCAL lock_timeout").kill()');
@ -115,7 +114,6 @@ SELECT master_move_shard_placement(101, 'localhost', :worker_2_proxy_port, 'loca
-- cleanup leftovers
SELECT citus.mitmproxy('conn.allow()');
SELECT public.wait_for_resource_cleanup();
CALL citus_cleanup_orphaned_shards();
-- cancellation on disabling subscription (right before dropping it)
SELECT citus.mitmproxy('conn.onQuery(query="^ALTER SUBSCRIPTION .* DISABLE").cancel(' || :pid || ')');
@ -128,6 +126,7 @@ SELECT public.wait_for_resource_cleanup();
-- disable maintenance daemon cleanup, to prevent the flaky test
ALTER SYSTEM SET citus.defer_shard_delete_interval TO -1;
SELECT pg_reload_conf();
SET citus.next_operation_id TO 777;
-- failure on dropping subscription
SELECT citus.mitmproxy('conn.onQuery(query="^DROP SUBSCRIPTION").killall()');
@ -136,10 +135,7 @@ SELECT master_move_shard_placement(101, 'localhost', :worker_1_port, 'localhost'
SELECT citus.mitmproxy('conn.allow()');
-- first, manually drop the subscsription object. But the record for it will remain on pg_dist_cleanup
-- we expect the drop query will succeed on only one node
SELECT COUNT(*)
FROM run_command_on_workers(
$$DROP SUBSCRIPTION citus_shard_move_subscription_10_19$$)
WHERE success AND result = 'DROP SUBSCRIPTION';
SELECT 1 FROM run_command_on_workers('DROP SUBSCRIPTION citus_shard_move_subscription_10_777');
-- reset back
ALTER SYSTEM RESET citus.defer_shard_delete_interval;
@ -204,4 +200,8 @@ SELECT master_move_shard_placement(101, 'localhost', :worker_1_port, 'localhost'
SELECT * FROM shards_in_workers;
SELECT count(*) FROM t;
-- cleanup leftovers
SELECT citus.mitmproxy('conn.allow()');
SELECT public.wait_for_resource_cleanup();
DROP SCHEMA move_shard CASCADE ;

View File

@ -45,14 +45,14 @@ SELECT master_move_shard_placement(15000009, 'localhost', :worker_1_port, 'local
SELECT count(*) FROM referencing_table2;
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
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;
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
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 TABLE reference_table_commands (id int UNIQUE);

View File

@ -1,140 +0,0 @@
CREATE SCHEMA ignoring_orphaned_shards;
SET search_path TO ignoring_orphaned_shards;
-- Use a weird shard count that we don't use in any other tests
SET citus.shard_count TO 13;
SET citus.shard_replication_factor TO 1;
SET citus.next_shard_id TO 92448000;
CREATE TABLE ref(id int PRIMARY KEY);
SELECT * FROM create_reference_table('ref');
SET citus.next_shard_id TO 92448100;
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 92448100;
CREATE TABLE dist1(id int);
SELECT * FROM create_distributed_table('dist1', 'id');
SELECT logicalrelid FROM pg_dist_partition WHERE colocationid = 92448100 ORDER BY 1;
-- Move first shard, so that the first shard now has 2 placements. One that's
-- active and one that's orphaned.
SELECT citus_move_shard_placement(92448100, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'block_writes');
SELECT shardid, shardstate, nodeport FROM pg_dist_shard_placement WHERE shardid = 92448100 ORDER BY placementid;
-- Add a new table that should get colocated with dist1 automatically, but
-- should not get a shard for the orphaned placement.
SET citus.next_shard_id TO 92448200;
CREATE TABLE dist2(id int);
SELECT * FROM create_distributed_table('dist2', 'id');
SELECT logicalrelid FROM pg_dist_partition WHERE colocationid = 92448100 ORDER BY 1;
SELECT shardid, shardstate, nodeport FROM pg_dist_shard_placement WHERE shardid = 92448200 ORDER BY placementid;
-- uncolocate it
SELECT update_distributed_table_colocation('dist2', 'none');
SELECT logicalrelid FROM pg_dist_partition WHERE colocationid = 92448100 ORDER BY 1;
-- Make sure we can add it back to the colocation, even though it has a
-- different number of shard placements for the first shard.
SELECT update_distributed_table_colocation('dist2', 'dist1');
SELECT logicalrelid FROM pg_dist_partition WHERE colocationid = 92448100 ORDER BY 1;
-- Make sure that replication count check in FOR UPDATE ignores orphaned
-- shards.
SELECT * FROM dist1 WHERE id = 1 FOR UPDATE;
-- Make sure we don't send a query to the orphaned shard
BEGIN;
SET LOCAL citus.log_remote_commands TO ON;
INSERT INTO dist1 VALUES (1);
ROLLBACK;
-- Make sure we can create a foreign key on community edition, because
-- replication factor is 1
ALTER TABLE dist1
ADD CONSTRAINT dist1_ref_fk
FOREIGN KEY (id)
REFERENCES ref(id);
SET citus.shard_replication_factor TO 2;
SET citus.next_shard_id TO 92448300;
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 92448300;
CREATE TABLE rep1(id int);
SELECT * FROM create_distributed_table('rep1', 'id');
-- Add the coordinator, so we can have a replicated shard
SELECT 1 FROM citus_add_node('localhost', :master_port, 0);
SELECT 1 FROM citus_set_node_property('localhost', :master_port, 'shouldhaveshards', true);
SELECT logicalrelid FROM pg_dist_partition WHERE colocationid = 92448300 ORDER BY 1;
SELECT citus_move_shard_placement(92448300, 'localhost', :worker_1_port, 'localhost', :master_port, 'block_writes');
SELECT shardid, shardstate, nodeport FROM pg_dist_shard_placement WHERE shardid = 92448300 ORDER BY placementid;
-- Add a new table that should get colocated with rep1 automatically, but
-- should not get a shard for the orphaned placement.
SET citus.next_shard_id TO 92448400;
CREATE TABLE rep2(id int);
SELECT * FROM create_distributed_table('rep2', 'id');
SELECT logicalrelid FROM pg_dist_partition WHERE colocationid = 92448300 ORDER BY 1;
SELECT shardid, shardstate, nodeport FROM pg_dist_shard_placement WHERE shardid = 92448400 ORDER BY placementid;
-- uncolocate it
SELECT update_distributed_table_colocation('rep2', 'none');
SELECT logicalrelid FROM pg_dist_partition WHERE colocationid = 92448300 ORDER BY 1;
-- Make sure we can add it back to the colocation, even though it has a
-- different number of shard placements for the first shard.
SELECT update_distributed_table_colocation('rep2', 'rep1');
SELECT logicalrelid FROM pg_dist_partition WHERE colocationid = 92448300 ORDER BY 1;
-- Make sure we don't send a query to the orphaned shard
BEGIN;
SET LOCAL citus.log_remote_commands TO ON;
SET LOCAL citus.log_local_commands TO ON;
INSERT INTO rep1 VALUES (1);
ROLLBACK;
-- Cause the orphaned shard to be local
SELECT 1 FROM citus_drain_node('localhost', :master_port, 'block_writes');
SELECT shardid, shardstate, nodeport FROM pg_dist_shard_placement WHERE shardid = 92448300 ORDER BY placementid;
-- Make sure we don't send a query to the orphaned shard if it's local
BEGIN;
SET LOCAL citus.log_remote_commands TO ON;
SET LOCAL citus.log_local_commands TO ON;
INSERT INTO rep1 VALUES (1);
ROLLBACK;
SET citus.shard_replication_factor TO 1;
SET citus.next_shard_id TO 92448500;
CREATE TABLE range1(id int);
SELECT create_distributed_table('range1', 'id', 'range');
CALL public.create_range_partitioned_shards('range1', '{0,3}','{2,5}');
-- Move shard placement and clean it up
SELECT citus_move_shard_placement(92448500, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'block_writes');
CALL citus_cleanup_orphaned_shards();
SELECT shardid, shardstate, nodeport FROM pg_dist_shard_placement WHERE shardid = 92448300 ORDER BY placementid;
SET citus.next_shard_id TO 92448600;
CREATE TABLE range2(id int);
SELECT create_distributed_table('range2', 'id', 'range');
CALL public.create_range_partitioned_shards('range2', '{0,3}','{2,5}');
-- Mark tables co-located
UPDATE pg_dist_partition SET colocationid = 30001
WHERE logicalrelid = 'range1'::regclass OR logicalrelid = 'range2'::regclass;
-- Move shard placement and DON'T clean it up, now range1 and range2 are
-- colocated, but only range2 has an orphaned shard.
SELECT citus_move_shard_placement(92448600, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'block_writes');
SELECT shardid, shardstate, nodeport FROM pg_dist_shard_placement WHERE shardid = 92448600 ORDER BY placementid;
-- Make sure co-located join works
SELECT * FROM range1 JOIN range2 ON range1.id = range2.id;
-- Make sure we can create a foreign key on community edition, because
-- replication factor is 1
ALTER TABLE range1
ADD CONSTRAINT range1_ref_fk
FOREIGN KEY (id)
REFERENCES ref(id);
SET client_min_messages TO WARNING;
DROP SCHEMA ignoring_orphaned_shards CASCADE;

View File

@ -179,24 +179,10 @@ SELECT shardid, shardstate, nodename, nodeport FROM pg_dist_shard_placement WHER
SELECT master_remove_node('localhost', :worker_2_port);
SELECT master_get_active_worker_nodes();
-- mark all placements in the candidate node as to be deleted
UPDATE pg_dist_placement SET shardstate=4 WHERE groupid=:worker_2_group;
SELECT run_command_on_workers('UPDATE pg_dist_placement SET shardstate=4 WHERE groupid=' || :'worker_2_group');
SELECT shardid, shardstate, nodename, nodeport FROM pg_dist_shard_placement WHERE nodeport=:worker_2_port;
CREATE TABLE cluster_management_test_colocated (col_1 text, col_2 int);
-- Check that we warn the user about colocated shards that will not get created for shards that do not have active placements
SELECT create_distributed_table('cluster_management_test_colocated', 'col_1', 'hash', colocate_with => 'cluster_management_test');
-- Check that colocated shards don't get created for shards that are to be deleted
SELECT logicalrelid, shardid, shardstate, nodename, nodeport FROM pg_dist_shard_placement NATURAL JOIN pg_dist_shard ORDER BY shardstate, shardid;
-- clean-up
SELECT 1 FROM master_add_node('localhost', :worker_2_port);
UPDATE pg_dist_placement SET shardstate=1 WHERE groupid=:worker_2_group;
SELECT run_command_on_workers('UPDATE pg_dist_placement SET shardstate=1 WHERE groupid=' || :'worker_2_group');
SET client_min_messages TO ERROR;
DROP TABLE cluster_management_test_colocated;
RESET client_min_messages;
-- when there is no primary we should get a pretty error
UPDATE pg_dist_node SET noderole = 'secondary' WHERE nodeport=:worker_2_port;

View File

@ -46,6 +46,9 @@ SELECT citus_copy_shard_placement(13000000, 'localhost', :worker_1_port, 'localh
-- copy colocated shards
SELECT citus_copy_shard_placement(13000000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'force_logical');
-- error out if trying to move a shard that already has a placement on the target
SELECT citus_move_shard_placement(13000000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'force_logical');
-- status after shard copy
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
FROM
@ -286,7 +289,7 @@ INSERT INTO serial_move_test (key) VALUES (15) RETURNING *;
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, 'force_logical');
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
-- 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;
@ -311,7 +314,7 @@ DROP TABLE logical_failure_test_13000038;
-- should fail since the command wouldn't be able to connect to the worker_1
\c - - - :master_port
SELECT master_move_shard_placement(13000038, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'force_logical');
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
DROP TABLE logical_failure_test;
@ -323,12 +326,12 @@ SELECT create_distributed_table('test_with_pkey', 'key', colocate_with => 'none'
-- should succeed since there is a replica identity defined
SELECT master_move_shard_placement(13000042, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
-- should succeed since we still have a replica identity
ALTER TABLE test_with_pkey REPLICA IDENTITY FULL;
SELECT master_move_shard_placement(13000042, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'auto');
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
-- make sure we have the replica identity after the move
SELECT result FROM run_command_on_placements( 'test_with_pkey', 'SELECT relreplident FROM pg_class WHERE relname = ''%s''') WHERE shardid = 13000042;
@ -336,7 +339,7 @@ SELECT result FROM run_command_on_placements( 'test_with_pkey', 'SELECT relrepli
-- this time should fail since we don't have replica identity any more
ALTER TABLE test_with_pkey REPLICA IDENTITY NOTHING;
SELECT master_move_shard_placement(13000042, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'auto');
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
-- make sure we have the replica identity after the move
SELECT result FROM run_command_on_placements( 'test_with_pkey', 'SELECT relreplident FROM pg_class WHERE relname = ''%s''') WHERE shardid = 13000042;
@ -344,7 +347,7 @@ SELECT result FROM run_command_on_placements( 'test_with_pkey', 'SELECT relrepli
-- should succeed since we still have a replica identity
ALTER TABLE test_with_pkey REPLICA IDENTITY USING INDEX test_with_pkey_pkey;
SELECT master_move_shard_placement(13000042, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
-- make sure we have the replica identity after the move
SELECT result FROM run_command_on_placements( 'test_with_pkey', 'SELECT relreplident FROM pg_class WHERE relname = ''%s''') WHERE shardid = 13000042;
@ -353,14 +356,14 @@ SELECT result FROM run_command_on_placements( 'test_with_pkey', 'SELECT relrepli
CREATE UNIQUE INDEX req_rep_idx ON test_with_pkey(key, value);
ALTER TABLE test_with_pkey REPLICA IDENTITY USING INDEX req_rep_idx;
SELECT master_move_shard_placement(13000042, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'auto');
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
-- make sure we have the replica identity after the move
SELECT result FROM run_command_on_placements( 'test_with_pkey', 'SELECT relreplident FROM pg_class WHERE relname = ''%s''') WHERE shardid = 13000042;
ALTER TABLE test_with_pkey REPLICA IDENTITY NOTHING;
SELECT master_move_shard_placement(13000042, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'force_logical');
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
-- make sure we have the replica identity after the move
SELECT result FROM run_command_on_placements( 'test_with_pkey', 'SELECT relreplident FROM pg_class WHERE relname = ''%s''') WHERE shardid = 13000042;
@ -370,9 +373,9 @@ ALTER TABLE test_with_pkey REPLICA IDENTITY NOTHING;
SET client_min_messages TO DEBUG1;
SELECT master_move_shard_placement(13000042, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'block_writes');
CALL citus_cleanup_orphaned_shards();
SET client_min_messages TO DEFAULT;
SELECT public.wait_for_resource_cleanup();
-- we don't support multiple shard moves in a single transaction
SELECT
@ -380,7 +383,7 @@ SELECT
FROM
pg_dist_shard_placement where nodeport = :worker_1_port AND
shardid IN (SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_with_pkey'::regclass);
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
-- similar test with explicit transaction block
BEGIN;
@ -388,7 +391,7 @@ BEGIN;
SELECT master_move_shard_placement(13000042, 'localhost', :worker_1_port, 'localhost', :worker_2_port, shard_transfer_mode:='force_logical');
SELECT master_move_shard_placement(13000044, 'localhost', :worker_1_port, 'localhost', :worker_2_port, shard_transfer_mode:='force_logical');
COMMIT;
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
-- we do support the same with block writes
SELECT
@ -396,17 +399,17 @@ SELECT
FROM
pg_dist_shard_placement where nodeport = :worker_1_port AND
shardid IN (SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_with_pkey'::regclass);
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
-- we should be able to move shard placements after COMMIT/ABORT
BEGIN;
SELECT master_move_shard_placement(13000043, 'localhost', :worker_2_port, 'localhost', :worker_1_port, shard_transfer_mode:='force_logical');
COMMIT;
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
SELECT master_move_shard_placement(13000045, 'localhost', :worker_2_port, 'localhost', :worker_1_port, shard_transfer_mode:='force_logical');
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
BEGIN;
@ -414,7 +417,7 @@ BEGIN;
ABORT;
SELECT master_move_shard_placement(13000045, 'localhost', :worker_1_port, 'localhost', :worker_2_port, shard_transfer_mode:='force_logical');
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
-- we should be able to move shard placements of partitioend tables
CREATE SCHEMA move_partitions;
@ -442,14 +445,14 @@ FROM pg_dist_shard JOIN pg_dist_shard_placement USING (shardid)
WHERE logicalrelid = 'move_partitions.events'::regclass AND nodeport = :worker_2_port
AND shardstate != 4
ORDER BY shardid LIMIT 1;
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
-- force logical replication
SELECT master_move_shard_placement(shardid, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'force_logical')
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;
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
SELECT count(*) FROM move_partitions.events;
@ -461,7 +464,7 @@ SELECT master_move_shard_placement(shardid, 'localhost', :worker_2_port, 'localh
FROM pg_dist_shard JOIN pg_dist_shard_placement USING (shardid)
WHERE logicalrelid = 'move_partitions.events'::regclass AND nodeport = :worker_2_port AND shardstate != 4
ORDER BY shardid LIMIT 1;
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
SELECT count(*) FROM move_partitions.events;
@ -470,7 +473,7 @@ SELECT master_move_shard_placement(shardid, 'localhost', :worker_2_port, 'localh
FROM pg_dist_shard JOIN pg_dist_shard_placement USING (shardid)
WHERE logicalrelid = 'move_partitions.events'::regclass AND nodeport = :worker_2_port AND shardstate != 4
ORDER BY shardid LIMIT 1;
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
SELECT count(*) FROM move_partitions.events;
@ -489,3 +492,4 @@ DROP TABLE table1_group1;
DROP TABLE table5_groupX;
DROP TABLE table6_append;
DROP TABLE serial_move_test;
DROP SCHEMA move_partitions CASCADE;

View File

@ -525,12 +525,42 @@ SELECT * FROM multi_extension.print_extension_changes();
-- Test downgrade to 11.1-1 from 11.2-1
ALTER EXTENSION citus UPDATE TO '11.2-1';
-- create a table with orphaned shards to see if orphaned shards will be dropped
-- and cleanup records will be created for them
SET citus.shard_replication_factor to 1;
CREATE TABLE table_with_orphaned_shards (a int);
SELECT create_distributed_table('table_with_orphaned_shards', 'a');
-- show there are 4 placements
SELECT * FROM pg_dist_placement ORDER BY shardid;
-- mark two of them as orphaned
UPDATE pg_dist_placement SET shardstate = 4 WHERE shardid % 2 = 1;
ALTER EXTENSION citus UPDATE TO '11.1-1';
-- show placements and cleanup records
SELECT * FROM pg_dist_placement ORDER BY shardid;
SELECT * FROM pg_dist_cleanup;
-- Should be empty result since upgrade+downgrade should be a no-op
SELECT * FROM multi_extension.print_extension_changes();
-- Snapshot of state at 11.2-1
ALTER EXTENSION citus UPDATE TO '11.2-1';
-- verify that the placements are deleted and cleanup records are created
SELECT * FROM pg_dist_placement ORDER BY shardid;
SELECT * FROM pg_dist_cleanup;
-- error out as cleanup records remain
ALTER EXTENSION citus UPDATE TO '11.0-4';
-- cleanup
SET client_min_messages TO ERROR;
CALL citus_cleanup_orphaned_resources();
DROP TABLE table_with_orphaned_shards;
RESET client_min_messages;
SELECT * FROM multi_extension.print_extension_changes();
DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff;

View File

@ -169,7 +169,7 @@ $$ LANGUAGE plpgsql;
-- Introduce a function that waits until all cleanup records are deleted, for testing purposes
CREATE OR REPLACE FUNCTION wait_for_resource_cleanup() RETURNS void
SET client_min_messages TO WARNING
SET client_min_messages TO ERROR
AS $$
DECLARE
record_count integer;

View File

@ -101,7 +101,7 @@ SELECT attname || ' ' || attcompression::text FROM pg_attribute WHERE attrelid::
SELECT shardid INTO moving_shard FROM citus_shards WHERE table_name='col_compression'::regclass AND nodeport=:worker_1_port LIMIT 1;
SELECT citus_move_shard_placement((SELECT * FROM moving_shard), :'public_worker_1_host', :worker_1_port, :'public_worker_2_host', :worker_2_port, shard_transfer_mode := 'block_writes');
SELECT rebalance_table_shards('col_compression', rebalance_strategy := 'by_shard_count', shard_transfer_mode := 'block_writes');
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
SELECT result AS column_compression FROM run_command_on_workers($$SELECT ARRAY(
SELECT attname || ' ' || attcompression::text FROM pg_attribute WHERE attrelid::regclass::text LIKE 'pg14.col\_compression%' AND attnum > 0 ORDER BY 1
)$$);

View File

@ -150,7 +150,7 @@ indrelid IN
\c - - - :master_port
-- make sure that constrainst are moved sanely with logical replication
SELECT citus_move_shard_placement(8970000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, shard_transfer_mode:='force_logical');
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
\c - postgres - :worker_2_port
@ -258,8 +258,19 @@ SET search_path TO public;
SELECT citus_move_shard_placement(8970000, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'force_logical');
SELECT citus_move_shard_placement(8970000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'block_writes');
SELECT public.wait_for_resource_cleanup();
\c - postgres - :master_port
-- create a fake cleanup record
INSERT INTO pg_dist_cleanup
SELECT nextval('pg_dist_cleanup_recordid_seq'), 0, 1, shard_name(logicalrelid, shardid) AS object_name, -13 AS node_group_id, 0
FROM pg_dist_shard WHERE shardid = 8970000;
-- make sure we error out if there's a cleanup record for the shard to be moved
SELECT citus_move_shard_placement(8970000, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'force_logical');
-- delete the fake record
DELETE FROM pg_dist_cleanup WHERE node_group_id = -13;
-- stop and re-sync the metadata to make sure all works fine
SELECT stop_metadata_sync_to_node('localhost', :worker_1_port);
SELECT stop_metadata_sync_to_node('localhost', :worker_2_port);
@ -267,3 +278,4 @@ SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
SELECT start_metadata_sync_to_node('localhost', :worker_2_port);
DROP SCHEMA "shard Move Fkeys Indexes" CASCADE;
DROP ROLE mx_rebalancer_role_ent;

View File

@ -144,7 +144,7 @@ indrelid IN
\c - - - :master_port
-- make sure that constrainst are moved sanely with logical replication
SELECT citus_move_shard_placement(8970000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, shard_transfer_mode:='block_writes');
CALL citus_cleanup_orphaned_shards();
SELECT public.wait_for_resource_cleanup();
\c - postgres - :worker_2_port
@ -199,3 +199,4 @@ ALTER TABLE sensors_2020_01_01 DROP CONSTRAINT fkey_from_child_to_child;
-- cleanup
\c - postgres - :master_port
DROP SCHEMA "blocking shard Move Fkeys Indexes" CASCADE;
DROP ROLE mx_rebalancer_blocking_role_ent;

View File

@ -32,12 +32,15 @@ $cmd$);
-- Make sure this cannot be run in a transaction
BEGIN;
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
COMMIT;
-- execute delayed removal
-- citus_cleanup_orphaned_shards is deprecated
CALL citus_cleanup_orphaned_shards();
-- execute delayed removal
CALL citus_cleanup_orphaned_resources();
-- 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';
@ -56,6 +59,7 @@ SELECT pg_reload_conf();
-- Sleep 1 second to give Valgrind enough time to clear transactions
SELECT pg_sleep(1);
SELECT public.wait_for_resource_cleanup();
-- we expect the shard to be on only the first worker
SELECT run_command_on_workers($cmd$
@ -85,7 +89,7 @@ ROLLBACK;
-- see the cleanup record for the shard on the target node
-- https://github.com/citusdata/citus/issues/6580
select object_name, object_type from pg_dist_cleanup;
select object_name, object_type from pg_dist_cleanup where object_type = 1;
-- see the shard on both workers
SELECT run_command_on_workers($cmd$

View File

@ -17,9 +17,9 @@ SELECT 1 FROM master_add_node('localhost', :master_port, groupId=>0);
-- should just be noops even if we add the coordinator to the pg_dist_node
SELECT rebalance_table_shards('dist_table_test');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT rebalance_table_shards();
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
-- test that calling rebalance_table_shards without specifying relation
@ -30,7 +30,7 @@ SELECT citus_add_local_table_to_metadata('citus_local_table');
INSERT INTO citus_local_table VALUES (1, 2);
SELECT rebalance_table_shards();
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
-- Check that rebalance_table_shards and get_rebalance_table_shards_plan fail
-- for any type of table, but distributed tables.
@ -57,14 +57,14 @@ SELECT pg_reload_conf();
SELECT pg_sleep(.1); -- wait to make sure the config has changed before running the GUC
SELECT master_drain_node('localhost', :master_port);
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
ALTER SYSTEM RESET citus.local_hostname;
SELECT pg_reload_conf();
SELECT pg_sleep(.1); -- wait to make sure the config has changed before running the GUC
SELECT master_drain_node('localhost', :master_port);
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
-- show that citus local table shard is still on the coordinator
SELECT tablename FROM pg_catalog.pg_tables where tablename like 'citus_local_table_%';
@ -133,14 +133,14 @@ AS 'citus'
LANGUAGE C STRICT VOLATILE;
CREATE FUNCTION shard_placement_replication_array(worker_node_list json[],
shard_placement_list json[],
shard_replication_factor int)
CREATE OR REPLACE FUNCTION shard_placement_replication_array(worker_node_list json[],
shard_placement_list json[],
shard_replication_factor int)
RETURNS json[]
AS 'citus'
LANGUAGE C STRICT VOLATILE;
CREATE FUNCTION worker_node_responsive(worker_node_name text, worker_node_port int)
CREATE OR REPLACE FUNCTION worker_node_responsive(worker_node_name text, worker_node_port int)
RETURNS boolean
AS 'citus'
LANGUAGE C STRICT VOLATILE;
@ -369,7 +369,7 @@ ORDER BY logicalrelid::regclass, nodename, nodeport;
-- Create six shards with replication factor 1 and move them to the same
-- node to create an unbalanced cluster.
CREATE PROCEDURE create_unbalanced_shards(rel text)
CREATE OR REPLACE PROCEDURE create_unbalanced_shards(rel text)
LANGUAGE SQL
AS $$
SET citus.shard_replication_factor TO 1;
@ -382,7 +382,7 @@ AS $$
pg_dist_shard_placement src USING (shardid),
(SELECT nodename, nodeport FROM pg_dist_shard_placement ORDER BY nodeport DESC LIMIT 1) dst
WHERE src.nodeport < dst.nodeport AND s.logicalrelid = rel::regclass;
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
$$;
CALL create_unbalanced_shards('rebalance_test_table');
@ -411,7 +411,7 @@ FROM (
FROM pg_dist_shard
WHERE logicalrelid = 'rebalance_test_table'::regclass
) T;
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
ALTER SYSTEM RESET citus.local_hostname;
SELECT pg_reload_conf();
@ -428,7 +428,7 @@ FROM (
FROM pg_dist_shard
WHERE logicalrelid = 'rebalance_test_table'::regclass
) T;
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM table_placements_per_node;
@ -461,29 +461,29 @@ SET ROLE testrole;
SELECT rebalance_table_shards('rebalance_test_table',
shard_transfer_mode:='block_writes');
RESET ROLE;
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
-- Confirm no moves took place at all during these errors
SELECT * FROM table_placements_per_node;
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT rebalance_table_shards('rebalance_test_table',
threshold := 0, max_shard_moves := 1,
shard_transfer_mode:='block_writes');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM table_placements_per_node;
-- Check that threshold=1 doesn't move any shards
SELECT rebalance_table_shards('rebalance_test_table', threshold := 1, shard_transfer_mode:='block_writes');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM table_placements_per_node;
-- Move the remaining shards using threshold=0
SELECT rebalance_table_shards('rebalance_test_table', threshold := 0, shard_transfer_mode:='block_writes');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM table_placements_per_node;
@ -491,7 +491,7 @@ SELECT * FROM table_placements_per_node;
-- any effects.
SELECT rebalance_table_shards('rebalance_test_table', threshold := 0, shard_transfer_mode:='block_writes');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM table_placements_per_node;
@ -591,7 +591,7 @@ SELECT COUNT(*) FROM imbalanced_table;
-- Test rebalance operation
SELECT rebalance_table_shards('imbalanced_table', threshold:=0, shard_transfer_mode:='block_writes');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
-- Confirm rebalance
-- Shard counts in each node after rebalance
@ -605,6 +605,7 @@ DROP TABLE test_schema_support.imbalanced_table_local;
SET citus.shard_replication_factor TO 1;
SET citus.shard_count = 4;
ALTER SEQUENCE pg_catalog.pg_dist_placement_placementid_seq RESTART 136;
CREATE TABLE colocated_rebalance_test(id integer);
CREATE TABLE colocated_rebalance_test2(id integer);
@ -618,7 +619,7 @@ SELECT create_distributed_table('colocated_rebalance_test', 'id');
SELECT master_move_shard_placement(shardid, 'localhost', :worker_2_port, 'localhost', 10000, 'block_writes')
FROM pg_dist_shard_placement
WHERE nodeport = :worker_2_port;
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
-- Try to move shards to a node where shards are not allowed
SELECT * from master_set_node_property('localhost', :worker_1_port, 'shouldhaveshards', false);
@ -645,7 +646,7 @@ UPDATE pg_dist_node SET noderole = 'primary' WHERE nodeport = :worker_1_port;
SELECT master_move_shard_placement(shardid, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'block_writes')
FROM pg_dist_shard_placement
WHERE nodeport = :worker_2_port;
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT create_distributed_table('colocated_rebalance_test2', 'id');
@ -656,7 +657,7 @@ SELECT * FROM public.table_placements_per_node;
SELECT * FROM get_rebalance_table_shards_plan('colocated_rebalance_test', threshold := 0, drain_only := true);
-- Running with drain_only shouldn't do anything
SELECT * FROM rebalance_table_shards('colocated_rebalance_test', threshold := 0, shard_transfer_mode := 'block_writes', drain_only := true);
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
-- Confirm that nothing changed
SELECT * FROM public.table_placements_per_node;
@ -669,14 +670,14 @@ SELECT * FROM get_rebalance_table_shards_plan('colocated_rebalance_test', rebala
SELECT * FROM get_rebalance_progress();
-- Actually do the rebalance
SELECT * FROM rebalance_table_shards('colocated_rebalance_test', threshold := 0, shard_transfer_mode := 'block_writes');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
-- Check that we can call this function without a crash
SELECT * FROM get_rebalance_progress();
-- Confirm that the shards are now there
SELECT * FROM public.table_placements_per_node;
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
select * from pg_dist_placement ORDER BY placementid;
@ -688,7 +689,7 @@ WHERE nodeport = :worker_2_port AND logicalrelid = 'colocated_rebalance_test'::r
-- Confirm that the shards are now all on worker1
SELECT * FROM public.table_placements_per_node;
-- Explicitly don't run citus_cleanup_orphaned_shards, rebalance_table_shards
-- Explicitly don't run citus_cleanup_orphaned_resources, rebalance_table_shards
-- should do that for automatically.
SELECT * FROM rebalance_table_shards('colocated_rebalance_test', threshold := 0, shard_transfer_mode := 'block_writes');
@ -706,50 +707,54 @@ SELECT * from master_set_node_property('localhost', :worker_2_port, 'shouldhaves
SELECT * FROM get_rebalance_table_shards_plan('colocated_rebalance_test', threshold := 0);
SELECT * FROM rebalance_table_shards('colocated_rebalance_test', threshold := 0, shard_transfer_mode := 'block_writes');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
SELECT * FROM get_rebalance_table_shards_plan('non_colocated_rebalance_test', threshold := 0);
SELECT * FROM rebalance_table_shards('non_colocated_rebalance_test', threshold := 0, shard_transfer_mode := 'block_writes');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
-- Put shards back
SELECT * from master_set_node_property('localhost', :worker_2_port, 'shouldhaveshards', true);
SELECT * FROM rebalance_table_shards('colocated_rebalance_test', threshold := 0, shard_transfer_mode := 'block_writes');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
SELECT * FROM rebalance_table_shards('non_colocated_rebalance_test', threshold := 0, shard_transfer_mode := 'block_writes');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
-- testing behaviour when setting shouldhaveshards to false and rebalancing all
-- colocation groups with drain_only=true
SELECT * from master_set_node_property('localhost', :worker_2_port, 'shouldhaveshards', false);
SELECT * FROM get_rebalance_table_shards_plan(threshold := 0, drain_only := true);
-- we actually shouldn't need the ORDER BY clause as the output will be in execution order
-- but this one involves different colocation groups and which colocation group is first moved is not consistent
SELECT * FROM get_rebalance_table_shards_plan(threshold := 0, drain_only := true) ORDER BY shardid;
SELECT * FROM rebalance_table_shards(threshold := 0, shard_transfer_mode := 'block_writes', drain_only := true);
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
-- Put shards back
SELECT * from master_set_node_property('localhost', :worker_2_port, 'shouldhaveshards', true);
SELECT * FROM rebalance_table_shards(threshold := 0, shard_transfer_mode := 'block_writes');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
-- testing behaviour when setting shouldhaveshards to false and rebalancing all
-- colocation groups with drain_only=false
SELECT * from master_set_node_property('localhost', :worker_2_port, 'shouldhaveshards', false);
SELECT * FROM get_rebalance_table_shards_plan(threshold := 0);
-- we actually shouldn't need the ORDER BY clause as the output will be in execution order
-- but this one involves different colocation groups and which colocation group is first moved is not consistent
SELECT * FROM get_rebalance_table_shards_plan(threshold := 0) ORDER BY shardid;
SELECT * FROM rebalance_table_shards(threshold := 0, shard_transfer_mode := 'block_writes');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
-- Put shards back
SELECT * from master_set_node_property('localhost', :worker_2_port, 'shouldhaveshards', true);
SELECT * FROM rebalance_table_shards(threshold := 0, shard_transfer_mode := 'block_writes');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
-- Make it a data node again
@ -757,18 +762,19 @@ SELECT * from master_set_node_property('localhost', :worker_2_port, 'shouldhaves
-- testing behaviour of master_drain_node
SELECT * from master_drain_node('localhost', :worker_2_port, shard_transfer_mode := 'block_writes');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
select shouldhaveshards from pg_dist_node where nodeport = :worker_2_port;
SELECT * FROM public.table_placements_per_node;
-- Put shards back
SELECT * from master_set_node_property('localhost', :worker_2_port, 'shouldhaveshards', true);
SELECT * FROM rebalance_table_shards(threshold := 0, shard_transfer_mode := 'block_writes');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
-- Drop some tables for clear consistent error
DROP TABLE test_schema_support.non_colocated_rebalance_test;
DROP TABLE test_schema_support.colocated_rebalance_test2;
-- testing behaviour when a transfer fails when using master_drain_node
@ -806,7 +812,13 @@ VACUUM FULL tab;
ANALYZE tab;
\c - - - :worker_1_port
SELECT table_schema, table_name, row_estimate, total_bytes
SELECT table_schema, table_name, row_estimate,
CASE
WHEN total_bytes BETWEEN 1900000 AND 2300000 THEN 2179072
WHEN total_bytes BETWEEN 900000 AND 1200000 THEN 1089536
WHEN total_bytes BETWEEN 300000 AND 440000 THEN 368640
ELSE total_bytes
END
FROM (
SELECT *, total_bytes-index_bytes-COALESCE(toast_bytes,0) AS table_bytes FROM (
SELECT c.oid,nspname AS table_schema, relname AS TABLE_NAME
@ -822,7 +834,13 @@ SELECT table_schema, table_name, row_estimate, total_bytes
WHERE table_schema = 'public'
) a ORDER BY table_name;
\c - - - :worker_2_port
SELECT table_schema, table_name, row_estimate, total_bytes
SELECT table_schema, table_name, row_estimate,
CASE
WHEN total_bytes BETWEEN 1900000 AND 2300000 THEN 2179072
WHEN total_bytes BETWEEN 900000 AND 1200000 THEN 1089536
WHEN total_bytes BETWEEN 300000 AND 440000 THEN 368640
ELSE total_bytes
END
FROM (
SELECT *, total_bytes-index_bytes-COALESCE(toast_bytes,0) AS table_bytes FROM (
SELECT c.oid,nspname AS table_schema, relname AS TABLE_NAME
@ -845,15 +863,15 @@ SELECT * FROM get_rebalance_table_shards_plan('tab', rebalance_strategy := 'by_d
SELECT * FROM get_rebalance_table_shards_plan('tab', rebalance_strategy := 'by_disk_size', threshold := 0);
SELECT * FROM rebalance_table_shards('tab', shard_transfer_mode:='block_writes');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
SELECT * FROM rebalance_table_shards('tab', rebalance_strategy := 'by_disk_size', shard_transfer_mode:='block_writes');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
SELECT * FROM rebalance_table_shards('tab', rebalance_strategy := 'by_disk_size', shard_transfer_mode:='block_writes', threshold := 0);
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
-- Check that sizes of colocated tables are added together for rebalances
@ -869,7 +887,13 @@ VACUUM FULL tab, tab2;
ANALYZE tab, tab2;
\c - - - :worker_1_port
SELECT table_schema, table_name, row_estimate, total_bytes
SELECT table_schema, table_name, row_estimate,
CASE
WHEN total_bytes BETWEEN 1900000 AND 2300000 THEN 2179072
WHEN total_bytes BETWEEN 900000 AND 1200000 THEN 1089536
WHEN total_bytes BETWEEN 300000 AND 440000 THEN 368640
ELSE total_bytes
END
FROM (
SELECT *, total_bytes-index_bytes-COALESCE(toast_bytes,0) AS table_bytes FROM (
SELECT c.oid,nspname AS table_schema, relname AS TABLE_NAME
@ -885,7 +909,13 @@ SELECT table_schema, table_name, row_estimate, total_bytes
WHERE table_schema = 'public'
) a ORDER BY table_name;
\c - - - :worker_2_port
SELECT table_schema, table_name, row_estimate, total_bytes
SELECT table_schema, table_name, row_estimate,
CASE
WHEN total_bytes BETWEEN 1900000 AND 2300000 THEN 2179072
WHEN total_bytes BETWEEN 900000 AND 1200000 THEN 1089536
WHEN total_bytes BETWEEN 300000 AND 440000 THEN 368640
ELSE total_bytes
END
FROM (
SELECT *, total_bytes-index_bytes-COALESCE(toast_bytes,0) AS table_bytes FROM (
SELECT c.oid,nspname AS table_schema, relname AS TABLE_NAME
@ -902,17 +932,24 @@ WHERE table_schema = 'public'
) a ORDER BY table_name;
\c - - - :master_port
SELECT * FROM get_rebalance_table_shards_plan('tab', rebalance_strategy := 'by_disk_size');
-- disk sizes can be slightly different, so ORDER BY shardid gives us a consistent output
SELECT * FROM get_rebalance_table_shards_plan('tab', rebalance_strategy := 'by_disk_size') ORDER BY shardid;
-- supports improvement_threshold
SELECT * FROM get_rebalance_table_shards_plan('tab', rebalance_strategy := 'by_disk_size', improvement_threshold := 0);
SELECT * FROM rebalance_table_shards('tab', rebalance_strategy := 'by_disk_size', shard_transfer_mode:='block_writes');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
VACUUM FULL tab, tab2;
ANALYZE tab, tab2;
\c - - - :worker_1_port
SELECT table_schema, table_name, row_estimate, total_bytes
SELECT table_schema, table_name, row_estimate,
CASE
WHEN total_bytes BETWEEN 1900000 AND 2300000 THEN 2179072
WHEN total_bytes BETWEEN 900000 AND 1200000 THEN 1089536
WHEN total_bytes BETWEEN 300000 AND 440000 THEN 368640
ELSE total_bytes
END
FROM (
SELECT *, total_bytes-index_bytes-COALESCE(toast_bytes,0) AS table_bytes FROM (
SELECT c.oid,nspname AS table_schema, relname AS TABLE_NAME
@ -928,7 +965,13 @@ SELECT table_schema, table_name, row_estimate, total_bytes
WHERE table_schema = 'public'
) a ORDER BY table_name;
\c - - - :worker_2_port
SELECT table_schema, table_name, row_estimate, total_bytes
SELECT table_schema, table_name, row_estimate,
CASE
WHEN total_bytes BETWEEN 1900000 AND 2300000 THEN 2179072
WHEN total_bytes BETWEEN 900000 AND 1200000 THEN 1089536
WHEN total_bytes BETWEEN 300000 AND 440000 THEN 368640
ELSE total_bytes
END
FROM (
SELECT *, total_bytes-index_bytes-COALESCE(toast_bytes,0) AS table_bytes FROM (
SELECT c.oid,nspname AS table_schema, relname AS TABLE_NAME
@ -966,16 +1009,16 @@ SELECT citus_add_rebalance_strategy(
SELECT * FROM get_rebalance_table_shards_plan('tab', rebalance_strategy := 'capacity_high_worker_2');
SELECT * FROM rebalance_table_shards('tab', rebalance_strategy := 'capacity_high_worker_2', shard_transfer_mode:='block_writes');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
SELECT citus_set_default_rebalance_strategy('capacity_high_worker_2');
SELECT * FROM get_rebalance_table_shards_plan('tab');
SELECT * FROM rebalance_table_shards('tab', shard_transfer_mode:='block_writes');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
CREATE FUNCTION only_worker_1(shardid bigint, nodeidarg int)
CREATE OR REPLACE FUNCTION only_worker_1(shardid bigint, nodeidarg int)
RETURNS boolean AS $$
SELECT
(CASE WHEN nodeport = 57637 THEN TRUE ELSE FALSE END)
@ -993,7 +1036,7 @@ SELECT citus_add_rebalance_strategy(
SELECT citus_set_default_rebalance_strategy('only_worker_1');
SELECT * FROM get_rebalance_table_shards_plan('tab');
SELECT * FROM rebalance_table_shards('tab', shard_transfer_mode:='block_writes');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM public.table_placements_per_node;
SELECT citus_set_default_rebalance_strategy('by_shard_count');
@ -1002,18 +1045,18 @@ SELECT * FROM get_rebalance_table_shards_plan('tab');
-- Check all the error handling cases
SELECT * FROM get_rebalance_table_shards_plan('tab', rebalance_strategy := 'non_existing');
SELECT * FROM rebalance_table_shards('tab', rebalance_strategy := 'non_existing');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM master_drain_node('localhost', :worker_2_port, rebalance_strategy := 'non_existing');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT citus_set_default_rebalance_strategy('non_existing');
UPDATE pg_dist_rebalance_strategy SET default_strategy=false;
SELECT * FROM get_rebalance_table_shards_plan('tab');
SELECT * FROM rebalance_table_shards('tab');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT * FROM master_drain_node('localhost', :worker_2_port);
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
UPDATE pg_dist_rebalance_strategy SET default_strategy=true WHERE name='by_shard_count';
CREATE OR REPLACE FUNCTION shard_cost_no_arguments()
@ -1241,7 +1284,7 @@ SELECT 1 FROM master_add_node('localhost', :master_port, groupId=>0);
SELECT count(*) FROM pg_dist_shard NATURAL JOIN pg_dist_shard_placement WHERE logicalrelid = 'ref_table'::regclass;
SELECT rebalance_table_shards('rebalance_test_table', shard_transfer_mode:='block_writes');
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
SELECT count(*) FROM pg_dist_shard NATURAL JOIN pg_dist_shard_placement WHERE logicalrelid = 'ref_table'::regclass;
@ -1278,7 +1321,7 @@ SELECT 1 from master_add_node('localhost', :worker_2_port);
-- since r2 has no replica identity we expect an error here
SELECT rebalance_table_shards();
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
DROP TABLE t1, r1, r2;
@ -1295,6 +1338,7 @@ SELECT public.wait_until_metadata_sync(30000);
CREATE TABLE r1 (a int PRIMARY KEY, b int);
SELECT create_reference_table('r1');
ALTER SEQUENCE pg_dist_groupid_seq RESTART WITH 15;
SELECT 1 from master_add_node('localhost', :worker_2_port);
-- count the number of placements for the reference table to verify it is not available on
@ -1306,7 +1350,7 @@ WHERE logicalrelid = 'r1'::regclass;
-- rebalance with _only_ a reference table, this should trigger the copy
SELECT rebalance_table_shards();
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
-- verify the reference table is on all nodes after the rebalance
SELECT count(*)
@ -1366,11 +1410,11 @@ INSERT INTO test_rebalance_with_index VALUES(1);
INSERT INTO test_rebalance_with_index VALUES(2);
SELECT * FROM master_drain_node('localhost', :worker_2_port);
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
UPDATE pg_dist_node SET shouldhaveshards=true WHERE nodeport = :worker_2_port;
SELECT rebalance_table_shards();
CALL citus_cleanup_orphaned_shards();
CALL citus_cleanup_orphaned_resources();
DROP TABLE test_rebalance_with_index CASCADE;
@ -1388,7 +1432,7 @@ SELECT public.wait_until_metadata_sync(30000);
SELECT rebalance_table_shards('test_rebalance_with_disabled_worker');
SELECT citus_activate_node('localhost', :worker_2_port);
SELECT 1 FROM citus_activate_node('localhost', :worker_2_port);
DROP TABLE test_rebalance_with_disabled_worker;
@ -1427,3 +1471,17 @@ RESET client_min_messages;
DROP TABLE "events.Energy Added", colocated_t1, colocated_t2, colocated_t3;
RESET citus.shard_count;
DROP VIEW table_placements_per_node;
DELETE FROM pg_catalog.pg_dist_rebalance_strategy WHERE name='capacity_high_worker_2';
DELETE FROM pg_catalog.pg_dist_rebalance_strategy WHERE name='only_worker_1';
\c - - - :worker_1_port
SET citus.enable_ddl_propagation TO OFF;
REVOKE ALL ON SCHEMA public FROM testrole;
DROP USER testrole;
\c - - - :worker_2_port
SET citus.enable_ddl_propagation TO OFF;
REVOKE ALL ON SCHEMA public FROM testrole;
DROP USER testrole;
DROP TABLE test_rebalance_with_disabled_worker_433500, test_rebalance_with_disabled_worker_433501, test_rebalance_with_disabled_worker_433502, test_rebalance_with_disabled_worker_433503;