Remove do_repair option from citus_copy_shard_placement (#6299)

Co-authored-by: Marco Slot <marco.slot@gmail.com>
pull/6310/head
Marco Slot 2022-09-09 15:44:30 +02:00 committed by GitHub
parent 00a94c7f13
commit ba2fe3e3c4
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
57 changed files with 481 additions and 1652 deletions

View File

@ -60,10 +60,10 @@
#include "distributed/reference_table_utils.h"
#include "distributed/relation_access_tracking.h"
#include "distributed/remote_commands.h"
#include "distributed/repair_shards.h"
#include "distributed/resource_lock.h"
#include "distributed/shard_rebalancer.h"
#include "distributed/shard_split.h"
#include "distributed/shard_transfer.h"
#include "distributed/shared_library_init.h"
#include "distributed/shard_rebalancer.h"
#include "distributed/worker_protocol.h"

View File

@ -43,10 +43,10 @@
#include "distributed/pg_dist_rebalance_strategy.h"
#include "distributed/reference_table_utils.h"
#include "distributed/remote_commands.h"
#include "distributed/repair_shards.h"
#include "distributed/resource_lock.h"
#include "distributed/shard_rebalancer.h"
#include "distributed/shard_cleaner.h"
#include "distributed/shard_transfer.h"
#include "distributed/tuplestore.h"
#include "distributed/utils/array_type.h"
#include "distributed/worker_protocol.h"
@ -1621,7 +1621,6 @@ UpdateShardPlacement(PlacementUpdateEvent *placementUpdateEvent,
uint64 shardId = placementUpdateEvent->shardId;
WorkerNode *sourceNode = placementUpdateEvent->sourceNode;
WorkerNode *targetNode = placementUpdateEvent->targetNode;
const char *doRepair = "false";
Datum shardTranferModeLabelDatum =
DirectFunctionCall1(enum_out, shardReplicationModeOid);
@ -1665,13 +1664,12 @@ UpdateShardPlacement(PlacementUpdateEvent *placementUpdateEvent,
else if (updateType == PLACEMENT_UPDATE_COPY)
{
appendStringInfo(placementUpdateCommand,
"SELECT citus_copy_shard_placement(%ld,%s,%u,%s,%u,%s,%s)",
"SELECT citus_copy_shard_placement(%ld,%s,%u,%s,%u,%s)",
shardId,
quote_literal_cstr(sourceNode->workerName),
sourceNode->workerPort,
quote_literal_cstr(targetNode->workerName),
targetNode->workerPort,
doRepair,
quote_literal_cstr(shardTranferModeLabel));
}
else

View File

@ -29,7 +29,7 @@
#include "distributed/remote_commands.h"
#include "distributed/shard_split.h"
#include "distributed/reference_table_utils.h"
#include "distributed/repair_shards.h"
#include "distributed/shard_transfer.h"
#include "distributed/resource_lock.h"
#include "distributed/multi_partitioning_utils.h"
#include "distributed/worker_manager.h"
@ -233,9 +233,7 @@ ErrorIfCannotSplitShard(SplitOperation splitOperation, ShardInterval *sourceShar
"for the shard %lu",
SplitOperationName[splitOperation],
SplitTargetName[splitOperation],
relationName, shardId),
errhint("Use master_copy_shard_placement UDF to "
"repair the inactive shard placement.")));
relationName, shardId)));
}
}
}

View File

@ -1,9 +1,8 @@
/*-------------------------------------------------------------------------
*
* repair_shards.c
* shard_transfer.c
*
* This file contains functions to repair unhealthy shard placements using data
* from healthy ones.
* This file contains functions to transfer shards between nodes.
*
* Copyright (c) Citus Data, Inc.
*
@ -31,7 +30,6 @@
#include "distributed/listutils.h"
#include "distributed/shard_cleaner.h"
#include "distributed/coordinator_protocol.h"
#include "distributed/repair_shards.h"
#include "distributed/metadata_cache.h"
#include "distributed/metadata_sync.h"
#include "distributed/multi_join_order.h"
@ -43,6 +41,7 @@
#include "distributed/resource_lock.h"
#include "distributed/shard_rebalancer.h"
#include "distributed/shard_split.h"
#include "distributed/shard_transfer.h"
#include "distributed/worker_manager.h"
#include "distributed/worker_protocol.h"
#include "distributed/worker_transaction.h"
@ -76,9 +75,6 @@ static bool CanUseLogicalReplication(Oid relationId, char shardReplicationMode);
static void ErrorIfTableCannotBeReplicated(Oid relationId);
static void ErrorIfTargetNodeIsNotSafeToCopyTo(const char *targetNodeName,
int targetNodePort);
static void RepairShardPlacement(int64 shardId, const char *sourceNodeName,
int32 sourceNodePort, const char *targetNodeName,
int32 targetNodePort);
static void ReplicateColocatedShardPlacement(int64 shardId, char *sourceNodeName,
int32 sourceNodePort, char *targetNodeName,
int32 targetNodePort,
@ -95,12 +91,6 @@ static void CopyShardTablesViaLogicalReplication(List *shardIntervalList,
static void CopyShardTablesViaBlockWrites(List *shardIntervalList, char *sourceNodeName,
int32 sourceNodePort,
char *targetNodeName, int32 targetNodePort);
static List * CopyPartitionShardsCommandList(ShardInterval *shardInterval,
const char *sourceNodeName,
int32 sourceNodePort);
static void EnsureShardCanBeRepaired(int64 shardId, const char *sourceNodeName,
int32 sourceNodePort, const char *targetNodeName,
int32 targetNodePort);
static void EnsureShardCanBeCopied(int64 shardId, const char *sourceNodeName,
int32 sourceNodePort, const char *targetNodeName,
int32 targetNodePort);
@ -150,14 +140,8 @@ bool CheckAvailableSpaceBeforeMove = true;
/*
* citus_copy_shard_placement implements a user-facing UDF to repair data from
* a healthy (source) node to an inactive (target) node. To accomplish this it
* entirely recreates the table structure before copying all data. During this
* time all modifications are paused to the shard. After successful repair, the
* inactive placement is marked healthy and modifications may continue. If the
* repair fails at any point, this function throws an error, leaving the node
* in an unhealthy state. Please note that citus_copy_shard_placement copies
* given shard along with its co-located shards.
* citus_copy_shard_placement implements a user-facing UDF to copy a placement
* from a source node to a target node, including all co-located placements.
*/
Datum
citus_copy_shard_placement(PG_FUNCTION_ARGS)
@ -165,6 +149,35 @@ citus_copy_shard_placement(PG_FUNCTION_ARGS)
CheckCitusVersion(ERROR);
EnsureCoordinator();
int64 shardId = PG_GETARG_INT64(0);
text *sourceNodeNameText = PG_GETARG_TEXT_P(1);
int32 sourceNodePort = PG_GETARG_INT32(2);
text *targetNodeNameText = PG_GETARG_TEXT_P(3);
int32 targetNodePort = PG_GETARG_INT32(4);
Oid shardReplicationModeOid = PG_GETARG_OID(5);
char *sourceNodeName = text_to_cstring(sourceNodeNameText);
char *targetNodeName = text_to_cstring(targetNodeNameText);
char shardReplicationMode = LookupShardTransferMode(shardReplicationModeOid);
ReplicateColocatedShardPlacement(shardId, sourceNodeName, sourceNodePort,
targetNodeName, targetNodePort,
shardReplicationMode);
PG_RETURN_VOID();
}
/*
* master_copy_shard_placement is a wrapper function for old UDF name.
*/
Datum
master_copy_shard_placement(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
EnsureCoordinator();
int64 shardId = PG_GETARG_INT64(0);
text *sourceNodeNameText = PG_GETARG_TEXT_P(1);
int32 sourceNodePort = PG_GETARG_INT32(2);
@ -177,137 +190,21 @@ citus_copy_shard_placement(PG_FUNCTION_ARGS)
char *targetNodeName = text_to_cstring(targetNodeNameText);
char shardReplicationMode = LookupShardTransferMode(shardReplicationModeOid);
if (doRepair && shardReplicationMode == TRANSFER_MODE_FORCE_LOGICAL)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("logical replication cannot be used for repairs")));
}
ShardInterval *shardInterval = LoadShardInterval(shardId);
ErrorIfTableCannotBeReplicated(shardInterval->relationId);
ErrorIfTargetNodeIsNotSafeToCopyTo(targetNodeName, targetNodePort);
AcquirePlacementColocationLock(shardInterval->relationId, ExclusiveLock,
doRepair ? "repair" : "copy");
if (doRepair)
{
RepairShardPlacement(shardId, sourceNodeName, sourceNodePort, targetNodeName,
targetNodePort);
}
else
{
ReplicateColocatedShardPlacement(shardId, sourceNodeName, sourceNodePort,
targetNodeName, targetNodePort,
shardReplicationMode);
ereport(WARNING, (errmsg("do_repair argument is deprecated")));
}
ReplicateColocatedShardPlacement(shardId, sourceNodeName, sourceNodePort,
targetNodeName, targetNodePort,
shardReplicationMode);
PG_RETURN_VOID();
}
/*
* master_copy_shard_placement is a wrapper function for old UDF name.
*/
Datum
master_copy_shard_placement(PG_FUNCTION_ARGS)
{
return citus_copy_shard_placement(fcinfo);
}
/*
* ShardListSizeInBytes returns the size in bytes of a set of shard tables.
*/
uint64
ShardListSizeInBytes(List *shardList, char *workerNodeName, uint32
workerNodePort)
{
uint32 connectionFlag = 0;
/* we skip child tables of a partitioned table if this boolean variable is true */
bool optimizePartitionCalculations = true;
StringInfo tableSizeQuery = GenerateSizeQueryOnMultiplePlacements(shardList,
TOTAL_RELATION_SIZE,
optimizePartitionCalculations);
MultiConnection *connection = GetNodeConnection(connectionFlag, workerNodeName,
workerNodePort);
PGresult *result = NULL;
int queryResult = ExecuteOptionalRemoteCommand(connection, tableSizeQuery->data,
&result);
if (queryResult != RESPONSE_OKAY)
{
ereport(ERROR, (errcode(ERRCODE_CONNECTION_FAILURE),
errmsg("cannot get the size because of a connection error")));
}
List *sizeList = ReadFirstColumnAsText(result);
if (list_length(sizeList) != 1)
{
ereport(ERROR, (errmsg(
"received wrong number of rows from worker, expected 1 received %d",
list_length(sizeList))));
}
StringInfo totalSizeStringInfo = (StringInfo) linitial(sizeList);
char *totalSizeString = totalSizeStringInfo->data;
uint64 totalSize = SafeStringToUint64(totalSizeString);
PQclear(result);
ForgetResults(connection);
return totalSize;
}
/*
* CheckSpaceConstraints checks there is enough space to place the colocation
* on the node that the connection is connected to.
*/
static void
CheckSpaceConstraints(MultiConnection *connection, uint64 colocationSizeInBytes)
{
uint64 diskAvailableInBytes = 0;
uint64 diskSizeInBytes = 0;
bool success =
GetNodeDiskSpaceStatsForConnection(connection, &diskAvailableInBytes,
&diskSizeInBytes);
if (!success)
{
ereport(ERROR, (errmsg("Could not fetch disk stats for node: %s-%d",
connection->hostname, connection->port)));
}
uint64 diskAvailableInBytesAfterShardMove = 0;
if (diskAvailableInBytes < colocationSizeInBytes)
{
/*
* even though the space will be less than "0", we set it to 0 for convenience.
*/
diskAvailableInBytes = 0;
}
else
{
diskAvailableInBytesAfterShardMove = diskAvailableInBytes - colocationSizeInBytes;
}
uint64 desiredNewDiskAvailableInBytes = diskSizeInBytes *
(DesiredPercentFreeAfterMove / 100);
if (diskAvailableInBytesAfterShardMove < desiredNewDiskAvailableInBytes)
{
ereport(ERROR, (errmsg("not enough empty space on node if the shard is moved, "
"actual available space after move will be %ld bytes, "
"desired available space after move is %ld bytes,"
"estimated size increase on node after move is %ld bytes.",
diskAvailableInBytesAfterShardMove,
desiredNewDiskAvailableInBytes, colocationSizeInBytes),
errhint(
"consider lowering citus.desired_percent_disk_available_after_move.")));
}
}
/*
* citus_move_shard_placement moves given shard (and its co-located shards) from one
* node to the other node. To accomplish this it entirely recreates the table structure
@ -370,8 +267,8 @@ citus_move_shard_placement(PG_FUNCTION_ARGS)
{
char *relationName = get_rel_name(colocatedTableId);
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot repair shard"),
errdetail("Table %s is a foreign table. Repairing "
errmsg("cannot move shard"),
errdetail("Table %s is a foreign table. Moving "
"shards backed by foreign tables is "
"not supported.", relationName)));
}
@ -518,6 +415,98 @@ EnsureEnoughDiskSpaceForShardMove(List *colocatedShardList,
}
/*
* ShardListSizeInBytes returns the size in bytes of a set of shard tables.
*/
uint64
ShardListSizeInBytes(List *shardList, char *workerNodeName, uint32
workerNodePort)
{
uint32 connectionFlag = 0;
/* we skip child tables of a partitioned table if this boolean variable is true */
bool optimizePartitionCalculations = true;
StringInfo tableSizeQuery = GenerateSizeQueryOnMultiplePlacements(shardList,
TOTAL_RELATION_SIZE,
optimizePartitionCalculations);
MultiConnection *connection = GetNodeConnection(connectionFlag, workerNodeName,
workerNodePort);
PGresult *result = NULL;
int queryResult = ExecuteOptionalRemoteCommand(connection, tableSizeQuery->data,
&result);
if (queryResult != RESPONSE_OKAY)
{
ereport(ERROR, (errcode(ERRCODE_CONNECTION_FAILURE),
errmsg("cannot get the size because of a connection error")));
}
List *sizeList = ReadFirstColumnAsText(result);
if (list_length(sizeList) != 1)
{
ereport(ERROR, (errmsg(
"received wrong number of rows from worker, expected 1 received %d",
list_length(sizeList))));
}
StringInfo totalSizeStringInfo = (StringInfo) linitial(sizeList);
char *totalSizeString = totalSizeStringInfo->data;
uint64 totalSize = SafeStringToUint64(totalSizeString);
PQclear(result);
ForgetResults(connection);
return totalSize;
}
/*
* CheckSpaceConstraints checks there is enough space to place the colocation
* on the node that the connection is connected to.
*/
static void
CheckSpaceConstraints(MultiConnection *connection, uint64 colocationSizeInBytes)
{
uint64 diskAvailableInBytes = 0;
uint64 diskSizeInBytes = 0;
bool success =
GetNodeDiskSpaceStatsForConnection(connection, &diskAvailableInBytes,
&diskSizeInBytes);
if (!success)
{
ereport(ERROR, (errmsg("Could not fetch disk stats for node: %s-%d",
connection->hostname, connection->port)));
}
uint64 diskAvailableInBytesAfterShardMove = 0;
if (diskAvailableInBytes < colocationSizeInBytes)
{
/*
* even though the space will be less than "0", we set it to 0 for convenience.
*/
diskAvailableInBytes = 0;
}
else
{
diskAvailableInBytesAfterShardMove = diskAvailableInBytes - colocationSizeInBytes;
}
uint64 desiredNewDiskAvailableInBytes = diskSizeInBytes *
(DesiredPercentFreeAfterMove / 100);
if (diskAvailableInBytesAfterShardMove < desiredNewDiskAvailableInBytes)
{
ereport(ERROR, (errmsg("not enough empty space on node if the shard is moved, "
"actual available space after move will be %ld bytes, "
"desired available space after move is %ld bytes,"
"estimated size increase on node after move is %ld bytes.",
diskAvailableInBytesAfterShardMove,
desiredNewDiskAvailableInBytes, colocationSizeInBytes),
errhint(
"consider lowering citus.desired_percent_disk_available_after_move.")));
}
}
/*
* ErrorIfTargetNodeIsNotSafeToMove throws error if the target node is not
* eligible for moving shards.
@ -885,122 +874,6 @@ LookupShardTransferMode(Oid shardReplicationModeOid)
}
/*
* RepairShardPlacement repairs given shard from a source node to target node.
* This function is not co-location aware. It only repairs given shard.
*/
static void
RepairShardPlacement(int64 shardId, const char *sourceNodeName, int32 sourceNodePort,
const char *targetNodeName, int32 targetNodePort)
{
ShardInterval *shardInterval = LoadShardInterval(shardId);
Oid distributedTableId = shardInterval->relationId;
char *tableOwner = TableOwner(shardInterval->relationId);
/* prevent table from being dropped */
LockRelationOid(distributedTableId, AccessShareLock);
EnsureTableOwner(distributedTableId);
if (IsForeignTable(distributedTableId))
{
char *relationName = get_rel_name(distributedTableId);
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot repair shard"),
errdetail("Table %s is a foreign table. Repairing "
"shards backed by foreign tables is "
"not supported.", relationName)));
}
/*
* Let's not allow repairing partitions to prevent any edge cases.
* We're already not allowing any kind of modifications on the partitions
* so their placements are not likely to be marked as INVALID. The only
* possible case to mark placement of a partition as invalid is
* "ALTER TABLE parent_table DETACH PARTITION partition_table". But,
* given that the table would become a regular distributed table if the
* command succeeds, we're OK since the regular distributed tables can
* be repaired later on.
*/
EnsurePartitionTableNotReplicated(distributedTableId);
/*
* We take a lock on the referenced table if there is a foreign constraint
* during the copy procedure. If we do not block DMLs on the referenced
* table, we cannot avoid the inconsistency between the two copies of the
* data. Currently, we do not support replication factor > 1 on the tables
* with foreign constraints, so this command will fail for this case anyway.
* However, it is taken as a precaution in case we support it one day.
*/
LockReferencedReferenceShardDistributionMetadata(shardId, ExclusiveLock);
/*
* We plan to move the placement to the healthy state, so we need to grab a shard
* metadata lock (in exclusive mode).
*/
LockShardDistributionMetadata(shardId, ExclusiveLock);
/*
* For shard repair, there should be healthy placement in source node and unhealthy
* placement in the target node.
*/
EnsureShardCanBeRepaired(shardId, sourceNodeName, sourceNodePort, targetNodeName,
targetNodePort);
/*
* If the shard belongs to a partitioned table, we need to load the data after
* creating the partitions and the partitioning hierarcy.
*/
bool partitionedTable = PartitionedTableNoLock(distributedTableId);
bool includeData = !partitionedTable;
/* we generate necessary commands to recreate the shard in target node */
List *ddlCommandList =
CopyShardCommandList(shardInterval, sourceNodeName, sourceNodePort, includeData);
List *foreignConstraintCommandList = CopyShardForeignConstraintCommandList(
shardInterval);
ddlCommandList = list_concat(ddlCommandList, foreignConstraintCommandList);
/*
* CopyShardCommandList() drops the table which cascades to partitions if the
* table is a partitioned table. This means that we need to create both parent
* table and its partitions.
*
* We also skipped copying the data, so include it here.
*/
if (partitionedTable)
{
char *shardName = ConstructQualifiedShardName(shardInterval);
StringInfo copyShardDataCommand = makeStringInfo();
List *partitionCommandList =
CopyPartitionShardsCommandList(shardInterval, sourceNodeName, sourceNodePort);
ddlCommandList = list_concat(ddlCommandList, partitionCommandList);
/* finally copy the data as well */
appendStringInfo(copyShardDataCommand, WORKER_APPEND_TABLE_TO_SHARD,
quote_literal_cstr(shardName), /* table to append */
quote_literal_cstr(shardName), /* remote table name */
quote_literal_cstr(sourceNodeName), /* remote host */
sourceNodePort); /* remote port */
ddlCommandList = lappend(ddlCommandList, copyShardDataCommand->data);
}
EnsureNoModificationsHaveBeenDone();
SendCommandListToWorkerOutsideTransaction(targetNodeName, targetNodePort, tableOwner,
ddlCommandList);
/* after successful repair, we update shard state as healthy*/
List *placementList = ShardPlacementListWithoutOrphanedPlacements(shardId);
ShardPlacement *placement = SearchShardPlacementInListOrError(placementList,
targetNodeName,
targetNodePort);
UpdateShardPlacementState(placement->placementId, SHARD_STATE_ACTIVE);
}
/*
* ReplicateColocatedShardPlacement replicates the given shard and its
* colocated shards from a source node to target node.
@ -1013,6 +886,12 @@ ReplicateColocatedShardPlacement(int64 shardId, char *sourceNodeName,
ShardInterval *shardInterval = LoadShardInterval(shardId);
Oid distributedTableId = shardInterval->relationId;
ErrorIfTableCannotBeReplicated(shardInterval->relationId);
ErrorIfTargetNodeIsNotSafeToCopyTo(targetNodeName, targetNodePort);
EnsureNoModificationsHaveBeenDone();
AcquirePlacementColocationLock(shardInterval->relationId, ExclusiveLock, "copy");
List *colocatedTableList = ColocatedTableList(distributedTableId);
List *colocatedShardList = ColocatedShardIntervalList(shardInterval);
@ -1439,88 +1318,6 @@ CreateShardCopyCommand(ShardInterval *shard,
}
/*
* CopyPartitionShardsCommandList gets a shardInterval which is a shard that
* belongs to partitioned table (this is asserted).
*
* The function returns a list of commands which re-creates all the partitions
* of the input shardInterval.
*/
static List *
CopyPartitionShardsCommandList(ShardInterval *shardInterval, const char *sourceNodeName,
int32 sourceNodePort)
{
Oid distributedTableId = shardInterval->relationId;
List *ddlCommandList = NIL;
Assert(PartitionedTableNoLock(distributedTableId));
List *partitionList = PartitionList(distributedTableId);
Oid partitionOid = InvalidOid;
foreach_oid(partitionOid, partitionList)
{
uint64 partitionShardId =
ColocatedShardIdInRelation(partitionOid, shardInterval->shardIndex);
ShardInterval *partitionShardInterval = LoadShardInterval(partitionShardId);
bool includeData = false;
List *copyCommandList =
CopyShardCommandList(partitionShardInterval, sourceNodeName, sourceNodePort,
includeData);
ddlCommandList = list_concat(ddlCommandList, copyCommandList);
char *attachPartitionCommand =
GenerateAttachShardPartitionCommand(partitionShardInterval);
ddlCommandList = lappend(ddlCommandList, attachPartitionCommand);
}
return ddlCommandList;
}
/*
* EnsureShardCanBeRepaired checks if the given shard has a healthy placement in the source
* node and inactive node on the target node.
*/
static void
EnsureShardCanBeRepaired(int64 shardId, const char *sourceNodeName, int32 sourceNodePort,
const char *targetNodeName, int32 targetNodePort)
{
List *shardPlacementList =
ShardPlacementListIncludingOrphanedPlacements(shardId);
ShardPlacement *sourcePlacement = SearchShardPlacementInListOrError(
shardPlacementList,
sourceNodeName,
sourceNodePort);
if (sourcePlacement->shardState != SHARD_STATE_ACTIVE)
{
ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("source placement must be in active state")));
}
ShardPlacement *targetPlacement = SearchShardPlacementInListOrError(
shardPlacementList,
targetNodeName,
targetNodePort);
/*
* shardStateInactive is a legacy state for a placement. As of Citus 11,
* we never mark any placement as INACTIVE.
*
* Still, we prefer to keep this function/code here, as users may need
* to recover placements that are marked as inactive pre Citus 11.
*
*/
int shardStateInactive = 3;
if (targetPlacement->shardState != shardStateInactive)
{
ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("target placement must be in inactive state")));
}
}
/*
* EnsureShardCanBeCopied checks if the given shard has a healthy placement in the source
* node and no placements in the target node.
@ -1544,6 +1341,7 @@ EnsureShardCanBeCopied(int64 shardId, const char *sourceNodeName, int32 sourceNo
ShardPlacement *targetPlacement = SearchShardPlacementInList(shardPlacementList,
targetNodeName,
targetNodePort);
if (targetPlacement != NULL)
{
if (targetPlacement->shardState == SHARD_STATE_TO_DELETE)
@ -1901,7 +1699,7 @@ RecreateTableDDLCommandList(Oid relationId)
else
{
ereport(ERROR, (errcode(ERRCODE_WRONG_OBJECT_TYPE),
errmsg("repair target is not a regular, foreign or partitioned "
errmsg("target is not a regular, foreign or partitioned "
"table")));
}

View File

@ -44,9 +44,9 @@
#include "distributed/priority.h"
#include "distributed/distributed_planner.h"
#include "distributed/remote_commands.h"
#include "distributed/repair_shards.h"
#include "distributed/resource_lock.h"
#include "distributed/shard_rebalancer.h"
#include "distributed/shard_transfer.h"
#include "distributed/version_compat.h"
#include "nodes/bitmapset.h"
#include "parser/scansup.h"

View File

@ -73,9 +73,9 @@
#include "distributed/recursive_planning.h"
#include "distributed/reference_table_utils.h"
#include "distributed/relation_access_tracking.h"
#include "distributed/repair_shards.h"
#include "distributed/run_from_same_connection.h"
#include "distributed/shard_cleaner.h"
#include "distributed/shard_transfer.h"
#include "distributed/shared_connection_stats.h"
#include "distributed/shardsplit_shared_memory.h"
#include "distributed/query_pushdown_planning.h"

View File

@ -1,6 +1,7 @@
#include "udfs/citus_locks/11.1-1.sql"
#include "udfs/create_distributed_table_concurrently/11.1-1.sql"
#include "udfs/citus_internal_delete_partition_metadata/11.1-1.sql"
#include "udfs/citus_copy_shard_placement/11.1-1.sql"
DROP FUNCTION pg_catalog.worker_create_schema(bigint,text);
DROP FUNCTION pg_catalog.worker_cleanup_job_schema_cache();

View File

@ -105,7 +105,6 @@ DROP TABLE pg_catalog.pg_dist_cleanup;
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();
DROP FUNCTION pg_catalog.citus_job_cancel(bigint);
DROP FUNCTION pg_catalog.citus_job_wait(bigint, pg_catalog.citus_job_status);
DROP TABLE pg_catalog.pg_dist_background_task_depend;
@ -113,3 +112,5 @@ DROP TABLE pg_catalog.pg_dist_background_task;
DROP TYPE pg_catalog.citus_task_status;
DROP TABLE pg_catalog.pg_dist_background_job;
DROP TYPE pg_catalog.citus_job_status;
DROP FUNCTION pg_catalog.citus_copy_shard_placement;
#include "../udfs/citus_copy_shard_placement/10.0-1.sql"

View File

@ -0,0 +1,19 @@
DROP FUNCTION pg_catalog.citus_copy_shard_placement;
CREATE FUNCTION pg_catalog.citus_copy_shard_placement(
shard_id bigint,
source_node_name text,
source_node_port integer,
target_node_name text,
target_node_port integer,
transfer_mode citus.shard_transfer_mode default 'auto')
RETURNS void
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$citus_copy_shard_placement$$;
COMMENT ON FUNCTION pg_catalog.citus_copy_shard_placement(shard_id bigint,
source_node_name text,
source_node_port integer,
target_node_name text,
target_node_port integer,
shard_transfer_mode citus.shard_transfer_mode)
IS 'copy a shard from the source node to the destination node';

View File

@ -1,10 +1,10 @@
DROP FUNCTION pg_catalog.citus_copy_shard_placement;
CREATE FUNCTION pg_catalog.citus_copy_shard_placement(
shard_id bigint,
source_node_name text,
source_node_port integer,
target_node_name text,
target_node_port integer,
do_repair bool DEFAULT true,
transfer_mode citus.shard_transfer_mode default 'auto')
RETURNS void
LANGUAGE C STRICT
@ -15,6 +15,5 @@ COMMENT ON FUNCTION pg_catalog.citus_copy_shard_placement(shard_id bigint,
source_node_port integer,
target_node_name text,
target_node_port integer,
do_repair bool,
shard_transfer_mode citus.shard_transfer_mode)
IS 'copy a shard from the source node to the destination node';

View File

@ -83,7 +83,7 @@ replicate_reference_tables(PG_FUNCTION_ARGS)
/*
* EnsureReferenceTablesExistOnAllNodes ensures that a shard placement for every
* reference table exists on all nodes. If a node does not have a set of shard
* placements, then master_copy_shard_placement is called in a subtransaction
* placements, then citus_copy_shard_placement is called in a subtransaction
* to pull the data to the new node.
*/
void
@ -96,7 +96,7 @@ EnsureReferenceTablesExistOnAllNodes(void)
/*
* EnsureReferenceTablesExistOnAllNodesExtended ensures that a shard placement for every
* reference table exists on all nodes. If a node does not have a set of shard placements,
* then master_copy_shard_placement is called in a subtransaction to pull the data to the
* then citus_copy_shard_placement is called in a subtransaction to pull the data to the
* new node.
*
* The transferMode is passed on to the implementation of the copy to control the locks
@ -193,7 +193,7 @@ EnsureReferenceTablesExistOnAllNodesExtended(char transferMode)
}
/*
* master_copy_shard_placement triggers metadata sync-up, which tries to
* citus_copy_shard_placement triggers metadata sync-up, which tries to
* acquire a ShareLock on pg_dist_node. We do master_copy_shad_placement
* in a separate connection. If we have modified pg_dist_node in the
* current backend, this will cause a deadlock.
@ -207,7 +207,7 @@ EnsureReferenceTablesExistOnAllNodesExtended(char transferMode)
/*
* Modifications to reference tables in current transaction are not visible
* to master_copy_shard_placement, since it is done in a separate backend.
* to citus_copy_shard_placement, since it is done in a separate backend.
*/
if (AnyRelationsModifiedInTransaction(referenceTableIdList))
{
@ -235,7 +235,7 @@ EnsureReferenceTablesExistOnAllNodesExtended(char transferMode)
newWorkerNode->workerPort)));
/*
* Call master_copy_shard_placement using citus extension owner. Current
* Call citus_copy_shard_placement using citus extension owner. Current
* user might not have permissions to do the copy.
*/
const char *userName = CitusExtensionOwnerName();
@ -348,7 +348,7 @@ WorkersWithoutReferenceTablePlacement(uint64 shardId, LOCKMODE lockMode)
/*
* CopyShardPlacementToWorkerNodeQuery returns the master_copy_shard_placement
* CopyShardPlacementToWorkerNodeQuery returns the citus_copy_shard_placement
* command to copy the given shard placement to given node.
*/
static StringInfo
@ -364,8 +364,8 @@ CopyShardPlacementToWorkerNodeQuery(ShardPlacement *sourceShardPlacement,
"auto";
appendStringInfo(queryString,
"SELECT master_copy_shard_placement("
UINT64_FORMAT ", %s, %d, %s, %d, do_repair := false, "
"SELECT citus_copy_shard_placement("
UINT64_FORMAT ", %s, %d, %s, %d, "
"transfer_mode := %s)",
sourceShardPlacement->shardId,
quote_literal_cstr(sourceShardPlacement->nodeName),

View File

@ -47,7 +47,7 @@
#define CANDIDATE_NODE_FIELDS 2
#define WORKER_NODE_FIELDS 2
/* transfer mode for master_copy_shard_placement */
/* transfer mode for citus_copy_shard_placement */
#define TRANSFER_MODE_AUTOMATIC 'a'
#define TRANSFER_MODE_FORCE_LOGICAL 'l'
#define TRANSFER_MODE_BLOCK_WRITES 'b'
@ -286,9 +286,6 @@ extern Datum isolate_tenant_to_new_shard(PG_FUNCTION_ARGS);
/* function declarations for shard split functionality */
extern Datum citus_split_shard_by_split_points(PG_FUNCTION_ARGS);
/* function declarations for shard repair functionality */
extern Datum master_copy_shard_placement(PG_FUNCTION_ARGS);
/* function declarations for shard copy functinality */
extern List * CopyShardCommandList(ShardInterval *shardInterval, const
char *sourceNodeName,

View File

@ -1,4 +1,4 @@
-- Tests for master_copy_shard_placement, which can be used for adding replicas in statement-based replication
-- Tests for citus_copy_shard_placement, which can be used for adding replicas in statement-based replication
CREATE SCHEMA mcsp;
SET search_path TO mcsp;
SET citus.next_shard_id TO 8139000;
@ -35,7 +35,7 @@ SELECT create_distributed_table('history','key');
(1 row)
-- Mark tables as non-mx tables, in order to be able to test master_copy_shard_placement
-- Mark tables as non-mx tables, in order to be able to test citus_copy_shard_placement
UPDATE pg_dist_partition SET repmodel='c' WHERE logicalrelid IN
('data'::regclass, 'history'::regclass);
INSERT INTO data VALUES ('key-1', 'value-1');
@ -43,48 +43,43 @@ INSERT INTO data VALUES ('key-2', 'value-2');
INSERT INTO history VALUES ('key-1', '2020-02-01', 'old');
INSERT INTO history VALUES ('key-1', '2019-10-01', 'older');
-- verify we error out if no healthy placement exists at source
SELECT master_copy_shard_placement(
SELECT citus_copy_shard_placement(
get_shard_id_for_distribution_column('data', 'key-1'),
'localhost', :worker_1_port,
'localhost', :worker_2_port,
do_repair := false,
transfer_mode := 'block_writes');
ERROR: could not find placement matching "localhost:xxxxx"
HINT: Confirm the placement still exists and try again.
-- verify we error out if source and destination are the same
SELECT master_copy_shard_placement(
SELECT citus_copy_shard_placement(
get_shard_id_for_distribution_column('data', 'key-1'),
'localhost', :worker_2_port,
'localhost', :worker_2_port,
do_repair := false,
transfer_mode := 'block_writes');
ERROR: shard xxxxx already exists in the target node
-- verify we error out if target already contains a healthy placement
SELECT master_copy_shard_placement(
SELECT citus_copy_shard_placement(
(SELECT shardid FROM pg_dist_shard WHERE logicalrelid='ref_table'::regclass::oid),
'localhost', :worker_1_port,
'localhost', :worker_2_port,
do_repair := false,
transfer_mode := 'block_writes');
ERROR: shard xxxxx already exists in the target node
-- verify we error out if table has foreign key constraints
INSERT INTO ref_table SELECT 1, value FROM data;
ALTER TABLE data ADD CONSTRAINT distfk FOREIGN KEY (value) REFERENCES ref_table (b) MATCH FULL;
SELECT master_copy_shard_placement(
SELECT citus_copy_shard_placement(
get_shard_id_for_distribution_column('data', 'key-1'),
'localhost', :worker_2_port,
'localhost', :worker_1_port,
do_repair := false);
'localhost', :worker_1_port);
ERROR: cannot replicate shards with foreign keys
ALTER TABLE data DROP CONSTRAINT distfk;
-- replicate shard that contains key-1
SELECT master_copy_shard_placement(
SELECT citus_copy_shard_placement(
get_shard_id_for_distribution_column('data', 'key-1'),
'localhost', :worker_2_port,
'localhost', :worker_1_port,
do_repair := false,
transfer_mode := 'block_writes');
master_copy_shard_placement
citus_copy_shard_placement
---------------------------------------------------------------------
(1 row)
@ -127,11 +122,10 @@ SELECT create_distributed_table('mx_table', 'a');
(1 row)
SELECT master_copy_shard_placement(
SELECT citus_copy_shard_placement(
get_shard_id_for_distribution_column('mx_table', '1'),
'localhost', :worker_1_port,
'localhost', :worker_2_port,
do_repair := false,
transfer_mode := 'block_writes');
ERROR: Table 'mx_table' is streaming replicated. Shards of streaming replicated tables cannot be copied
SET client_min_messages TO ERROR;

View File

@ -302,8 +302,8 @@ SELECT get_shard_id_for_distribution_column('citus_local_table_1');
1504014
(1 row)
-- master_copy_shard_placement is not supported
SELECT master_copy_shard_placement(shardid, 'localhost', :master_port, 'localhost', :worker_1_port, true)
-- citus_copy_shard_placement is not supported
SELECT citus_copy_shard_placement(shardid, 'localhost', :master_port, 'localhost', :worker_1_port)
FROM (SELECT shardid FROM pg_dist_shard WHERE logicalrelid='citus_local_table_1'::regclass) as shardid;
ERROR: Table 'citus_local_table_1' is a local table. Replicating shard of a local table added to metadata currently is not supported
-- undistribute_table is supported

View File

@ -24,12 +24,12 @@ SELECT citus_add_local_table_to_metadata('citus_local_table');
-- isolate_tenant_to_new_shard is not supported
SELECT isolate_tenant_to_new_shard('citus_local_table', 100, shard_transfer_mode => 'block_writes');
ERROR: cannot isolate tenant because tenant isolation is only support for hash distributed tables
-- master_copy_shard_placement is not supported
SELECT master_copy_shard_placement(shardid, 'localhost', :master_port, 'localhost', :worker_1_port, false)
-- citus_copy_shard_placement is not supported
SELECT citus_copy_shard_placement(shardid, 'localhost', :master_port, 'localhost', :worker_1_port, false)
FROM (SELECT shardid FROM pg_dist_shard WHERE logicalrelid='citus_local_table'::regclass) as shardid;
ERROR: Table 'citus_local_table' is a local table. Replicating shard of a local table added to metadata currently is not supported
-- master_move_shard_placement is not supported
SELECT master_move_shard_placement(shardid, 'localhost', :master_port, 'localhost', :worker_1_port)
ERROR: function citus_copy_shard_placement(bigint, unknown, integer, unknown, integer, boolean) does not exist at character 8
-- citus_move_shard_placement is not supported
SELECT citus_move_shard_placement(shardid, 'localhost', :master_port, 'localhost', :worker_1_port)
FROM (SELECT shardid FROM pg_dist_shard WHERE logicalrelid='citus_local_table'::regclass) as shardid;
ERROR: table citus_local_tables_ent.citus_local_table is a local table, moving shard of a local table added to metadata is currently not supported
-- replicate_table_shards is not suported

View File

@ -213,36 +213,9 @@ SELECT logicalrelid FROM pg_dist_partition WHERE colocationid = 92448300 ORDER B
rep2
(2 rows)
UPDATE pg_dist_placement SET shardstate = 3 WHERE shardid = 92448300 AND groupid = 0;
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 | 3 | 57636
(3 rows)
-- cannot copy from an orphaned shard
SELECT * FROM citus_copy_shard_placement(92448300, 'localhost', :worker_1_port, 'localhost', :master_port);
ERROR: source placement must be in active state
-- cannot copy to an orphaned shard
SELECT * FROM citus_copy_shard_placement(92448300, 'localhost', :worker_2_port, 'localhost', :worker_1_port);
ERROR: target placement must be in inactive state
-- can still copy to an inactive shard
SELECT * FROM citus_copy_shard_placement(92448300, 'localhost', :worker_2_port, 'localhost', :master_port);
citus_copy_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)
-- Make sure we don't send a query to the orphaned shard
BEGIN;
SET LOCAL citus.log_remote_commands TO ON;

View File

@ -1,53 +1,53 @@
Parsed test spec with 2 sessions
starting permutation: s1-load-cache s2-load-cache s2-set-placement-inactive s2-begin s2-repair-placement s1-repair-placement s2-commit
starting permutation: s1-load-cache s2-load-cache s2-delete-inactive s2-begin s2-repair-placement s1-repair-placement s2-commit
step s1-load-cache:
COPY test_hash_table FROM PROGRAM 'echo 1,1 && echo 2,2 && echo 3,3 && echo 4,4 && echo 5,5' WITH CSV;
COPY test_hash_table FROM PROGRAM 'echo 1,1 && echo 2,2 && echo 3,3 && echo 4,4 && echo 5,5' WITH CSV;
step s2-load-cache:
COPY test_hash_table FROM PROGRAM 'echo 1,1 && echo 2,2 && echo 3,3 && echo 4,4 && echo 5,5' WITH CSV;
COPY test_hash_table FROM PROGRAM 'echo 1,1 && echo 2,2 && echo 3,3 && echo 4,4 && echo 5,5' WITH CSV;
step s2-set-placement-inactive:
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE shardid IN (SELECT * FROM selected_shard_for_test_table) AND nodeport = 57638;
step s2-delete-inactive:
DELETE FROM pg_dist_shard_placement WHERE shardid IN (SELECT * FROM selected_shard_for_test_table) AND nodeport = 57638;
step s2-begin:
BEGIN;
BEGIN;
step s2-repair-placement:
SELECT master_copy_shard_placement((SELECT * FROM selected_shard_for_test_table), 'localhost', 57637, 'localhost', 57638);
SELECT citus_copy_shard_placement((SELECT * FROM selected_shard_for_test_table), 'localhost', 57637, 'localhost', 57638, transfer_mode := 'block_writes');
master_copy_shard_placement
citus_copy_shard_placement
---------------------------------------------------------------------
(1 row)
step s1-repair-placement:
SELECT master_copy_shard_placement((SELECT * FROM selected_shard_for_test_table), 'localhost', 57637, 'localhost', 57638);
SELECT citus_copy_shard_placement((SELECT * FROM selected_shard_for_test_table), 'localhost', 57637, 'localhost', 57638);
ERROR: could not acquire the lock required to repair public.test_hash_table
ERROR: could not acquire the lock required to copy public.test_hash_table
step s2-commit:
COMMIT;
COMMIT;
starting permutation: s2-set-placement-inactive s2-begin s2-repair-placement s1-repair-placement s2-commit
step s2-set-placement-inactive:
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE shardid IN (SELECT * FROM selected_shard_for_test_table) AND nodeport = 57638;
starting permutation: s2-delete-inactive s2-begin s2-repair-placement s1-repair-placement s2-commit
step s2-delete-inactive:
DELETE FROM pg_dist_shard_placement WHERE shardid IN (SELECT * FROM selected_shard_for_test_table) AND nodeport = 57638;
step s2-begin:
BEGIN;
BEGIN;
step s2-repair-placement:
SELECT master_copy_shard_placement((SELECT * FROM selected_shard_for_test_table), 'localhost', 57637, 'localhost', 57638);
SELECT citus_copy_shard_placement((SELECT * FROM selected_shard_for_test_table), 'localhost', 57637, 'localhost', 57638, transfer_mode := 'block_writes');
master_copy_shard_placement
citus_copy_shard_placement
---------------------------------------------------------------------
(1 row)
step s1-repair-placement:
SELECT master_copy_shard_placement((SELECT * FROM selected_shard_for_test_table), 'localhost', 57637, 'localhost', 57638);
SELECT citus_copy_shard_placement((SELECT * FROM selected_shard_for_test_table), 'localhost', 57637, 'localhost', 57638);
ERROR: could not acquire the lock required to repair public.test_hash_table
ERROR: could not acquire the lock required to copy public.test_hash_table
step s2-commit:
COMMIT;
COMMIT;

View File

@ -1,6 +1,6 @@
Parsed test spec with 2 sessions
starting permutation: s1-load-cache s1-insert s1-begin s1-select s2-set-placement-inactive s2-begin s2-repair-placement s1-update s2-commit s1-commit s2-print-content
starting permutation: s1-load-cache s1-insert s1-begin s1-select s2-delete-inactive s2-begin s2-repair-placement s1-update s2-commit s1-commit s2-print-content
step s1-load-cache:
TRUNCATE test_repair_placement_vs_modification;
@ -19,16 +19,16 @@ count
1
(1 row)
step s2-set-placement-inactive:
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE shardid IN (SELECT * FROM selected_shard) AND nodeport = 57638;
step s2-delete-inactive:
DELETE FROM pg_dist_shard_placement WHERE shardid IN (SELECT * FROM selected_shard) AND nodeport = 57638;
step s2-begin:
BEGIN;
step s2-repair-placement:
SELECT master_copy_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638);
SELECT citus_copy_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638, transfer_mode := 'block_writes');
master_copy_shard_placement
citus_copy_shard_placement
---------------------------------------------------------------------
(1 row)
@ -60,7 +60,7 @@ nodeport|success|result
(2 rows)
starting permutation: s1-load-cache s1-insert s1-begin s1-select s2-set-placement-inactive s2-begin s2-repair-placement s1-delete s2-commit s1-commit s2-print-content
starting permutation: s1-load-cache s1-insert s1-begin s1-select s2-delete-inactive s2-begin s2-repair-placement s1-delete s2-commit s1-commit s2-print-content
step s1-load-cache:
TRUNCATE test_repair_placement_vs_modification;
@ -79,16 +79,16 @@ count
1
(1 row)
step s2-set-placement-inactive:
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE shardid IN (SELECT * FROM selected_shard) AND nodeport = 57638;
step s2-delete-inactive:
DELETE FROM pg_dist_shard_placement WHERE shardid IN (SELECT * FROM selected_shard) AND nodeport = 57638;
step s2-begin:
BEGIN;
step s2-repair-placement:
SELECT master_copy_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638);
SELECT citus_copy_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638, transfer_mode := 'block_writes');
master_copy_shard_placement
citus_copy_shard_placement
---------------------------------------------------------------------
(1 row)
@ -120,7 +120,7 @@ nodeport|success|result
(2 rows)
starting permutation: s1-load-cache s1-begin s1-select s2-set-placement-inactive s2-begin s2-repair-placement s1-insert s2-commit s1-commit s2-print-content
starting permutation: s1-load-cache s1-begin s1-select s2-delete-inactive s2-begin s2-repair-placement s1-insert s2-commit s1-commit s2-print-content
step s1-load-cache:
TRUNCATE test_repair_placement_vs_modification;
@ -136,16 +136,16 @@ count
0
(1 row)
step s2-set-placement-inactive:
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE shardid IN (SELECT * FROM selected_shard) AND nodeport = 57638;
step s2-delete-inactive:
DELETE FROM pg_dist_shard_placement WHERE shardid IN (SELECT * FROM selected_shard) AND nodeport = 57638;
step s2-begin:
BEGIN;
step s2-repair-placement:
SELECT master_copy_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638);
SELECT citus_copy_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638, transfer_mode := 'block_writes');
master_copy_shard_placement
citus_copy_shard_placement
---------------------------------------------------------------------
(1 row)
@ -177,7 +177,7 @@ nodeport|success|result
(2 rows)
starting permutation: s1-load-cache s1-begin s1-select s2-set-placement-inactive s2-begin s2-repair-placement s1-copy s2-commit s1-commit s2-print-content
starting permutation: s1-load-cache s1-begin s1-select s2-delete-inactive s2-begin s2-repair-placement s1-copy s2-commit s1-commit s2-print-content
step s1-load-cache:
TRUNCATE test_repair_placement_vs_modification;
@ -193,16 +193,16 @@ count
0
(1 row)
step s2-set-placement-inactive:
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE shardid IN (SELECT * FROM selected_shard) AND nodeport = 57638;
step s2-delete-inactive:
DELETE FROM pg_dist_shard_placement WHERE shardid IN (SELECT * FROM selected_shard) AND nodeport = 57638;
step s2-begin:
BEGIN;
step s2-repair-placement:
SELECT master_copy_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638);
SELECT citus_copy_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638, transfer_mode := 'block_writes');
master_copy_shard_placement
citus_copy_shard_placement
---------------------------------------------------------------------
(1 row)
@ -234,7 +234,7 @@ nodeport|success|result
(2 rows)
starting permutation: s1-load-cache s1-begin s1-select s2-set-placement-inactive s2-begin s2-repair-placement s1-ddl s2-commit s1-commit s2-print-index-count
starting permutation: s1-load-cache s1-begin s1-select s2-delete-inactive s2-begin s2-repair-placement s1-ddl s2-commit s1-commit s2-print-index-count
step s1-load-cache:
TRUNCATE test_repair_placement_vs_modification;
@ -250,16 +250,16 @@ count
0
(1 row)
step s2-set-placement-inactive:
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE shardid IN (SELECT * FROM selected_shard) AND nodeport = 57638;
step s2-delete-inactive:
DELETE FROM pg_dist_shard_placement WHERE shardid IN (SELECT * FROM selected_shard) AND nodeport = 57638;
step s2-begin:
BEGIN;
step s2-repair-placement:
SELECT master_copy_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638);
SELECT citus_copy_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638, transfer_mode := 'block_writes');
master_copy_shard_placement
citus_copy_shard_placement
---------------------------------------------------------------------
(1 row)
@ -291,7 +291,7 @@ nodeport|success|result
(4 rows)
starting permutation: s1-insert s1-begin s1-select s2-set-placement-inactive s2-begin s2-repair-placement s1-update s2-commit s1-commit s2-print-content
starting permutation: s1-insert s1-begin s1-select s2-delete-inactive s2-begin s2-repair-placement s1-update s2-commit s1-commit s2-print-content
step s1-insert:
INSERT INTO test_repair_placement_vs_modification VALUES (5, 10);
@ -307,16 +307,16 @@ count
1
(1 row)
step s2-set-placement-inactive:
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE shardid IN (SELECT * FROM selected_shard) AND nodeport = 57638;
step s2-delete-inactive:
DELETE FROM pg_dist_shard_placement WHERE shardid IN (SELECT * FROM selected_shard) AND nodeport = 57638;
step s2-begin:
BEGIN;
step s2-repair-placement:
SELECT master_copy_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638);
SELECT citus_copy_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638, transfer_mode := 'block_writes');
master_copy_shard_placement
citus_copy_shard_placement
---------------------------------------------------------------------
(1 row)
@ -348,7 +348,7 @@ nodeport|success|result
(2 rows)
starting permutation: s1-insert s1-begin s1-select s2-set-placement-inactive s2-begin s2-repair-placement s1-delete s2-commit s1-commit s2-print-content
starting permutation: s1-insert s1-begin s1-select s2-delete-inactive s2-begin s2-repair-placement s1-delete s2-commit s1-commit s2-print-content
step s1-insert:
INSERT INTO test_repair_placement_vs_modification VALUES (5, 10);
@ -364,16 +364,16 @@ count
1
(1 row)
step s2-set-placement-inactive:
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE shardid IN (SELECT * FROM selected_shard) AND nodeport = 57638;
step s2-delete-inactive:
DELETE FROM pg_dist_shard_placement WHERE shardid IN (SELECT * FROM selected_shard) AND nodeport = 57638;
step s2-begin:
BEGIN;
step s2-repair-placement:
SELECT master_copy_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638);
SELECT citus_copy_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638, transfer_mode := 'block_writes');
master_copy_shard_placement
citus_copy_shard_placement
---------------------------------------------------------------------
(1 row)
@ -405,7 +405,7 @@ nodeport|success|result
(2 rows)
starting permutation: s1-begin s1-select s2-set-placement-inactive s2-begin s2-repair-placement s1-insert s2-commit s1-commit s2-print-content
starting permutation: s1-begin s1-select s2-delete-inactive s2-begin s2-repair-placement s1-insert s2-commit s1-commit s2-print-content
step s1-begin:
BEGIN;
SET LOCAL citus.select_opens_transaction_block TO off;
@ -418,16 +418,16 @@ count
0
(1 row)
step s2-set-placement-inactive:
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE shardid IN (SELECT * FROM selected_shard) AND nodeport = 57638;
step s2-delete-inactive:
DELETE FROM pg_dist_shard_placement WHERE shardid IN (SELECT * FROM selected_shard) AND nodeport = 57638;
step s2-begin:
BEGIN;
step s2-repair-placement:
SELECT master_copy_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638);
SELECT citus_copy_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638, transfer_mode := 'block_writes');
master_copy_shard_placement
citus_copy_shard_placement
---------------------------------------------------------------------
(1 row)
@ -459,7 +459,7 @@ nodeport|success|result
(2 rows)
starting permutation: s1-begin s1-select s2-set-placement-inactive s2-begin s2-repair-placement s1-copy s2-commit s1-commit s2-print-content
starting permutation: s1-begin s1-select s2-delete-inactive s2-begin s2-repair-placement s1-copy s2-commit s1-commit s2-print-content
step s1-begin:
BEGIN;
SET LOCAL citus.select_opens_transaction_block TO off;
@ -472,16 +472,16 @@ count
0
(1 row)
step s2-set-placement-inactive:
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE shardid IN (SELECT * FROM selected_shard) AND nodeport = 57638;
step s2-delete-inactive:
DELETE FROM pg_dist_shard_placement WHERE shardid IN (SELECT * FROM selected_shard) AND nodeport = 57638;
step s2-begin:
BEGIN;
step s2-repair-placement:
SELECT master_copy_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638);
SELECT citus_copy_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638, transfer_mode := 'block_writes');
master_copy_shard_placement
citus_copy_shard_placement
---------------------------------------------------------------------
(1 row)
@ -513,7 +513,7 @@ nodeport|success|result
(2 rows)
starting permutation: s1-begin s1-select s2-set-placement-inactive s2-begin s2-repair-placement s1-ddl s2-commit s1-commit s2-print-index-count
starting permutation: s1-begin s1-select s2-delete-inactive s2-begin s2-repair-placement s1-ddl s2-commit s1-commit s2-print-index-count
step s1-begin:
BEGIN;
SET LOCAL citus.select_opens_transaction_block TO off;
@ -526,16 +526,16 @@ count
0
(1 row)
step s2-set-placement-inactive:
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE shardid IN (SELECT * FROM selected_shard) AND nodeport = 57638;
step s2-delete-inactive:
DELETE FROM pg_dist_shard_placement WHERE shardid IN (SELECT * FROM selected_shard) AND nodeport = 57638;
step s2-begin:
BEGIN;
step s2-repair-placement:
SELECT master_copy_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638);
SELECT citus_copy_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638, transfer_mode := 'block_writes');
master_copy_shard_placement
citus_copy_shard_placement
---------------------------------------------------------------------
(1 row)

View File

@ -585,12 +585,12 @@ create_distributed_table_concurrently
(1 row)
step s4-print-colocations:
SELECT * FROM pg_dist_colocation ORDER BY colocationid;
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation ORDER BY colocationid;
colocationid|shardcount|replicationfactor|distributioncolumntype|distributioncolumncollation
shardcount|replicationfactor|distributioncolumntype|distributioncolumncollation
---------------------------------------------------------------------
123173| 4| 1| 21| 0
123174| 4| 1| 23| 0
4| 1| 21| 0
4| 1| 23| 0
(2 rows)
citus_remove_node
@ -651,12 +651,12 @@ create_distributed_table_concurrently
(1 row)
step s4-print-colocations:
SELECT * FROM pg_dist_colocation ORDER BY colocationid;
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation ORDER BY colocationid;
colocationid|shardcount|replicationfactor|distributioncolumntype|distributioncolumncollation
shardcount|replicationfactor|distributioncolumntype|distributioncolumncollation
---------------------------------------------------------------------
123175| 4| 1| 23| 0
123176| 4| 1| 21| 0
4| 1| 23| 0
4| 1| 21| 0
(2 rows)
citus_remove_node
@ -700,12 +700,12 @@ create_distributed_table_concurrently
(1 row)
step s4-print-colocations:
SELECT * FROM pg_dist_colocation ORDER BY colocationid;
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation ORDER BY colocationid;
colocationid|shardcount|replicationfactor|distributioncolumntype|distributioncolumncollation
shardcount|replicationfactor|distributioncolumntype|distributioncolumncollation
---------------------------------------------------------------------
123177| 4| 1| 21| 0
123178| 4| 1| 23| 0
4| 1| 21| 0
4| 1| 23| 0
(2 rows)
citus_remove_node
@ -748,12 +748,12 @@ create_distributed_table_concurrently
(1 row)
step s4-print-colocations:
SELECT * FROM pg_dist_colocation ORDER BY colocationid;
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation ORDER BY colocationid;
colocationid|shardcount|replicationfactor|distributioncolumntype|distributioncolumncollation
shardcount|replicationfactor|distributioncolumntype|distributioncolumncollation
---------------------------------------------------------------------
123179| 4| 1| 21| 0
123180| 4| 1| 23| 0
4| 1| 21| 0
4| 1| 23| 0
(2 rows)
citus_remove_node

View File

@ -1,236 +0,0 @@
Parsed test spec with 2 sessions
starting permutation: s2-invalidate-57637 s1-begin s1-insertone s2-repair s1-commit
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s2-invalidate-57637:
UPDATE pg_dist_shard_placement SET shardstate = '3' WHERE shardid = (SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_dml_vs_repair'::regclass) AND nodeport = 57637;
step s1-begin:
BEGIN;
step s1-insertone:
INSERT INTO test_dml_vs_repair VALUES(1, 1);
step s2-repair:
SELECT master_copy_shard_placement((SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_dml_vs_repair'::regclass), 'localhost', 57638, 'localhost', 57637);
<waiting ...>
step s1-commit:
COMMIT;
step s2-repair: <... completed>
master_copy_shard_placement
---------------------------------------------------------------------
(1 row)
starting permutation: s1-insertone s2-invalidate-57637 s1-begin s1-insertall s2-repair s1-commit
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-insertone:
INSERT INTO test_dml_vs_repair VALUES(1, 1);
step s2-invalidate-57637:
UPDATE pg_dist_shard_placement SET shardstate = '3' WHERE shardid = (SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_dml_vs_repair'::regclass) AND nodeport = 57637;
step s1-begin:
BEGIN;
step s1-insertall:
INSERT INTO test_dml_vs_repair SELECT test_id, data+1 FROM test_dml_vs_repair;
step s2-repair:
SELECT master_copy_shard_placement((SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_dml_vs_repair'::regclass), 'localhost', 57638, 'localhost', 57637);
<waiting ...>
step s1-commit:
COMMIT;
step s2-repair: <... completed>
master_copy_shard_placement
---------------------------------------------------------------------
(1 row)
starting permutation: s2-invalidate-57637 s2-begin s2-repair s1-insertone s2-commit s2-invalidate-57638 s1-display s2-invalidate-57637 s2-revalidate-57638 s1-display
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s2-invalidate-57637:
UPDATE pg_dist_shard_placement SET shardstate = '3' WHERE shardid = (SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_dml_vs_repair'::regclass) AND nodeport = 57637;
step s2-begin:
BEGIN;
step s2-repair:
SELECT master_copy_shard_placement((SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_dml_vs_repair'::regclass), 'localhost', 57638, 'localhost', 57637);
master_copy_shard_placement
---------------------------------------------------------------------
(1 row)
step s1-insertone:
INSERT INTO test_dml_vs_repair VALUES(1, 1);
<waiting ...>
step s2-commit:
COMMIT;
step s1-insertone: <... completed>
step s2-invalidate-57638:
UPDATE pg_dist_shard_placement SET shardstate = '3' WHERE shardid = (SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_dml_vs_repair'::regclass) AND nodeport = 57638;
step s1-display:
SELECT * FROM test_dml_vs_repair WHERE test_id = 1 ORDER BY test_id;
test_id|data
---------------------------------------------------------------------
1| 1
(1 row)
step s2-invalidate-57637:
UPDATE pg_dist_shard_placement SET shardstate = '3' WHERE shardid = (SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_dml_vs_repair'::regclass) AND nodeport = 57637;
step s2-revalidate-57638:
UPDATE pg_dist_shard_placement SET shardstate = '1' WHERE shardid = (SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_dml_vs_repair'::regclass) AND nodeport = 57638;
step s1-display:
SELECT * FROM test_dml_vs_repair WHERE test_id = 1 ORDER BY test_id;
test_id|data
---------------------------------------------------------------------
1| 1
(1 row)
starting permutation: s2-invalidate-57637 s1-prepared-insertone s2-begin s2-repair s1-prepared-insertone s2-commit s2-invalidate-57638 s1-display s2-invalidate-57637 s2-revalidate-57638 s1-display
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s2-invalidate-57637:
UPDATE pg_dist_shard_placement SET shardstate = '3' WHERE shardid = (SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_dml_vs_repair'::regclass) AND nodeport = 57637;
step s1-prepared-insertone:
EXECUTE insertone;
step s2-begin:
BEGIN;
step s2-repair:
SELECT master_copy_shard_placement((SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_dml_vs_repair'::regclass), 'localhost', 57638, 'localhost', 57637);
master_copy_shard_placement
---------------------------------------------------------------------
(1 row)
step s1-prepared-insertone:
EXECUTE insertone;
<waiting ...>
step s2-commit:
COMMIT;
step s1-prepared-insertone: <... completed>
step s2-invalidate-57638:
UPDATE pg_dist_shard_placement SET shardstate = '3' WHERE shardid = (SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_dml_vs_repair'::regclass) AND nodeport = 57638;
step s1-display:
SELECT * FROM test_dml_vs_repair WHERE test_id = 1 ORDER BY test_id;
test_id|data
---------------------------------------------------------------------
1| 1
1| 1
(2 rows)
step s2-invalidate-57637:
UPDATE pg_dist_shard_placement SET shardstate = '3' WHERE shardid = (SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_dml_vs_repair'::regclass) AND nodeport = 57637;
step s2-revalidate-57638:
UPDATE pg_dist_shard_placement SET shardstate = '1' WHERE shardid = (SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_dml_vs_repair'::regclass) AND nodeport = 57638;
step s1-display:
SELECT * FROM test_dml_vs_repair WHERE test_id = 1 ORDER BY test_id;
test_id|data
---------------------------------------------------------------------
1| 1
1| 1
(2 rows)
starting permutation: s2-invalidate-57637 s1-insertone s1-prepared-insertall s2-begin s2-repair s1-prepared-insertall s2-commit s2-invalidate-57638 s1-display s2-invalidate-57637 s2-revalidate-57638 s1-display
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s2-invalidate-57637:
UPDATE pg_dist_shard_placement SET shardstate = '3' WHERE shardid = (SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_dml_vs_repair'::regclass) AND nodeport = 57637;
step s1-insertone:
INSERT INTO test_dml_vs_repair VALUES(1, 1);
step s1-prepared-insertall:
EXECUTE insertall;
step s2-begin:
BEGIN;
step s2-repair:
SELECT master_copy_shard_placement((SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_dml_vs_repair'::regclass), 'localhost', 57638, 'localhost', 57637);
master_copy_shard_placement
---------------------------------------------------------------------
(1 row)
step s1-prepared-insertall:
EXECUTE insertall;
<waiting ...>
step s2-commit:
COMMIT;
step s1-prepared-insertall: <... completed>
step s2-invalidate-57638:
UPDATE pg_dist_shard_placement SET shardstate = '3' WHERE shardid = (SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_dml_vs_repair'::regclass) AND nodeport = 57638;
step s1-display:
SELECT * FROM test_dml_vs_repair WHERE test_id = 1 ORDER BY test_id;
test_id|data
---------------------------------------------------------------------
1| 1
1| 2
1| 2
1| 3
(4 rows)
step s2-invalidate-57637:
UPDATE pg_dist_shard_placement SET shardstate = '3' WHERE shardid = (SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_dml_vs_repair'::regclass) AND nodeport = 57637;
step s2-revalidate-57638:
UPDATE pg_dist_shard_placement SET shardstate = '1' WHERE shardid = (SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_dml_vs_repair'::regclass) AND nodeport = 57638;
step s1-display:
SELECT * FROM test_dml_vs_repair WHERE test_id = 1 ORDER BY test_id;
test_id|data
---------------------------------------------------------------------
1| 1
1| 2
1| 2
1| 3
(4 rows)

View File

@ -59,7 +59,7 @@ SELECT master_create_empty_shard('table6_append');
13000021
(1 row)
-- Mark tables as non-mx tables, in order to be able to test master_copy_shard_placement
-- Mark tables as non-mx tables, in order to be able to test citus_copy_shard_placement
UPDATE pg_dist_partition SET repmodel='c' WHERE logicalrelid IN
('table1_group1'::regclass, 'table2_group1'::regclass, 'table5_groupX'::regclass);
-- test copy
@ -90,13 +90,13 @@ ORDER BY s.shardid, sp.nodeport;
(12 rows)
-- try to copy colocated shards without a replica identity
SELECT master_copy_shard_placement(13000000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, false);
SELECT citus_copy_shard_placement(13000000, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
ERROR: cannot use logical replication to transfer shards of the relation table2_group1 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'.
-- copy colocated shards
SELECT master_copy_shard_placement(13000000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, false, 'force_logical');
master_copy_shard_placement
SELECT citus_copy_shard_placement(13000000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'force_logical');
citus_copy_shard_placement
---------------------------------------------------------------------
(1 row)
@ -144,7 +144,7 @@ SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.table2_
\c - - - :master_port
-- copy colocated shards again to see error message
SELECT master_copy_shard_placement(13000000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, false, 'force_logical');
SELECT citus_copy_shard_placement(13000000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'force_logical');
ERROR: shard xxxxx already exists in the target node
-- test copying NOT colocated shard
-- status before shard copy
@ -169,8 +169,8 @@ ORDER BY s.shardid, sp.nodeport;
(8 rows)
-- copy NOT colocated shard
SELECT master_copy_shard_placement(13000012, 'localhost', :worker_1_port, 'localhost', :worker_2_port, false, 'force_logical');
master_copy_shard_placement
SELECT citus_copy_shard_placement(13000012, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'force_logical');
citus_copy_shard_placement
---------------------------------------------------------------------
(1 row)
@ -214,8 +214,8 @@ ORDER BY s.shardid, sp.nodeport;
(2 rows)
-- copy shard in append distributed table
SELECT master_copy_shard_placement(13000020, 'localhost', :worker_2_port, 'localhost', :worker_1_port, false, 'force_logical');
master_copy_shard_placement
SELECT citus_copy_shard_placement(13000020, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'force_logical');
citus_copy_shard_placement
---------------------------------------------------------------------
(1 row)
@ -524,7 +524,7 @@ SELECT "Constraint", "Definition" FROM table_fkeys
\c - - - :master_port
-- test shard copy with foreign constraints
-- we expect it to error out because we do not support foreign constraints with replication factor > 1
SELECT master_copy_shard_placement(13000022, 'localhost', :worker_2_port, 'localhost', :worker_1_port, false);
SELECT citus_copy_shard_placement(13000022, 'localhost', :worker_2_port, 'localhost', :worker_1_port);
ERROR: cannot replicate shards with foreign keys
-- lets also test that master_move_shard_placement doesn't break serials
CREATE TABLE serial_move_test (key int, other_val serial);

View File

@ -1,231 +0,0 @@
--
-- MULTI_COLOCATED_SHARD_TRANSFER
--
-- These tables are created in multi_colocation_utils test
-- test repair
-- manually set shardstate as inactive
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE nodeport = :worker_2_port AND (shardid = 1300000 OR shardid = 1300004);
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE nodeport = :worker_2_port AND shardid = 1300016;
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE nodeport = :worker_2_port AND shardid = 1300020;
-- test repairing colocated shards
-- status before shard repair
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport, p.colocationid, sp.shardstate
FROM
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
WHERE
p.logicalrelid = s.logicalrelid AND
s.shardid = sp.shardid AND
colocationid = (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'table1_group1'::regclass)
ORDER BY s.shardid, sp.nodeport;
shardid | logicalrelid | nodeport | colocationid | shardstate
---------------------------------------------------------------------
1300000 | table1_group1 | 57637 | 1000 | 1
1300000 | table1_group1 | 57638 | 1000 | 3
1300001 | table1_group1 | 57637 | 1000 | 1
1300001 | table1_group1 | 57638 | 1000 | 1
1300002 | table1_group1 | 57637 | 1000 | 1
1300002 | table1_group1 | 57638 | 1000 | 1
1300003 | table1_group1 | 57637 | 1000 | 1
1300003 | table1_group1 | 57638 | 1000 | 1
1300004 | table2_group1 | 57637 | 1000 | 1
1300004 | table2_group1 | 57638 | 1000 | 3
1300005 | table2_group1 | 57637 | 1000 | 1
1300005 | table2_group1 | 57638 | 1000 | 1
1300006 | table2_group1 | 57637 | 1000 | 1
1300006 | table2_group1 | 57638 | 1000 | 1
1300007 | table2_group1 | 57637 | 1000 | 1
1300007 | table2_group1 | 57638 | 1000 | 1
(16 rows)
-- repair colocated shards
SELECT master_copy_shard_placement(1300000, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
master_copy_shard_placement
---------------------------------------------------------------------
(1 row)
-- status after shard repair
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport, p.colocationid, sp.shardstate
FROM
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
WHERE
p.logicalrelid = s.logicalrelid AND
s.shardid = sp.shardid AND
colocationid = (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'table1_group1'::regclass)
ORDER BY s.shardid, sp.nodeport;
shardid | logicalrelid | nodeport | colocationid | shardstate
---------------------------------------------------------------------
1300000 | table1_group1 | 57637 | 1000 | 1
1300000 | table1_group1 | 57638 | 1000 | 1
1300001 | table1_group1 | 57637 | 1000 | 1
1300001 | table1_group1 | 57638 | 1000 | 1
1300002 | table1_group1 | 57637 | 1000 | 1
1300002 | table1_group1 | 57638 | 1000 | 1
1300003 | table1_group1 | 57637 | 1000 | 1
1300003 | table1_group1 | 57638 | 1000 | 1
1300004 | table2_group1 | 57637 | 1000 | 1
1300004 | table2_group1 | 57638 | 1000 | 3
1300005 | table2_group1 | 57637 | 1000 | 1
1300005 | table2_group1 | 57638 | 1000 | 1
1300006 | table2_group1 | 57637 | 1000 | 1
1300006 | table2_group1 | 57638 | 1000 | 1
1300007 | table2_group1 | 57637 | 1000 | 1
1300007 | table2_group1 | 57638 | 1000 | 1
(16 rows)
-- test repairing NOT colocated shard
-- status before shard repair
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport, p.colocationid, sp.shardstate
FROM
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
WHERE
p.logicalrelid = s.logicalrelid AND
s.shardid = sp.shardid AND
p.logicalrelid = 'table5_groupX'::regclass
ORDER BY s.shardid, sp.nodeport;
shardid | logicalrelid | nodeport | colocationid | shardstate
---------------------------------------------------------------------
1300016 | table5_groupx | 57637 | 0 | 1
1300016 | table5_groupx | 57638 | 0 | 3
1300017 | table5_groupx | 57637 | 0 | 1
1300017 | table5_groupx | 57638 | 0 | 1
1300018 | table5_groupx | 57637 | 0 | 1
1300018 | table5_groupx | 57638 | 0 | 1
1300019 | table5_groupx | 57637 | 0 | 1
1300019 | table5_groupx | 57638 | 0 | 1
(8 rows)
-- repair NOT colocated shard
SELECT master_copy_shard_placement(1300016, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
master_copy_shard_placement
---------------------------------------------------------------------
(1 row)
-- status after shard repair
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport, p.colocationid, sp.shardstate
FROM
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
WHERE
p.logicalrelid = s.logicalrelid AND
s.shardid = sp.shardid AND
p.logicalrelid = 'table5_groupX'::regclass
ORDER BY s.shardid, sp.nodeport;
shardid | logicalrelid | nodeport | colocationid | shardstate
---------------------------------------------------------------------
1300016 | table5_groupx | 57637 | 0 | 1
1300016 | table5_groupx | 57638 | 0 | 1
1300017 | table5_groupx | 57637 | 0 | 1
1300017 | table5_groupx | 57638 | 0 | 1
1300018 | table5_groupx | 57637 | 0 | 1
1300018 | table5_groupx | 57638 | 0 | 1
1300019 | table5_groupx | 57637 | 0 | 1
1300019 | table5_groupx | 57638 | 0 | 1
(8 rows)
-- test repairing shard in append distributed table
-- status before shard repair
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport, p.colocationid, sp.shardstate
FROM
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
WHERE
p.logicalrelid = s.logicalrelid AND
s.shardid = sp.shardid AND
p.logicalrelid = 'table6_append'::regclass
ORDER BY s.shardid, sp.nodeport;
shardid | logicalrelid | nodeport | colocationid | shardstate
---------------------------------------------------------------------
1300020 | table6_append | 57637 | 0 | 1
1300020 | table6_append | 57638 | 0 | 3
1300021 | table6_append | 57637 | 0 | 1
1300021 | table6_append | 57638 | 0 | 1
(4 rows)
-- repair shard in append distributed table
SELECT master_copy_shard_placement(1300020, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
master_copy_shard_placement
---------------------------------------------------------------------
(1 row)
-- status after shard repair
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport, p.colocationid, sp.shardstate
FROM
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
WHERE
p.logicalrelid = s.logicalrelid AND
s.shardid = sp.shardid AND
p.logicalrelid = 'table6_append'::regclass
ORDER BY s.shardid, sp.nodeport;
shardid | logicalrelid | nodeport | colocationid | shardstate
---------------------------------------------------------------------
1300020 | table6_append | 57637 | 0 | 1
1300020 | table6_append | 57638 | 0 | 1
1300021 | table6_append | 57637 | 0 | 1
1300021 | table6_append | 57638 | 0 | 1
(4 rows)
-- test repair while all placements of one shard in colocation group is unhealthy
-- manually set shardstate as inactive
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE shardid = 1300000;
-- status before shard repair
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport, p.colocationid, sp.shardstate
FROM
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
WHERE
p.logicalrelid = s.logicalrelid AND
s.shardid = sp.shardid AND
colocationid = (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'table1_group1'::regclass)
ORDER BY s.shardid, sp.nodeport;
shardid | logicalrelid | nodeport | colocationid | shardstate
---------------------------------------------------------------------
1300000 | table1_group1 | 57637 | 1000 | 3
1300000 | table1_group1 | 57638 | 1000 | 3
1300001 | table1_group1 | 57637 | 1000 | 1
1300001 | table1_group1 | 57638 | 1000 | 1
1300002 | table1_group1 | 57637 | 1000 | 1
1300002 | table1_group1 | 57638 | 1000 | 1
1300003 | table1_group1 | 57637 | 1000 | 1
1300003 | table1_group1 | 57638 | 1000 | 1
1300004 | table2_group1 | 57637 | 1000 | 1
1300004 | table2_group1 | 57638 | 1000 | 3
1300005 | table2_group1 | 57637 | 1000 | 1
1300005 | table2_group1 | 57638 | 1000 | 1
1300006 | table2_group1 | 57637 | 1000 | 1
1300006 | table2_group1 | 57638 | 1000 | 1
1300007 | table2_group1 | 57637 | 1000 | 1
1300007 | table2_group1 | 57638 | 1000 | 1
(16 rows)
-- repair while all placements of one shard in colocation group is unhealthy
SELECT master_copy_shard_placement(1300000, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
ERROR: source placement must be in active state
-- status after shard repair
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport, p.colocationid, sp.shardstate
FROM
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
WHERE
p.logicalrelid = s.logicalrelid AND
s.shardid = sp.shardid AND
colocationid = (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'table1_group1'::regclass)
ORDER BY s.shardid, sp.nodeport;
shardid | logicalrelid | nodeport | colocationid | shardstate
---------------------------------------------------------------------
1300000 | table1_group1 | 57637 | 1000 | 3
1300000 | table1_group1 | 57638 | 1000 | 3
1300001 | table1_group1 | 57637 | 1000 | 1
1300001 | table1_group1 | 57638 | 1000 | 1
1300002 | table1_group1 | 57637 | 1000 | 1
1300002 | table1_group1 | 57638 | 1000 | 1
1300003 | table1_group1 | 57637 | 1000 | 1
1300003 | table1_group1 | 57638 | 1000 | 1
1300004 | table2_group1 | 57637 | 1000 | 1
1300004 | table2_group1 | 57638 | 1000 | 3
1300005 | table2_group1 | 57637 | 1000 | 1
1300005 | table2_group1 | 57638 | 1000 | 1
1300006 | table2_group1 | 57637 | 1000 | 1
1300006 | table2_group1 | 57638 | 1000 | 1
1300007 | table2_group1 | 57637 | 1000 | 1
1300007 | table2_group1 | 57638 | 1000 | 1
(16 rows)

View File

@ -1111,59 +1111,61 @@ ERROR: extension "citus" already exists
-- Snapshot of state at 11.1-1
ALTER EXTENSION citus UPDATE TO '11.1-1';
SELECT * FROM multi_extension.print_extension_changes();
previous_object | current_object
previous_object | current_object
---------------------------------------------------------------------
access method columnar |
function alter_columnar_table_reset(regclass,boolean,boolean,boolean,boolean) void |
function alter_columnar_table_set(regclass,integer,integer,name,integer) void |
function citus_internal.columnar_ensure_am_depends_catalog() void |
function citus_internal.downgrade_columnar_storage(regclass) void |
function citus_internal.upgrade_columnar_storage(regclass) void |
function columnar.columnar_handler(internal) table_am_handler |
function isolate_tenant_to_new_shard(regclass,"any",text) bigint |
function replicate_reference_tables() void |
function worker_cleanup_job_schema_cache() void |
function worker_create_schema(bigint,text) void |
function worker_fetch_foreign_file(text,text,bigint,text[],integer[]) void |
function worker_fetch_partition_file(bigint,integer,integer,integer,text,integer) void |
function worker_hash_partition_table(bigint,integer,text,text,oid,anyarray) void |
function worker_merge_files_into_table(bigint,integer,text[],text[]) void |
function worker_range_partition_table(bigint,integer,text,text,oid,anyarray) void |
function worker_repartition_cleanup(bigint) void |
schema columnar |
sequence columnar.storageid_seq |
table columnar.chunk |
table columnar.chunk_group |
table columnar.options |
table columnar.stripe |
| function citus_cleanup_orphaned_resources()
| function citus_internal_delete_partition_metadata(regclass) void
| function citus_job_cancel(bigint) void
| function citus_job_wait(bigint,citus_job_status) void
| function citus_locks() SETOF record
| function citus_split_shard_by_split_points(bigint,text[],integer[],citus.shard_transfer_mode) void
| function create_distributed_table_concurrently(regclass,text,citus.distribution_type,text,integer) void
| function isolate_tenant_to_new_shard(regclass,"any",text,citus.shard_transfer_mode) bigint
| function replicate_reference_tables(citus.shard_transfer_mode) void
| function worker_copy_table_to_node(regclass,integer) void
| function worker_split_copy(bigint,text,split_copy_info[]) void
| function worker_split_shard_release_dsm() void
| function worker_split_shard_replication_setup(split_shard_info[]) SETOF replication_slot_info
| sequence pg_dist_background_job_job_id_seq
| sequence pg_dist_background_task_task_id_seq
| sequence pg_dist_cleanup_recordid_seq
| sequence pg_dist_operationid_seq
| table pg_dist_background_job
| table pg_dist_background_task
| table pg_dist_background_task_depend
| table pg_dist_cleanup
| type citus_job_status
| type citus_task_status
| type replication_slot_info
| type split_copy_info
| type split_shard_info
| view citus_locks
(50 rows)
access method columnar |
function alter_columnar_table_reset(regclass,boolean,boolean,boolean,boolean) void |
function alter_columnar_table_set(regclass,integer,integer,name,integer) void |
function citus_copy_shard_placement(bigint,text,integer,text,integer,boolean,citus.shard_transfer_mode) void |
function citus_internal.columnar_ensure_am_depends_catalog() void |
function citus_internal.downgrade_columnar_storage(regclass) void |
function citus_internal.upgrade_columnar_storage(regclass) void |
function columnar.columnar_handler(internal) table_am_handler |
function isolate_tenant_to_new_shard(regclass,"any",text) bigint |
function replicate_reference_tables() void |
function worker_cleanup_job_schema_cache() void |
function worker_create_schema(bigint,text) void |
function worker_fetch_foreign_file(text,text,bigint,text[],integer[]) void |
function worker_fetch_partition_file(bigint,integer,integer,integer,text,integer) void |
function worker_hash_partition_table(bigint,integer,text,text,oid,anyarray) void |
function worker_merge_files_into_table(bigint,integer,text[],text[]) void |
function worker_range_partition_table(bigint,integer,text,text,oid,anyarray) void |
function worker_repartition_cleanup(bigint) void |
schema columnar |
sequence columnar.storageid_seq |
table columnar.chunk |
table columnar.chunk_group |
table columnar.options |
table columnar.stripe |
| function citus_cleanup_orphaned_resources()
| function citus_copy_shard_placement(bigint,text,integer,text,integer,citus.shard_transfer_mode) void
| function citus_internal_delete_partition_metadata(regclass) void
| function citus_job_cancel(bigint) void
| function citus_job_wait(bigint,citus_job_status) void
| function citus_locks() SETOF record
| function citus_split_shard_by_split_points(bigint,text[],integer[],citus.shard_transfer_mode) void
| function create_distributed_table_concurrently(regclass,text,citus.distribution_type,text,integer) void
| function isolate_tenant_to_new_shard(regclass,"any",text,citus.shard_transfer_mode) bigint
| function replicate_reference_tables(citus.shard_transfer_mode) void
| function worker_copy_table_to_node(regclass,integer) void
| function worker_split_copy(bigint,text,split_copy_info[]) void
| function worker_split_shard_release_dsm() void
| function worker_split_shard_replication_setup(split_shard_info[]) SETOF replication_slot_info
| sequence pg_dist_background_job_job_id_seq
| sequence pg_dist_background_task_task_id_seq
| sequence pg_dist_cleanup_recordid_seq
| sequence pg_dist_operationid_seq
| table pg_dist_background_job
| table pg_dist_background_task
| table pg_dist_background_task_depend
| table pg_dist_cleanup
| type citus_job_status
| type citus_task_status
| type replication_slot_info
| type split_copy_info
| type split_shard_info
| view citus_locks
(52 rows)
DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff;
-- show running version

View File

@ -100,9 +100,9 @@ ORDER BY
(12 rows)
\c - - - :master_port
-- Check that master_copy_shard_placement cannot be run with MX tables
-- Check that citus_copy_shard_placement cannot be run with MX tables
SELECT
master_copy_shard_placement(shardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port, false, 'force_logical')
citus_copy_shard_placement(shardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'force_logical')
FROM
pg_dist_shard NATURAL JOIN pg_dist_shard_placement
WHERE
@ -229,7 +229,7 @@ ORDER BY
-- Check that the UDFs cannot be called from the workers
SELECT
master_copy_shard_placement(shardid, 'localhost', :worker_2_port, 'localhost', :worker_1_port, false, 'force_logical')
citus_copy_shard_placement(shardid, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'force_logical')
FROM
pg_dist_shard NATURAL JOIN pg_dist_shard_placement
WHERE

View File

@ -1128,7 +1128,7 @@ CREATE OR REPLACE FUNCTION immutable_bleat(text) RETURNS int LANGUAGE plpgsql IM
CREATE TABLE test_table (test_id integer NOT NULL, data text);
SET citus.shard_count TO 2;
SET citus.shard_replication_factor TO 2;
SELECT create_distributed_table('test_table', 'test_id', 'hash');
SELECT create_distributed_table('test_table', 'test_id', 'hash', colocate_with := 'none');
create_distributed_table
---------------------------------------------------------------------
@ -1157,7 +1157,7 @@ EXECUTE countsome; -- no replanning
(0 rows)
-- invalidate half of the placements using SQL, should invalidate via trigger
UPDATE pg_dist_shard_placement SET shardstate = '3'
DELETE FROM pg_dist_shard_placement
WHERE shardid IN (
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_table'::regclass)
AND nodeport = :worker_1_port;
@ -1172,13 +1172,13 @@ EXECUTE countsome; -- no replanning
---------------------------------------------------------------------
(0 rows)
-- repair shards, should invalidate via master_metadata_utility.c
SELECT master_copy_shard_placement(shardid, 'localhost', :worker_2_port, 'localhost', :worker_1_port)
-- copy shards, should invalidate via master_metadata_utility.c
SELECT citus_copy_shard_placement(shardid, 'localhost', :worker_2_port, 'localhost', :worker_1_port, transfer_mode := 'block_writes')
FROM pg_dist_shard_placement
WHERE shardid IN (
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_table'::regclass)
AND nodeport = :worker_1_port;
master_copy_shard_placement
AND nodeport = :worker_2_port;
citus_copy_shard_placement
---------------------------------------------------------------------

View File

@ -1421,23 +1421,6 @@ SELECT master_get_table_ddl_events('reference_schema.reference_table_ddl');
ALTER TABLE reference_schema.reference_table_ddl OWNER TO postgres
(2 rows)
-- in reality, we wouldn't need to repair any reference table shard placements
-- however, the test could be relevant for other purposes
SELECT placementid AS a_placement_id FROM pg_dist_shard_placement WHERE shardid = :a_shard_id AND nodeport = :worker_1_port \gset
SELECT placementid AS b_placement_id FROM pg_dist_shard_placement WHERE shardid = :a_shard_id AND nodeport = :worker_2_port \gset
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE placementid = :a_placement_id;
SELECT master_copy_shard_placement(:a_shard_id, 'localhost', :worker_2_port, 'localhost', :worker_1_port);
master_copy_shard_placement
---------------------------------------------------------------------
(1 row)
SELECT shardid, shardstate FROM pg_dist_shard_placement WHERE placementid = :a_placement_id;
shardid | shardstate
---------------------------------------------------------------------
1250019 | 1
(1 row)
-- some queries that are captured in functions
CREATE OR REPLACE FUNCTION select_count_all() RETURNS bigint AS '
SELECT

View File

@ -1,94 +0,0 @@
SET citus.next_shard_id TO 820000;
SELECT groupid AS worker_2_group FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
SELECT groupid AS worker_1_group FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
-- ===================================================================
-- test shard repair functionality
-- ===================================================================
-- create a table and create its distribution metadata
CREATE TABLE customer_engagements ( id integer, created_at date, event_data text );
-- add some indexes
CREATE INDEX ON customer_engagements (id);
CREATE INDEX ON customer_engagements (created_at);
CREATE INDEX ON customer_engagements (event_data);
-- distribute the table
-- create a single shard on the first worker
SET citus.shard_count TO 1;
SET citus.shard_replication_factor TO 2;
SELECT create_distributed_table('customer_engagements', 'id', 'hash');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- ingest some data for the tests
INSERT INTO customer_engagements VALUES (1, '01-01-2015', 'first event');
INSERT INTO customer_engagements VALUES (2, '02-01-2015', 'second event');
INSERT INTO customer_engagements VALUES (1, '03-01-2015', 'third event');
-- the following queries does the following:
-- (i) create a new shard
-- (ii) mark the second shard placements as unhealthy
-- (iii) do basic checks i.e., only allow copy from healthy placement to unhealthy ones
-- (iv) do a successful master_copy_shard_placement from the first placement to the second
-- (v) mark the first placement as unhealthy and execute a query that is routed to the second placement
-- get the newshardid
SELECT shardid as newshardid FROM pg_dist_shard WHERE logicalrelid = 'customer_engagements'::regclass
\gset
-- now, update the second placement as unhealthy
UPDATE pg_dist_placement SET shardstate = 3 WHERE shardid = :newshardid
AND groupid = :worker_2_group;
-- cannot repair a shard after a modification (transaction still open during repair)
BEGIN;
ALTER TABLE customer_engagements ADD COLUMN value float;
SELECT master_copy_shard_placement(:newshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
ERROR: cannot open new connections after the first modification command within a transaction
ROLLBACK;
BEGIN;
INSERT INTO customer_engagements VALUES (4, '04-01-2015', 'fourth event');
SELECT master_copy_shard_placement(:newshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
ERROR: cannot open new connections after the first modification command within a transaction
ROLLBACK;
-- modifications after reparing a shard are fine (will use new metadata)
BEGIN;
SELECT master_copy_shard_placement(:newshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
master_copy_shard_placement
---------------------------------------------------------------------
(1 row)
ALTER TABLE customer_engagements ADD COLUMN value float;
ROLLBACK;
BEGIN;
SELECT master_copy_shard_placement(:newshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
master_copy_shard_placement
---------------------------------------------------------------------
(1 row)
INSERT INTO customer_engagements VALUES (4, '04-01-2015', 'fourth event');
ROLLBACK;
-- deactivate placement
UPDATE pg_dist_placement SET shardstate = 1 WHERE groupid = :worker_2_group and shardid = :newshardid;
SELECT master_copy_shard_placement(:newshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
ERROR: target placement must be in inactive state
UPDATE pg_dist_placement SET shardstate = 3 WHERE groupid = :worker_2_group and shardid = :newshardid;
-- also try to copy from an inactive placement
SELECT master_copy_shard_placement(:newshardid, 'localhost', :worker_2_port, 'localhost', :worker_1_port);
ERROR: source placement must be in active state
-- "copy" this shard from the first placement to the second one
SELECT master_copy_shard_placement(:newshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
master_copy_shard_placement
---------------------------------------------------------------------
(1 row)
-- now, update first placement as unhealthy (and raise a notice) so that queries are not routed to there
UPDATE pg_dist_placement SET shardstate = 3 WHERE shardid = :newshardid AND groupid = :worker_1_group;
-- get the data from the second placement
SELECT * FROM customer_engagements;
id | created_at | event_data
---------------------------------------------------------------------
1 | 01-01-2015 | first event
2 | 02-01-2015 | second event
1 | 03-01-2015 | third event
(3 rows)

View File

@ -787,7 +787,6 @@ SELECT citus_copy_shard_placement(
(SELECT shardid FROM pg_dist_shard WHERE logicalrelid='ref_table'::regclass::oid),
'localhost', :worker_1_port,
'localhost', :worker_2_port,
do_repair := false,
transfer_mode := 'block_writes');
ERROR: Copying shards to a non-existing node is not supported
HINT: Add the target node via SELECT citus_add_node('localhost', 57638);
@ -802,7 +801,6 @@ SELECT citus_copy_shard_placement(
(SELECT shardid FROM pg_dist_shard WHERE logicalrelid='ref_table'::regclass::oid),
'localhost', :worker_1_port,
'localhost', :worker_2_port,
do_repair := false,
transfer_mode := 'block_writes');
ERROR: Copying shards to a secondary (e.g., replica) node is not supported
SELECT citus_remove_node('localhost', :worker_2_port);
@ -822,7 +820,6 @@ SELECT citus_copy_shard_placement(
(SELECT shardid FROM pg_dist_shard WHERE logicalrelid='ref_table'::regclass::oid),
'localhost', :worker_1_port,
'localhost', :worker_2_port,
do_repair := false,
transfer_mode := 'block_writes');
ERROR: Copying shards to a non-active node is not supported
HINT: Activate the target node via SELECT citus_activate_node('localhost', 57638);
@ -1005,7 +1002,7 @@ SELECT min(result) = max(result) AS consistent FROM run_command_on_placements('r
t
(1 row)
-- test that metadata is synced when master_copy_shard_placement replicates
-- test that metadata is synced when citus_copy_shard_placement replicates
-- reference table shards
SET citus.replicate_reference_tables_on_activate TO off;
SELECT 1 FROM master_remove_node('localhost', :worker_2_port);
@ -1021,13 +1018,12 @@ SELECT 1 FROM master_add_node('localhost', :worker_2_port);
(1 row)
SET citus.shard_replication_factor TO 1;
SELECT master_copy_shard_placement(
SELECT citus_copy_shard_placement(
:ref_table_shard,
'localhost', :worker_1_port,
'localhost', :worker_2_port,
do_repair := false,
transfer_mode := 'block_writes');
master_copy_shard_placement
citus_copy_shard_placement
---------------------------------------------------------------------
(1 row)

View File

@ -19,8 +19,8 @@ CREATE TABLE test_schema_support.nation_append(
n_regionkey integer not null,
n_comment varchar(152)
);
SELECT master_create_distributed_table('test_schema_support.nation_append', 'n_nationkey', 'append');
master_create_distributed_table
SELECT create_distributed_table('test_schema_support.nation_append', 'n_nationkey', 'append');
create_distributed_table
---------------------------------------------------------------------
(1 row)
@ -41,8 +41,8 @@ CREATE TABLE test_schema_support."nation._'append" (
n_name char(25) not null,
n_regionkey integer not null,
n_comment varchar(152));
SELECT master_create_distributed_table('test_schema_support."nation._''append"', 'n_nationkey', 'append');
master_create_distributed_table
SELECT create_distributed_table('test_schema_support."nation._''append"', 'n_nationkey', 'append');
create_distributed_table
---------------------------------------------------------------------
(1 row)
@ -351,6 +351,7 @@ SET search_path TO public;
SELECT quote_ident(current_setting('lc_collate')) as current_locale \gset
CREATE COLLATION test_schema_support.english (LOCALE = :current_locale);
\c - - - :master_port
SET citus.shard_replication_factor TO 2;
CREATE TABLE test_schema_support.nation_hash_collation(
n_nationkey integer not null,
n_name char(25) not null COLLATE test_schema_support.english,
@ -364,14 +365,8 @@ SELECT master_get_table_ddl_events('test_schema_support.nation_hash_collation')
CREATE TABLE test_schema_support.nation_hash_collation (n_nationkey integer NOT NULL, n_name character(25) NOT NULL COLLATE test_schema_support.english, n_regionkey integer NOT NULL, n_comment character varying(152))
(2 rows)
SELECT master_create_distributed_table('test_schema_support.nation_hash_collation', 'n_nationkey', 'hash');
master_create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT master_create_worker_shards('test_schema_support.nation_hash_collation', 4, 2);
master_create_worker_shards
SELECT create_distributed_table('test_schema_support.nation_hash_collation', 'n_nationkey', 'hash', shard_count := 4, colocate_with := 'none');
create_distributed_table
---------------------------------------------------------------------
(1 row)
@ -407,14 +402,9 @@ CREATE TABLE nation_hash_collation_search_path(
n_regionkey integer not null,
n_comment varchar(152)
);
SELECT master_create_distributed_table('nation_hash_collation_search_path', 'n_nationkey', 'hash');
master_create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT master_create_worker_shards('nation_hash_collation_search_path', 4, 2);
master_create_worker_shards
SET citus.shard_replication_factor TO 2;
SELECT create_distributed_table('nation_hash_collation_search_path', 'n_nationkey', 'hash', shard_count := 4, colocate_with := 'none');
create_distributed_table
---------------------------------------------------------------------
(1 row)
@ -452,14 +442,8 @@ CREATE TABLE test_schema_support.nation_hash_composite_types(
n_comment varchar(152),
test_col test_schema_support.new_composite_type
);
SELECT master_create_distributed_table('test_schema_support.nation_hash_composite_types', 'n_nationkey', 'hash');
master_create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT master_create_worker_shards('test_schema_support.nation_hash_composite_types', 4, 2);
master_create_worker_shards
SELECT create_distributed_table('test_schema_support.nation_hash_composite_types', 'n_nationkey', 'hash', shard_count := 4, colocate_with := 'none');
create_distributed_table
---------------------------------------------------------------------
(1 row)
@ -638,12 +622,12 @@ DROP INDEX index1;
\c - - - :worker_1_port
\d test_schema_support.index1_1190003
\c - - - :master_port
-- test master_copy_shard_placement with schemas
-- test citus_copy_shard_placement with schemas
SET search_path TO public;
-- mark shard as inactive
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE shardid = 1190000 and nodeport = :worker_1_port;
SELECT master_copy_shard_placement(1190000, 'localhost', :worker_2_port, 'localhost', :worker_1_port);
master_copy_shard_placement
-- delete placements
DELETE FROM pg_dist_shard_placement WHERE shardid = 1190000 and nodeport = :worker_1_port;
SELECT citus_copy_shard_placement(1190000, 'localhost', :worker_2_port, 'localhost', :worker_1_port, transfer_mode := 'block_writes');
citus_copy_shard_placement
---------------------------------------------------------------------
(1 row)
@ -659,9 +643,9 @@ SELECT shardstate, nodename, nodeport FROM pg_dist_shard_placement WHERE shardid
--test with search_path is set
SET search_path TO test_schema_support;
-- mark shard as inactive
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE shardid = 1190000 and nodeport = :worker_1_port;
SELECT master_copy_shard_placement(1190000, 'localhost', :worker_2_port, 'localhost', :worker_1_port);
master_copy_shard_placement
DELETE FROM pg_dist_shard_placement WHERE shardid = 1190000 and nodeport = :worker_1_port;
SELECT citus_copy_shard_placement(1190000, 'localhost', :worker_2_port, 'localhost', :worker_1_port, transfer_mode := 'block_writes');
citus_copy_shard_placement
---------------------------------------------------------------------
(1 row)

View File

@ -667,7 +667,6 @@ SET search_path to "Tenant Isolation";
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE nodeport = :worker_1_port;
SELECT isolate_tenant_to_new_shard('lineitem_date', '1997-08-08', shard_transfer_mode => 'block_writes');
ERROR: cannot isolate tenant because relation "lineitem_date" has an inactive shard placement for the shard xxxxx
HINT: Use master_copy_shard_placement UDF to repair the inactive shard placement.
UPDATE pg_dist_shard_placement SET shardstate = 1 WHERE nodeport = :worker_1_port;
\c - mx_isolation_role_ent - :master_port
SET search_path to "Tenant Isolation";

View File

@ -667,7 +667,6 @@ SET search_path to "Tenant Isolation";
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE nodeport = :worker_1_port;
SELECT isolate_tenant_to_new_shard('lineitem_date', '1997-08-08', shard_transfer_mode => 'force_logical');
ERROR: cannot isolate tenant because relation "lineitem_date" has an inactive shard placement for the shard xxxxx
HINT: Use master_copy_shard_placement UDF to repair the inactive shard placement.
UPDATE pg_dist_shard_placement SET shardstate = 1 WHERE nodeport = :worker_1_port;
\c - mx_isolation_role_ent - :master_port
SET search_path to "Tenant Isolation";

View File

@ -302,7 +302,7 @@ SELECT count(*) FROM mx_table;
5
(1 row)
-- master_copy_shard_placement
-- citus_copy_shard_placement
SELECT logicalrelid, shardid AS testshardid, nodename, nodeport
FROM pg_dist_shard NATURAL JOIN pg_dist_shard_placement
WHERE logicalrelid = 'mx_table'::regclass AND nodeport=:worker_1_port
@ -311,7 +311,7 @@ LIMIT 1 \gset
SELECT groupid AS worker_2_group FROM pg_dist_node WHERE nodeport = :worker_2_port \gset
INSERT INTO pg_dist_placement (groupid, shardid, shardstate, shardlength)
VALUES (:worker_2_group, :testshardid, 3, 0);
SELECT master_copy_shard_placement(:testshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
SELECT citus_copy_shard_placement(:testshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
ERROR: operation is not allowed on this node
HINT: Connect to the coordinator and run it again.
SELECT shardid, nodename, nodeport, shardstate

View File

@ -203,7 +203,7 @@ SELECT create_distributed_table('collections_agg', 'key');
INSERT INTO collections_agg SELECT key, sum(key) FROM collections_1 GROUP BY key;
-- coordinator roll-up
INSERT INTO collections_agg SELECT collection_id, sum(key) FROM collections_1 GROUP BY collection_id;
-- now make sure that repair functionality works fine
-- now make sure that copy functionality works fine
-- create a table and create its distribution metadata
CREATE TABLE customer_engagements (id integer, event_id int) PARTITION BY LIST ( event_id );
CREATE TABLE customer_engagements_1
@ -220,7 +220,7 @@ CREATE INDEX ON customer_engagements (id, event_id);
-- create a single shard on the first worker
SET citus.shard_count TO 1;
SET citus.shard_replication_factor TO 2;
SELECT create_distributed_table('customer_engagements', 'id', 'hash');
SELECT create_distributed_table('customer_engagements', 'id', 'hash', colocate_with := 'none');
create_distributed_table
---------------------------------------------------------------------
@ -231,30 +231,22 @@ INSERT INTO customer_engagements VALUES (1, 1);
INSERT INTO customer_engagements VALUES (2, 1);
INSERT INTO customer_engagements VALUES (1, 2);
INSERT INTO customer_engagements VALUES (2, 2);
-- the following queries does the following:
-- (i) create a new shard
-- (ii) mark the second shard placements as unhealthy
-- (iii) do basic checks i.e., only allow copy from healthy placement to unhealthy ones
-- (iv) do a successful master_copy_shard_placement from the first placement to the second
-- (v) mark the first placement as unhealthy and execute a query that is routed to the second placement
SELECT groupid AS worker_2_group FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
SELECT groupid AS worker_1_group FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
-- get the newshardid
SELECT shardid as newshardid FROM pg_dist_shard WHERE logicalrelid = 'customer_engagements'::regclass
\gset
-- now, update the second placement as unhealthy
UPDATE pg_dist_placement SET shardstate = 3 WHERE shardid = :newshardid
AND groupid = :worker_2_group;
-- cannot repair a shard after a modification (transaction still open during repair)
-- delete all the placements on the second node belonging to partitioning hierarchy
DELETE FROM pg_dist_shard_placement p USING pg_dist_shard s
WHERE s.shardid = p.shardid AND nodeport = :worker_2_port AND logicalrelid::text LIKE 'customer_engagements%';
-- cannot copy a shard after a modification (transaction still open during copy)
BEGIN;
INSERT INTO customer_engagements VALUES (1, 1);
SELECT master_copy_shard_placement(:newshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
SELECT citus_copy_shard_placement(:newshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port, transfer_mode := 'block_writes');
ERROR: cannot open new connections after the first modification command within a transaction
ROLLBACK;
-- modifications after reparing a shard are fine (will use new metadata)
-- modifications after copying a shard are fine (will use new metadata)
BEGIN;
SELECT master_copy_shard_placement(:newshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
master_copy_shard_placement
SELECT citus_copy_shard_placement(:newshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port, transfer_mode := 'block_writes');
citus_copy_shard_placement
---------------------------------------------------------------------
(1 row)
@ -271,8 +263,8 @@ SELECT * FROM customer_engagements ORDER BY 1,2,3;
ROLLBACK;
BEGIN;
SELECT master_copy_shard_placement(:newshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
master_copy_shard_placement
SELECT citus_copy_shard_placement(:newshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port, transfer_mode := 'block_writes');
citus_copy_shard_placement
---------------------------------------------------------------------
(1 row)

View File

@ -164,7 +164,7 @@ SELECT * FROM master_get_table_ddl_events('test_range_dist');
(2 rows)
--
-- Test master_copy_shard_placement with a fake_am table
-- Test copy_copy_shard_placement with a fake_am table
--
select a.shardid, a.nodeport
FROM pg_dist_shard b, pg_dist_shard_placement a
@ -178,15 +178,14 @@ ORDER BY a.shardid, nodeport;
60003 | 57638
(4 rows)
-- Change repmodel to allow master_copy_shard_placement
-- Change repmodel to allow copy_copy_shard_placement
UPDATE pg_dist_partition SET repmodel='c' WHERE logicalrelid = 'test_hash_dist'::regclass;
SELECT master_copy_shard_placement(
SELECT citus_copy_shard_placement(
get_shard_id_for_distribution_column('test_hash_dist', '1'),
'localhost', :worker_1_port,
'localhost', :worker_2_port,
do_repair := false,
transfer_mode := 'block_writes');
master_copy_shard_placement
citus_copy_shard_placement
---------------------------------------------------------------------
(1 row)

View File

@ -35,7 +35,7 @@ ORDER BY 1;
function citus_cleanup_orphaned_shards()
function citus_conninfo_cache_invalidate()
function citus_coordinator_nodeid()
function citus_copy_shard_placement(bigint,text,integer,text,integer,boolean,citus.shard_transfer_mode)
function citus_copy_shard_placement(bigint,text,integer,text,integer,citus.shard_transfer_mode)
function citus_create_restore_point(text)
function citus_disable_node(text,integer,boolean)
function citus_dist_local_group_cache_invalidate()

View File

@ -26,7 +26,6 @@ test: isolation_citus_dist_activity
test: isolation_remove_coordinator
test: isolation_insert_select_repartition
test: isolation_dml_vs_repair
test: isolation_copy_placement_vs_copy_placement
test: isolation_concurrent_dml

View File

@ -180,7 +180,7 @@ test: multi_modifications
test: multi_distribution_metadata
test: multi_prune_shard_list
test: multi_upsert multi_simple_queries multi_data_types
test: master_copy_shard_placement
test: citus_copy_shard_placement
# multi_utilities cannot be run in parallel with other tests because it checks
# global locks
test: multi_utilities
@ -188,7 +188,7 @@ test: foreign_key_to_reference_table validate_constraint
test: multi_repartition_udt multi_repartitioned_subquery_udf multi_subtransactions
test: multi_modifying_xacts
test: multi_generate_ddl_commands multi_repair_shards
test: multi_generate_ddl_commands
test: multi_create_shards
test: multi_transaction_recovery
@ -253,10 +253,8 @@ test: multi_truncate
# ----------
# multi_colocation_utils tests utility functions written for co-location feature & internal API
# multi_colocated_shard_transfer tests master_copy_shard_placement with colocated tables.
# ----------
test: multi_colocation_utils
test: multi_colocated_shard_transfer
# ----------
# node_conninfo_reload tests that node_conninfo changes take effect

View File

@ -28,7 +28,7 @@ step "s1-load-cache"
step "s1-repair-placement"
{
SELECT master_copy_shard_placement((SELECT * FROM selected_shard_for_test_table), 'localhost', 57637, 'localhost', 57638);
SELECT citus_copy_shard_placement((SELECT * FROM selected_shard_for_test_table), 'localhost', 57637, 'localhost', 57638);
}
session "s2"
@ -38,14 +38,14 @@ step "s2-begin"
BEGIN;
}
step "s2-set-placement-inactive"
step "s2-delete-inactive"
{
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE shardid IN (SELECT * FROM selected_shard_for_test_table) AND nodeport = 57638;
DELETE FROM pg_dist_shard_placement WHERE shardid IN (SELECT * FROM selected_shard_for_test_table) AND nodeport = 57638;
}
step "s2-repair-placement"
{
SELECT master_copy_shard_placement((SELECT * FROM selected_shard_for_test_table), 'localhost', 57637, 'localhost', 57638);
SELECT citus_copy_shard_placement((SELECT * FROM selected_shard_for_test_table), 'localhost', 57637, 'localhost', 57638, transfer_mode := 'block_writes');
}
// since test_hash_table has rep > 1 simple select query doesn't hit all placements
@ -65,7 +65,7 @@ step "s2-commit"
// note that "s1-repair-placement" errors out but that is expected
// given that "s2-repair-placement" succeeds and the placement is
// already repaired
permutation "s1-load-cache" "s2-load-cache" "s2-set-placement-inactive" "s2-begin" "s2-repair-placement" "s1-repair-placement" "s2-commit"
permutation "s1-load-cache" "s2-load-cache" "s2-delete-inactive" "s2-begin" "s2-repair-placement" "s1-repair-placement" "s2-commit"
// the same test without the load caches
permutation "s2-set-placement-inactive" "s2-begin" "s2-repair-placement" "s1-repair-placement" "s2-commit"
permutation "s2-delete-inactive" "s2-begin" "s2-repair-placement" "s1-repair-placement" "s2-commit"

View File

@ -73,14 +73,14 @@ step "s2-begin"
BEGIN;
}
step "s2-set-placement-inactive"
step "s2-delete-inactive"
{
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE shardid IN (SELECT * FROM selected_shard) AND nodeport = 57638;
DELETE FROM pg_dist_shard_placement WHERE shardid IN (SELECT * FROM selected_shard) AND nodeport = 57638;
}
step "s2-repair-placement"
{
SELECT master_copy_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638);
SELECT citus_copy_shard_placement((SELECT * FROM selected_shard), 'localhost', 57637, 'localhost', 57638, transfer_mode := 'block_writes');
}
step "s2-commit"
@ -113,16 +113,16 @@ step "s2-print-index-count"
// repair a placement while concurrently performing an update/delete/insert/copy
// note that at some points we use "s1-select" just after "s1-begin" given that BEGIN
// may invalidate cache at certain cases
permutation "s1-load-cache" "s1-insert" "s1-begin" "s1-select" "s2-set-placement-inactive" "s2-begin" "s2-repair-placement" "s1-update" "s2-commit" "s1-commit" "s2-print-content"
permutation "s1-load-cache" "s1-insert" "s1-begin" "s1-select" "s2-set-placement-inactive" "s2-begin" "s2-repair-placement" "s1-delete" "s2-commit" "s1-commit" "s2-print-content"
permutation "s1-load-cache" "s1-begin" "s1-select" "s2-set-placement-inactive" "s2-begin" "s2-repair-placement" "s1-insert" "s2-commit" "s1-commit" "s2-print-content"
permutation "s1-load-cache" "s1-begin" "s1-select" "s2-set-placement-inactive" "s2-begin" "s2-repair-placement" "s1-copy" "s2-commit" "s1-commit" "s2-print-content"
permutation "s1-load-cache" "s1-begin" "s1-select" "s2-set-placement-inactive" "s2-begin" "s2-repair-placement" "s1-ddl" "s2-commit" "s1-commit" "s2-print-index-count"
permutation "s1-load-cache" "s1-insert" "s1-begin" "s1-select" "s2-delete-inactive" "s2-begin" "s2-repair-placement" "s1-update" "s2-commit" "s1-commit" "s2-print-content"
permutation "s1-load-cache" "s1-insert" "s1-begin" "s1-select" "s2-delete-inactive" "s2-begin" "s2-repair-placement" "s1-delete" "s2-commit" "s1-commit" "s2-print-content"
permutation "s1-load-cache" "s1-begin" "s1-select" "s2-delete-inactive" "s2-begin" "s2-repair-placement" "s1-insert" "s2-commit" "s1-commit" "s2-print-content"
permutation "s1-load-cache" "s1-begin" "s1-select" "s2-delete-inactive" "s2-begin" "s2-repair-placement" "s1-copy" "s2-commit" "s1-commit" "s2-print-content"
permutation "s1-load-cache" "s1-begin" "s1-select" "s2-delete-inactive" "s2-begin" "s2-repair-placement" "s1-ddl" "s2-commit" "s1-commit" "s2-print-index-count"
// the same tests without loading the cache at first
permutation "s1-insert" "s1-begin" "s1-select" "s2-set-placement-inactive" "s2-begin" "s2-repair-placement" "s1-update" "s2-commit" "s1-commit" "s2-print-content"
permutation "s1-insert" "s1-begin" "s1-select" "s2-set-placement-inactive" "s2-begin" "s2-repair-placement" "s1-delete" "s2-commit" "s1-commit" "s2-print-content"
permutation "s1-begin" "s1-select" "s2-set-placement-inactive" "s2-begin" "s2-repair-placement" "s1-insert" "s2-commit" "s1-commit" "s2-print-content"
permutation "s1-begin" "s1-select" "s2-set-placement-inactive" "s2-begin" "s2-repair-placement" "s1-copy" "s2-commit" "s1-commit" "s2-print-content"
permutation "s1-begin" "s1-select" "s2-set-placement-inactive" "s2-begin" "s2-repair-placement" "s1-ddl" "s2-commit" "s1-commit" "s2-print-index-count"
permutation "s1-insert" "s1-begin" "s1-select" "s2-delete-inactive" "s2-begin" "s2-repair-placement" "s1-update" "s2-commit" "s1-commit" "s2-print-content"
permutation "s1-insert" "s1-begin" "s1-select" "s2-delete-inactive" "s2-begin" "s2-repair-placement" "s1-delete" "s2-commit" "s1-commit" "s2-print-content"
permutation "s1-begin" "s1-select" "s2-delete-inactive" "s2-begin" "s2-repair-placement" "s1-insert" "s2-commit" "s1-commit" "s2-print-content"
permutation "s1-begin" "s1-select" "s2-delete-inactive" "s2-begin" "s2-repair-placement" "s1-copy" "s2-commit" "s1-commit" "s2-print-content"
permutation "s1-begin" "s1-select" "s2-delete-inactive" "s2-begin" "s2-repair-placement" "s1-ddl" "s2-commit" "s1-commit" "s2-print-index-count"

View File

@ -169,7 +169,7 @@ step "s4-print-waiting-advisory-locks"
step "s4-print-colocations"
{
SELECT * FROM pg_dist_colocation ORDER BY colocationid;
SELECT shardcount, replicationfactor, distributioncolumntype, distributioncolumncollation FROM pg_dist_colocation ORDER BY colocationid;
}
// show concurrent insert is NOT blocked by create_distributed_table_concurrently

View File

@ -1,104 +0,0 @@
setup
{
CREATE TABLE test_dml_vs_repair (test_id integer NOT NULL, data int);
SET citus.shard_replication_factor TO 2;
SELECT create_distributed_table('test_dml_vs_repair', 'test_id', 'hash', shard_count:=1);
}
teardown
{
DROP TABLE IF EXISTS test_dml_vs_repair CASCADE;
}
session "s1"
setup
{
DEALLOCATE all;
TRUNCATE test_dml_vs_repair;
PREPARE insertone AS INSERT INTO test_dml_vs_repair VALUES(1, 1);
PREPARE insertall AS INSERT INTO test_dml_vs_repair SELECT test_id, data+1 FROM test_dml_vs_repair;
}
step "s1-begin"
{
BEGIN;
}
step "s1-insertone"
{
INSERT INTO test_dml_vs_repair VALUES(1, 1);
}
step "s1-prepared-insertone"
{
EXECUTE insertone;
}
step "s1-insertall"
{
INSERT INTO test_dml_vs_repair SELECT test_id, data+1 FROM test_dml_vs_repair;
}
step "s1-prepared-insertall"
{
EXECUTE insertall;
}
step "s1-display"
{
SELECT * FROM test_dml_vs_repair WHERE test_id = 1 ORDER BY test_id;
}
step "s1-commit"
{
COMMIT;
}
session "s2"
step "s2-begin"
{
BEGIN;
}
step "s2-invalidate-57637"
{
UPDATE pg_dist_shard_placement SET shardstate = '3' WHERE shardid = (SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_dml_vs_repair'::regclass) AND nodeport = 57637;
}
step "s2-invalidate-57638"
{
UPDATE pg_dist_shard_placement SET shardstate = '3' WHERE shardid = (SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_dml_vs_repair'::regclass) AND nodeport = 57638;
}
step "s2-revalidate-57638"
{
UPDATE pg_dist_shard_placement SET shardstate = '1' WHERE shardid = (SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_dml_vs_repair'::regclass) AND nodeport = 57638;
}
step "s2-repair"
{
SELECT master_copy_shard_placement((SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_dml_vs_repair'::regclass), 'localhost', 57638, 'localhost', 57637);
}
step "s2-commit"
{
COMMIT;
}
// verify that repair is blocked by ongoing modifying simple transaction
permutation "s2-invalidate-57637" "s1-begin" "s1-insertone" "s2-repair" "s1-commit"
// verify that repair is blocked by ongoing modifying insert...select transaction
permutation "s1-insertone" "s2-invalidate-57637" "s1-begin" "s1-insertall" "s2-repair" "s1-commit"
// verify that modifications wait for shard repair
permutation "s2-invalidate-57637" "s2-begin" "s2-repair" "s1-insertone" "s2-commit" "s2-invalidate-57638" "s1-display" "s2-invalidate-57637" "s2-revalidate-57638" "s1-display"
// verify that prepared plain modifications wait for shard repair
permutation "s2-invalidate-57637" "s1-prepared-insertone" "s2-begin" "s2-repair" "s1-prepared-insertone" "s2-commit" "s2-invalidate-57638" "s1-display" "s2-invalidate-57637" "s2-revalidate-57638" "s1-display"
// verify that prepared INSERT ... SELECT waits for shard repair
permutation "s2-invalidate-57637" "s1-insertone" "s1-prepared-insertall" "s2-begin" "s2-repair" "s1-prepared-insertall" "s2-commit" "s2-invalidate-57638" "s1-display" "s2-invalidate-57637" "s2-revalidate-57638" "s1-display"

View File

@ -1,4 +1,4 @@
-- Tests for master_copy_shard_placement, which can be used for adding replicas in statement-based replication
-- Tests for citus_copy_shard_placement, which can be used for adding replicas in statement-based replication
CREATE SCHEMA mcsp;
SET search_path TO mcsp;
SET citus.next_shard_id TO 8139000;
@ -24,7 +24,7 @@ CREATE TABLE history_p1 PARTITION OF history FOR VALUES FROM ('2019-01-01') TO (
CREATE TABLE history_p2 PARTITION OF history FOR VALUES FROM ('2020-01-01') TO ('2021-01-01');
SELECT create_distributed_table('history','key');
-- Mark tables as non-mx tables, in order to be able to test master_copy_shard_placement
-- Mark tables as non-mx tables, in order to be able to test citus_copy_shard_placement
UPDATE pg_dist_partition SET repmodel='c' WHERE logicalrelid IN
('data'::regclass, 'history'::regclass);
@ -35,47 +35,42 @@ INSERT INTO history VALUES ('key-1', '2020-02-01', 'old');
INSERT INTO history VALUES ('key-1', '2019-10-01', 'older');
-- verify we error out if no healthy placement exists at source
SELECT master_copy_shard_placement(
SELECT citus_copy_shard_placement(
get_shard_id_for_distribution_column('data', 'key-1'),
'localhost', :worker_1_port,
'localhost', :worker_2_port,
do_repair := false,
transfer_mode := 'block_writes');
-- verify we error out if source and destination are the same
SELECT master_copy_shard_placement(
SELECT citus_copy_shard_placement(
get_shard_id_for_distribution_column('data', 'key-1'),
'localhost', :worker_2_port,
'localhost', :worker_2_port,
do_repair := false,
transfer_mode := 'block_writes');
-- verify we error out if target already contains a healthy placement
SELECT master_copy_shard_placement(
SELECT citus_copy_shard_placement(
(SELECT shardid FROM pg_dist_shard WHERE logicalrelid='ref_table'::regclass::oid),
'localhost', :worker_1_port,
'localhost', :worker_2_port,
do_repair := false,
transfer_mode := 'block_writes');
-- verify we error out if table has foreign key constraints
INSERT INTO ref_table SELECT 1, value FROM data;
ALTER TABLE data ADD CONSTRAINT distfk FOREIGN KEY (value) REFERENCES ref_table (b) MATCH FULL;
SELECT master_copy_shard_placement(
SELECT citus_copy_shard_placement(
get_shard_id_for_distribution_column('data', 'key-1'),
'localhost', :worker_2_port,
'localhost', :worker_1_port,
do_repair := false);
'localhost', :worker_1_port);
ALTER TABLE data DROP CONSTRAINT distfk;
-- replicate shard that contains key-1
SELECT master_copy_shard_placement(
SELECT citus_copy_shard_placement(
get_shard_id_for_distribution_column('data', 'key-1'),
'localhost', :worker_2_port,
'localhost', :worker_1_port,
do_repair := false,
transfer_mode := 'block_writes');
-- forcefully mark the old replica as inactive
@ -102,11 +97,10 @@ RESET client_min_messages;
CREATE TABLE mx_table(a int);
SELECT create_distributed_table('mx_table', 'a');
SELECT master_copy_shard_placement(
SELECT citus_copy_shard_placement(
get_shard_id_for_distribution_column('mx_table', '1'),
'localhost', :worker_1_port,
'localhost', :worker_2_port,
do_repair := false,
transfer_mode := 'block_writes');
SET client_min_messages TO ERROR;

View File

@ -223,8 +223,8 @@ SELECT master_create_empty_shard('citus_local_table_1');
-- get_shard_id_for_distribution_column is supported
SELECT get_shard_id_for_distribution_column('citus_local_table_1', 'not_checking_this_arg_for_non_dist_tables');
SELECT get_shard_id_for_distribution_column('citus_local_table_1');
-- master_copy_shard_placement is not supported
SELECT master_copy_shard_placement(shardid, 'localhost', :master_port, 'localhost', :worker_1_port, true)
-- citus_copy_shard_placement is not supported
SELECT citus_copy_shard_placement(shardid, 'localhost', :master_port, 'localhost', :worker_1_port)
FROM (SELECT shardid FROM pg_dist_shard WHERE logicalrelid='citus_local_table_1'::regclass) as shardid;
-- undistribute_table is supported
BEGIN;

View File

@ -19,12 +19,12 @@ SELECT citus_add_local_table_to_metadata('citus_local_table');
-- isolate_tenant_to_new_shard is not supported
SELECT isolate_tenant_to_new_shard('citus_local_table', 100, shard_transfer_mode => 'block_writes');
-- master_copy_shard_placement is not supported
SELECT master_copy_shard_placement(shardid, 'localhost', :master_port, 'localhost', :worker_1_port, false)
-- citus_copy_shard_placement is not supported
SELECT citus_copy_shard_placement(shardid, 'localhost', :master_port, 'localhost', :worker_1_port, false)
FROM (SELECT shardid FROM pg_dist_shard WHERE logicalrelid='citus_local_table'::regclass) as shardid;
-- master_move_shard_placement is not supported
SELECT master_move_shard_placement(shardid, 'localhost', :master_port, 'localhost', :worker_1_port)
-- citus_move_shard_placement is not supported
SELECT citus_move_shard_placement(shardid, 'localhost', :master_port, 'localhost', :worker_1_port)
FROM (SELECT shardid FROM pg_dist_shard WHERE logicalrelid='citus_local_table'::regclass) as shardid;
-- replicate_table_shards is not suported

View File

@ -82,16 +82,8 @@ SELECT logicalrelid FROM pg_dist_partition WHERE colocationid = 92448300 ORDER B
SELECT update_distributed_table_colocation('rep2', 'rep1');
SELECT logicalrelid FROM pg_dist_partition WHERE colocationid = 92448300 ORDER BY 1;
UPDATE pg_dist_placement SET shardstate = 3 WHERE shardid = 92448300 AND groupid = 0;
SELECT shardid, shardstate, nodeport FROM pg_dist_shard_placement WHERE shardid = 92448300 ORDER BY placementid;
-- cannot copy from an orphaned shard
SELECT * FROM citus_copy_shard_placement(92448300, 'localhost', :worker_1_port, 'localhost', :master_port);
-- cannot copy to an orphaned shard
SELECT * FROM citus_copy_shard_placement(92448300, 'localhost', :worker_2_port, 'localhost', :worker_1_port);
-- can still copy to an inactive shard
SELECT * FROM citus_copy_shard_placement(92448300, 'localhost', :worker_2_port, 'localhost', :master_port);
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
BEGIN;

View File

@ -38,7 +38,7 @@ SELECT master_create_distributed_table('table6_append', 'id', 'append');
SELECT master_create_empty_shard('table6_append');
SELECT master_create_empty_shard('table6_append');
-- Mark tables as non-mx tables, in order to be able to test master_copy_shard_placement
-- Mark tables as non-mx tables, in order to be able to test citus_copy_shard_placement
UPDATE pg_dist_partition SET repmodel='c' WHERE logicalrelid IN
('table1_group1'::regclass, 'table2_group1'::regclass, 'table5_groupX'::regclass);
@ -56,10 +56,10 @@ WHERE
ORDER BY s.shardid, sp.nodeport;
-- try to copy colocated shards without a replica identity
SELECT master_copy_shard_placement(13000000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, false);
SELECT citus_copy_shard_placement(13000000, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
-- copy colocated shards
SELECT master_copy_shard_placement(13000000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, false, 'force_logical');
SELECT citus_copy_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
@ -79,7 +79,7 @@ SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.table2_
-- copy colocated shards again to see error message
SELECT master_copy_shard_placement(13000000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, false, 'force_logical');
SELECT citus_copy_shard_placement(13000000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'force_logical');
-- test copying NOT colocated shard
@ -94,7 +94,7 @@ WHERE
ORDER BY s.shardid, sp.nodeport;
-- copy NOT colocated shard
SELECT master_copy_shard_placement(13000012, 'localhost', :worker_1_port, 'localhost', :worker_2_port, false, 'force_logical');
SELECT citus_copy_shard_placement(13000012, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'force_logical');
-- status after shard copy
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
@ -119,7 +119,7 @@ WHERE
ORDER BY s.shardid, sp.nodeport;
-- copy shard in append distributed table
SELECT master_copy_shard_placement(13000020, 'localhost', :worker_2_port, 'localhost', :worker_1_port, false, 'force_logical');
SELECT citus_copy_shard_placement(13000020, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'force_logical');
-- status after shard copy
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport
@ -281,7 +281,7 @@ SELECT "Constraint", "Definition" FROM table_fkeys
-- test shard copy with foreign constraints
-- we expect it to error out because we do not support foreign constraints with replication factor > 1
SELECT master_copy_shard_placement(13000022, 'localhost', :worker_2_port, 'localhost', :worker_1_port, false);
SELECT citus_copy_shard_placement(13000022, 'localhost', :worker_2_port, 'localhost', :worker_1_port);
-- lets also test that master_move_shard_placement doesn't break serials

View File

@ -1,114 +0,0 @@
--
-- MULTI_COLOCATED_SHARD_TRANSFER
--
-- These tables are created in multi_colocation_utils test
-- test repair
-- manually set shardstate as inactive
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE nodeport = :worker_2_port AND (shardid = 1300000 OR shardid = 1300004);
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE nodeport = :worker_2_port AND shardid = 1300016;
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE nodeport = :worker_2_port AND shardid = 1300020;
-- test repairing colocated shards
-- status before shard repair
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport, p.colocationid, sp.shardstate
FROM
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
WHERE
p.logicalrelid = s.logicalrelid AND
s.shardid = sp.shardid AND
colocationid = (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'table1_group1'::regclass)
ORDER BY s.shardid, sp.nodeport;
-- repair colocated shards
SELECT master_copy_shard_placement(1300000, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
-- status after shard repair
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport, p.colocationid, sp.shardstate
FROM
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
WHERE
p.logicalrelid = s.logicalrelid AND
s.shardid = sp.shardid AND
colocationid = (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'table1_group1'::regclass)
ORDER BY s.shardid, sp.nodeport;
-- test repairing NOT colocated shard
-- status before shard repair
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport, p.colocationid, sp.shardstate
FROM
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
WHERE
p.logicalrelid = s.logicalrelid AND
s.shardid = sp.shardid AND
p.logicalrelid = 'table5_groupX'::regclass
ORDER BY s.shardid, sp.nodeport;
-- repair NOT colocated shard
SELECT master_copy_shard_placement(1300016, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
-- status after shard repair
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport, p.colocationid, sp.shardstate
FROM
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
WHERE
p.logicalrelid = s.logicalrelid AND
s.shardid = sp.shardid AND
p.logicalrelid = 'table5_groupX'::regclass
ORDER BY s.shardid, sp.nodeport;
-- test repairing shard in append distributed table
-- status before shard repair
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport, p.colocationid, sp.shardstate
FROM
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
WHERE
p.logicalrelid = s.logicalrelid AND
s.shardid = sp.shardid AND
p.logicalrelid = 'table6_append'::regclass
ORDER BY s.shardid, sp.nodeport;
-- repair shard in append distributed table
SELECT master_copy_shard_placement(1300020, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
-- status after shard repair
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport, p.colocationid, sp.shardstate
FROM
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
WHERE
p.logicalrelid = s.logicalrelid AND
s.shardid = sp.shardid AND
p.logicalrelid = 'table6_append'::regclass
ORDER BY s.shardid, sp.nodeport;
-- test repair while all placements of one shard in colocation group is unhealthy
-- manually set shardstate as inactive
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE shardid = 1300000;
-- status before shard repair
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport, p.colocationid, sp.shardstate
FROM
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
WHERE
p.logicalrelid = s.logicalrelid AND
s.shardid = sp.shardid AND
colocationid = (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'table1_group1'::regclass)
ORDER BY s.shardid, sp.nodeport;
-- repair while all placements of one shard in colocation group is unhealthy
SELECT master_copy_shard_placement(1300000, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
-- status after shard repair
SELECT s.shardid, s.logicalrelid::regclass, sp.nodeport, p.colocationid, sp.shardstate
FROM
pg_dist_partition p, pg_dist_shard s, pg_dist_shard_placement sp
WHERE
p.logicalrelid = s.logicalrelid AND
s.shardid = sp.shardid AND
colocationid = (SELECT colocationid FROM pg_dist_partition WHERE logicalrelid = 'table1_group1'::regclass)
ORDER BY s.shardid, sp.nodeport;

View File

@ -54,9 +54,9 @@ ORDER BY
logicalrelid, shardid;
\c - - - :master_port
-- Check that master_copy_shard_placement cannot be run with MX tables
-- Check that citus_copy_shard_placement cannot be run with MX tables
SELECT
master_copy_shard_placement(shardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port, false, 'force_logical')
citus_copy_shard_placement(shardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port, 'force_logical')
FROM
pg_dist_shard NATURAL JOIN pg_dist_shard_placement
WHERE
@ -146,7 +146,7 @@ ORDER BY
-- Check that the UDFs cannot be called from the workers
SELECT
master_copy_shard_placement(shardid, 'localhost', :worker_2_port, 'localhost', :worker_1_port, false, 'force_logical')
citus_copy_shard_placement(shardid, 'localhost', :worker_2_port, 'localhost', :worker_1_port, 'force_logical')
FROM
pg_dist_shard NATURAL JOIN pg_dist_shard_placement
WHERE

View File

@ -614,7 +614,7 @@ CREATE OR REPLACE FUNCTION immutable_bleat(text) RETURNS int LANGUAGE plpgsql IM
CREATE TABLE test_table (test_id integer NOT NULL, data text);
SET citus.shard_count TO 2;
SET citus.shard_replication_factor TO 2;
SELECT create_distributed_table('test_table', 'test_id', 'hash');
SELECT create_distributed_table('test_table', 'test_id', 'hash', colocate_with := 'none');
-- avoid 9.6+ only context messages
\set VERBOSITY terse
@ -627,19 +627,19 @@ EXECUTE countsome; -- should indicate planning
EXECUTE countsome; -- no replanning
-- invalidate half of the placements using SQL, should invalidate via trigger
UPDATE pg_dist_shard_placement SET shardstate = '3'
DELETE FROM pg_dist_shard_placement
WHERE shardid IN (
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_table'::regclass)
AND nodeport = :worker_1_port;
EXECUTE countsome; -- should indicate replanning
EXECUTE countsome; -- no replanning
-- repair shards, should invalidate via master_metadata_utility.c
SELECT master_copy_shard_placement(shardid, 'localhost', :worker_2_port, 'localhost', :worker_1_port)
-- copy shards, should invalidate via master_metadata_utility.c
SELECT citus_copy_shard_placement(shardid, 'localhost', :worker_2_port, 'localhost', :worker_1_port, transfer_mode := 'block_writes')
FROM pg_dist_shard_placement
WHERE shardid IN (
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'test_table'::regclass)
AND nodeport = :worker_1_port;
AND nodeport = :worker_2_port;
EXECUTE countsome; -- should indicate replanning
EXECUTE countsome; -- no replanning

View File

@ -905,15 +905,6 @@ SELECT master_update_shard_statistics(:a_shard_id);
SELECT master_get_table_ddl_events('reference_schema.reference_table_ddl');
-- in reality, we wouldn't need to repair any reference table shard placements
-- however, the test could be relevant for other purposes
SELECT placementid AS a_placement_id FROM pg_dist_shard_placement WHERE shardid = :a_shard_id AND nodeport = :worker_1_port \gset
SELECT placementid AS b_placement_id FROM pg_dist_shard_placement WHERE shardid = :a_shard_id AND nodeport = :worker_2_port \gset
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE placementid = :a_placement_id;
SELECT master_copy_shard_placement(:a_shard_id, 'localhost', :worker_2_port, 'localhost', :worker_1_port);
SELECT shardid, shardstate FROM pg_dist_shard_placement WHERE placementid = :a_placement_id;
-- some queries that are captured in functions
CREATE OR REPLACE FUNCTION select_count_all() RETURNS bigint AS '
SELECT

View File

@ -1,82 +0,0 @@
SET citus.next_shard_id TO 820000;
SELECT groupid AS worker_2_group FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
SELECT groupid AS worker_1_group FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
-- ===================================================================
-- test shard repair functionality
-- ===================================================================
-- create a table and create its distribution metadata
CREATE TABLE customer_engagements ( id integer, created_at date, event_data text );
-- add some indexes
CREATE INDEX ON customer_engagements (id);
CREATE INDEX ON customer_engagements (created_at);
CREATE INDEX ON customer_engagements (event_data);
-- distribute the table
-- create a single shard on the first worker
SET citus.shard_count TO 1;
SET citus.shard_replication_factor TO 2;
SELECT create_distributed_table('customer_engagements', 'id', 'hash');
-- ingest some data for the tests
INSERT INTO customer_engagements VALUES (1, '01-01-2015', 'first event');
INSERT INTO customer_engagements VALUES (2, '02-01-2015', 'second event');
INSERT INTO customer_engagements VALUES (1, '03-01-2015', 'third event');
-- the following queries does the following:
-- (i) create a new shard
-- (ii) mark the second shard placements as unhealthy
-- (iii) do basic checks i.e., only allow copy from healthy placement to unhealthy ones
-- (iv) do a successful master_copy_shard_placement from the first placement to the second
-- (v) mark the first placement as unhealthy and execute a query that is routed to the second placement
-- get the newshardid
SELECT shardid as newshardid FROM pg_dist_shard WHERE logicalrelid = 'customer_engagements'::regclass
\gset
-- now, update the second placement as unhealthy
UPDATE pg_dist_placement SET shardstate = 3 WHERE shardid = :newshardid
AND groupid = :worker_2_group;
-- cannot repair a shard after a modification (transaction still open during repair)
BEGIN;
ALTER TABLE customer_engagements ADD COLUMN value float;
SELECT master_copy_shard_placement(:newshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
ROLLBACK;
BEGIN;
INSERT INTO customer_engagements VALUES (4, '04-01-2015', 'fourth event');
SELECT master_copy_shard_placement(:newshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
ROLLBACK;
-- modifications after reparing a shard are fine (will use new metadata)
BEGIN;
SELECT master_copy_shard_placement(:newshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
ALTER TABLE customer_engagements ADD COLUMN value float;
ROLLBACK;
BEGIN;
SELECT master_copy_shard_placement(:newshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
INSERT INTO customer_engagements VALUES (4, '04-01-2015', 'fourth event');
ROLLBACK;
-- deactivate placement
UPDATE pg_dist_placement SET shardstate = 1 WHERE groupid = :worker_2_group and shardid = :newshardid;
SELECT master_copy_shard_placement(:newshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
UPDATE pg_dist_placement SET shardstate = 3 WHERE groupid = :worker_2_group and shardid = :newshardid;
-- also try to copy from an inactive placement
SELECT master_copy_shard_placement(:newshardid, 'localhost', :worker_2_port, 'localhost', :worker_1_port);
-- "copy" this shard from the first placement to the second one
SELECT master_copy_shard_placement(:newshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
-- now, update first placement as unhealthy (and raise a notice) so that queries are not routed to there
UPDATE pg_dist_placement SET shardstate = 3 WHERE shardid = :newshardid AND groupid = :worker_1_group;
-- get the data from the second placement
SELECT * FROM customer_engagements;

View File

@ -515,7 +515,6 @@ SELECT citus_copy_shard_placement(
(SELECT shardid FROM pg_dist_shard WHERE logicalrelid='ref_table'::regclass::oid),
'localhost', :worker_1_port,
'localhost', :worker_2_port,
do_repair := false,
transfer_mode := 'block_writes');
-- verify direct call to citus_copy_shard_placement errors if target node is secondary
@ -524,7 +523,6 @@ SELECT citus_copy_shard_placement(
(SELECT shardid FROM pg_dist_shard WHERE logicalrelid='ref_table'::regclass::oid),
'localhost', :worker_1_port,
'localhost', :worker_2_port,
do_repair := false,
transfer_mode := 'block_writes');
SELECT citus_remove_node('localhost', :worker_2_port);
@ -534,7 +532,6 @@ SELECT citus_copy_shard_placement(
(SELECT shardid FROM pg_dist_shard WHERE logicalrelid='ref_table'::regclass::oid),
'localhost', :worker_1_port,
'localhost', :worker_2_port,
do_repair := false,
transfer_mode := 'block_writes');
SELECT 1 FROM master_activate_node('localhost', :worker_2_port);
@ -620,7 +617,7 @@ SELECT count(*) - :ref_table_placements FROM pg_dist_shard_placement WHERE shard
SELECT min(result) = max(result) AS consistent FROM run_command_on_placements('ref_table', 'SELECT sum(a) FROM %s');
-- test that metadata is synced when master_copy_shard_placement replicates
-- test that metadata is synced when citus_copy_shard_placement replicates
-- reference table shards
SET citus.replicate_reference_tables_on_activate TO off;
SELECT 1 FROM master_remove_node('localhost', :worker_2_port);
@ -628,11 +625,10 @@ SELECT 1 FROM master_add_node('localhost', :worker_2_port);
SET citus.shard_replication_factor TO 1;
SELECT master_copy_shard_placement(
SELECT citus_copy_shard_placement(
:ref_table_shard,
'localhost', :worker_1_port,
'localhost', :worker_2_port,
do_repair := false,
transfer_mode := 'block_writes');
SELECT result::int - :ref_table_placements

View File

@ -32,7 +32,7 @@ CREATE TABLE test_schema_support.nation_append(
n_regionkey integer not null,
n_comment varchar(152)
);
SELECT master_create_distributed_table('test_schema_support.nation_append', 'n_nationkey', 'append');
SELECT create_distributed_table('test_schema_support.nation_append', 'n_nationkey', 'append');
SELECT master_create_empty_shard('test_schema_support.nation_append') as simple_shardid \gset
-- append table to shard
@ -55,7 +55,7 @@ CREATE TABLE test_schema_support."nation._'append" (
n_regionkey integer not null,
n_comment varchar(152));
SELECT master_create_distributed_table('test_schema_support."nation._''append"', 'n_nationkey', 'append');
SELECT create_distributed_table('test_schema_support."nation._''append"', 'n_nationkey', 'append');
SELECT master_create_empty_shard('test_schema_support."nation._''append"') as special_shardid \gset
copy test_schema_support."nation._'append" FROM STDIN with (append_to_shard :special_shardid, delimiter '|');
@ -298,6 +298,7 @@ SELECT quote_ident(current_setting('lc_collate')) as current_locale \gset
CREATE COLLATION test_schema_support.english (LOCALE = :current_locale);
\c - - - :master_port
SET citus.shard_replication_factor TO 2;
CREATE TABLE test_schema_support.nation_hash_collation(
n_nationkey integer not null,
@ -306,8 +307,7 @@ CREATE TABLE test_schema_support.nation_hash_collation(
n_comment varchar(152)
);
SELECT master_get_table_ddl_events('test_schema_support.nation_hash_collation') ORDER BY 1;
SELECT master_create_distributed_table('test_schema_support.nation_hash_collation', 'n_nationkey', 'hash');
SELECT master_create_worker_shards('test_schema_support.nation_hash_collation', 4, 2);
SELECT create_distributed_table('test_schema_support.nation_hash_collation', 'n_nationkey', 'hash', shard_count := 4, colocate_with := 'none');
\copy test_schema_support.nation_hash_collation FROM STDIN with delimiter '|';
0|ALGERIA|0|haggle. carefully final deposits detect slyly agai
@ -329,8 +329,8 @@ CREATE TABLE nation_hash_collation_search_path(
n_regionkey integer not null,
n_comment varchar(152)
);
SELECT master_create_distributed_table('nation_hash_collation_search_path', 'n_nationkey', 'hash');
SELECT master_create_worker_shards('nation_hash_collation_search_path', 4, 2);
SET citus.shard_replication_factor TO 2;
SELECT create_distributed_table('nation_hash_collation_search_path', 'n_nationkey', 'hash', shard_count := 4, colocate_with := 'none');
\copy nation_hash_collation_search_path FROM STDIN with delimiter '|';
0|ALGERIA|0|haggle. carefully final deposits detect slyly agai
@ -355,8 +355,7 @@ CREATE TABLE test_schema_support.nation_hash_composite_types(
n_comment varchar(152),
test_col test_schema_support.new_composite_type
);
SELECT master_create_distributed_table('test_schema_support.nation_hash_composite_types', 'n_nationkey', 'hash');
SELECT master_create_worker_shards('test_schema_support.nation_hash_composite_types', 4, 2);
SELECT create_distributed_table('test_schema_support.nation_hash_composite_types', 'n_nationkey', 'hash', shard_count := 4, colocate_with := 'none');
-- insert some data to verify composite type queries
\copy test_schema_support.nation_hash_composite_types FROM STDIN with delimiter '|';
@ -463,12 +462,12 @@ DROP INDEX index1;
\c - - - :master_port
-- test master_copy_shard_placement with schemas
-- test citus_copy_shard_placement with schemas
SET search_path TO public;
-- mark shard as inactive
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE shardid = 1190000 and nodeport = :worker_1_port;
SELECT master_copy_shard_placement(1190000, 'localhost', :worker_2_port, 'localhost', :worker_1_port);
-- delete placements
DELETE FROM pg_dist_shard_placement WHERE shardid = 1190000 and nodeport = :worker_1_port;
SELECT citus_copy_shard_placement(1190000, 'localhost', :worker_2_port, 'localhost', :worker_1_port, transfer_mode := 'block_writes');
-- verify shardstate
SELECT shardstate, nodename, nodeport FROM pg_dist_shard_placement WHERE shardid = 1190000 ORDER BY nodeport;
@ -478,8 +477,8 @@ SELECT shardstate, nodename, nodeport FROM pg_dist_shard_placement WHERE shardid
SET search_path TO test_schema_support;
-- mark shard as inactive
UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE shardid = 1190000 and nodeport = :worker_1_port;
SELECT master_copy_shard_placement(1190000, 'localhost', :worker_2_port, 'localhost', :worker_1_port);
DELETE FROM pg_dist_shard_placement WHERE shardid = 1190000 and nodeport = :worker_1_port;
SELECT citus_copy_shard_placement(1190000, 'localhost', :worker_2_port, 'localhost', :worker_1_port, transfer_mode := 'block_writes');
-- verify shardstate
SELECT shardstate, nodename, nodeport FROM pg_dist_shard_placement WHERE shardid = 1190000 ORDER BY nodeport;

View File

@ -182,7 +182,7 @@ SELECT master_remove_distributed_table_metadata_from_workers('mx_table'::regclas
SELECT master_remove_partition_metadata('mx_table'::regclass, 'public', 'mx_table');
SELECT count(*) FROM mx_table;
-- master_copy_shard_placement
-- citus_copy_shard_placement
SELECT logicalrelid, shardid AS testshardid, nodename, nodeport
FROM pg_dist_shard NATURAL JOIN pg_dist_shard_placement
WHERE logicalrelid = 'mx_table'::regclass AND nodeport=:worker_1_port
@ -193,7 +193,7 @@ SELECT groupid AS worker_2_group FROM pg_dist_node WHERE nodeport = :worker_2_po
INSERT INTO pg_dist_placement (groupid, shardid, shardstate, shardlength)
VALUES (:worker_2_group, :testshardid, 3, 0);
SELECT master_copy_shard_placement(:testshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
SELECT citus_copy_shard_placement(:testshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
SELECT shardid, nodename, nodeport, shardstate
FROM pg_dist_shard_placement

View File

@ -152,7 +152,7 @@ INSERT INTO collections_agg SELECT key, sum(key) FROM collections_1 GROUP BY key
-- coordinator roll-up
INSERT INTO collections_agg SELECT collection_id, sum(key) FROM collections_1 GROUP BY collection_id;
-- now make sure that repair functionality works fine
-- now make sure that copy functionality works fine
-- create a table and create its distribution metadata
CREATE TABLE customer_engagements (id integer, event_id int) PARTITION BY LIST ( event_id );
@ -173,7 +173,7 @@ CREATE INDEX ON customer_engagements (id, event_id);
-- create a single shard on the first worker
SET citus.shard_count TO 1;
SET citus.shard_replication_factor TO 2;
SELECT create_distributed_table('customer_engagements', 'id', 'hash');
SELECT create_distributed_table('customer_engagements', 'id', 'hash', colocate_with := 'none');
-- ingest some data for the tests
INSERT INTO customer_engagements VALUES (1, 1);
@ -181,39 +181,29 @@ INSERT INTO customer_engagements VALUES (2, 1);
INSERT INTO customer_engagements VALUES (1, 2);
INSERT INTO customer_engagements VALUES (2, 2);
-- the following queries does the following:
-- (i) create a new shard
-- (ii) mark the second shard placements as unhealthy
-- (iii) do basic checks i.e., only allow copy from healthy placement to unhealthy ones
-- (iv) do a successful master_copy_shard_placement from the first placement to the second
-- (v) mark the first placement as unhealthy and execute a query that is routed to the second placement
SELECT groupid AS worker_2_group FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
SELECT groupid AS worker_1_group FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
-- get the newshardid
SELECT shardid as newshardid FROM pg_dist_shard WHERE logicalrelid = 'customer_engagements'::regclass
\gset
-- now, update the second placement as unhealthy
UPDATE pg_dist_placement SET shardstate = 3 WHERE shardid = :newshardid
AND groupid = :worker_2_group;
-- delete all the placements on the second node belonging to partitioning hierarchy
DELETE FROM pg_dist_shard_placement p USING pg_dist_shard s
WHERE s.shardid = p.shardid AND nodeport = :worker_2_port AND logicalrelid::text LIKE 'customer_engagements%';
-- cannot repair a shard after a modification (transaction still open during repair)
-- cannot copy a shard after a modification (transaction still open during copy)
BEGIN;
INSERT INTO customer_engagements VALUES (1, 1);
SELECT master_copy_shard_placement(:newshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
SELECT citus_copy_shard_placement(:newshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port, transfer_mode := 'block_writes');
ROLLBACK;
-- modifications after reparing a shard are fine (will use new metadata)
-- modifications after copying a shard are fine (will use new metadata)
BEGIN;
SELECT master_copy_shard_placement(:newshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
SELECT citus_copy_shard_placement(:newshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port, transfer_mode := 'block_writes');
ALTER TABLE customer_engagements ADD COLUMN value float DEFAULT 1.0;
SELECT * FROM customer_engagements ORDER BY 1,2,3;
ROLLBACK;
BEGIN;
SELECT master_copy_shard_placement(:newshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port);
SELECT citus_copy_shard_placement(:newshardid, 'localhost', :worker_1_port, 'localhost', :worker_2_port, transfer_mode := 'block_writes');
INSERT INTO customer_engagements VALUES (1, 1);
SELECT count(*) FROM customer_engagements;
ROLLBACK;

View File

@ -81,7 +81,7 @@ COPY test_range_dist FROM PROGRAM 'echo 25, 16 && echo 26, 1 && echo 27, 4 && ec
SELECT * FROM master_get_table_ddl_events('test_range_dist');
--
-- Test master_copy_shard_placement with a fake_am table
-- Test copy_copy_shard_placement with a fake_am table
--
select a.shardid, a.nodeport
@ -89,14 +89,13 @@ FROM pg_dist_shard b, pg_dist_shard_placement a
WHERE a.shardid=b.shardid AND logicalrelid = 'test_hash_dist'::regclass::oid
ORDER BY a.shardid, nodeport;
-- Change repmodel to allow master_copy_shard_placement
-- Change repmodel to allow copy_copy_shard_placement
UPDATE pg_dist_partition SET repmodel='c' WHERE logicalrelid = 'test_hash_dist'::regclass;
SELECT master_copy_shard_placement(
SELECT citus_copy_shard_placement(
get_shard_id_for_distribution_column('test_hash_dist', '1'),
'localhost', :worker_1_port,
'localhost', :worker_2_port,
do_repair := false,
transfer_mode := 'block_writes');
select a.shardid, a.nodeport