Add status column to get_rebalance_progress() (#6403)

DESCRIPTION: Adds status column to get_rebalance_progress()

Introduces a new column named `status` for the function
`get_rebalance_progress()`. For each ongoing shard move, this column
will reveal information about that shard move operation's current
status.

For now, candidate status messages could be one of the below.

* Not Started
* Setting Up
* Copying Data
* Catching Up
* Creating Constraints
* Final Catchup
* Creating Foreign Keys
* Completing
* Completed
pull/6429/head
Ahmet Gedemenli 2022-10-17 16:55:31 +03:00 committed by GitHub
parent 8323f4f12c
commit 96912d9ba1
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
13 changed files with 303 additions and 86 deletions

View File

@ -283,6 +283,18 @@ static const char *PlacementUpdateTypeNames[] = {
[PLACEMENT_UPDATE_COPY] = "copy", [PLACEMENT_UPDATE_COPY] = "copy",
}; };
static const char *PlacementUpdateStatusNames[] = {
[PLACEMENT_UPDATE_STATUS_NOT_STARTED_YET] = "Not Started Yet",
[PLACEMENT_UPDATE_STATUS_SETTING_UP] = "Setting Up",
[PLACEMENT_UPDATE_STATUS_COPYING_DATA] = "Copying Data",
[PLACEMENT_UPDATE_STATUS_CATCHING_UP] = "Catching Up",
[PLACEMENT_UPDATE_STATUS_CREATING_CONSTRAINTS] = "Creating Constraints",
[PLACEMENT_UPDATE_STATUS_FINAL_CATCH_UP] = "Final Catchup",
[PLACEMENT_UPDATE_STATUS_CREATING_FOREIGN_KEYS] = "Creating Foreign Keys",
[PLACEMENT_UPDATE_STATUS_COMPLETING] = "Completing",
[PLACEMENT_UPDATE_STATUS_COMPLETED] = "Completed",
};
#ifdef USE_ASSERT_CHECKING #ifdef USE_ASSERT_CHECKING
/* /*
@ -797,7 +809,8 @@ ExecutePlacementUpdates(List *placementUpdateList, Oid shardReplicationModeOid,
void void
SetupRebalanceMonitor(List *placementUpdateList, SetupRebalanceMonitor(List *placementUpdateList,
Oid relationId, Oid relationId,
uint64 initialProgressState) uint64 initialProgressState,
PlacementUpdateStatus initialStatus)
{ {
List *colocatedUpdateList = GetColocatedRebalanceSteps(placementUpdateList); List *colocatedUpdateList = GetColocatedRebalanceSteps(placementUpdateList);
ListCell *colocatedUpdateCell = NULL; ListCell *colocatedUpdateCell = NULL;
@ -822,6 +835,7 @@ SetupRebalanceMonitor(List *placementUpdateList,
event->sourcePort = colocatedUpdate->sourceNode->workerPort; event->sourcePort = colocatedUpdate->sourceNode->workerPort;
event->targetPort = colocatedUpdate->targetNode->workerPort; event->targetPort = colocatedUpdate->targetNode->workerPort;
event->updateType = colocatedUpdate->updateType; event->updateType = colocatedUpdate->updateType;
pg_atomic_init_u64(&event->updateStatus, initialStatus);
pg_atomic_init_u64(&event->progress, initialProgressState); pg_atomic_init_u64(&event->progress, initialProgressState);
eventIndex++; eventIndex++;
@ -1261,8 +1275,8 @@ get_rebalance_progress(PG_FUNCTION_ARGS)
shardSize = shardSizesStat->totalSize; shardSize = shardSizesStat->totalSize;
} }
Datum values[14]; Datum values[15];
bool nulls[14]; bool nulls[15];
memset(values, 0, sizeof(values)); memset(values, 0, sizeof(values));
memset(nulls, 0, sizeof(nulls)); memset(nulls, 0, sizeof(nulls));
@ -1282,6 +1296,10 @@ get_rebalance_progress(PG_FUNCTION_ARGS)
cstring_to_text(PlacementUpdateTypeNames[step->updateType])); cstring_to_text(PlacementUpdateTypeNames[step->updateType]));
values[12] = LSNGetDatum(sourceLSN); values[12] = LSNGetDatum(sourceLSN);
values[13] = LSNGetDatum(targetLSN); values[13] = LSNGetDatum(targetLSN);
values[14] = PointerGetDatum(cstring_to_text(
PlacementUpdateStatusNames[
pg_atomic_read_u64(
&step->updateStatus)]));
tuplestore_putvalues(tupstore, tupdesc, values, nulls); tuplestore_putvalues(tupstore, tupdesc, values, nulls);
} }
@ -1794,7 +1812,8 @@ RebalanceTableShards(RebalanceOptions *options, Oid shardReplicationModeOid)
* purposes so it does not really matter which to show * purposes so it does not really matter which to show
*/ */
SetupRebalanceMonitor(placementUpdateList, linitial_oid(options->relationIdList), SetupRebalanceMonitor(placementUpdateList, linitial_oid(options->relationIdList),
REBALANCE_PROGRESS_WAITING); REBALANCE_PROGRESS_WAITING,
PLACEMENT_UPDATE_STATUS_NOT_STARTED_YET);
ExecutePlacementUpdates(placementUpdateList, shardReplicationModeOid, "Moving"); ExecutePlacementUpdates(placementUpdateList, shardReplicationModeOid, "Moving");
FinalizeCurrentProgressMonitor(); FinalizeCurrentProgressMonitor();
} }

View File

@ -338,9 +338,16 @@ citus_move_shard_placement(PG_FUNCTION_ARGS)
placementUpdateEvent->sourceNode = sourceNode; placementUpdateEvent->sourceNode = sourceNode;
placementUpdateEvent->targetNode = targetNode; placementUpdateEvent->targetNode = targetNode;
SetupRebalanceMonitor(list_make1(placementUpdateEvent), relationId, SetupRebalanceMonitor(list_make1(placementUpdateEvent), relationId,
REBALANCE_PROGRESS_MOVING); REBALANCE_PROGRESS_MOVING,
PLACEMENT_UPDATE_STATUS_SETTING_UP);
} }
UpdatePlacementUpdateStatusForShardIntervalList(
colocatedShardList,
sourceNodeName,
sourceNodePort,
PLACEMENT_UPDATE_STATUS_SETTING_UP);
/* /*
* At this point of the shard moves, we don't need to block the writes to * At this point of the shard moves, we don't need to block the writes to
* shards when logical replication is used. * shards when logical replication is used.
@ -408,6 +415,12 @@ citus_move_shard_placement(PG_FUNCTION_ARGS)
sourceNodePort, targetNodeName, sourceNodePort, targetNodeName,
targetNodePort); targetNodePort);
UpdatePlacementUpdateStatusForShardIntervalList(
colocatedShardList,
sourceNodeName,
sourceNodePort,
PLACEMENT_UPDATE_STATUS_COMPLETED);
FinalizeCurrentProgressMonitor(); FinalizeCurrentProgressMonitor();
PG_RETURN_VOID(); PG_RETURN_VOID();
} }
@ -1007,8 +1020,14 @@ ReplicateColocatedShardPlacement(int64 shardId, char *sourceNodeName,
placementUpdateEvent->sourceNode = sourceNode; placementUpdateEvent->sourceNode = sourceNode;
placementUpdateEvent->targetNode = targetNode; placementUpdateEvent->targetNode = targetNode;
SetupRebalanceMonitor(list_make1(placementUpdateEvent), relationId, SetupRebalanceMonitor(list_make1(placementUpdateEvent), relationId,
REBALANCE_PROGRESS_MOVING); REBALANCE_PROGRESS_MOVING,
PLACEMENT_UPDATE_STATUS_SETTING_UP);
UpdatePlacementUpdateStatusForShardIntervalList(
colocatedShardList,
sourceNodeName,
sourceNodePort,
PLACEMENT_UPDATE_STATUS_SETTING_UP);
/* /*
* At this point of the shard replication, we don't need to block the writes to * At this point of the shard replication, we don't need to block the writes to
@ -1078,6 +1097,13 @@ ReplicateColocatedShardPlacement(int64 shardId, char *sourceNodeName,
SendCommandToWorkersWithMetadata(placementCommand); SendCommandToWorkersWithMetadata(placementCommand);
} }
} }
UpdatePlacementUpdateStatusForShardIntervalList(
colocatedShardList,
sourceNodeName,
sourceNodePort,
PLACEMENT_UPDATE_STATUS_COMPLETED);
FinalizeCurrentProgressMonitor(); FinalizeCurrentProgressMonitor();
} }
@ -1257,8 +1283,20 @@ CopyShardTablesViaBlockWrites(List *shardIntervalList, char *sourceNodeName,
tableOwner, ddlCommandList); tableOwner, ddlCommandList);
} }
UpdatePlacementUpdateStatusForShardIntervalList(
shardIntervalList,
sourceNodeName,
sourceNodePort,
PLACEMENT_UPDATE_STATUS_COPYING_DATA);
CopyShardsToNode(sourceNode, targetNode, shardIntervalList, NULL); CopyShardsToNode(sourceNode, targetNode, shardIntervalList, NULL);
UpdatePlacementUpdateStatusForShardIntervalList(
shardIntervalList,
sourceNodeName,
sourceNodePort,
PLACEMENT_UPDATE_STATUS_CREATING_CONSTRAINTS);
foreach_ptr(shardInterval, shardIntervalList) foreach_ptr(shardInterval, shardIntervalList)
{ {
List *ddlCommandList = List *ddlCommandList =
@ -1291,6 +1329,12 @@ CopyShardTablesViaBlockWrites(List *shardIntervalList, char *sourceNodeName,
} }
} }
UpdatePlacementUpdateStatusForShardIntervalList(
shardIntervalList,
sourceNodeName,
sourceNodePort,
PLACEMENT_UPDATE_STATUS_CREATING_FOREIGN_KEYS);
/* /*
* Iterate through the colocated shards and create DDL commamnds * Iterate through the colocated shards and create DDL commamnds
* to create the foreign constraints. * to create the foreign constraints.
@ -1322,6 +1366,12 @@ CopyShardTablesViaBlockWrites(List *shardIntervalList, char *sourceNodeName,
shardCommandList->ddlCommandList); shardCommandList->ddlCommandList);
} }
UpdatePlacementUpdateStatusForShardIntervalList(
shardIntervalList,
sourceNodeName,
sourceNodePort,
PLACEMENT_UPDATE_STATUS_COMPLETING);
MemoryContextReset(localContext); MemoryContextReset(localContext);
MemoryContextSwitchTo(oldContext); MemoryContextSwitchTo(oldContext);
} }
@ -1937,3 +1987,59 @@ WorkerApplyShardDDLCommandList(List *ddlCommandList, int64 shardId)
return applyDDLCommandList; return applyDDLCommandList;
} }
/*
* UpdatePlacementUpdateStatusForShardIntervalList updates the status field for shards
* in the given shardInterval list.
*/
void
UpdatePlacementUpdateStatusForShardIntervalList(List *shardIntervalList,
char *sourceName, int sourcePort,
PlacementUpdateStatus status)
{
List *segmentList = NIL;
List *rebalanceMonitorList = NULL;
if (!HasProgressMonitor())
{
rebalanceMonitorList = ProgressMonitorList(REBALANCE_ACTIVITY_MAGIC_NUMBER,
&segmentList);
}
else
{
rebalanceMonitorList = list_make1(GetCurrentProgressMonitor());
}
ProgressMonitorData *monitor = NULL;
foreach_ptr(monitor, rebalanceMonitorList)
{
PlacementUpdateEventProgress *steps = ProgressMonitorSteps(monitor);
for (int moveIndex = 0; moveIndex < monitor->stepCount; moveIndex++)
{
PlacementUpdateEventProgress *step = steps + moveIndex;
uint64 currentShardId = step->shardId;
bool foundInList = false;
ShardInterval *candidateShard = NULL;
foreach_ptr(candidateShard, shardIntervalList)
{
if (candidateShard->shardId == currentShardId)
{
foundInList = true;
break;
}
}
if (foundInList &&
strcmp(step->sourceName, sourceName) == 0 &&
step->sourcePort == sourcePort)
{
pg_atomic_write_u64(&step->updateStatus, status);
}
}
}
DetachFromDSMSegments(segmentList);
}

View File

@ -133,6 +133,17 @@ FinalizeCurrentProgressMonitor(void)
} }
/*
* HasProgressMonitor returns true if there is a current progress monitor,
* by checking the variable currentProgressDSMHandle. Returns false otherwise.
*/
bool
HasProgressMonitor(void)
{
return currentProgressDSMHandle != DSM_HANDLE_INVALID;
}
/* /*
* ProgressMonitorList returns the addresses of monitors of ongoing commands, associated * ProgressMonitorList returns the addresses of monitors of ongoing commands, associated
* with the given identifier magic number. The function takes a pass in * with the given identifier magic number. The function takes a pass in

View File

@ -258,6 +258,12 @@ LogicallyReplicateShards(List *shardList, char *sourceNodeName, int sourceNodePo
*/ */
CreateReplicaIdentities(logicalRepTargetList); CreateReplicaIdentities(logicalRepTargetList);
UpdatePlacementUpdateStatusForShardIntervalList(
shardList,
sourceNodeName,
sourceNodePort,
PLACEMENT_UPDATE_STATUS_COPYING_DATA);
CopyShardsToNode(sourceNode, targetNode, shardList, snapshot); CopyShardsToNode(sourceNode, targetNode, shardList, snapshot);
/* /*
@ -374,6 +380,12 @@ CompleteNonBlockingShardTransfer(List *shardList,
/* Start applying the changes from the replication slots to catch up. */ /* Start applying the changes from the replication slots to catch up. */
EnableSubscriptions(logicalRepTargetList); EnableSubscriptions(logicalRepTargetList);
UpdatePlacementUpdateStatusForShardIntervalList(
shardList,
sourceConnection->hostname,
sourceConnection->port,
PLACEMENT_UPDATE_STATUS_CATCHING_UP);
/* /*
* The following check is a leftover from when used subscriptions with * The following check is a leftover from when used subscriptions with
* copy_data=true. It's probably not really necessary anymore, but it * copy_data=true. It's probably not really necessary anymore, but it
@ -392,12 +404,25 @@ CompleteNonBlockingShardTransfer(List *shardList,
*/ */
WaitForAllSubscriptionsToCatchUp(sourceConnection, groupedLogicalRepTargetsHash); WaitForAllSubscriptionsToCatchUp(sourceConnection, groupedLogicalRepTargetsHash);
UpdatePlacementUpdateStatusForShardIntervalList(
shardList,
sourceConnection->hostname,
sourceConnection->port,
PLACEMENT_UPDATE_STATUS_CREATING_CONSTRAINTS);
/* /*
* Now lets create the post-load objects, such as the indexes, constraints * Now lets create the post-load objects, such as the indexes, constraints
* and partitioning hierarchy. Once they are done, wait until the replication * and partitioning hierarchy. Once they are done, wait until the replication
* catches up again. So we don't block writes too long. * catches up again. So we don't block writes too long.
*/ */
CreatePostLogicalReplicationDataLoadObjects(logicalRepTargetList, type); CreatePostLogicalReplicationDataLoadObjects(logicalRepTargetList, type);
UpdatePlacementUpdateStatusForShardIntervalList(
shardList,
sourceConnection->hostname,
sourceConnection->port,
PLACEMENT_UPDATE_STATUS_FINAL_CATCH_UP);
WaitForAllSubscriptionsToCatchUp(sourceConnection, groupedLogicalRepTargetsHash); WaitForAllSubscriptionsToCatchUp(sourceConnection, groupedLogicalRepTargetsHash);
@ -419,6 +444,12 @@ CompleteNonBlockingShardTransfer(List *shardList,
if (type != SHARD_SPLIT) if (type != SHARD_SPLIT)
{ {
UpdatePlacementUpdateStatusForShardIntervalList(
shardList,
sourceConnection->hostname,
sourceConnection->port,
PLACEMENT_UPDATE_STATUS_CREATING_FOREIGN_KEYS);
/* /*
* We're creating the foreign constraints to reference tables after the * We're creating the foreign constraints to reference tables after the
* data is already replicated and all the necessary locks are acquired. * data is already replicated and all the necessary locks are acquired.
@ -432,6 +463,12 @@ CompleteNonBlockingShardTransfer(List *shardList,
CreateUncheckedForeignKeyConstraints(logicalRepTargetList); CreateUncheckedForeignKeyConstraints(logicalRepTargetList);
} }
UpdatePlacementUpdateStatusForShardIntervalList(
shardList,
sourceConnection->hostname,
sourceConnection->port,
PLACEMENT_UPDATE_STATUS_COMPLETING);
/* we're done, cleanup the publication and subscription */ /* we're done, cleanup the publication and subscription */
DropSubscriptions(logicalRepTargetList); DropSubscriptions(logicalRepTargetList);
DropReplicationSlots(sourceConnection, logicalRepTargetList); DropReplicationSlots(sourceConnection, logicalRepTargetList);

View File

@ -14,7 +14,8 @@ CREATE OR REPLACE FUNCTION pg_catalog.get_rebalance_progress()
target_shard_size bigint, target_shard_size bigint,
operation_type text, operation_type text,
source_lsn pg_lsn, source_lsn pg_lsn,
target_lsn pg_lsn target_lsn pg_lsn,
status text
) )
AS 'MODULE_PATHNAME' AS 'MODULE_PATHNAME'
LANGUAGE C STRICT; LANGUAGE C STRICT;

View File

@ -14,7 +14,8 @@ CREATE OR REPLACE FUNCTION pg_catalog.get_rebalance_progress()
target_shard_size bigint, target_shard_size bigint,
operation_type text, operation_type text,
source_lsn pg_lsn, source_lsn pg_lsn,
target_lsn pg_lsn target_lsn pg_lsn,
status text
) )
AS 'MODULE_PATHNAME' AS 'MODULE_PATHNAME'
LANGUAGE C STRICT; LANGUAGE C STRICT;

View File

@ -34,6 +34,7 @@ extern void RegisterProgressMonitor(uint64 progressTypeMagicNumber,
dsm_handle dsmHandle); dsm_handle dsmHandle);
extern ProgressMonitorData * GetCurrentProgressMonitor(void); extern ProgressMonitorData * GetCurrentProgressMonitor(void);
extern void FinalizeCurrentProgressMonitor(void); extern void FinalizeCurrentProgressMonitor(void);
extern bool HasProgressMonitor(void);
extern List * ProgressMonitorList(uint64 commandTypeMagicNumber, extern List * ProgressMonitorList(uint64 commandTypeMagicNumber,
List **attachedDSMSegmentList); List **attachedDSMSegmentList);
extern void DetachFromDSMSegments(List *dsmSegmentList); extern void DetachFromDSMSegments(List *dsmSegmentList);

View File

@ -85,6 +85,19 @@ typedef enum
PLACEMENT_UPDATE_COPY = 2 PLACEMENT_UPDATE_COPY = 2
} PlacementUpdateType; } PlacementUpdateType;
typedef enum
{
PLACEMENT_UPDATE_STATUS_NOT_STARTED_YET = 0,
PLACEMENT_UPDATE_STATUS_SETTING_UP = 1,
PLACEMENT_UPDATE_STATUS_COPYING_DATA = 2,
PLACEMENT_UPDATE_STATUS_CATCHING_UP = 3,
PLACEMENT_UPDATE_STATUS_CREATING_CONSTRAINTS = 4,
PLACEMENT_UPDATE_STATUS_FINAL_CATCH_UP = 5,
PLACEMENT_UPDATE_STATUS_CREATING_FOREIGN_KEYS = 6,
PLACEMENT_UPDATE_STATUS_COMPLETING = 7,
PLACEMENT_UPDATE_STATUS_COMPLETED = 8,
} PlacementUpdateStatus;
/* /*
* PlacementUpdateEvent represents a logical unit of work that copies or * PlacementUpdateEvent represents a logical unit of work that copies or
@ -108,6 +121,7 @@ typedef struct PlacementUpdateEventProgress
int targetPort; int targetPort;
PlacementUpdateType updateType; PlacementUpdateType updateType;
pg_atomic_uint64 progress; pg_atomic_uint64 progress;
pg_atomic_uint64 updateStatus;
} PlacementUpdateEventProgress; } PlacementUpdateEventProgress;
typedef struct NodeFillState typedef struct NodeFillState
@ -199,6 +213,7 @@ extern void AcquirePlacementColocationLock(Oid relationId, int lockMode,
extern void SetupRebalanceMonitor(List *placementUpdateList, extern void SetupRebalanceMonitor(List *placementUpdateList,
Oid relationId, Oid relationId,
uint64 initialProgressState); uint64 initialProgressState,
PlacementUpdateStatus initialStatus);
#endif /* SHARD_REBALANCER_H */ #endif /* SHARD_REBALANCER_H */

View File

@ -1,6 +1,6 @@
/*------------------------------------------------------------------------- /*-------------------------------------------------------------------------
* *
* repair_shards.h * shard_transfer.h
* Code used to move shards around. * Code used to move shards around.
* *
* Copyright (c) Citus Data, Inc. * Copyright (c) Citus Data, Inc.
@ -9,6 +9,7 @@
#include "postgres.h" #include "postgres.h"
#include "distributed/shard_rebalancer.h"
#include "nodes/pg_list.h" #include "nodes/pg_list.h"
extern uint64 ShardListSizeInBytes(List *colocatedShardList, extern uint64 ShardListSizeInBytes(List *colocatedShardList,
@ -18,3 +19,7 @@ extern void CopyShardsToNode(WorkerNode *sourceNode, WorkerNode *targetNode,
List *shardIntervalList, char *snapshotName); List *shardIntervalList, char *snapshotName);
extern void VerifyTablesHaveReplicaIdentity(List *colocatedTableList); extern void VerifyTablesHaveReplicaIdentity(List *colocatedTableList);
extern bool RelationCanPublishAllModifications(Oid relationId); extern bool RelationCanPublishAllModifications(Oid relationId);
extern void UpdatePlacementUpdateStatusForShardIntervalList(List *shardIntervalList,
char *sourceName,
int sourcePort,
PlacementUpdateStatus status);

View File

@ -33,15 +33,16 @@ step s7-get-progress:
operation_type, operation_type,
source_lsn >= target_lsn as lsn_sanity_check, source_lsn >= target_lsn as lsn_sanity_check,
source_lsn > '0/0' as source_lsn_available, source_lsn > '0/0' as source_lsn_available,
target_lsn > '0/0' as target_lsn_available target_lsn > '0/0' as target_lsn_available,
status
FROM get_rebalance_progress(); FROM get_rebalance_progress();
table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available|status
--------------------------------------------------------------------- ---------------------------------------------------------------------
colocated1|1500001| 50000|localhost | 57637| 50000|localhost | 57638| 0| 1|move |t |t |f colocated1|1500001| 50000|localhost | 57637| 50000|localhost | 57638| 0| 1|move |t |t |f |Setting Up
colocated2|1500005| 400000|localhost | 57637| 400000|localhost | 57638| 0| 1|move |t |t |f colocated2|1500005| 400000|localhost | 57637| 400000|localhost | 57638| 0| 1|move |t |t |f |Setting Up
colocated1|1500002| 200000|localhost | 57637| 200000|localhost | 57638| 0| 0|move |t |t |f colocated1|1500002| 200000|localhost | 57637| 200000|localhost | 57638| 0| 0|move |t |t |f |Not Started Yet
colocated2|1500006| 8000|localhost | 57637| 8000|localhost | 57638| 0| 0|move |t |t |f colocated2|1500006| 8000|localhost | 57637| 8000|localhost | 57638| 0| 0|move |t |t |f |Not Started Yet
(4 rows) (4 rows)
step s2-unlock-1-start: step s2-unlock-1-start:
@ -81,10 +82,11 @@ step s7-get-progress:
operation_type, operation_type,
source_lsn >= target_lsn as lsn_sanity_check, source_lsn >= target_lsn as lsn_sanity_check,
source_lsn > '0/0' as source_lsn_available, source_lsn > '0/0' as source_lsn_available,
target_lsn > '0/0' as target_lsn_available target_lsn > '0/0' as target_lsn_available,
status
FROM get_rebalance_progress(); FROM get_rebalance_progress();
table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available|status
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (0 rows)
@ -124,15 +126,16 @@ step s7-get-progress:
operation_type, operation_type,
source_lsn >= target_lsn as lsn_sanity_check, source_lsn >= target_lsn as lsn_sanity_check,
source_lsn > '0/0' as source_lsn_available, source_lsn > '0/0' as source_lsn_available,
target_lsn > '0/0' as target_lsn_available target_lsn > '0/0' as target_lsn_available,
status
FROM get_rebalance_progress(); FROM get_rebalance_progress();
table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available|status
--------------------------------------------------------------------- ---------------------------------------------------------------------
colocated1|1500001| 50000|localhost | 57637| 0|localhost | 57638| 50000| 2|move |t |t |f colocated1|1500001| 50000|localhost | 57637| 0|localhost | 57638| 50000| 2|move |t |t |f |Completed
colocated2|1500005| 400000|localhost | 57637| 0|localhost | 57638| 400000| 2|move |t |t |f colocated2|1500005| 400000|localhost | 57637| 0|localhost | 57638| 400000| 2|move |t |t |f |Completed
colocated1|1500002| 200000|localhost | 57637| 200000|localhost | 57638| 0| 1|move |t |t |f colocated1|1500002| 200000|localhost | 57637| 200000|localhost | 57638| 0| 1|move |t |t |f |Setting Up
colocated2|1500006| 8000|localhost | 57637| 8000|localhost | 57638| 0| 1|move |t |t |f colocated2|1500006| 8000|localhost | 57637| 8000|localhost | 57638| 0| 1|move |t |t |f |Setting Up
(4 rows) (4 rows)
step s3-unlock-2-start: step s3-unlock-2-start:
@ -172,10 +175,11 @@ step s7-get-progress:
operation_type, operation_type,
source_lsn >= target_lsn as lsn_sanity_check, source_lsn >= target_lsn as lsn_sanity_check,
source_lsn > '0/0' as source_lsn_available, source_lsn > '0/0' as source_lsn_available,
target_lsn > '0/0' as target_lsn_available target_lsn > '0/0' as target_lsn_available,
status
FROM get_rebalance_progress(); FROM get_rebalance_progress();
table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available|status
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (0 rows)
@ -227,15 +231,16 @@ step s7-get-progress:
operation_type, operation_type,
source_lsn >= target_lsn as lsn_sanity_check, source_lsn >= target_lsn as lsn_sanity_check,
source_lsn > '0/0' as source_lsn_available, source_lsn > '0/0' as source_lsn_available,
target_lsn > '0/0' as target_lsn_available target_lsn > '0/0' as target_lsn_available,
status
FROM get_rebalance_progress(); FROM get_rebalance_progress();
table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available|status
--------------------------------------------------------------------- ---------------------------------------------------------------------
colocated1|1500001| 50000|localhost | 57637| 50000|localhost | 57638| 50000| 1|move |t |t |f colocated1|1500001| 50000|localhost | 57637| 50000|localhost | 57638| 50000| 1|move |t |t |f |Completing
colocated2|1500005| 400000|localhost | 57637| 400000|localhost | 57638| 400000| 1|move |t |t |f colocated2|1500005| 400000|localhost | 57637| 400000|localhost | 57638| 400000| 1|move |t |t |f |Completing
colocated1|1500002| 200000|localhost | 57637| 200000|localhost | 57638| 0| 0|move |t |t |f colocated1|1500002| 200000|localhost | 57637| 200000|localhost | 57638| 0| 0|move |t |t |f |Not Started Yet
colocated2|1500006| 8000|localhost | 57637| 8000|localhost | 57638| 0| 0|move |t |t |f colocated2|1500006| 8000|localhost | 57637| 8000|localhost | 57638| 0| 0|move |t |t |f |Not Started Yet
(4 rows) (4 rows)
step s7-release-lock: step s7-release-lock:
@ -275,10 +280,11 @@ step s7-get-progress:
operation_type, operation_type,
source_lsn >= target_lsn as lsn_sanity_check, source_lsn >= target_lsn as lsn_sanity_check,
source_lsn > '0/0' as source_lsn_available, source_lsn > '0/0' as source_lsn_available,
target_lsn > '0/0' as target_lsn_available target_lsn > '0/0' as target_lsn_available,
status
FROM get_rebalance_progress(); FROM get_rebalance_progress();
table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available|status
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (0 rows)
@ -322,15 +328,16 @@ step s7-get-progress:
operation_type, operation_type,
source_lsn >= target_lsn as lsn_sanity_check, source_lsn >= target_lsn as lsn_sanity_check,
source_lsn > '0/0' as source_lsn_available, source_lsn > '0/0' as source_lsn_available,
target_lsn > '0/0' as target_lsn_available target_lsn > '0/0' as target_lsn_available,
status
FROM get_rebalance_progress(); FROM get_rebalance_progress();
table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available|status
--------------------------------------------------------------------- ---------------------------------------------------------------------
colocated1|1500001| 50000|localhost | 57637| 50000|localhost | 57638| 8000| 1|move |t |t |f colocated1|1500001| 50000|localhost | 57637| 50000|localhost | 57638| 8000| 1|move |t |t |f |Setting Up
colocated2|1500005| 400000|localhost | 57637| 400000|localhost | 57638| 8000| 1|move |t |t |f colocated2|1500005| 400000|localhost | 57637| 400000|localhost | 57638| 8000| 1|move |t |t |f |Setting Up
colocated1|1500002| 200000|localhost | 57637| 200000|localhost | 57638| 0| 0|move |t |t |f colocated1|1500002| 200000|localhost | 57637| 200000|localhost | 57638| 0| 0|move |t |t |f |Not Started Yet
colocated2|1500006| 8000|localhost | 57637| 8000|localhost | 57638| 0| 0|move |t |t |f colocated2|1500006| 8000|localhost | 57637| 8000|localhost | 57638| 0| 0|move |t |t |f |Not Started Yet
(4 rows) (4 rows)
step s6-release-advisory-lock: step s6-release-advisory-lock:
@ -375,10 +382,11 @@ step s7-get-progress:
operation_type, operation_type,
source_lsn >= target_lsn as lsn_sanity_check, source_lsn >= target_lsn as lsn_sanity_check,
source_lsn > '0/0' as source_lsn_available, source_lsn > '0/0' as source_lsn_available,
target_lsn > '0/0' as target_lsn_available target_lsn > '0/0' as target_lsn_available,
status
FROM get_rebalance_progress(); FROM get_rebalance_progress();
table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available|status
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (0 rows)
@ -418,13 +426,14 @@ step s7-get-progress:
operation_type, operation_type,
source_lsn >= target_lsn as lsn_sanity_check, source_lsn >= target_lsn as lsn_sanity_check,
source_lsn > '0/0' as source_lsn_available, source_lsn > '0/0' as source_lsn_available,
target_lsn > '0/0' as target_lsn_available target_lsn > '0/0' as target_lsn_available,
status
FROM get_rebalance_progress(); FROM get_rebalance_progress();
table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available|status
--------------------------------------------------------------------- ---------------------------------------------------------------------
colocated1|1500001| 50000|localhost | 57637| 50000|localhost | 57638| 0| 1|move |t |t |f colocated1|1500001| 50000|localhost | 57637| 50000|localhost | 57638| 0| 1|move |t |t |f |Setting Up
colocated2|1500005| 400000|localhost | 57637| 400000|localhost | 57638| 0| 1|move |t |t |f colocated2|1500005| 400000|localhost | 57637| 400000|localhost | 57638| 0| 1|move |t |t |f |Setting Up
(2 rows) (2 rows)
step s2-unlock-1-start: step s2-unlock-1-start:
@ -456,10 +465,11 @@ step s7-get-progress:
operation_type, operation_type,
source_lsn >= target_lsn as lsn_sanity_check, source_lsn >= target_lsn as lsn_sanity_check,
source_lsn > '0/0' as source_lsn_available, source_lsn > '0/0' as source_lsn_available,
target_lsn > '0/0' as target_lsn_available target_lsn > '0/0' as target_lsn_available,
status
FROM get_rebalance_progress(); FROM get_rebalance_progress();
table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available|status
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (0 rows)
@ -510,13 +520,14 @@ step s7-get-progress:
operation_type, operation_type,
source_lsn >= target_lsn as lsn_sanity_check, source_lsn >= target_lsn as lsn_sanity_check,
source_lsn > '0/0' as source_lsn_available, source_lsn > '0/0' as source_lsn_available,
target_lsn > '0/0' as target_lsn_available target_lsn > '0/0' as target_lsn_available,
status
FROM get_rebalance_progress(); FROM get_rebalance_progress();
table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available|status
--------------------------------------------------------------------- ---------------------------------------------------------------------
colocated1|1500001| 50000|localhost | 57637| 50000|localhost | 57638| 50000| 1|move |t |t |f colocated1|1500001| 50000|localhost | 57637| 50000|localhost | 57638| 50000| 1|move |t |t |f |Completing
colocated2|1500005| 400000|localhost | 57637| 400000|localhost | 57638| 400000| 1|move |t |t |f colocated2|1500005| 400000|localhost | 57637| 400000|localhost | 57638| 400000| 1|move |t |t |f |Completing
(2 rows) (2 rows)
step s7-release-lock: step s7-release-lock:
@ -548,10 +559,11 @@ step s7-get-progress:
operation_type, operation_type,
source_lsn >= target_lsn as lsn_sanity_check, source_lsn >= target_lsn as lsn_sanity_check,
source_lsn > '0/0' as source_lsn_available, source_lsn > '0/0' as source_lsn_available,
target_lsn > '0/0' as target_lsn_available target_lsn > '0/0' as target_lsn_available,
status
FROM get_rebalance_progress(); FROM get_rebalance_progress();
table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available|status
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (0 rows)
@ -592,13 +604,14 @@ step s7-get-progress:
operation_type, operation_type,
source_lsn >= target_lsn as lsn_sanity_check, source_lsn >= target_lsn as lsn_sanity_check,
source_lsn > '0/0' as source_lsn_available, source_lsn > '0/0' as source_lsn_available,
target_lsn > '0/0' as target_lsn_available target_lsn > '0/0' as target_lsn_available,
status
FROM get_rebalance_progress(); FROM get_rebalance_progress();
table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available|status
--------------------------------------------------------------------- ---------------------------------------------------------------------
colocated1|1500001| 50000|localhost | 57637| 50000|localhost | 57638| 0| 1|copy |t |t |f colocated1|1500001| 50000|localhost | 57637| 50000|localhost | 57638| 0| 1|copy |t |t |f |Setting Up
colocated2|1500005| 400000|localhost | 57637| 400000|localhost | 57638| 0| 1|copy |t |t |f colocated2|1500005| 400000|localhost | 57637| 400000|localhost | 57638| 0| 1|copy |t |t |f |Setting Up
(2 rows) (2 rows)
step s2-unlock-1-start: step s2-unlock-1-start:
@ -649,13 +662,14 @@ step s7-get-progress:
operation_type, operation_type,
source_lsn >= target_lsn as lsn_sanity_check, source_lsn >= target_lsn as lsn_sanity_check,
source_lsn > '0/0' as source_lsn_available, source_lsn > '0/0' as source_lsn_available,
target_lsn > '0/0' as target_lsn_available target_lsn > '0/0' as target_lsn_available,
status
FROM get_rebalance_progress(); FROM get_rebalance_progress();
table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available|status
--------------------------------------------------------------------- ---------------------------------------------------------------------
colocated1|1500001| 50000|localhost | 57637| 50000|localhost | 57638| 8000| 1|move |t |t |f colocated1|1500001| 50000|localhost | 57637| 50000|localhost | 57638| 8000| 1|move |t |t |f |Setting Up
colocated2|1500005| 400000|localhost | 57637| 400000|localhost | 57638| 8000| 1|move |t |t |f colocated2|1500005| 400000|localhost | 57637| 400000|localhost | 57638| 8000| 1|move |t |t |f |Setting Up
(2 rows) (2 rows)
step s6-release-advisory-lock: step s6-release-advisory-lock:
@ -692,10 +706,11 @@ step s7-get-progress:
operation_type, operation_type,
source_lsn >= target_lsn as lsn_sanity_check, source_lsn >= target_lsn as lsn_sanity_check,
source_lsn > '0/0' as source_lsn_available, source_lsn > '0/0' as source_lsn_available,
target_lsn > '0/0' as target_lsn_available target_lsn > '0/0' as target_lsn_available,
status
FROM get_rebalance_progress(); FROM get_rebalance_progress();
table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available|status
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (0 rows)
@ -739,13 +754,14 @@ step s7-get-progress:
operation_type, operation_type,
source_lsn >= target_lsn as lsn_sanity_check, source_lsn >= target_lsn as lsn_sanity_check,
source_lsn > '0/0' as source_lsn_available, source_lsn > '0/0' as source_lsn_available,
target_lsn > '0/0' as target_lsn_available target_lsn > '0/0' as target_lsn_available,
status
FROM get_rebalance_progress(); FROM get_rebalance_progress();
table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available|status
--------------------------------------------------------------------- ---------------------------------------------------------------------
colocated1|1500001| 50000|localhost | 57637| 50000|localhost | 57638| 8000| 1|copy |t |t |f colocated1|1500001| 50000|localhost | 57637| 50000|localhost | 57638| 8000| 1|copy |t |t |f |Setting Up
colocated2|1500005| 400000|localhost | 57637| 400000|localhost | 57638| 8000| 1|copy |t |t |f colocated2|1500005| 400000|localhost | 57637| 400000|localhost | 57638| 8000| 1|copy |t |t |f |Setting Up
(2 rows) (2 rows)
step s6-release-advisory-lock: step s6-release-advisory-lock:
@ -802,14 +818,15 @@ step s7-get-progress:
operation_type, operation_type,
source_lsn >= target_lsn as lsn_sanity_check, source_lsn >= target_lsn as lsn_sanity_check,
source_lsn > '0/0' as source_lsn_available, source_lsn > '0/0' as source_lsn_available,
target_lsn > '0/0' as target_lsn_available target_lsn > '0/0' as target_lsn_available,
status
FROM get_rebalance_progress(); FROM get_rebalance_progress();
table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available|status
--------------------------------------------------------------------- ---------------------------------------------------------------------
colocated1|1500001| 50000|localhost | 57637| 50000|localhost | 57638| 0| 1|move |t |t |f colocated1|1500001| 50000|localhost | 57637| 50000|localhost | 57638| 0| 1|move |t |t |f |Setting Up
colocated2|1500005| 400000|localhost | 57637| 400000|localhost | 57638| 0| 1|move |t |t |f colocated2|1500005| 400000|localhost | 57637| 400000|localhost | 57638| 0| 1|move |t |t |f |Setting Up
separate |1500009| 50000|localhost | 57637| 50000|localhost | 57638| 0| 1|move |t |t |f separate |1500009| 50000|localhost | 57637| 50000|localhost | 57638| 0| 1|move |t |t |f |Setting Up
(3 rows) (3 rows)
step s2-unlock-1-start: step s2-unlock-1-start:
@ -850,10 +867,11 @@ step s7-get-progress:
operation_type, operation_type,
source_lsn >= target_lsn as lsn_sanity_check, source_lsn >= target_lsn as lsn_sanity_check,
source_lsn > '0/0' as source_lsn_available, source_lsn > '0/0' as source_lsn_available,
target_lsn > '0/0' as target_lsn_available target_lsn > '0/0' as target_lsn_available,
status
FROM get_rebalance_progress(); FROM get_rebalance_progress();
table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available|status
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (0 rows)
@ -908,14 +926,15 @@ step s7-get-progress:
operation_type, operation_type,
source_lsn >= target_lsn as lsn_sanity_check, source_lsn >= target_lsn as lsn_sanity_check,
source_lsn > '0/0' as source_lsn_available, source_lsn > '0/0' as source_lsn_available,
target_lsn > '0/0' as target_lsn_available target_lsn > '0/0' as target_lsn_available,
status
FROM get_rebalance_progress(); FROM get_rebalance_progress();
table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available|status
--------------------------------------------------------------------- ---------------------------------------------------------------------
colocated1|1500001| 50000|localhost | 57637| 50000|localhost | 57638| 50000| 1|move |t |t |f colocated1|1500001| 50000|localhost | 57637| 50000|localhost | 57638| 50000| 1|move |t |t |f |Completing
colocated2|1500005| 400000|localhost | 57637| 400000|localhost | 57638| 400000| 1|move |t |t |f colocated2|1500005| 400000|localhost | 57637| 400000|localhost | 57638| 400000| 1|move |t |t |f |Completing
separate |1500009| 50000|localhost | 57637| 50000|localhost | 57638| 200000| 1|move |t |t |f separate |1500009| 50000|localhost | 57637| 50000|localhost | 57638| 200000| 1|move |t |t |f |Completing
(3 rows) (3 rows)
step s7-release-lock: step s7-release-lock:
@ -956,10 +975,11 @@ step s7-get-progress:
operation_type, operation_type,
source_lsn >= target_lsn as lsn_sanity_check, source_lsn >= target_lsn as lsn_sanity_check,
source_lsn > '0/0' as source_lsn_available, source_lsn > '0/0' as source_lsn_available,
target_lsn > '0/0' as target_lsn_available target_lsn > '0/0' as target_lsn_available,
status
FROM get_rebalance_progress(); FROM get_rebalance_progress();
table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress|operation_type|lsn_sanity_check|source_lsn_available|target_lsn_available|status
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (0 rows)

View File

@ -1202,7 +1202,7 @@ SELECT * FROM multi_extension.print_extension_changes();
previous_object | current_object previous_object | current_object
--------------------------------------------------------------------- ---------------------------------------------------------------------
function get_rebalance_progress() TABLE(sessionid integer, table_name regclass, shardid bigint, shard_size bigint, sourcename text, sourceport integer, targetname text, targetport integer, progress bigint, source_shard_size bigint, target_shard_size bigint, operation_type text) | function get_rebalance_progress() TABLE(sessionid integer, table_name regclass, shardid bigint, shard_size bigint, sourcename text, sourceport integer, targetname text, targetport integer, progress bigint, source_shard_size bigint, target_shard_size bigint, operation_type text) |
| function get_rebalance_progress() TABLE(sessionid integer, table_name regclass, shardid bigint, shard_size bigint, sourcename text, sourceport integer, targetname text, targetport integer, progress bigint, source_shard_size bigint, target_shard_size bigint, operation_type text, source_lsn pg_lsn, target_lsn pg_lsn) | function get_rebalance_progress() TABLE(sessionid integer, table_name regclass, shardid bigint, shard_size bigint, sourcename text, sourceport integer, targetname text, targetport integer, progress bigint, source_shard_size bigint, target_shard_size bigint, operation_type text, source_lsn pg_lsn, target_lsn pg_lsn, status text)
(2 rows) (2 rows)
DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff; DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff;

View File

@ -1044,7 +1044,7 @@ SELECT * FROM get_rebalance_table_shards_plan('colocated_rebalance_test', rebala
-- Check that we can call this function -- Check that we can call this function
SELECT * FROM get_rebalance_progress(); 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 sessionid | table_name | shardid | shard_size | sourcename | sourceport | targetname | targetport | progress | source_shard_size | target_shard_size | operation_type | source_lsn | target_lsn | status
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (0 rows)
@ -1058,7 +1058,7 @@ SELECT * FROM rebalance_table_shards('colocated_rebalance_test', threshold := 0,
CALL citus_cleanup_orphaned_shards(); CALL citus_cleanup_orphaned_shards();
-- Check that we can call this function without a crash -- Check that we can call this function without a crash
SELECT * FROM get_rebalance_progress(); 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 sessionid | table_name | shardid | shard_size | sourcename | sourceport | targetname | targetport | progress | source_shard_size | target_shard_size | operation_type | source_lsn | target_lsn | status
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (0 rows)

View File

@ -175,7 +175,8 @@ step "s7-get-progress"
operation_type, operation_type,
source_lsn >= target_lsn as lsn_sanity_check, source_lsn >= target_lsn as lsn_sanity_check,
source_lsn > '0/0' as source_lsn_available, source_lsn > '0/0' as source_lsn_available,
target_lsn > '0/0' as target_lsn_available target_lsn > '0/0' as target_lsn_available,
status
FROM get_rebalance_progress(); FROM get_rebalance_progress();
} }