diff --git a/src/backend/distributed/metadata/metadata_cache.c b/src/backend/distributed/metadata/metadata_cache.c index 69e794f09..e6cdcd148 100644 --- a/src/backend/distributed/metadata/metadata_cache.c +++ b/src/backend/distributed/metadata/metadata_cache.c @@ -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; diff --git a/src/backend/distributed/metadata/metadata_sync.c b/src/backend/distributed/metadata/metadata_sync.c index ff9300152..121531682 100644 --- a/src/backend/distributed/metadata/metadata_sync.c +++ b/src/backend/distributed/metadata/metadata_sync.c @@ -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) { diff --git a/src/backend/distributed/metadata/metadata_utility.c b/src/backend/distributed/metadata/metadata_utility.c index dce66965a..781760316 100644 --- a/src/backend/distributed/metadata/metadata_utility.c +++ b/src/backend/distributed/metadata/metadata_utility.c @@ -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. diff --git a/src/backend/distributed/metadata/node_metadata.c b/src/backend/distributed/metadata/node_metadata.c index d133c50b2..d8f4958df 100644 --- a/src/backend/distributed/metadata/node_metadata.c +++ b/src/backend/distributed/metadata/node_metadata.c @@ -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) diff --git a/src/backend/distributed/operations/create_shards.c b/src/backend/distributed/operations/create_shards.c index 71ab13139..d4978a6c4 100644 --- a/src/backend/distributed/operations/create_shards.c +++ b/src/backend/distributed/operations/create_shards.c @@ -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, diff --git a/src/backend/distributed/operations/delete_protocol.c b/src/backend/distributed/operations/delete_protocol.c index 17666bb9e..abed39272 100644 --- a/src/backend/distributed/operations/delete_protocol.c +++ b/src/backend/distributed/operations/delete_protocol.c @@ -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; } diff --git a/src/backend/distributed/operations/shard_cleaner.c b/src/backend/distributed/operations/shard_cleaner.c index d324d220b..d98b30c17 100644 --- a/src/backend/distributed/operations/shard_cleaner.c +++ b/src/backend/distributed/operations/shard_cleaner.c @@ -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. */ diff --git a/src/backend/distributed/operations/shard_split.c b/src/backend/distributed/operations/shard_split.c index fe1765122..01aba671f 100644 --- a/src/backend/distributed/operations/shard_split.c +++ b/src/backend/distributed/operations/shard_split.c @@ -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. diff --git a/src/backend/distributed/operations/shard_transfer.c b/src/backend/distributed/operations/shard_transfer.c index 3c935d16d..850196d1c 100644 --- a/src/backend/distributed/operations/shard_transfer.c +++ b/src/backend/distributed/operations/shard_transfer.c @@ -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); } } diff --git a/src/backend/distributed/sql/citus--11.1-1--11.2-1.sql b/src/backend/distributed/sql/citus--11.1-1--11.2-1.sql index 3ffe979d5..05c2a8ba6 100644 --- a/src/backend/distributed/sql/citus--11.1-1--11.2-1.sql +++ b/src/backend/distributed/sql/citus--11.1-1--11.2-1.sql @@ -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; diff --git a/src/backend/distributed/sql/downgrades/citus--11.1-1--11.0-4.sql b/src/backend/distributed/sql/downgrades/citus--11.1-1--11.0-4.sql index 54ebcf4f3..54317790c 100644 --- a/src/backend/distributed/sql/downgrades/citus--11.1-1--11.0-4.sql +++ b/src/backend/distributed/sql/downgrades/citus--11.1-1--11.0-4.sql @@ -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(); diff --git a/src/backend/distributed/test/distribution_metadata.c b/src/backend/distributed/test/distribution_metadata.c index 6d769ef27..cb378e9e3 100644 --- a/src/backend/distributed/test/distribution_metadata.c +++ b/src/backend/distributed/test/distribution_metadata.c @@ -133,7 +133,7 @@ load_shard_placement_array(PG_FUNCTION_ARGS) } else { - placementList = ShardPlacementListIncludingOrphanedPlacements(shardId); + placementList = ShardPlacementList(shardId); } placementList = SortList(placementList, CompareShardPlacementsByWorker); diff --git a/src/backend/distributed/test/shard_rebalancer.c b/src/backend/distributed/test/shard_rebalancer.c index 5cfc4c59f..9b496e1c9 100644 --- a/src/backend/distributed/test/shard_rebalancer.c +++ b/src/backend/distributed/test/shard_rebalancer.c @@ -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(); } diff --git a/src/backend/distributed/utils/colocation_utils.c b/src/backend/distributed/utils/colocation_utils.c index a5a32db0e..3421c2b35 100644 --- a/src/backend/distributed/utils/colocation_utils.c +++ b/src/backend/distributed/utils/colocation_utils.c @@ -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)) diff --git a/src/backend/distributed/utils/maintenanced.c b/src/backend/distributed/utils/maintenanced.c index 444dd9377..5f49de20a 100644 --- a/src/backend/distributed/utils/maintenanced.c +++ b/src/backend/distributed/utils/maintenanced.c @@ -645,8 +645,7 @@ CitusMaintenanceDaemonMain(Datum main_arg) */ lastShardCleanTime = GetCurrentTimestamp(); - bool waitForLocks = false; - numberOfDroppedResources = TryDropOrphanedResources(waitForLocks); + numberOfDroppedResources = TryDropOrphanedResources(); } CommitTransactionCommand(); diff --git a/src/backend/distributed/utils/shardinterval_utils.c b/src/backend/distributed/utils/shardinterval_utils.c index 54b3b348d..2980d11a4 100644 --- a/src/backend/distributed/utils/shardinterval_utils.c +++ b/src/backend/distributed/utils/shardinterval_utils.c @@ -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) { diff --git a/src/backend/distributed/worker/worker_drop_protocol.c b/src/backend/distributed/worker/worker_drop_protocol.c index 53edac74a..16b7bb66a 100644 --- a/src/backend/distributed/worker/worker_drop_protocol.c +++ b/src/backend/distributed/worker/worker_drop_protocol.c @@ -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) { diff --git a/src/include/distributed/metadata_cache.h b/src/include/distributed/metadata_cache.h index bd16e535a..07fa50e64 100644 --- a/src/include/distributed/metadata_cache.h +++ b/src/include/distributed/metadata_cache.h @@ -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); diff --git a/src/include/distributed/metadata_utility.h b/src/include/distributed/metadata_utility.h index 81d95f868..6ade9f743 100644 --- a/src/include/distributed/metadata_utility.h +++ b/src/include/distributed/metadata_utility.h @@ -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, diff --git a/src/include/distributed/relay_utility.h b/src/include/distributed/relay_utility.h index 35c66761d..b1e96811e 100644 --- a/src/include/distributed/relay_utility.h +++ b/src/include/distributed/relay_utility.h @@ -35,7 +35,6 @@ typedef enum { SHARD_STATE_INVALID_FIRST = 0, SHARD_STATE_ACTIVE = 1, - SHARD_STATE_TO_DELETE = 4, } ShardState; diff --git a/src/include/distributed/shard_cleaner.h b/src/include/distributed/shard_cleaner.h index b7bbed88c..e7d3dea1b 100644 --- a/src/include/distributed/shard_cleaner.h +++ b/src/include/distributed/shard_cleaner.h @@ -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; diff --git a/src/include/distributed/shard_transfer.h b/src/include/distributed/shard_transfer.h index aa2bd7002..00a31af29 100644 --- a/src/include/distributed/shard_transfer.h +++ b/src/include/distributed/shard_transfer.h @@ -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); diff --git a/src/test/regress/bin/normalize.sed b/src/test/regress/bin/normalize.sed index 175411a4e..4007f9ac3 100644 --- a/src/test/regress/bin/normalize.sed +++ b/src/test/regress/bin/normalize.sed @@ -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 diff --git a/src/test/regress/expected/citus_drain_node.out b/src/test/regress/expected/citus_drain_node.out index 48d60d45a..1957535b4 100644 --- a/src/test/regress/expected/citus_drain_node.out +++ b/src/test/regress/expected/citus_drain_node.out @@ -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 diff --git a/src/test/regress/expected/citus_non_blocking_split_shards.out b/src/test/regress/expected/citus_non_blocking_split_shards.out index af65563c2..d6dde8b7a 100644 --- a/src/test/regress/expected/citus_non_blocking_split_shards.out +++ b/src/test/regress/expected/citus_non_blocking_split_shards.out @@ -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 diff --git a/src/test/regress/expected/citus_split_shard_by_split_points.out b/src/test/regress/expected/citus_split_shard_by_split_points.out index 6fb6270c7..87f50da31 100644 --- a/src/test/regress/expected/citus_split_shard_by_split_points.out +++ b/src/test/regress/expected/citus_split_shard_by_split_points.out @@ -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 diff --git a/src/test/regress/expected/failure_offline_move_shard_placement.out b/src/test/regress/expected/failure_offline_move_shard_placement.out index bdd45449b..b2b539cd8 100644 --- a/src/test/regress/expected/failure_offline_move_shard_placement.out +++ b/src/test/regress/expected/failure_offline_move_shard_placement.out @@ -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 --------------------------------------------------------------------- diff --git a/src/test/regress/expected/failure_online_move_shard_placement.out b/src/test/regress/expected/failure_online_move_shard_placement.out index ed9537106..cf5890f35 100644 --- a/src/test/regress/expected/failure_online_move_shard_placement.out +++ b/src/test/regress/expected/failure_online_move_shard_placement.out @@ -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 diff --git a/src/test/regress/expected/foreign_key_to_reference_shard_rebalance.out b/src/test/regress/expected/foreign_key_to_reference_shard_rebalance.out index 54054d99b..2dc329153 100644 --- a/src/test/regress/expected/foreign_key_to_reference_shard_rebalance.out +++ b/src/test/regress/expected/foreign_key_to_reference_shard_rebalance.out @@ -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 --------------------------------------------------------------------- diff --git a/src/test/regress/expected/ignoring_orphaned_shards.out b/src/test/regress/expected/ignoring_orphaned_shards.out deleted file mode 100644 index a43dd9d59..000000000 --- a/src/test/regress/expected/ignoring_orphaned_shards.out +++ /dev/null @@ -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; diff --git a/src/test/regress/expected/isolation_move_placement_vs_move_placement.out b/src/test/regress/expected/isolation_move_placement_vs_move_placement.out index 307b08b00..a2058b225 100644 --- a/src/test/regress/expected/isolation_move_placement_vs_move_placement.out +++ b/src/test/regress/expected/isolation_move_placement_vs_move_placement.out @@ -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'); + +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) + diff --git a/src/test/regress/expected/isolation_rebalancer_deferred_drop.out b/src/test/regress/expected/isolation_rebalancer_deferred_drop.out index d915703f7..8adecd298 100644 --- a/src/test/regress/expected/isolation_rebalancer_deferred_drop.out +++ b/src/test/regress/expected/isolation_rebalancer_deferred_drop.out @@ -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(); - -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_%'; -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); - -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; - diff --git a/src/test/regress/expected/multi_cluster_management.out b/src/test/regress/expected/multi_cluster_management.out index c4a3976a4..56235cc91 100644 --- a/src/test/regress/expected/multi_cluster_management.out +++ b/src/test/regress/expected/multi_cluster_management.out @@ -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; diff --git a/src/test/regress/expected/multi_colocated_shard_rebalance.out b/src/test/regress/expected/multi_colocated_shard_rebalance.out index 749a565cf..784202918 100644 --- a/src/test/regress/expected/multi_colocated_shard_rebalance.out +++ b/src/test/regress/expected/multi_colocated_shard_rebalance.out @@ -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; diff --git a/src/test/regress/expected/multi_extension.out b/src/test/regress/expected/multi_extension.out index 7ac4c30b1..bce90883c 100644 --- a/src/test/regress/expected/multi_extension.out +++ b/src/test/regress/expected/multi_extension.out @@ -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 --------------------------------------------------------------------- diff --git a/src/test/regress/expected/multi_test_helpers.out b/src/test/regress/expected/multi_test_helpers.out index f997e40d2..55f286d1f 100644 --- a/src/test/regress/expected/multi_test_helpers.out +++ b/src/test/regress/expected/multi_test_helpers.out @@ -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; diff --git a/src/test/regress/expected/pg14.out b/src/test/regress/expected/pg14.out index 7d73284ca..ae5fe8bdc 100644 --- a/src/test/regress/expected/pg14.out +++ b/src/test/regress/expected/pg14.out @@ -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 )$$); diff --git a/src/test/regress/expected/shard_move_constraints.out b/src/test/regress/expected/shard_move_constraints.out index 8635f00b4..0862a4b4f 100644 --- a/src/test/regress/expected/shard_move_constraints.out +++ b/src/test/regress/expected/shard_move_constraints.out @@ -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; diff --git a/src/test/regress/expected/shard_move_constraints_blocking.out b/src/test/regress/expected/shard_move_constraints_blocking.out index 3eae6b41c..82d925821 100644 --- a/src/test/regress/expected/shard_move_constraints_blocking.out +++ b/src/test/regress/expected/shard_move_constraints_blocking.out @@ -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; diff --git a/src/test/regress/expected/shard_move_deferred_delete.out b/src/test/regress/expected/shard_move_deferred_delete.out index 0f025e3c1..e87cd0f97 100644 --- a/src/test/regress/expected/shard_move_deferred_delete.out +++ b/src/test/regress/expected/shard_move_deferred_delete.out @@ -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 diff --git a/src/test/regress/expected/shard_rebalancer.out b/src/test/regress/expected/shard_rebalancer.out index 88c13b958..84ba28402 100644 --- a/src/test/regress/expected/shard_rebalancer.out +++ b/src/test/regress/expected/shard_rebalancer.out @@ -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: -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: -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; diff --git a/src/test/regress/expected/shard_rebalancer_unit.out b/src/test/regress/expected/shard_rebalancer_unit.out index fe88968fe..156edfc6b 100644 --- a/src/test/regress/expected/shard_rebalancer_unit.out +++ b/src/test/regress/expected/shard_rebalancer_unit.out @@ -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 diff --git a/src/test/regress/operations_schedule b/src/test/regress/operations_schedule index 2c767eb42..15afd9e18 100644 --- a/src/test/regress/operations_schedule +++ b/src/test/regress/operations_schedule @@ -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 diff --git a/src/test/regress/spec/isolation_logical_replication_nonsu_nonbypassrls.spec b/src/test/regress/spec/isolation_logical_replication_nonsu_nonbypassrls.spec index 1eaeee98f..103b5e70f 100644 --- a/src/test/regress/spec/isolation_logical_replication_nonsu_nonbypassrls.spec +++ b/src/test/regress/spec/isolation_logical_replication_nonsu_nonbypassrls.spec @@ -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; diff --git a/src/test/regress/spec/isolation_move_placement_vs_move_placement.spec b/src/test/regress/spec/isolation_move_placement_vs_move_placement.spec index ebb2e1761..b55ab6e54 100644 --- a/src/test/regress/spec/isolation_move_placement_vs_move_placement.spec +++ b/src/test/regress/spec/isolation_move_placement_vs_move_placement.spec @@ -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" diff --git a/src/test/regress/spec/isolation_rebalancer_deferred_drop.spec b/src/test/regress/spec/isolation_rebalancer_deferred_drop.spec index fa4c25ad1..e5185680e 100644 --- a/src/test/regress/spec/isolation_rebalancer_deferred_drop.spec +++ b/src/test/regress/spec/isolation_rebalancer_deferred_drop.spec @@ -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" diff --git a/src/test/regress/spec/isolation_shard_rebalancer_progress.spec b/src/test/regress/spec/isolation_shard_rebalancer_progress.spec index e24a0a0df..d7c540101 100644 --- a/src/test/regress/spec/isolation_shard_rebalancer_progress.spec +++ b/src/test/regress/spec/isolation_shard_rebalancer_progress.spec @@ -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'); diff --git a/src/test/regress/sql/citus_drain_node.sql b/src/test/regress/sql/citus_drain_node.sql index 6ee16a29f..c105a0245 100644 --- a/src/test/regress/sql/citus_drain_node.sql +++ b/src/test/regress/sql/citus_drain_node.sql @@ -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, diff --git a/src/test/regress/sql/citus_non_blocking_split_shards.sql b/src/test/regress/sql/citus_non_blocking_split_shards.sql index 05f2e7dfc..11275a342 100644 --- a/src/test/regress/sql/citus_non_blocking_split_shards.sql +++ b/src/test/regress/sql/citus_non_blocking_split_shards.sql @@ -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 diff --git a/src/test/regress/sql/citus_split_shard_by_split_points.sql b/src/test/regress/sql/citus_split_shard_by_split_points.sql index f50130276..f5e7f005a 100644 --- a/src/test/regress/sql/citus_split_shard_by_split_points.sql +++ b/src/test/regress/sql/citus_split_shard_by_split_points.sql @@ -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 diff --git a/src/test/regress/sql/failure_offline_move_shard_placement.sql b/src/test/regress/sql/failure_offline_move_shard_placement.sql index 1b02da1e9..359cceac3 100644 --- a/src/test/regress/sql/failure_offline_move_shard_placement.sql +++ b/src/test/regress/sql/failure_offline_move_shard_placement.sql @@ -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; diff --git a/src/test/regress/sql/failure_online_move_shard_placement.sql b/src/test/regress/sql/failure_online_move_shard_placement.sql index 5007c194f..3ddf3eb53 100644 --- a/src/test/regress/sql/failure_online_move_shard_placement.sql +++ b/src/test/regress/sql/failure_online_move_shard_placement.sql @@ -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 ; diff --git a/src/test/regress/sql/foreign_key_to_reference_shard_rebalance.sql b/src/test/regress/sql/foreign_key_to_reference_shard_rebalance.sql index dc70f8563..7791001e0 100644 --- a/src/test/regress/sql/foreign_key_to_reference_shard_rebalance.sql +++ b/src/test/regress/sql/foreign_key_to_reference_shard_rebalance.sql @@ -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); diff --git a/src/test/regress/sql/ignoring_orphaned_shards.sql b/src/test/regress/sql/ignoring_orphaned_shards.sql deleted file mode 100644 index 937587e2e..000000000 --- a/src/test/regress/sql/ignoring_orphaned_shards.sql +++ /dev/null @@ -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; diff --git a/src/test/regress/sql/multi_cluster_management.sql b/src/test/regress/sql/multi_cluster_management.sql index d8beba8e0..f9aa81836 100644 --- a/src/test/regress/sql/multi_cluster_management.sql +++ b/src/test/regress/sql/multi_cluster_management.sql @@ -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; diff --git a/src/test/regress/sql/multi_colocated_shard_rebalance.sql b/src/test/regress/sql/multi_colocated_shard_rebalance.sql index a0655229c..c22af2cc7 100644 --- a/src/test/regress/sql/multi_colocated_shard_rebalance.sql +++ b/src/test/regress/sql/multi_colocated_shard_rebalance.sql @@ -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; diff --git a/src/test/regress/sql/multi_extension.sql b/src/test/regress/sql/multi_extension.sql index 4f4bcd8c3..a44282521 100644 --- a/src/test/regress/sql/multi_extension.sql +++ b/src/test/regress/sql/multi_extension.sql @@ -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; diff --git a/src/test/regress/sql/multi_test_helpers.sql b/src/test/regress/sql/multi_test_helpers.sql index dd8067adc..ba6d6f17f 100644 --- a/src/test/regress/sql/multi_test_helpers.sql +++ b/src/test/regress/sql/multi_test_helpers.sql @@ -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; diff --git a/src/test/regress/sql/pg14.sql b/src/test/regress/sql/pg14.sql index 7c5faa94f..77f3e1cc5 100644 --- a/src/test/regress/sql/pg14.sql +++ b/src/test/regress/sql/pg14.sql @@ -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 )$$); diff --git a/src/test/regress/sql/shard_move_constraints.sql b/src/test/regress/sql/shard_move_constraints.sql index b66dd84b9..e64962d5b 100644 --- a/src/test/regress/sql/shard_move_constraints.sql +++ b/src/test/regress/sql/shard_move_constraints.sql @@ -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; diff --git a/src/test/regress/sql/shard_move_constraints_blocking.sql b/src/test/regress/sql/shard_move_constraints_blocking.sql index 6caad1eb0..df394820c 100644 --- a/src/test/regress/sql/shard_move_constraints_blocking.sql +++ b/src/test/regress/sql/shard_move_constraints_blocking.sql @@ -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; diff --git a/src/test/regress/sql/shard_move_deferred_delete.sql b/src/test/regress/sql/shard_move_deferred_delete.sql index 73a403ddc..9cbad588e 100644 --- a/src/test/regress/sql/shard_move_deferred_delete.sql +++ b/src/test/regress/sql/shard_move_deferred_delete.sql @@ -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$ diff --git a/src/test/regress/sql/shard_rebalancer.sql b/src/test/regress/sql/shard_rebalancer.sql index a5141813b..e6ed5fe62 100644 --- a/src/test/regress/sql/shard_rebalancer.sql +++ b/src/test/regress/sql/shard_rebalancer.sql @@ -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;