Merge branch 'main' into baby_step_pg_15

pull/6022/head
Önder Kalacı 2022-07-18 15:02:39 +02:00 committed by GitHub
commit 90b1afe31e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
47 changed files with 6543 additions and 195 deletions

View File

@ -643,7 +643,19 @@ workflows:
image_tag: '<< pipeline.parameters.pg13_version >>'
make: check-enterprise-failure
requires: [build-13]
- test-citus:
name: 'test-13_check-split'
pg_major: 13
image_tag: '<< pipeline.parameters.pg13_version >>'
make: check-split
requires: [build-13]
- test-citus:
name: 'test-14_check-split'
pg_major: 14
image_tag: '<< pipeline.parameters.pg14_version >>'
make: check-split
requires: [build-14]
- test-citus:
name: 'test-14_check-enterprise'
pg_major: 14

View File

@ -262,7 +262,6 @@ static StringInfo ConstructCopyStatement(CopyStmt *copyStatement, int64 shardId)
static void SendCopyDataToAll(StringInfo dataBuffer, int64 shardId, List *connectionList);
static void SendCopyDataToPlacement(StringInfo dataBuffer, int64 shardId,
MultiConnection *connection);
static void ReportCopyError(MultiConnection *connection, PGresult *result);
static uint32 AvailableColumnCount(TupleDesc tupleDescriptor);
static Oid TypeForColumnName(Oid relationId, TupleDesc tupleDescriptor, char *columnName);
@ -1201,7 +1200,7 @@ EndRemoteCopy(int64 shardId, List *connectionList)
* ReportCopyError tries to report a useful error message for the user from
* the remote COPY error messages.
*/
static void
void
ReportCopyError(MultiConnection *connection, PGresult *result)
{
char *remoteMessage = PQresultErrorField(result, PG_DIAG_MESSAGE_PRIMARY);

View File

@ -74,8 +74,6 @@ static void PartitioningTupleDestPutTuple(TupleDestination *self, Task *task,
HeapTuple heapTuple, uint64 tupleLibpqSize);
static TupleDesc PartitioningTupleDestTupleDescForQuery(TupleDestination *self, int
queryNumber);
static ArrayType * CreateArrayFromDatums(Datum *datumArray, bool *nullsArray, int
datumCount, Oid typeId);
static char * SourceShardPrefix(const char *resultPrefix, uint64 shardId);
static DistributedResultFragment * TupleToDistributedResultFragment(HeapTuple heapTuple,
TupleDesc tupleDesc,
@ -372,7 +370,7 @@ ShardMinMaxValueArrays(ShardInterval **shardIntervalArray, int shardCount,
/*
* CreateArrayFromDatums creates an array consisting of given values and nulls.
*/
static ArrayType *
ArrayType *
CreateArrayFromDatums(Datum *datumArray, bool *nullsArray, int datumCount, Oid typeId)
{
bool typeByValue = false;

View File

@ -95,18 +95,6 @@ typedef struct PartitionedResultDestReceiver
} PartitionedResultDestReceiver;
static Portal StartPortalForQueryExecution(const char *queryString);
static CitusTableCacheEntry * QueryTupleShardSearchInfo(ArrayType *minValuesArray,
ArrayType *maxValuesArray,
char partitionMethod,
Var *partitionColumn);
static DestReceiver * CreatePartitionedResultDestReceiver(int partitionColumnIndex,
int partitionCount,
CitusTableCacheEntry *
shardSearchInfo,
DestReceiver **
partitionedDestReceivers,
bool lazyStartup,
bool allowNullPartitionValues);
static void PartitionedResultDestReceiverStartup(DestReceiver *dest, int operation,
TupleDesc inputTupleDescriptor);
static bool PartitionedResultDestReceiverReceive(TupleTableSlot *slot,
@ -319,7 +307,7 @@ StartPortalForQueryExecution(const char *queryString)
* information so that FindShardInterval() can find the shard corresponding
* to a tuple.
*/
static CitusTableCacheEntry *
CitusTableCacheEntry *
QueryTupleShardSearchInfo(ArrayType *minValuesArray, ArrayType *maxValuesArray,
char partitionMethod, Var *partitionColumn)
{
@ -408,7 +396,7 @@ QueryTupleShardSearchInfo(ArrayType *minValuesArray, ArrayType *maxValuesArray,
/*
* CreatePartitionedResultDestReceiver sets up a partitioned dest receiver.
*/
static DestReceiver *
DestReceiver *
CreatePartitionedResultDestReceiver(int partitionColumnIndex,
int partitionCount,
CitusTableCacheEntry *shardSearchInfo,

View File

@ -0,0 +1,100 @@
/*-------------------------------------------------------------------------
*
* citus_split_shard_by_split_points.c
*
* This file contains functions to split a shard.
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "catalog/pg_type.h"
#include "nodes/pg_list.h"
#include "lib/stringinfo.h"
#include "utils/builtins.h"
#include "utils/lsyscache.h"
#include "distributed/utils/array_type.h"
#include "distributed/colocation_utils.h"
#include "distributed/metadata_cache.h"
#include "distributed/shardinterval_utils.h"
#include "distributed/coordinator_protocol.h"
#include "distributed/connection_management.h"
#include "distributed/remote_commands.h"
#include "distributed/shard_split.h"
/* declarations for dynamic loading */
PG_FUNCTION_INFO_V1(citus_split_shard_by_split_points);
static SplitMode LookupSplitMode(Oid shardTransferModeOid);
/*
* citus_split_shard_by_split_points(shard_id bigint, split_points text[], node_ids integer[], shard_transfer_mode citus.shard_transfer_mode)
* Split source shard into multiple shards using the given split points.
* 'shard_id' is the id of source shard to split.
* 'split_points' is an array that represents the split points.
* 'node_ids' is an array that represents the placement node ids of the new shards.
* 'shard_transfer_mode citus.shard_transfer_mode' is the transfer mode for split.
*/
Datum
citus_split_shard_by_split_points(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
EnsureCoordinator();
uint64 shardIdToSplit = DatumGetUInt64(PG_GETARG_DATUM(0));
ArrayType *splitPointsArrayObject = PG_GETARG_ARRAYTYPE_P(1);
List *shardSplitPointsList = TextArrayTypeToIntegerList(splitPointsArrayObject);
ArrayType *nodeIdsArrayObject = PG_GETARG_ARRAYTYPE_P(2);
List *nodeIdsForPlacementList = IntegerArrayTypeToList(nodeIdsArrayObject);
Oid shardTransferModeOid = PG_GETARG_OID(3);
SplitMode shardSplitMode = LookupSplitMode(shardTransferModeOid);
SplitShard(
shardSplitMode,
SHARD_SPLIT_API,
shardIdToSplit,
shardSplitPointsList,
nodeIdsForPlacementList);
PG_RETURN_VOID();
}
/*
* LookupSplitMode maps the oids of citus.shard_transfer_mode to SplitMode enum.
*/
SplitMode
LookupSplitMode(Oid shardTransferModeOid)
{
SplitMode shardSplitMode = BLOCKING_SPLIT;
Datum enumLabelDatum = DirectFunctionCall1(enum_out, shardTransferModeOid);
char *enumLabel = DatumGetCString(enumLabelDatum);
/* Extend with other modes as we support them */
if (strncmp(enumLabel, "block_writes", NAMEDATALEN) == 0)
{
shardSplitMode = BLOCKING_SPLIT;
}
else if (strncmp(enumLabel, "auto", NAMEDATALEN) == 0 ||
strncmp(enumLabel, "force_logical", NAMEDATALEN) == 0)
{
ereport(ERROR, (errmsg(
"Shard Tranfer mode: '%s' is not supported. Please use 'block_writes' instead.",
enumLabel)));
}
else
{
/* We will not get here as postgres will validate the enum value. */
ereport(ERROR, (errmsg(
"Invalid shard tranfer mode: '%s'. Expected split mode is 'block_writes'.",
enumLabel)));
}
return shardSplitMode;
}

View File

@ -32,6 +32,7 @@
#include "distributed/worker_protocol.h"
#include "distributed/worker_transaction.h"
#include "distributed/version_compat.h"
#include "distributed/shard_split.h"
#include "nodes/pg_list.h"
#include "storage/lock.h"
#include "utils/builtins.h"
@ -48,7 +49,6 @@ PG_FUNCTION_INFO_V1(worker_hash);
/* local function forward declarations */
static uint64 SplitShardByValue(ShardInterval *sourceShard, Datum distributionValueDatum);
static void ErrorIfCannotSplitShard(ShardInterval *sourceShard);
static void CreateSplitOffShards(ShardInterval *sourceShard, int hashedValue,
List **splitOffShardList, int *isolatedShardId);
static List * ShardTemplateList(ShardInterval *sourceShard, int hashedValue,
@ -62,7 +62,6 @@ static void InsertSplitOffShardMetadata(List *splitOffShardList,
List *sourcePlacementList);
static void CreateForeignConstraints(List *splitOffShardList, List *sourcePlacementList);
static void ExecuteCommandListOnWorker(char *nodeName, int nodePort, List *commandList);
static void DropShardList(List *shardIntervalList);
/*
@ -245,7 +244,7 @@ SplitShardByValue(ShardInterval *sourceShard, Datum distributionValueDatum)
/* get locks */
BlockWritesToShardList(colocatedShardList);
ErrorIfCannotSplitShard(sourceShard);
ErrorIfCannotSplitShard(ISOLATE_TENANT_TO_NEW_SHARD, sourceShard);
/* get hash function name */
CitusTableCacheEntry *cacheEntry = GetCitusTableCacheEntry(relationId);
@ -372,65 +371,6 @@ ExecuteCommandListOnWorker(char *nodeName, int nodePort, List *commandList)
}
/*
* ErrorIfCannotSplitShard checks relation kind and invalid shards. It errors
* out if we are not able to split the given shard.
*/
static void
ErrorIfCannotSplitShard(ShardInterval *sourceShard)
{
Oid relationId = sourceShard->relationId;
ListCell *colocatedTableCell = NULL;
ListCell *colocatedShardCell = NULL;
/* checks for table ownership and foreign tables */
List *colocatedTableList = ColocatedTableList(relationId);
foreach(colocatedTableCell, colocatedTableList)
{
Oid colocatedTableId = lfirst_oid(colocatedTableCell);
/* check that user has owner rights in all co-located tables */
EnsureTableOwner(colocatedTableId);
char relationKind = get_rel_relkind(colocatedTableId);
if (relationKind == RELKIND_FOREIGN_TABLE)
{
char *relationName = get_rel_name(colocatedTableId);
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot isolate tenant because \"%s\" is a "
"foreign table", relationName),
errdetail("Isolating shards backed by foreign tables "
"is not supported.")));
}
}
/* check shards with inactive placements */
List *colocatedShardList = ColocatedShardIntervalList(sourceShard);
foreach(colocatedShardCell, colocatedShardList)
{
ShardInterval *shardInterval = (ShardInterval *) lfirst(colocatedShardCell);
uint64 shardId = shardInterval->shardId;
ListCell *shardPlacementCell = NULL;
List *shardPlacementList = ShardPlacementListWithoutOrphanedPlacements(shardId);
foreach(shardPlacementCell, shardPlacementList)
{
ShardPlacement *placement = (ShardPlacement *) lfirst(shardPlacementCell);
if (placement->shardState != SHARD_STATE_ACTIVE)
{
char *relationName = get_rel_name(shardInterval->relationId);
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot isolate tenant because relation "
"\"%s\" has an inactive shard placement "
"for the shard %lu", relationName, shardId),
errhint("Use master_copy_shard_placement UDF to "
"repair the inactive shard placement.")));
}
}
}
}
/*
* CreateSplitOffShards gets a shard and a hashed value to pick the split point.
* First, it creates templates to create new shards. Then, for every colocated
@ -754,69 +694,3 @@ InsertSplitOffShardMetadata(List *splitOffShardList, List *sourcePlacementList)
SendCommandToWorkersWithMetadata(command);
}
}
/*
* DropShardList drops shards and their metadata from both the coordinator and
* mx nodes.
*/
static void
DropShardList(List *shardIntervalList)
{
ListCell *shardIntervalCell = NULL;
foreach(shardIntervalCell, shardIntervalList)
{
ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell);
ListCell *shardPlacementCell = NULL;
Oid relationId = shardInterval->relationId;
uint64 oldShardId = shardInterval->shardId;
/* delete metadata from synced nodes */
if (ShouldSyncTableMetadata(relationId))
{
ListCell *commandCell = NULL;
/* send the commands one by one */
List *shardMetadataDeleteCommandList = ShardDeleteCommandList(shardInterval);
foreach(commandCell, shardMetadataDeleteCommandList)
{
char *command = (char *) lfirst(commandCell);
SendCommandToWorkersWithMetadata(command);
}
}
/* delete shard placements and drop shards */
List *shardPlacementList = ActiveShardPlacementList(oldShardId);
foreach(shardPlacementCell, shardPlacementList)
{
ShardPlacement *placement = (ShardPlacement *) lfirst(shardPlacementCell);
char *workerName = placement->nodeName;
uint32 workerPort = placement->nodePort;
StringInfo dropQuery = makeStringInfo();
DeleteShardPlacementRow(placement->placementId);
/* get shard name */
char *qualifiedShardName = ConstructQualifiedShardName(shardInterval);
char storageType = shardInterval->storageType;
if (storageType == SHARD_STORAGE_TABLE)
{
appendStringInfo(dropQuery, DROP_REGULAR_TABLE_COMMAND,
qualifiedShardName);
}
else if (storageType == SHARD_STORAGE_FOREIGN)
{
appendStringInfo(dropQuery, DROP_FOREIGN_TABLE_COMMAND,
qualifiedShardName);
}
/* drop old shard */
SendCommandToWorker(workerName, workerPort, dropQuery->data);
}
/* delete shard row */
DeleteShardRow(oldShardId);
}
}

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,483 @@
/*-------------------------------------------------------------------------
*
* worker_shard_copy.c
* Functions for copying a shard to destination.
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "libpq-fe.h"
#include "postgres.h"
#include "commands/copy.h"
#include "nodes/makefuncs.h"
#include "parser/parse_relation.h"
#include "utils/lsyscache.h"
#include "utils/builtins.h"
#include "distributed/remote_commands.h"
#include "distributed/worker_shard_copy.h"
#include "distributed/commands/multi_copy.h"
#include "distributed/local_multi_copy.h"
#include "distributed/worker_manager.h"
#include "distributed/connection_management.h"
#include "distributed/relation_utils.h"
#include "distributed/version_compat.h"
#include "distributed/local_executor.h"
/*
* LocalCopyBuffer is used in copy callback to return the copied rows.
* The reason this is a global variable is that we cannot pass an additional
* argument to the copy callback.
*/
static StringInfo LocalCopyBuffer;
typedef struct ShardCopyDestReceiver
{
/* public DestReceiver interface */
DestReceiver pub;
/* Destination Relation Name */
List *destinationShardFullyQualifiedName;
/* descriptor of the tuples that are sent to the worker */
TupleDesc tupleDescriptor;
/* state on how to copy out data types */
CopyOutState copyOutState;
FmgrInfo *columnOutputFunctions;
/* number of tuples sent */
int64 tuplesSent;
/* destination node id */
uint32_t destinationNodeId;
/* local copy if destination shard in same node */
bool useLocalCopy;
/* EState for per-tuple memory allocation */
EState *executorState;
/*
* Connection for destination shard (NULL if useLocalCopy is true)
*/
MultiConnection *connection;
} ShardCopyDestReceiver;
static bool ShardCopyDestReceiverReceive(TupleTableSlot *slot, DestReceiver *dest);
static void ShardCopyDestReceiverStartup(DestReceiver *dest, int operation,
TupleDesc inputTupleDescriptor);
static void ShardCopyDestReceiverShutdown(DestReceiver *destReceiver);
static void ShardCopyDestReceiverDestroy(DestReceiver *destReceiver);
static bool CanUseLocalCopy(uint32_t destinationNodeId);
static StringInfo ConstructShardCopyStatement(List *destinationShardFullyQualifiedName,
bool
useBinaryFormat);
static void WriteLocalTuple(TupleTableSlot *slot, ShardCopyDestReceiver *copyDest);
static int ReadFromLocalBufferCallback(void *outBuf, int minRead, int maxRead);
static void LocalCopyToShard(ShardCopyDestReceiver *copyDest, CopyOutState
localCopyOutState);
static void ConnectToRemoteAndStartCopy(ShardCopyDestReceiver *copyDest);
static bool
CanUseLocalCopy(uint32_t destinationNodeId)
{
/* If destination node is same as source, use local copy */
return GetLocalNodeId() == (int32) destinationNodeId;
}
/* Connect to node with source shard and trigger copy start. */
static void
ConnectToRemoteAndStartCopy(ShardCopyDestReceiver *copyDest)
{
int connectionFlags = OUTSIDE_TRANSACTION;
char *currentUser = CurrentUserName();
WorkerNode *workerNode = FindNodeWithNodeId(copyDest->destinationNodeId,
false /* missingOk */);
copyDest->connection = GetNodeUserDatabaseConnection(connectionFlags,
workerNode->workerName,
workerNode->workerPort,
currentUser,
NULL /* database (current) */);
ClaimConnectionExclusively(copyDest->connection);
StringInfo copyStatement = ConstructShardCopyStatement(
copyDest->destinationShardFullyQualifiedName,
copyDest->copyOutState->binary);
if (!SendRemoteCommand(copyDest->connection, copyStatement->data))
{
ReportConnectionError(copyDest->connection, ERROR);
}
PGresult *result = GetRemoteCommandResult(copyDest->connection,
true /* raiseInterrupts */);
if (PQresultStatus(result) != PGRES_COPY_IN)
{
ReportResultError(copyDest->connection, result, ERROR);
}
PQclear(result);
}
/*
* CreateShardCopyDestReceiver creates a DestReceiver that copies into
* a destinationShardFullyQualifiedName on destinationNodeId.
*/
DestReceiver *
CreateShardCopyDestReceiver(EState *executorState,
List *destinationShardFullyQualifiedName,
uint32_t destinationNodeId)
{
ShardCopyDestReceiver *copyDest = (ShardCopyDestReceiver *) palloc0(
sizeof(ShardCopyDestReceiver));
/* set up the DestReceiver function pointers */
copyDest->pub.receiveSlot = ShardCopyDestReceiverReceive;
copyDest->pub.rStartup = ShardCopyDestReceiverStartup;
copyDest->pub.rShutdown = ShardCopyDestReceiverShutdown;
copyDest->pub.rDestroy = ShardCopyDestReceiverDestroy;
copyDest->pub.mydest = DestCopyOut;
copyDest->executorState = executorState;
copyDest->destinationNodeId = destinationNodeId;
copyDest->destinationShardFullyQualifiedName = destinationShardFullyQualifiedName;
copyDest->tuplesSent = 0;
copyDest->connection = NULL;
copyDest->useLocalCopy = CanUseLocalCopy(destinationNodeId);
return (DestReceiver *) copyDest;
}
/*
* ShardCopyDestReceiverReceive implements the receiveSlot function of
* ShardCopyDestReceiver. It takes a TupleTableSlot and sends the contents to
* the appropriate destination node.
*/
static bool
ShardCopyDestReceiverReceive(TupleTableSlot *slot, DestReceiver *dest)
{
ShardCopyDestReceiver *copyDest = (ShardCopyDestReceiver *) dest;
/*
* Switch to a per-tuple memory memory context. When used in
* context of Split Copy, this is a no-op as switch is already done.
*/
EState *executorState = copyDest->executorState;
MemoryContext executorTupleContext = GetPerTupleMemoryContext(executorState);
MemoryContext oldContext = MemoryContextSwitchTo(executorTupleContext);
/* If remote copy, connect lazily and initiate copy */
if (copyDest->tuplesSent == 0 && (!copyDest->useLocalCopy))
{
ConnectToRemoteAndStartCopy(copyDest);
}
slot_getallattrs(slot);
Datum *columnValues = slot->tts_values;
bool *columnNulls = slot->tts_isnull;
CopyOutState copyOutState = copyDest->copyOutState;
if (copyDest->useLocalCopy)
{
WriteLocalTuple(slot, copyDest);
if (copyOutState->fe_msgbuf->len > LocalCopyFlushThresholdByte)
{
LocalCopyToShard(copyDest, copyOutState);
}
}
else
{
resetStringInfo(copyOutState->fe_msgbuf);
if (copyDest->copyOutState->binary && copyDest->tuplesSent == 0)
{
AppendCopyBinaryHeaders(copyDest->copyOutState);
}
AppendCopyRowData(columnValues,
columnNulls,
copyDest->tupleDescriptor,
copyOutState,
copyDest->columnOutputFunctions,
NULL /* columnCoercionPaths */);
if (!PutRemoteCopyData(copyDest->connection, copyOutState->fe_msgbuf->data,
copyOutState->fe_msgbuf->len))
{
char *destinationShardSchemaName = linitial(
copyDest->destinationShardFullyQualifiedName);
char *destinationShardRelationName = lsecond(
copyDest->destinationShardFullyQualifiedName);
char *errorMessage = PQerrorMessage(copyDest->connection->pgConn);
ereport(ERROR, (errcode(ERRCODE_IO_ERROR),
errmsg("Failed to COPY to shard %s.%s : %s,",
destinationShardSchemaName,
destinationShardRelationName,
errorMessage),
errdetail("failed to send %d bytes %s on node %u",
copyOutState->fe_msgbuf->len,
copyOutState->fe_msgbuf->data,
copyDest->destinationNodeId)));
}
}
MemoryContextSwitchTo(oldContext);
ResetPerTupleExprContext(executorState);
copyDest->tuplesSent++;
return true;
}
/*
* ShardCopyDestReceiverStartup implements the rStartup interface of ShardCopyDestReceiver.
*/
static void
ShardCopyDestReceiverStartup(DestReceiver *dest, int operation, TupleDesc
inputTupleDescriptor)
{
ShardCopyDestReceiver *copyDest = (ShardCopyDestReceiver *) dest;
copyDest->tupleDescriptor = inputTupleDescriptor;
copyDest->tuplesSent = 0;
const char *delimiterCharacter = "\t";
const char *nullPrintCharacter = "\\N";
/* define how tuples will be serialised */
CopyOutState copyOutState = (CopyOutState) palloc0(sizeof(CopyOutStateData));
copyOutState->binary = EnableBinaryProtocol && CanUseBinaryCopyFormat(
inputTupleDescriptor);
copyOutState->null_print = (char *) nullPrintCharacter;
copyOutState->null_print_client = (char *) nullPrintCharacter;
copyOutState->fe_msgbuf = makeStringInfo();
copyOutState->delim = (char *) delimiterCharacter;
copyOutState->rowcontext = GetPerTupleMemoryContext(copyDest->executorState);
copyDest->columnOutputFunctions = ColumnOutputFunctions(inputTupleDescriptor,
copyOutState->binary);
copyDest->copyOutState = copyOutState;
}
/*
* ShardCopyDestReceiverShutdown implements the rShutdown interface of
* ShardCopyDestReceiver. It ends all open COPY operations, copying any pending
* data in buffer.
*/
static void
ShardCopyDestReceiverShutdown(DestReceiver *dest)
{
ShardCopyDestReceiver *copyDest = (ShardCopyDestReceiver *) dest;
if (copyDest->useLocalCopy)
{
if (copyDest->copyOutState != NULL &&
copyDest->copyOutState->fe_msgbuf->len > 0)
{
/* end the COPY input */
LocalCopyToShard(copyDest, copyDest->copyOutState);
}
}
else if (copyDest->connection != NULL)
{
resetStringInfo(copyDest->copyOutState->fe_msgbuf);
if (copyDest->copyOutState->binary)
{
AppendCopyBinaryFooters(copyDest->copyOutState);
}
/* end the COPY input */
if (!PutRemoteCopyEnd(copyDest->connection, NULL /* errormsg */))
{
char *destinationShardSchemaName = linitial(
copyDest->destinationShardFullyQualifiedName);
char *destinationShardRelationName = lsecond(
copyDest->destinationShardFullyQualifiedName);
ereport(ERROR, (errcode(ERRCODE_IO_ERROR),
errmsg("Failed to COPY to destination shard %s.%s",
destinationShardSchemaName,
destinationShardRelationName),
errdetail("failed to send %d bytes %s on node %u",
copyDest->copyOutState->fe_msgbuf->len,
copyDest->copyOutState->fe_msgbuf->data,
copyDest->destinationNodeId)));
}
/* check whether there were any COPY errors */
PGresult *result = GetRemoteCommandResult(copyDest->connection,
true /* raiseInterrupts */);
if (PQresultStatus(result) != PGRES_COMMAND_OK)
{
ReportCopyError(copyDest->connection, result);
}
PQclear(result);
ForgetResults(copyDest->connection);
CloseConnection(copyDest->connection);
}
}
/*
* ShardCopyDestReceiverDestroy frees the DestReceiver.
*/
static void
ShardCopyDestReceiverDestroy(DestReceiver *dest)
{
ShardCopyDestReceiver *copyDest = (ShardCopyDestReceiver *) dest;
if (copyDest->copyOutState)
{
pfree(copyDest->copyOutState);
}
if (copyDest->columnOutputFunctions)
{
pfree(copyDest->columnOutputFunctions);
}
pfree(copyDest);
}
/*
* ConstructShardCopyStatement constructs the text of a COPY statement
* for copying into a result table
*/
static StringInfo
ConstructShardCopyStatement(List *destinationShardFullyQualifiedName, bool
useBinaryFormat)
{
char *destinationShardSchemaName = linitial(destinationShardFullyQualifiedName);
char *destinationShardRelationName = lsecond(destinationShardFullyQualifiedName);
StringInfo command = makeStringInfo();
appendStringInfo(command, "COPY %s.%s FROM STDIN",
quote_identifier(destinationShardSchemaName), quote_identifier(
destinationShardRelationName));
if (useBinaryFormat)
{
appendStringInfo(command, " WITH (format binary);");
}
else
{
appendStringInfo(command, ";");
}
return command;
}
/* Write Tuple to Local Shard. */
static void
WriteLocalTuple(TupleTableSlot *slot, ShardCopyDestReceiver *copyDest)
{
CopyOutState localCopyOutState = copyDest->copyOutState;
/*
* Since we are doing a local copy, the following statements should
* use local execution to see the changes
*/
SetLocalExecutionStatus(LOCAL_EXECUTION_REQUIRED);
bool isBinaryCopy = localCopyOutState->binary;
bool shouldAddBinaryHeaders = (isBinaryCopy && localCopyOutState->fe_msgbuf->len ==
0);
if (shouldAddBinaryHeaders)
{
AppendCopyBinaryHeaders(localCopyOutState);
}
Datum *columnValues = slot->tts_values;
bool *columnNulls = slot->tts_isnull;
FmgrInfo *columnOutputFunctions = copyDest->columnOutputFunctions;
AppendCopyRowData(columnValues, columnNulls, copyDest->tupleDescriptor,
localCopyOutState, columnOutputFunctions,
NULL /* columnCoercionPaths */);
}
/*
* LocalCopyToShard performs local copy for the given destination shard.
*/
static void
LocalCopyToShard(ShardCopyDestReceiver *copyDest, CopyOutState localCopyOutState)
{
bool isBinaryCopy = localCopyOutState->binary;
if (isBinaryCopy)
{
AppendCopyBinaryFooters(localCopyOutState);
}
/*
* Set the buffer as a global variable to allow ReadFromLocalBufferCallback
* to read from it. We cannot pass additional arguments to
* ReadFromLocalBufferCallback.
*/
LocalCopyBuffer = localCopyOutState->fe_msgbuf;
char *destinationShardSchemaName = linitial(
copyDest->destinationShardFullyQualifiedName);
char *destinationShardRelationName = lsecond(
copyDest->destinationShardFullyQualifiedName);
Oid destinationSchemaOid = get_namespace_oid(destinationShardSchemaName,
false /* missing_ok */);
Oid destinationShardOid = get_relname_relid(destinationShardRelationName,
destinationSchemaOid);
DefElem *binaryFormatOption = NULL;
if (isBinaryCopy)
{
binaryFormatOption = makeDefElem("format", (Node *) makeString("binary"), -1);
}
Relation shard = table_open(destinationShardOid, RowExclusiveLock);
ParseState *pState = make_parsestate(NULL /* parentParseState */);
(void) addRangeTableEntryForRelation(pState, shard, AccessShareLock,
NULL /* alias */, false /* inh */,
false /* inFromCl */);
List *options = (isBinaryCopy) ? list_make1(binaryFormatOption) : NULL;
CopyFromState cstate = BeginCopyFrom_compat(pState, shard,
NULL /* whereClause */,
NULL /* fileName */,
false /* is_program */,
ReadFromLocalBufferCallback,
NULL /* attlist (NULL is all columns) */,
options);
CopyFrom(cstate);
EndCopyFrom(cstate);
resetStringInfo(localCopyOutState->fe_msgbuf);
table_close(shard, NoLock);
free_parsestate(pState);
}
/*
* ReadFromLocalBufferCallback is the copy callback.
* It always tries to copy maxRead bytes.
*/
static int
ReadFromLocalBufferCallback(void *outBuf, int minRead, int maxRead)
{
int bytesRead = 0;
int avail = LocalCopyBuffer->len - LocalCopyBuffer->cursor;
int bytesToRead = Min(avail, maxRead);
if (bytesToRead > 0)
{
memcpy_s(outBuf, bytesToRead,
&LocalCopyBuffer->data[LocalCopyBuffer->cursor], bytesToRead);
}
bytesRead += bytesToRead;
LocalCopyBuffer->cursor += bytesToRead;
return bytesRead;
}

View File

@ -0,0 +1,264 @@
/*-------------------------------------------------------------------------
*
* worker_split_copy_udf.c
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "pg_version_compat.h"
#include "utils/lsyscache.h"
#include "utils/array.h"
#include "utils/builtins.h"
#include "distributed/utils/array_type.h"
#include "distributed/listutils.h"
#include "distributed/multi_executor.h"
#include "distributed/worker_shard_copy.h"
#include "distributed/intermediate_results.h"
#include "distributed/citus_ruleutils.h"
PG_FUNCTION_INFO_V1(worker_split_copy);
typedef struct SplitCopyInfo
{
uint64 destinationShardId; /* destination shard id */
Datum destinationShardMinHashValue; /* min hash value of destination shard */
Datum destinationShardMaxHashValue; /* max hash value of destination shard */
uint32_t destinationShardNodeId; /* node where split child shard is to be placed */
} SplitCopyInfo;
static void ParseSplitCopyInfoDatum(Datum splitCopyInfoDatum,
SplitCopyInfo **splitCopyInfo);
static DestReceiver ** CreateShardCopyDestReceivers(EState *estate,
ShardInterval *
shardIntervalToSplitCopy,
List *splitCopyInfoList);
static DestReceiver * CreatePartitionedSplitCopyDestReceiver(EState *executor,
ShardInterval *
shardIntervalToSplitCopy,
List *splitCopyInfoList);
static void BuildMinMaxRangeArrays(List *splitCopyInfoList, ArrayType **minValueArray,
ArrayType **maxValueArray);
/*
* worker_split_copy(source_shard_id bigint, splitCopyInfo pg_catalog.split_copy_info[])
* UDF to split copy shard to list of destination shards.
* 'source_shard_id' : Source ShardId to split copy.
* 'splitCopyInfos' : Array of Split Copy Info (destination_shard's id, min/max ranges and node_id)
*/
Datum
worker_split_copy(PG_FUNCTION_ARGS)
{
uint64 shardIdToSplitCopy = DatumGetUInt64(PG_GETARG_DATUM(0));
ShardInterval *shardIntervalToSplitCopy = LoadShardInterval(shardIdToSplitCopy);
ArrayType *splitCopyInfoArrayObject = PG_GETARG_ARRAYTYPE_P(1);
bool arrayHasNull = ARR_HASNULL(splitCopyInfoArrayObject);
if (arrayHasNull)
{
ereport(ERROR, (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
errmsg(
"pg_catalog.split_copy_info array cannot contain null values")));
}
const int slice_ndim = 0;
ArrayMetaState *mState = NULL;
ArrayIterator copyInfo_iterator = array_create_iterator(splitCopyInfoArrayObject,
slice_ndim,
mState);
Datum copyInfoDatum = 0;
bool isnull = false;
List *splitCopyInfoList = NIL;
while (array_iterate(copyInfo_iterator, &copyInfoDatum, &isnull))
{
SplitCopyInfo *splitCopyInfo = NULL;
ParseSplitCopyInfoDatum(copyInfoDatum, &splitCopyInfo);
splitCopyInfoList = lappend(splitCopyInfoList, splitCopyInfo);
}
EState *executor = CreateExecutorState();
DestReceiver *splitCopyDestReceiver = CreatePartitionedSplitCopyDestReceiver(executor,
shardIntervalToSplitCopy,
splitCopyInfoList);
Oid sourceShardToCopySchemaOId = get_rel_namespace(
shardIntervalToSplitCopy->relationId);
char *sourceShardToCopySchemaName = get_namespace_name(sourceShardToCopySchemaOId);
char *sourceShardToCopyName = get_rel_name(shardIntervalToSplitCopy->relationId);
AppendShardIdToName(&sourceShardToCopyName, shardIdToSplitCopy);
char *sourceShardToCopyQualifiedName = quote_qualified_identifier(
sourceShardToCopySchemaName,
sourceShardToCopyName);
StringInfo selectShardQueryForCopy = makeStringInfo();
appendStringInfo(selectShardQueryForCopy,
"SELECT * FROM %s;", sourceShardToCopyQualifiedName);
ParamListInfo params = NULL;
ExecuteQueryStringIntoDestReceiver(selectShardQueryForCopy->data, params,
(DestReceiver *) splitCopyDestReceiver);
FreeExecutorState(executor);
PG_RETURN_VOID();
}
/* Parse a single SplitCopyInfo Tuple */
static void
ParseSplitCopyInfoDatum(Datum splitCopyInfoDatum, SplitCopyInfo **splitCopyInfo)
{
HeapTupleHeader dataTuple = DatumGetHeapTupleHeader(splitCopyInfoDatum);
SplitCopyInfo *copyInfo = palloc0(sizeof(SplitCopyInfo));
bool isnull = false;
Datum destinationShardIdDatum = GetAttributeByName(dataTuple, "destination_shard_id",
&isnull);
if (isnull)
{
ereport(ERROR, (errmsg(
"destination_shard_id for pg_catalog.split_copy_info cannot be null.")));
}
copyInfo->destinationShardId = DatumGetUInt64(destinationShardIdDatum);
Datum minValueDatum = GetAttributeByName(dataTuple, "destination_shard_min_value",
&isnull);
if (isnull)
{
ereport(ERROR, (errmsg(
"destination_shard_min_value for pg_catalog.split_copy_info cannot be null.")));
}
copyInfo->destinationShardMinHashValue = minValueDatum;
Datum maxValueDatum = GetAttributeByName(dataTuple, "destination_shard_max_value",
&isnull);
if (isnull)
{
ereport(ERROR, (errmsg(
"destination_shard_max_value for pg_catalog.split_copy_info cannot be null.")));
}
copyInfo->destinationShardMaxHashValue = maxValueDatum;
Datum nodeIdDatum = GetAttributeByName(dataTuple, "destination_shard_node_id",
&isnull);
if (isnull)
{
ereport(ERROR, (errmsg(
"destination_shard_node_id for pg_catalog.split_copy_info cannot be null.")));
}
copyInfo->destinationShardNodeId = DatumGetInt32(nodeIdDatum);
*splitCopyInfo = copyInfo;
}
/* Build 'min/max' hash range arrays for PartitionedResultDestReceiver */
static void
BuildMinMaxRangeArrays(List *splitCopyInfoList, ArrayType **minValueArray,
ArrayType **maxValueArray)
{
int partitionCount = list_length(splitCopyInfoList);
Datum *minValues = palloc0(partitionCount * sizeof(Datum));
bool *minValueNulls = palloc0(partitionCount * sizeof(bool));
Datum *maxValues = palloc0(partitionCount * sizeof(Datum));
bool *maxValueNulls = palloc0(partitionCount * sizeof(bool));
SplitCopyInfo *splitCopyInfo = NULL;
int index = 0;
foreach_ptr(splitCopyInfo, splitCopyInfoList)
{
minValues[index] = splitCopyInfo->destinationShardMinHashValue;
maxValues[index] = splitCopyInfo->destinationShardMaxHashValue;
/* Caller enforces that min/max values will be not-null */
minValueNulls[index] = false;
maxValueNulls[index] = false;
index++;
}
*minValueArray = CreateArrayFromDatums(minValues, minValueNulls, partitionCount,
TEXTOID);
*maxValueArray = CreateArrayFromDatums(maxValues, maxValueNulls, partitionCount,
TEXTOID);
}
/*
* Create underlying ShardCopyDestReceivers for PartitionedResultDestReceiver
* Each ShardCopyDestReceivers will be responsible for copying tuples from source shard,
* that fall under its min/max range, to specified destination shard.
*/
static DestReceiver **
CreateShardCopyDestReceivers(EState *estate, ShardInterval *shardIntervalToSplitCopy,
List *splitCopyInfoList)
{
DestReceiver **shardCopyDests = palloc0(splitCopyInfoList->length *
sizeof(DestReceiver *));
SplitCopyInfo *splitCopyInfo = NULL;
int index = 0;
char *sourceShardNamePrefix = get_rel_name(shardIntervalToSplitCopy->relationId);
foreach_ptr(splitCopyInfo, splitCopyInfoList)
{
Oid destinationShardSchemaOid = get_rel_namespace(
shardIntervalToSplitCopy->relationId);
char *destinationShardSchemaName = get_namespace_name(destinationShardSchemaOid);
char *destinationShardNameCopy = pstrdup(sourceShardNamePrefix);
AppendShardIdToName(&destinationShardNameCopy, splitCopyInfo->destinationShardId);
DestReceiver *shardCopyDest = CreateShardCopyDestReceiver(
estate,
list_make2(destinationShardSchemaName, destinationShardNameCopy),
splitCopyInfo->destinationShardNodeId);
shardCopyDests[index] = shardCopyDest;
index++;
}
return shardCopyDests;
}
/* Create PartitionedSplitCopyDestReceiver along with underlying ShardCopyDestReceivers */
static DestReceiver *
CreatePartitionedSplitCopyDestReceiver(EState *estate,
ShardInterval *shardIntervalToSplitCopy,
List *splitCopyInfoList)
{
/* Create underlying ShardCopyDestReceivers */
DestReceiver **shardCopyDestReceivers = CreateShardCopyDestReceivers(
estate,
shardIntervalToSplitCopy,
splitCopyInfoList);
/* construct an artificial CitusTableCacheEntry for routing tuples to appropriate ShardCopyReceiver */
ArrayType *minValuesArray = NULL;
ArrayType *maxValuesArray = NULL;
BuildMinMaxRangeArrays(splitCopyInfoList, &minValuesArray, &maxValuesArray);
CitusTableCacheEntry *cacheEntry = GetCitusTableCacheEntry(
shardIntervalToSplitCopy->relationId);
char partitionMethod = cacheEntry->partitionMethod;
Var *partitionColumn = cacheEntry->partitionColumn;
CitusTableCacheEntry *shardSearchInfo =
QueryTupleShardSearchInfo(minValuesArray, maxValuesArray,
partitionMethod, partitionColumn);
/* Construct PartitionedResultDestReceiver from cache and underlying ShardCopyDestReceivers */
int partitionColumnIndex = partitionColumn->varattno - 1;
int partitionCount = splitCopyInfoList->length;
DestReceiver *splitCopyDestReceiver = CreatePartitionedResultDestReceiver(
partitionColumnIndex,
partitionCount,
shardSearchInfo,
shardCopyDestReceivers,
true /* lazyStartup */,
false /* allowNullPartitionColumnValues */);
return splitCopyDestReceiver;
}

View File

@ -65,3 +65,5 @@ DROP FUNCTION pg_catalog.get_all_active_transactions(OUT datid oid, OUT process_
OUT worker_query BOOL, OUT transaction_number int8, OUT transaction_stamp timestamptz,
OUT global_pid int8);
#include "udfs/get_all_active_transactions/11.1-1.sql"
#include "udfs/citus_split_shard_by_split_points/11.1-1.sql"
#include "udfs/worker_split_copy/11.1-1.sql"

View File

@ -63,6 +63,15 @@ ALTER EXTENSION citus ADD FUNCTION citus_internal.upgrade_columnar_storage;
ALTER EXTENSION citus ADD FUNCTION citus_internal.downgrade_columnar_storage;
ALTER EXTENSION citus ADD FUNCTION citus_internal.columnar_ensure_am_depends_catalog;
DROP FUNCTION pg_catalog.citus_split_shard_by_split_points(
shard_id bigint,
split_points text[],
node_ids integer[],
shard_transfer_mode citus.shard_transfer_mode);
DROP FUNCTION pg_catalog.worker_split_copy(
source_shard_id bigint,
splitCopyInfos pg_catalog.split_copy_info[]);
DROP TYPE pg_catalog.split_copy_info;
DROP FUNCTION pg_catalog.get_all_active_transactions(OUT datid oid, OUT process_id int, OUT initiator_node_identifier int4,
OUT worker_query BOOL, OUT transaction_number int8, OUT transaction_stamp timestamptz,

View File

@ -0,0 +1,14 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_split_shard_by_split_points(
shard_id bigint,
split_points text[],
-- A 'nodeId' is a uint32 in CITUS [1, 4294967296] but postgres does not have unsigned type support.
-- Use integer (consistent with other previously defined UDFs that take nodeId as integer) as for all practical purposes it is big enough.
node_ids integer[],
-- Three modes to be implemented: block_writes, force_logical and auto.
-- Currently, the default / only supported mode is block_writes.
shard_transfer_mode citus.shard_transfer_mode default 'block_writes')
RETURNS void
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$citus_split_shard_by_split_points$$;
COMMENT ON FUNCTION pg_catalog.citus_split_shard_by_split_points(shard_id bigint, split_points text[], nodeIds integer[], citus.shard_transfer_mode)
IS 'split a shard using split mode.';

View File

@ -0,0 +1,14 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_split_shard_by_split_points(
shard_id bigint,
split_points text[],
-- A 'nodeId' is a uint32 in CITUS [1, 4294967296] but postgres does not have unsigned type support.
-- Use integer (consistent with other previously defined UDFs that take nodeId as integer) as for all practical purposes it is big enough.
node_ids integer[],
-- Three modes to be implemented: block_writes, force_logical and auto.
-- Currently, the default / only supported mode is block_writes.
shard_transfer_mode citus.shard_transfer_mode default 'block_writes')
RETURNS void
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$citus_split_shard_by_split_points$$;
COMMENT ON FUNCTION pg_catalog.citus_split_shard_by_split_points(shard_id bigint, split_points text[], nodeIds integer[], citus.shard_transfer_mode)
IS 'split a shard using split mode.';

View File

@ -0,0 +1,22 @@
-- We want to create the type in pg_catalog but doing that leads to an error
-- "ERROR: permission denied to create "pg_catalog.split_copy_info"
-- "DETAIL: System catalog modifications are currently disallowed. ""
-- As a workaround, we create the type in the citus schema and then later modify it to pg_catalog.
DROP TYPE IF EXISTS citus.split_copy_info;
CREATE TYPE citus.split_copy_info AS (
destination_shard_id bigint,
destination_shard_min_value text,
destination_shard_max_value text,
-- A 'nodeId' is a uint32 in CITUS [1, 4294967296] but postgres does not have unsigned type support.
-- Use integer (consistent with other previously defined UDFs that take nodeId as integer) as for all practical purposes it is big enough.
destination_shard_node_id integer);
ALTER TYPE citus.split_copy_info SET SCHEMA pg_catalog;
CREATE OR REPLACE FUNCTION pg_catalog.worker_split_copy(
source_shard_id bigint,
splitCopyInfos pg_catalog.split_copy_info[])
RETURNS void
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$worker_split_copy$$;
COMMENT ON FUNCTION pg_catalog.worker_split_copy(source_shard_id bigint, splitCopyInfos pg_catalog.split_copy_info[])
IS 'Perform split copy for shard';

View File

@ -0,0 +1,22 @@
-- We want to create the type in pg_catalog but doing that leads to an error
-- "ERROR: permission denied to create "pg_catalog.split_copy_info"
-- "DETAIL: System catalog modifications are currently disallowed. ""
-- As a workaround, we create the type in the citus schema and then later modify it to pg_catalog.
DROP TYPE IF EXISTS citus.split_copy_info;
CREATE TYPE citus.split_copy_info AS (
destination_shard_id bigint,
destination_shard_min_value text,
destination_shard_max_value text,
-- A 'nodeId' is a uint32 in CITUS [1, 4294967296] but postgres does not have unsigned type support.
-- Use integer (consistent with other previously defined UDFs that take nodeId as integer) as for all practical purposes it is big enough.
destination_shard_node_id integer);
ALTER TYPE citus.split_copy_info SET SCHEMA pg_catalog;
CREATE OR REPLACE FUNCTION pg_catalog.worker_split_copy(
source_shard_id bigint,
splitCopyInfos pg_catalog.split_copy_info[])
RETURNS void
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$worker_split_copy$$;
COMMENT ON FUNCTION pg_catalog.worker_split_copy(source_shard_id bigint, splitCopyInfos pg_catalog.split_copy_info[])
IS 'Perform split copy for shard';

View File

@ -12,8 +12,12 @@
#include "postgres.h"
#include "miscadmin.h"
#include "pg_version_compat.h"
#include "catalog/pg_type.h"
#include "nodes/pg_list.h"
#include "distributed/utils/array_type.h"
#include "utils/array.h"
#include "utils/builtins.h"
#include "utils/lsyscache.h"
@ -96,3 +100,42 @@ DatumArrayToArrayType(Datum *datumArray, int datumCount, Oid datumTypeId)
return arrayObject;
}
/*
* Converts ArrayType to List.
*/
List *
IntegerArrayTypeToList(ArrayType *arrayObject)
{
List *list = NULL;
Datum *datumObjectArray = DeconstructArrayObject(arrayObject);
int arrayObjectCount = ArrayObjectCount(arrayObject);
for (int index = 0; index < arrayObjectCount; index++)
{
list = lappend_int(list, datumObjectArray[index]);
}
return list;
}
/*
* Converts Text ArrayType to Integer List.
*/
extern List *
TextArrayTypeToIntegerList(ArrayType *arrayObject)
{
List *list = NULL;
Datum *datumObjectArray = DeconstructArrayObject(arrayObject);
int arrayObjectCount = ArrayObjectCount(arrayObject);
for (int index = 0; index < arrayObjectCount; index++)
{
char *intAsStr = text_to_cstring(DatumGetTextP(datumObjectArray[index]));
list = lappend_int(list, pg_strtoint32(intAsStr));
}
return list;
}

View File

@ -664,37 +664,6 @@ LockTransactionRecovery(LOCKMODE lockmode)
}
/*
* LockJobResource acquires a lock for creating resources associated with the
* given jobId. This resource is typically a job schema (namespace), and less
* commonly a partition task directory.
*/
void
LockJobResource(uint64 jobId, LOCKMODE lockmode)
{
LOCKTAG tag;
const bool sessionLock = false;
const bool dontWait = false;
SET_LOCKTAG_JOB_RESOURCE(tag, MyDatabaseId, jobId);
(void) LockAcquire(&tag, lockmode, sessionLock, dontWait);
}
/* Releases the lock for resources associated with the given job id. */
void
UnlockJobResource(uint64 jobId, LOCKMODE lockmode)
{
LOCKTAG tag;
const bool sessionLock = false;
SET_LOCKTAG_JOB_RESOURCE(tag, MyDatabaseId, jobId);
LockRelease(&tag, lockmode, sessionLock);
}
/*
* LockShardListMetadata takes shared locks on the metadata of all shards in
* shardIntervalList to prevents concurrent placement changes.

View File

@ -182,6 +182,7 @@ extern void CheckCopyPermissions(CopyStmt *copyStatement);
extern bool IsCopyResultStmt(CopyStmt *copyStatement);
extern void ConversionPathForTypes(Oid inputType, Oid destType, CopyCoercionData *result);
extern Datum CoerceColumnValue(Datum inputValue, CopyCoercionData *coercionPath);
extern void ReportCopyError(MultiConnection *connection, PGresult *result);
#endif /* MULTI_COPY_H */

View File

@ -283,6 +283,9 @@ extern int MasterDropAllShards(Oid relationId, char *schemaName, char *relationN
extern Datum master_create_worker_shards(PG_FUNCTION_ARGS);
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);

View File

@ -69,12 +69,26 @@ typedef struct NodeToNodeFragmentsTransfer
List *fragmentList;
} NodeToNodeFragmentsTransfer;
/* Forward Declarations */
struct CitusTableCacheEntry;
/* intermediate_results.c */
extern DestReceiver * CreateRemoteFileDestReceiver(const char *resultId,
EState *executorState,
List *initialNodeList, bool
writeLocalFile);
extern DestReceiver * CreatePartitionedResultDestReceiver(int partitionColumnIndex,
int partitionCount,
CitusTableCacheEntry *
shardSearchInfo,
DestReceiver **
partitionedDestReceivers,
bool lazyStartup,
bool allowNullPartitionValues);
extern CitusTableCacheEntry * QueryTupleShardSearchInfo(ArrayType *minValuesArray,
ArrayType *maxValuesArray,
char partitionMethod,
Var *partitionColumn);
extern void WriteToLocalFile(StringInfo copyData, FileCompat *fileCompat);
extern uint64 RemoteFileDestReceiverBytesSent(DestReceiver *destReceiver);
extern void SendQueryResultViaCopy(const char *resultId);
@ -83,6 +97,9 @@ extern void RemoveIntermediateResultsDirectories(void);
extern int64 IntermediateResultSize(const char *resultId);
extern char * QueryResultFileName(const char *resultId);
extern char * CreateIntermediateResultsDirectory(void);
extern ArrayType * CreateArrayFromDatums(Datum *datumArray, bool *nullsArray, int
datumCount, Oid typeId);
/* distributed_intermediate_results.c */
extern List ** RedistributeTaskListResults(const char *resultIdPrefix,

View File

@ -74,14 +74,6 @@ typedef enum CitusOperations
(uint32) (shardid), \
ADV_LOCKTAG_CLASS_CITUS_SHARD)
/* reuse advisory lock, but with different, unused field 4 (6) */
#define SET_LOCKTAG_JOB_RESOURCE(tag, db, jobid) \
SET_LOCKTAG_ADVISORY(tag, \
db, \
(uint32) ((jobid) >> 32), \
(uint32) (jobid), \
ADV_LOCKTAG_CLASS_CITUS_JOB)
/* reuse advisory lock, but with different, unused field 4 (7)
* Also it has the database hardcoded to MyDatabaseId, to ensure the locks
* are local to each database */
@ -157,10 +149,6 @@ extern void LockReferencedReferenceShardDistributionMetadata(uint64 shardId,
/* Lock shard data, for DML commands or remote fetches */
extern void LockShardResource(uint64 shardId, LOCKMODE lockmode);
/* Lock a job schema or partition task directory */
extern void LockJobResource(uint64 jobId, LOCKMODE lockmode);
extern void UnlockJobResource(uint64 jobId, LOCKMODE lockmode);
/* Lock a co-location group */
extern void LockColocationId(int colocationId, LOCKMODE lockMode);
extern void UnlockColocationId(int colocationId, LOCKMODE lockMode);

View File

@ -0,0 +1,46 @@
/*-------------------------------------------------------------------------
*
* shard_split.h
*
* API for shard splits.
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#ifndef SHARDSPLIT_H_
#define SHARDSPLIT_H_
/* Split Modes supported by Shard Split API */
typedef enum SplitMode
{
BLOCKING_SPLIT = 0
} SplitMode;
/*
* User Scenario calling Split Shard API.
* The 'SplitOperation' type is used to customize info/error messages based on user scenario.
*/
typedef enum SplitOperation
{
SHARD_SPLIT_API = 0,
ISOLATE_TENANT_TO_NEW_SHARD
} SplitOperation;
/*
* SplitShard API to split a given shard (or shard group) using split mode and
* specified split points to a set of destination nodes.
*/
extern void SplitShard(SplitMode splitMode,
SplitOperation splitOperation,
uint64 shardIdToSplit,
List *shardSplitPointsList,
List *nodeIdsForPlacementList);
/* TODO(niupre): Make all these APIs private when all consumers (Example : ISOLATE_TENANT_TO_NEW_SHARD) directly call 'SplitShard' API. */
extern void ErrorIfCannotSplitShard(SplitOperation splitOperation,
ShardInterval *sourceShard);
extern void DropShardList(List *shardIntervalList);
#endif /* SHARDSPLIT_H_ */

View File

@ -20,6 +20,7 @@ extern Datum * DeconstructArrayObject(ArrayType *arrayObject);
extern int32 ArrayObjectCount(ArrayType *arrayObject);
extern ArrayType * DatumArrayToArrayType(Datum *datumArray, int datumCount,
Oid datumTypeId);
extern List * IntegerArrayTypeToList(ArrayType *arrayObject);
extern List * TextArrayTypeToIntegerList(ArrayType *arrayObject);
#endif /* CITUS_ARRAY_TYPE_H */

View File

@ -0,0 +1,22 @@
/*-------------------------------------------------------------------------
*
* worker_shard_copy.c
* Copy data to destination shard in a push approach.
*
* Copyright (c) Citus Data, Inc.
*
*
*-------------------------------------------------------------------------
*/
#ifndef WORKER_SHARD_COPY_H_
#define WORKER_SHARD_COPY_H_
/* GUC, determining whether Binary Copy is enabled */
extern bool EnableBinaryProtocol;
extern DestReceiver * CreateShardCopyDestReceiver(EState *executorState,
List *destinationShardFullyQualifiedName,
uint32_t destinationNodeId);
#endif /* WORKER_SHARD_COPY_H_ */

View File

@ -43,7 +43,7 @@ output_files := $(patsubst $(citus_abs_srcdir)/output/%.source,expected/%.out, $
# intermediate, for muscle memory backward compatibility.
check: check-full check-enterprise-full
# check-full triggers all tests that ought to be run routinely
check-full: check-multi check-multi-mx check-multi-1 check-operations check-follower-cluster check-isolation check-failure
check-full: check-multi check-multi-mx check-multi-1 check-operations check-follower-cluster check-isolation check-failure check-split
# check-enterprise-full triggers all enterprise specific tests
check-enterprise-full: check-enterprise check-enterprise-isolation check-enterprise-failure
@ -216,6 +216,10 @@ check-columnar-isolation: all $(isolation_test_files)
$(pg_regress_multi_check) --load-extension=citus --isolationtester \
-- $(MULTI_REGRESS_OPTS) --inputdir=$(citus_abs_srcdir)/build --schedule=$(citus_abs_srcdir)/columnar_isolation_schedule $(EXTRA_TESTS)
check-split: all
$(pg_regress_multi_check) --load-extension=citus \
-- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/split_schedule $(EXTRA_TESTS)
check-failure: all
$(pg_regress_multi_check) --load-extension=citus --mitmproxy \
-- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/failure_schedule $(EXTRA_TESTS)

View File

@ -13,3 +13,5 @@ test: isolation_ref2ref_foreign_keys_enterprise
test: isolation_pg_send_cancellation
test: isolation_shard_move_vs_start_metadata_sync
test: isolation_tenant_isolation
test: isolation_blocking_shard_split
test: isolation_blocking_shard_split_with_fkey_to_reference

View File

@ -0,0 +1,459 @@
/*
Citus Shard Split Test.The test is model similar to 'shard_move_constraints'.
Here is a high level overview of test plan:
1. Create a table 'sensors' (ShardCount = 2) to be split. Add indexes and statistics on this table.
2. Create two other tables: 'reference_table' and 'colocated_dist_table', co-located with sensors.
3. Create Foreign key constraints between the two co-located distributed tables.
4. Load data into the three tables.
5. Move one of the shards for 'sensors' to test ShardMove -> Split.
6. Trigger Split on both shards of 'sensors'. This will also split co-located tables.
7. Move one of the split shard to test Split -> ShardMove.
8. Split an already split shard second time on a different schema.
*/
CREATE SCHEMA "citus_split_test_schema";
CREATE ROLE test_split_role WITH LOGIN;
GRANT USAGE, CREATE ON SCHEMA "citus_split_test_schema" TO test_split_role;
SET ROLE test_split_role;
SET search_path TO "citus_split_test_schema";
SET citus.next_shard_id TO 8981000;
SET citus.next_placement_id TO 8610000;
SET citus.shard_count TO 2;
SET citus.shard_replication_factor TO 1;
-- BEGIN: Create table to split, along with other co-located tables. Add indexes, statistics etc.
CREATE TABLE sensors(
measureid integer,
eventdatetime date,
measure_data jsonb,
meaure_quantity decimal(15, 2),
measure_status char(1),
measure_comment varchar(44),
PRIMARY KEY (measureid, eventdatetime, measure_data));
CREATE INDEX index_on_sensors ON sensors(lower(measureid::text));
ALTER INDEX index_on_sensors ALTER COLUMN 1 SET STATISTICS 1000;
CREATE INDEX hash_index_on_sensors ON sensors USING HASH((measure_data->'IsFailed'));
CREATE INDEX index_with_include_on_sensors ON sensors ((measure_data->'IsFailed')) INCLUDE (measure_data, eventdatetime, measure_status);
CREATE STATISTICS stats_on_sensors (dependencies) ON measureid, eventdatetime FROM sensors;
SELECT create_distributed_table('sensors', 'measureid', colocate_with:='none');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- END: Create table to split, along with other co-located tables. Add indexes, statistics etc.
-- BEGIN: Create co-located distributed and reference tables.
CREATE TABLE reference_table (measureid integer PRIMARY KEY);
SELECT create_reference_table('reference_table');
create_reference_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE colocated_dist_table (measureid integer PRIMARY KEY);
CLUSTER colocated_dist_table USING colocated_dist_table_pkey;
SELECT create_distributed_table('colocated_dist_table', 'measureid', colocate_with:='sensors');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE table_with_index_rep_identity(key int NOT NULL);
CREATE UNIQUE INDEX uqx ON table_with_index_rep_identity(key);
ALTER TABLE table_with_index_rep_identity REPLICA IDENTITY USING INDEX uqx;
CLUSTER table_with_index_rep_identity USING uqx;
SELECT create_distributed_table('table_with_index_rep_identity', 'key', colocate_with:='sensors');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- END: Create co-located distributed and reference tables.
-- BEGIN : Create Foreign key constraints.
ALTER TABLE sensors ADD CONSTRAINT fkey_table_to_dist FOREIGN KEY (measureid) REFERENCES colocated_dist_table(measureid);
-- END : Create Foreign key constraints.
-- BEGIN : Load data into tables.
INSERT INTO reference_table SELECT i FROM generate_series(0,1000)i;
INSERT INTO colocated_dist_table SELECT i FROM generate_series(0,1000)i;
INSERT INTO sensors SELECT i, '2020-01-05', '{}', 11011.10, 'A', 'I <3 Citus' FROM generate_series(0,1000)i;
SELECT COUNT(*) FROM sensors;
count
---------------------------------------------------------------------
1001
(1 row)
SELECT COUNT(*) FROM reference_table;
count
---------------------------------------------------------------------
1001
(1 row)
SELECT COUNT(*) FROM colocated_dist_table;
count
---------------------------------------------------------------------
1001
(1 row)
-- END: Load data into tables.
-- BEGIN : Display current state.
SELECT shard.shardid, logicalrelid, shardminvalue, shardmaxvalue, nodename, nodeport
FROM pg_dist_shard AS shard
INNER JOIN pg_dist_placement placement ON shard.shardid = placement.shardid
INNER JOIN pg_dist_node node ON placement.groupid = node.groupid
INNER JOIN pg_catalog.pg_class cls ON shard.logicalrelid = cls.oid
WHERE node.noderole = 'primary' AND (logicalrelid = 'sensors'::regclass OR logicalrelid = 'colocated_dist_table'::regclass OR logicalrelid = 'table_with_index_rep_identity'::regclass)
ORDER BY logicalrelid, shardminvalue::BIGINT;
shardid | logicalrelid | shardminvalue | shardmaxvalue | nodename | nodeport
---------------------------------------------------------------------
8981000 | sensors | -2147483648 | -1 | localhost | 57637
8981001 | sensors | 0 | 2147483647 | localhost | 57638
8981003 | colocated_dist_table | -2147483648 | -1 | localhost | 57637
8981004 | colocated_dist_table | 0 | 2147483647 | localhost | 57638
8981005 | table_with_index_rep_identity | -2147483648 | -1 | localhost | 57637
8981006 | table_with_index_rep_identity | 0 | 2147483647 | localhost | 57638
(6 rows)
\c - - - :worker_1_port
SET search_path TO "citus_split_test_schema", public, pg_catalog;
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like 'sensors_%'
ORDER BY 1, 2;
relname | Constraint | Definition
---------------------------------------------------------------------
sensors_8981000 | fkey_table_to_dist_8981000 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8981003(measureid)
(1 row)
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'sensors_%' ORDER BY 1,2;
tablename | indexdef
---------------------------------------------------------------------
sensors_8981000 | CREATE INDEX hash_index_on_sensors_8981000 ON citus_split_test_schema.sensors_8981000 USING hash (((measure_data -> 'IsFailed'::text)))
sensors_8981000 | CREATE INDEX index_on_sensors_8981000 ON citus_split_test_schema.sensors_8981000 USING btree (lower((measureid)::text))
sensors_8981000 | CREATE INDEX index_with_include_on_sensors_8981000 ON citus_split_test_schema.sensors_8981000 USING btree (((measure_data -> 'IsFailed'::text))) INCLUDE (measure_data, eventdatetime, measure_status)
sensors_8981000 | CREATE UNIQUE INDEX sensors_pkey_8981000 ON citus_split_test_schema.sensors_8981000 USING btree (measureid, eventdatetime, measure_data)
(4 rows)
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'table_with_index_rep_identity_%' ORDER BY 1,2;
tablename | indexdef
---------------------------------------------------------------------
table_with_index_rep_identity_8981005 | CREATE UNIQUE INDEX uqx_8981005 ON citus_split_test_schema.table_with_index_rep_identity_8981005 USING btree (key)
(1 row)
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema')
)
ORDER BY stxname ASC;
stxname
---------------------------------------------------------------------
stats_on_sensors
stats_on_sensors_8981000
(2 rows)
\c - - - :worker_2_port
SET search_path TO "citus_split_test_schema", public, pg_catalog;
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like 'sensors_%'
ORDER BY 1, 2;
relname | Constraint | Definition
---------------------------------------------------------------------
sensors_8981001 | fkey_table_to_dist_8981001 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8981004(measureid)
(1 row)
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'sensors_%' ORDER BY 1,2;
tablename | indexdef
---------------------------------------------------------------------
sensors_8981001 | CREATE INDEX hash_index_on_sensors_8981001 ON citus_split_test_schema.sensors_8981001 USING hash (((measure_data -> 'IsFailed'::text)))
sensors_8981001 | CREATE INDEX index_on_sensors_8981001 ON citus_split_test_schema.sensors_8981001 USING btree (lower((measureid)::text))
sensors_8981001 | CREATE INDEX index_with_include_on_sensors_8981001 ON citus_split_test_schema.sensors_8981001 USING btree (((measure_data -> 'IsFailed'::text))) INCLUDE (measure_data, eventdatetime, measure_status)
sensors_8981001 | CREATE UNIQUE INDEX sensors_pkey_8981001 ON citus_split_test_schema.sensors_8981001 USING btree (measureid, eventdatetime, measure_data)
(4 rows)
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'table_with_index_rep_identity_%' ORDER BY 1,2;
tablename | indexdef
---------------------------------------------------------------------
table_with_index_rep_identity_8981006 | CREATE UNIQUE INDEX uqx_8981006 ON citus_split_test_schema.table_with_index_rep_identity_8981006 USING btree (key)
(1 row)
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema')
)
ORDER BY stxname ASC;
stxname
---------------------------------------------------------------------
stats_on_sensors
stats_on_sensors_8981001
(2 rows)
-- END : Display current state
-- BEGIN : Move one shard before we split it.
\c - postgres - :master_port
SET ROLE test_split_role;
SET search_path TO "citus_split_test_schema";
SET citus.next_shard_id TO 8981007;
SET citus.defer_drop_after_shard_move TO OFF;
SELECT citus_move_shard_placement(8981000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, shard_transfer_mode:='force_logical');
citus_move_shard_placement
---------------------------------------------------------------------
(1 row)
-- END : Move one shard before we split it.
-- BEGIN : Set node id variables
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
-- END : Set node id variables
-- BEGIN : Split two shards : One with move and One without move.
-- Perform 2 way split
SELECT pg_catalog.citus_split_shard_by_split_points(
8981000,
ARRAY['-1073741824'],
ARRAY[:worker_1_node, :worker_2_node],
'block_writes');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
-- Perform 3 way split
SELECT pg_catalog.citus_split_shard_by_split_points(
8981001,
ARRAY['536870911', '1610612735'],
ARRAY[:worker_1_node, :worker_1_node, :worker_2_node],
'block_writes');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
-- END : Split two shards : One with move and One without move.
-- BEGIN : Move a shard post split.
SELECT citus_move_shard_placement(8981007, 'localhost', :worker_1_port, 'localhost', :worker_2_port, shard_transfer_mode:='block_writes');
citus_move_shard_placement
---------------------------------------------------------------------
(1 row)
-- END : Move a shard post split.
-- BEGIN : Display current state.
SELECT shard.shardid, logicalrelid, shardminvalue, shardmaxvalue, nodename, nodeport
FROM pg_dist_shard AS shard
INNER JOIN pg_dist_placement placement ON shard.shardid = placement.shardid
INNER JOIN pg_dist_node node ON placement.groupid = node.groupid
INNER JOIN pg_catalog.pg_class cls ON shard.logicalrelid = cls.oid
WHERE node.noderole = 'primary' AND (logicalrelid = 'sensors'::regclass OR logicalrelid = 'colocated_dist_table'::regclass OR logicalrelid = 'table_with_index_rep_identity'::regclass)
ORDER BY logicalrelid, shardminvalue::BIGINT;
shardid | logicalrelid | shardminvalue | shardmaxvalue | nodename | nodeport
---------------------------------------------------------------------
8981007 | sensors | -2147483648 | -1073741824 | localhost | 57638
8981008 | sensors | -1073741823 | -1 | localhost | 57638
8981013 | sensors | 0 | 536870911 | localhost | 57637
8981014 | sensors | 536870912 | 1610612735 | localhost | 57637
8981015 | sensors | 1610612736 | 2147483647 | localhost | 57638
8981009 | colocated_dist_table | -2147483648 | -1073741824 | localhost | 57638
8981010 | colocated_dist_table | -1073741823 | -1 | localhost | 57638
8981016 | colocated_dist_table | 0 | 536870911 | localhost | 57637
8981017 | colocated_dist_table | 536870912 | 1610612735 | localhost | 57637
8981018 | colocated_dist_table | 1610612736 | 2147483647 | localhost | 57638
8981011 | table_with_index_rep_identity | -2147483648 | -1073741824 | localhost | 57638
8981012 | table_with_index_rep_identity | -1073741823 | -1 | localhost | 57638
8981019 | table_with_index_rep_identity | 0 | 536870911 | localhost | 57637
8981020 | table_with_index_rep_identity | 536870912 | 1610612735 | localhost | 57637
8981021 | table_with_index_rep_identity | 1610612736 | 2147483647 | localhost | 57638
(15 rows)
\c - - - :worker_1_port
SET search_path TO "citus_split_test_schema", public, pg_catalog;
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like 'sensors_%'
ORDER BY 1, 2;
relname | Constraint | Definition
---------------------------------------------------------------------
sensors_8981013 | fkey_table_to_dist_8981013 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8981016(measureid)
sensors_8981014 | fkey_table_to_dist_8981014 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8981017(measureid)
(2 rows)
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'sensors_%' ORDER BY 1,2;
tablename | indexdef
---------------------------------------------------------------------
sensors_8981013 | CREATE INDEX hash_index_on_sensors_8981013 ON citus_split_test_schema.sensors_8981013 USING hash (((measure_data -> 'IsFailed'::text)))
sensors_8981013 | CREATE INDEX index_on_sensors_8981013 ON citus_split_test_schema.sensors_8981013 USING btree (lower((measureid)::text))
sensors_8981013 | CREATE INDEX index_with_include_on_sensors_8981013 ON citus_split_test_schema.sensors_8981013 USING btree (((measure_data -> 'IsFailed'::text))) INCLUDE (measure_data, eventdatetime, measure_status)
sensors_8981013 | CREATE UNIQUE INDEX sensors_pkey_8981013 ON citus_split_test_schema.sensors_8981013 USING btree (measureid, eventdatetime, measure_data)
sensors_8981014 | CREATE INDEX hash_index_on_sensors_8981014 ON citus_split_test_schema.sensors_8981014 USING hash (((measure_data -> 'IsFailed'::text)))
sensors_8981014 | CREATE INDEX index_on_sensors_8981014 ON citus_split_test_schema.sensors_8981014 USING btree (lower((measureid)::text))
sensors_8981014 | CREATE INDEX index_with_include_on_sensors_8981014 ON citus_split_test_schema.sensors_8981014 USING btree (((measure_data -> 'IsFailed'::text))) INCLUDE (measure_data, eventdatetime, measure_status)
sensors_8981014 | CREATE UNIQUE INDEX sensors_pkey_8981014 ON citus_split_test_schema.sensors_8981014 USING btree (measureid, eventdatetime, measure_data)
(8 rows)
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'table_with_index_rep_identity_%' ORDER BY 1,2;
tablename | indexdef
---------------------------------------------------------------------
table_with_index_rep_identity_8981019 | CREATE UNIQUE INDEX uqx_8981019 ON citus_split_test_schema.table_with_index_rep_identity_8981019 USING btree (key)
table_with_index_rep_identity_8981020 | CREATE UNIQUE INDEX uqx_8981020 ON citus_split_test_schema.table_with_index_rep_identity_8981020 USING btree (key)
(2 rows)
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema')
)
ORDER BY stxname ASC;
stxname
---------------------------------------------------------------------
stats_on_sensors
stats_on_sensors_8981013
stats_on_sensors_8981014
(3 rows)
\c - - - :worker_2_port
SET search_path TO "citus_split_test_schema", public, pg_catalog;
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like 'sensors_%'
ORDER BY 1, 2;
relname | Constraint | Definition
---------------------------------------------------------------------
sensors_8981007 | fkey_table_to_dist_8981007 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8981009(measureid)
sensors_8981008 | fkey_table_to_dist_8981008 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8981010(measureid)
sensors_8981015 | fkey_table_to_dist_8981015 | FOREIGN KEY (measureid) REFERENCES colocated_dist_table_8981018(measureid)
(3 rows)
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'sensors_%' ORDER BY 1,2;
tablename | indexdef
---------------------------------------------------------------------
sensors_8981007 | CREATE INDEX hash_index_on_sensors_8981007 ON citus_split_test_schema.sensors_8981007 USING hash (((measure_data -> 'IsFailed'::text)))
sensors_8981007 | CREATE INDEX index_on_sensors_8981007 ON citus_split_test_schema.sensors_8981007 USING btree (lower((measureid)::text))
sensors_8981007 | CREATE INDEX index_with_include_on_sensors_8981007 ON citus_split_test_schema.sensors_8981007 USING btree (((measure_data -> 'IsFailed'::text))) INCLUDE (measure_data, eventdatetime, measure_status)
sensors_8981007 | CREATE UNIQUE INDEX sensors_pkey_8981007 ON citus_split_test_schema.sensors_8981007 USING btree (measureid, eventdatetime, measure_data)
sensors_8981008 | CREATE INDEX hash_index_on_sensors_8981008 ON citus_split_test_schema.sensors_8981008 USING hash (((measure_data -> 'IsFailed'::text)))
sensors_8981008 | CREATE INDEX index_on_sensors_8981008 ON citus_split_test_schema.sensors_8981008 USING btree (lower((measureid)::text))
sensors_8981008 | CREATE INDEX index_with_include_on_sensors_8981008 ON citus_split_test_schema.sensors_8981008 USING btree (((measure_data -> 'IsFailed'::text))) INCLUDE (measure_data, eventdatetime, measure_status)
sensors_8981008 | CREATE UNIQUE INDEX sensors_pkey_8981008 ON citus_split_test_schema.sensors_8981008 USING btree (measureid, eventdatetime, measure_data)
sensors_8981015 | CREATE INDEX hash_index_on_sensors_8981015 ON citus_split_test_schema.sensors_8981015 USING hash (((measure_data -> 'IsFailed'::text)))
sensors_8981015 | CREATE INDEX index_on_sensors_8981015 ON citus_split_test_schema.sensors_8981015 USING btree (lower((measureid)::text))
sensors_8981015 | CREATE INDEX index_with_include_on_sensors_8981015 ON citus_split_test_schema.sensors_8981015 USING btree (((measure_data -> 'IsFailed'::text))) INCLUDE (measure_data, eventdatetime, measure_status)
sensors_8981015 | CREATE UNIQUE INDEX sensors_pkey_8981015 ON citus_split_test_schema.sensors_8981015 USING btree (measureid, eventdatetime, measure_data)
(12 rows)
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'table_with_index_rep_identity_%' ORDER BY 1,2;
tablename | indexdef
---------------------------------------------------------------------
table_with_index_rep_identity_8981011 | CREATE UNIQUE INDEX uqx_8981011 ON citus_split_test_schema.table_with_index_rep_identity_8981011 USING btree (key)
table_with_index_rep_identity_8981012 | CREATE UNIQUE INDEX uqx_8981012 ON citus_split_test_schema.table_with_index_rep_identity_8981012 USING btree (key)
table_with_index_rep_identity_8981021 | CREATE UNIQUE INDEX uqx_8981021 ON citus_split_test_schema.table_with_index_rep_identity_8981021 USING btree (key)
(3 rows)
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema')
)
ORDER BY stxname ASC;
stxname
---------------------------------------------------------------------
stats_on_sensors
stats_on_sensors_8981007
stats_on_sensors_8981008
stats_on_sensors_8981015
(4 rows)
-- END : Display current state
-- BEGIN: Should be able to change/drop constraints
\c - postgres - :master_port
SET ROLE test_split_role;
SET search_path TO "citus_split_test_schema";
ALTER INDEX index_on_sensors RENAME TO index_on_sensors_renamed;
ALTER INDEX index_on_sensors_renamed ALTER COLUMN 1 SET STATISTICS 200;
DROP STATISTICS stats_on_sensors;
DROP INDEX index_on_sensors_renamed;
ALTER TABLE sensors DROP CONSTRAINT fkey_table_to_dist;
-- END: Should be able to change/drop constraints
-- BEGIN: Split second time on another schema
SET search_path TO public;
SET citus.next_shard_id TO 8981031;
SELECT pg_catalog.citus_split_shard_by_split_points(
8981007,
ARRAY['-2100000000'],
ARRAY[:worker_1_node, :worker_2_node],
'block_writes');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
SET search_path TO "citus_split_test_schema";
SELECT shard.shardid, logicalrelid, shardminvalue, shardmaxvalue, nodename, nodeport
FROM pg_dist_shard AS shard
INNER JOIN pg_dist_placement placement ON shard.shardid = placement.shardid
INNER JOIN pg_dist_node node ON placement.groupid = node.groupid
INNER JOIN pg_catalog.pg_class cls ON shard.logicalrelid = cls.oid
WHERE node.noderole = 'primary' AND (logicalrelid = 'sensors'::regclass OR logicalrelid = 'colocated_dist_table'::regclass OR logicalrelid = 'table_with_index_rep_identity'::regclass)
ORDER BY logicalrelid, shardminvalue::BIGINT;
shardid | logicalrelid | shardminvalue | shardmaxvalue | nodename | nodeport
---------------------------------------------------------------------
8981031 | sensors | -2147483648 | -2100000000 | localhost | 57637
8981032 | sensors | -2099999999 | -1073741824 | localhost | 57638
8981008 | sensors | -1073741823 | -1 | localhost | 57638
8981013 | sensors | 0 | 536870911 | localhost | 57637
8981014 | sensors | 536870912 | 1610612735 | localhost | 57637
8981015 | sensors | 1610612736 | 2147483647 | localhost | 57638
8981033 | colocated_dist_table | -2147483648 | -2100000000 | localhost | 57637
8981034 | colocated_dist_table | -2099999999 | -1073741824 | localhost | 57638
8981010 | colocated_dist_table | -1073741823 | -1 | localhost | 57638
8981016 | colocated_dist_table | 0 | 536870911 | localhost | 57637
8981017 | colocated_dist_table | 536870912 | 1610612735 | localhost | 57637
8981018 | colocated_dist_table | 1610612736 | 2147483647 | localhost | 57638
8981035 | table_with_index_rep_identity | -2147483648 | -2100000000 | localhost | 57637
8981036 | table_with_index_rep_identity | -2099999999 | -1073741824 | localhost | 57638
8981012 | table_with_index_rep_identity | -1073741823 | -1 | localhost | 57638
8981019 | table_with_index_rep_identity | 0 | 536870911 | localhost | 57637
8981020 | table_with_index_rep_identity | 536870912 | 1610612735 | localhost | 57637
8981021 | table_with_index_rep_identity | 1610612736 | 2147483647 | localhost | 57638
(18 rows)
-- END: Split second time on another schema
-- BEGIN: Validate Data Count
SELECT COUNT(*) FROM sensors;
count
---------------------------------------------------------------------
1001
(1 row)
SELECT COUNT(*) FROM reference_table;
count
---------------------------------------------------------------------
1001
(1 row)
SELECT COUNT(*) FROM colocated_dist_table;
count
---------------------------------------------------------------------
1001
(1 row)
-- END: Validate Data Count
--BEGIN : Cleanup
\c - postgres - :master_port
DROP SCHEMA "citus_split_test_schema" CASCADE;
NOTICE: drop cascades to 4 other objects
DETAIL: drop cascades to table citus_split_test_schema.sensors
drop cascades to table citus_split_test_schema.reference_table
drop cascades to table citus_split_test_schema.colocated_dist_table
drop cascades to table citus_split_test_schema.table_with_index_rep_identity
--END : Cleanup

View File

@ -0,0 +1,105 @@
CREATE SCHEMA "citus_split_failure_test_schema";
SET search_path TO "citus_split_failure_test_schema";
SET citus.shard_count TO 1;
SET citus.next_shard_id TO 890000;
SET citus.shard_replication_factor TO 1;
-- BEGIN: Create table to split
CREATE TABLE sensors(
measureid integer,
eventdatetime date);
CREATE TABLE sensors_colocated(
measureid integer,
eventdatetime2 date);
SELECT create_distributed_table('sensors', 'measureid');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table('sensors_colocated', 'measureid', colocate_with:='sensors');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- END: Create table to split
-- BEGIN : Switch to worker and create split shards already so workflow fails.
\c - - - :worker_1_port
SET search_path TO "citus_split_failure_test_schema";
-- Don't create sensors_8981001, workflow will create and clean it.
-- Create rest of the shards so that the workflow fails, but will not clean them.
CREATE TABLE sensors_8981002(
measureid integer,
eventdatetime date);
CREATE TABLE sensors_colocated_8981003(
measureid integer,
eventdatetime date);
CREATE TABLE sensors_colocated_8981004(
measureid integer,
eventdatetime date);
-- A random table which should not be deleted.
CREATE TABLE sensors_nodelete(
measureid integer,
eventdatetime date);
-- List tables in worker.
SET search_path TO "citus_split_failure_test_schema";
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname
FROM pg_catalog.pg_class tbl
WHERE tbl.relname like 'sensors%'
ORDER BY 1;
relname
---------------------------------------------------------------------
sensors
sensors_890000
sensors_8981002
sensors_colocated
sensors_colocated_890001
sensors_colocated_8981003
sensors_colocated_8981004
sensors_nodelete
(8 rows)
-- END : Switch to worker and create split shards already so workflow fails.
-- BEGIN : Set node id variables
\c - postgres - :master_port
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
-- END : Set node id variables
-- BEGIN : Split Shard, which is expected to fail.
SET citus.next_shard_id TO 8981001;
SELECT pg_catalog.citus_split_shard_by_split_points(
890000,
ARRAY['-1073741824'],
ARRAY[:worker_1_node, :worker_1_node],
'block_writes');
ERROR: relation "sensors_8981002" already exists
CONTEXT: while executing command on localhost:xxxxx
-- BEGIN : Split Shard, which is expected to fail.
-- BEGIN : Ensure tables were cleaned from worker
\c - - - :worker_1_port
SET search_path TO "citus_split_failure_test_schema";
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname
FROM pg_catalog.pg_class tbl
WHERE tbl.relname like 'sensors%'
ORDER BY 1;
relname
---------------------------------------------------------------------
sensors
sensors_890000
sensors_8981002
sensors_colocated
sensors_colocated_890001
sensors_colocated_8981003
sensors_colocated_8981004
sensors_nodelete
(8 rows)
-- END : Ensure tables were cleaned from worker
--BEGIN : Cleanup
\c - postgres - :master_port
DROP SCHEMA "citus_split_failure_test_schema" CASCADE;
NOTICE: drop cascades to 2 other objects
DETAIL: drop cascades to table citus_split_failure_test_schema.sensors
drop cascades to table citus_split_failure_test_schema.sensors_colocated
--END : Cleanup

View File

@ -0,0 +1,173 @@
-- Negative test cases for citus_split_shard_by_split_points UDF.
CREATE SCHEMA citus_split_shard_by_split_points_negative;
SET search_path TO citus_split_shard_by_split_points_negative;
SET citus.shard_count TO 4;
SET citus.shard_replication_factor TO 1;
SET citus.next_shard_id TO 60761300;
CREATE TABLE range_paritioned_table_to_split(rid bigserial PRIMARY KEY, value char);
SELECT create_distributed_table('range_paritioned_table_to_split', 'rid', 'range');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- Shards are not created automatically for range distributed table.
SELECT master_create_empty_shard('range_paritioned_table_to_split');
master_create_empty_shard
---------------------------------------------------------------------
60761300
(1 row)
SET citus.next_shard_id TO 49761300;
CREATE TABLE table_to_split (id bigserial PRIMARY KEY, value char);
-- Shard1 | -2147483648 | -1073741825
-- Shard2 | -1073741824 | -1
-- Shard3 | 0 | 1073741823
-- Shard4 | 1073741824 | 2147483647
SELECT create_distributed_table('table_to_split','id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
-- UDF fails for any other shard_transfer_mode other than block_writes.
SELECT citus_split_shard_by_split_points(
49761302,
ARRAY['50'],
ARRAY[101, 201],
'auto');
ERROR: Shard Tranfer mode: 'auto' is not supported. Please use 'block_writes' instead.
SELECT citus_split_shard_by_split_points(
49761302,
ARRAY['50'],
ARRAY[101, 201],
'force_logical');
ERROR: Shard Tranfer mode: 'force_logical' is not supported. Please use 'block_writes' instead.
SELECT citus_split_shard_by_split_points(
49761302,
ARRAY['50'],
ARRAY[101, 201],
'gibberish');
ERROR: invalid input value for enum citus.shard_transfer_mode: "gibberish"
-- UDF fails for range partitioned tables.
SELECT citus_split_shard_by_split_points(
60761300,
ARRAY['-1073741826'],
ARRAY[:worker_1_node, :worker_2_node]);
ERROR: Cannot split shard as operation is only supported for hash distributed tables.
-- UDF fails if number of placement node list does not exceed split points by one.
-- Example: One split point defines two way split (2 worker nodes needed).
SELECT citus_split_shard_by_split_points(
49761300,
-- 2 split points defined making it a 3 way split but we only specify 2 placement lists.
ARRAY['-1073741826', '-107374182'],
ARRAY[:worker_1_node, :worker_2_node]); -- 2 worker nodes.
ERROR: Number of worker node ids should be one greater split points. NodeId count is '2' and SplitPoint count is '2'.
-- UDF fails if split ranges specified are not within the shard id to split.
SELECT citus_split_shard_by_split_points(
49761300, -- Shard range is from (-2147483648, -1073741825)
ARRAY['0'], -- The range we specified is 0 which is not in the range.
ARRAY[:worker_1_node, :worker_2_node]);
ERROR: Split point 0 is outside the min/max range(-2147483648, -1073741825) for shard id 49761300.
-- UDF fails if split points are not strictly increasing.
SELECT citus_split_shard_by_split_points(
49761302,
ARRAY['50', '35'],
ARRAY[:worker_1_node, :worker_2_node, :worker_1_node]);
ERROR: Invalid Split Points '50' followed by '35'. All split points should be strictly increasing.
SELECT citus_split_shard_by_split_points(
49761302,
ARRAY['50', '50'],
ARRAY[:worker_1_node, :worker_2_node, :worker_1_node]);
ERROR: Invalid Split Points '50' followed by '50'. All split points should be strictly increasing.
-- UDF fails if nodeIds are < 1 or Invalid.
SELECT citus_split_shard_by_split_points(
49761302,
ARRAY['50'],
ARRAY[0, :worker_2_node]);
ERROR: Invalid Node Id '0'.
SELECT citus_split_shard_by_split_points(
49761302,
ARRAY['50'],
ARRAY[101, 201]);
ERROR: Invalid Node Id '101'.
-- UDF fails if split point specified is equal to the max value in the range.
-- Example: ShardId 81060002 range is from (-2147483648, -1073741825)
-- '-1073741825' as split point is invalid.
-- '-1073741826' is valid and will split to: (-2147483648, -1073741826) and (-1073741825, -1073741825)
SELECT citus_split_shard_by_split_points(
49761300, -- Shard range is from (-2147483648, -1073741825)
ARRAY['-1073741825'], -- Split point equals shard's max value.
ARRAY[:worker_1_node, :worker_2_node]);
ERROR: Invalid split point -1073741825, as split points should be inclusive. Please use -1073741826 instead.
-- UDF fails if resulting shard count from split greater than MAX_SHARD_COUNT (64000)
-- 64000 split point definee 64000+1 way split (64001 worker nodes needed).
WITH shard_ranges AS (SELECT ((-2147483648 + indx))::text as split_points, :worker_1_node as node_ids FROM generate_series(1,64000) indx )
SELECT citus_split_shard_by_split_points(
49761300,
array_agg(split_points),
array_agg(node_ids) || :worker_1_node) --placement node list should exceed split points by one.
FROM shard_ranges;
ERROR: Resulting shard count '64001' with split is greater than max shard count '64000' limit.
-- UDF fails where source shard cannot be split further i.e min and max range is equal.
-- Create a Shard where range cannot be split further
SELECT isolate_tenant_to_new_shard('table_to_split', 1);
isolate_tenant_to_new_shard
---------------------------------------------------------------------
49761305
(1 row)
SELECT citus_split_shard_by_split_points(
49761305,
ARRAY['-1073741826'],
ARRAY[:worker_1_node, :worker_2_node]);
ERROR: Cannot split shard id "49761305" as min/max range are equal: ('-1905060026', '-1905060026').
-- Create distributed table with replication factor > 1
SET citus.shard_replication_factor TO 2;
SET citus.next_shard_id TO 51261400;
CREATE TABLE table_to_split_replication_factor_2 (id bigserial PRIMARY KEY, value char);
SELECT create_distributed_table('table_to_split_replication_factor_2','id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- UDF fails for replication factor > 1
SELECT citus_split_shard_by_split_points(
51261400,
ARRAY['-1073741826'],
ARRAY[:worker_1_node, :worker_2_node]);
ERROR: Operation split not supported for shard as replication factor '2' is greater than 1.
-- Create distributed table with columnar type.
SET citus.next_shard_id TO 51271400;
CREATE TABLE table_to_split_columnar (id bigserial PRIMARY KEY, value char) USING columnar;
SELECT create_distributed_table('table_to_split_columnar','id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- UDF fails for columnar table.
SELECT citus_split_shard_by_split_points(
51271400,
ARRAY['-1073741826'],
ARRAY[:worker_1_node, :worker_2_node]);
ERROR: Cannot split shard as operation is not supported for Columnar tables.
-- Create distributed table which is partitioned.
SET citus.next_shard_id TO 51271900;
CREATE TABLE table_to_split_partitioned(id integer, dt date) PARTITION BY RANGE(dt);
SELECT create_distributed_table('table_to_split_partitioned','id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- UDF fails for partitioned table.
SELECT citus_split_shard_by_split_points(
51271900,
ARRAY['-1073741826'],
ARRAY[:worker_1_node, :worker_2_node]);
ERROR: cannot split of 'table_to_split_partitioned', because it is a partitioned table
DETAIL: In colocation group of 'table_to_split_partitioned', a partitioned relation exists: 'table_to_split_partitioned'. Citus does not support split of partitioned tables.

View File

@ -0,0 +1,951 @@
Parsed test spec with 2 sessions
starting permutation: s1-load-cache s1-insert s1-begin s1-select s2-begin s2-blocking-shard-split s1-update s2-commit s1-commit s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-load-cache:
-- Indirect way to load cache.
TRUNCATE to_split_table;
step s1-insert:
-- Id '123456789' maps to shard xxxxx.
SELECT get_shard_id_for_distribution_column('to_split_table', 123456789);
INSERT INTO to_split_table VALUES (123456789, 1);
get_shard_id_for_distribution_column
---------------------------------------------------------------------
1500002
(1 row)
step s1-begin:
BEGIN;
-- the tests are written with the logic where single shard SELECTs
-- do not to open transaction blocks
SET citus.select_opens_transaction_block TO false;
step s1-select:
SELECT count(*) FROM to_split_table WHERE id = 123456789;
count
---------------------------------------------------------------------
1
(1 row)
step s2-begin:
BEGIN;
step s2-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500002,
ARRAY['1073741824'],
ARRAY[1, 2],
'block_writes');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-update:
UPDATE to_split_table SET value = 111 WHERE id = 123456789;
<waiting ...>
step s2-commit:
COMMIT;
step s1-update: <... completed>
ERROR: could not find valid entry for shard xxxxx
step s1-commit:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 0
57637|1500003|t | 1
57638|1500004|t | 0
(3 rows)
id|value
---------------------------------------------------------------------
123456789| 1
(1 row)
starting permutation: s1-load-cache s1-insert s1-begin s1-select s2-begin s2-blocking-shard-split s1-delete s2-commit s1-commit s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-load-cache:
-- Indirect way to load cache.
TRUNCATE to_split_table;
step s1-insert:
-- Id '123456789' maps to shard xxxxx.
SELECT get_shard_id_for_distribution_column('to_split_table', 123456789);
INSERT INTO to_split_table VALUES (123456789, 1);
get_shard_id_for_distribution_column
---------------------------------------------------------------------
1500002
(1 row)
step s1-begin:
BEGIN;
-- the tests are written with the logic where single shard SELECTs
-- do not to open transaction blocks
SET citus.select_opens_transaction_block TO false;
step s1-select:
SELECT count(*) FROM to_split_table WHERE id = 123456789;
count
---------------------------------------------------------------------
1
(1 row)
step s2-begin:
BEGIN;
step s2-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500002,
ARRAY['1073741824'],
ARRAY[1, 2],
'block_writes');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-delete:
DELETE FROM to_split_table WHERE id = 123456789;
<waiting ...>
step s2-commit:
COMMIT;
step s1-delete: <... completed>
ERROR: could not find valid entry for shard xxxxx
step s1-commit:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 0
57637|1500003|t | 1
57638|1500004|t | 0
(3 rows)
id|value
---------------------------------------------------------------------
123456789| 1
(1 row)
starting permutation: s1-load-cache s1-begin s1-select s2-begin s2-blocking-shard-split s1-insert s2-commit s1-commit s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-load-cache:
-- Indirect way to load cache.
TRUNCATE to_split_table;
step s1-begin:
BEGIN;
-- the tests are written with the logic where single shard SELECTs
-- do not to open transaction blocks
SET citus.select_opens_transaction_block TO false;
step s1-select:
SELECT count(*) FROM to_split_table WHERE id = 123456789;
count
---------------------------------------------------------------------
0
(1 row)
step s2-begin:
BEGIN;
step s2-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500002,
ARRAY['1073741824'],
ARRAY[1, 2],
'block_writes');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-insert:
-- Id '123456789' maps to shard xxxxx.
SELECT get_shard_id_for_distribution_column('to_split_table', 123456789);
INSERT INTO to_split_table VALUES (123456789, 1);
<waiting ...>
step s2-commit:
COMMIT;
step s1-insert: <... completed>
get_shard_id_for_distribution_column
---------------------------------------------------------------------
1500002
(1 row)
ERROR: could not find valid entry for shard xxxxx
step s1-commit:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 0
57637|1500003|t | 0
57638|1500004|t | 0
(3 rows)
id|value
---------------------------------------------------------------------
(0 rows)
starting permutation: s1-load-cache s1-begin s1-select s2-begin s2-blocking-shard-split s1-copy s2-commit s1-commit s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-load-cache:
-- Indirect way to load cache.
TRUNCATE to_split_table;
step s1-begin:
BEGIN;
-- the tests are written with the logic where single shard SELECTs
-- do not to open transaction blocks
SET citus.select_opens_transaction_block TO false;
step s1-select:
SELECT count(*) FROM to_split_table WHERE id = 123456789;
count
---------------------------------------------------------------------
0
(1 row)
step s2-begin:
BEGIN;
step s2-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500002,
ARRAY['1073741824'],
ARRAY[1, 2],
'block_writes');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-copy:
COPY to_split_table FROM PROGRAM 'echo "1,1\n2,2\n3,3\n4,4\n5,5"' WITH CSV;
<waiting ...>
step s2-commit:
COMMIT;
step s1-copy: <... completed>
ERROR: could not find valid entry for shard xxxxx
step s1-commit:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 0
57637|1500003|t | 0
57638|1500004|t | 0
(3 rows)
id|value
---------------------------------------------------------------------
(0 rows)
starting permutation: s1-insert s1-begin s1-select s2-begin s2-blocking-shard-split s1-update s2-commit s1-commit s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-insert:
-- Id '123456789' maps to shard xxxxx.
SELECT get_shard_id_for_distribution_column('to_split_table', 123456789);
INSERT INTO to_split_table VALUES (123456789, 1);
get_shard_id_for_distribution_column
---------------------------------------------------------------------
1500002
(1 row)
step s1-begin:
BEGIN;
-- the tests are written with the logic where single shard SELECTs
-- do not to open transaction blocks
SET citus.select_opens_transaction_block TO false;
step s1-select:
SELECT count(*) FROM to_split_table WHERE id = 123456789;
count
---------------------------------------------------------------------
1
(1 row)
step s2-begin:
BEGIN;
step s2-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500002,
ARRAY['1073741824'],
ARRAY[1, 2],
'block_writes');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-update:
UPDATE to_split_table SET value = 111 WHERE id = 123456789;
<waiting ...>
step s2-commit:
COMMIT;
step s1-update: <... completed>
ERROR: could not find valid entry for shard xxxxx
step s1-commit:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 0
57637|1500003|t | 1
57638|1500004|t | 0
(3 rows)
id|value
---------------------------------------------------------------------
123456789| 1
(1 row)
starting permutation: s1-insert s1-begin s1-select s2-begin s2-blocking-shard-split s1-delete s2-commit s1-commit s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-insert:
-- Id '123456789' maps to shard xxxxx.
SELECT get_shard_id_for_distribution_column('to_split_table', 123456789);
INSERT INTO to_split_table VALUES (123456789, 1);
get_shard_id_for_distribution_column
---------------------------------------------------------------------
1500002
(1 row)
step s1-begin:
BEGIN;
-- the tests are written with the logic where single shard SELECTs
-- do not to open transaction blocks
SET citus.select_opens_transaction_block TO false;
step s1-select:
SELECT count(*) FROM to_split_table WHERE id = 123456789;
count
---------------------------------------------------------------------
1
(1 row)
step s2-begin:
BEGIN;
step s2-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500002,
ARRAY['1073741824'],
ARRAY[1, 2],
'block_writes');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-delete:
DELETE FROM to_split_table WHERE id = 123456789;
<waiting ...>
step s2-commit:
COMMIT;
step s1-delete: <... completed>
ERROR: could not find valid entry for shard xxxxx
step s1-commit:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 0
57637|1500003|t | 1
57638|1500004|t | 0
(3 rows)
id|value
---------------------------------------------------------------------
123456789| 1
(1 row)
starting permutation: s1-begin s1-select s2-begin s2-blocking-shard-split s1-insert s2-commit s1-commit s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
-- the tests are written with the logic where single shard SELECTs
-- do not to open transaction blocks
SET citus.select_opens_transaction_block TO false;
step s1-select:
SELECT count(*) FROM to_split_table WHERE id = 123456789;
count
---------------------------------------------------------------------
0
(1 row)
step s2-begin:
BEGIN;
step s2-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500002,
ARRAY['1073741824'],
ARRAY[1, 2],
'block_writes');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-insert:
-- Id '123456789' maps to shard xxxxx.
SELECT get_shard_id_for_distribution_column('to_split_table', 123456789);
INSERT INTO to_split_table VALUES (123456789, 1);
<waiting ...>
step s2-commit:
COMMIT;
step s1-insert: <... completed>
get_shard_id_for_distribution_column
---------------------------------------------------------------------
1500002
(1 row)
ERROR: could not find valid entry for shard xxxxx
step s1-commit:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 0
57637|1500003|t | 0
57638|1500004|t | 0
(3 rows)
id|value
---------------------------------------------------------------------
(0 rows)
starting permutation: s1-begin s1-select s2-begin s2-blocking-shard-split s1-copy s2-commit s1-commit s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
-- the tests are written with the logic where single shard SELECTs
-- do not to open transaction blocks
SET citus.select_opens_transaction_block TO false;
step s1-select:
SELECT count(*) FROM to_split_table WHERE id = 123456789;
count
---------------------------------------------------------------------
0
(1 row)
step s2-begin:
BEGIN;
step s2-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500002,
ARRAY['1073741824'],
ARRAY[1, 2],
'block_writes');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-copy:
COPY to_split_table FROM PROGRAM 'echo "1,1\n2,2\n3,3\n4,4\n5,5"' WITH CSV;
<waiting ...>
step s2-commit:
COMMIT;
step s1-copy: <... completed>
ERROR: could not find valid entry for shard xxxxx
step s1-commit:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 0
57637|1500003|t | 0
57638|1500004|t | 0
(3 rows)
id|value
---------------------------------------------------------------------
(0 rows)
starting permutation: s1-load-cache s1-insert s1-begin s1-blocking-shard-split s2-blocking-shard-split s1-commit s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-load-cache:
-- Indirect way to load cache.
TRUNCATE to_split_table;
step s1-insert:
-- Id '123456789' maps to shard xxxxx.
SELECT get_shard_id_for_distribution_column('to_split_table', 123456789);
INSERT INTO to_split_table VALUES (123456789, 1);
get_shard_id_for_distribution_column
---------------------------------------------------------------------
1500002
(1 row)
step s1-begin:
BEGIN;
-- the tests are written with the logic where single shard SELECTs
-- do not to open transaction blocks
SET citus.select_opens_transaction_block TO false;
step s1-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500001,
ARRAY['-1073741824'],
ARRAY[1, 2],
'block_writes');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s2-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500002,
ARRAY['1073741824'],
ARRAY[1, 2],
'block_writes');
<waiting ...>
step s1-commit:
COMMIT;
step s2-blocking-shard-split: <... completed>
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500003|t | 0
57637|1500005|t | 1
57638|1500004|t | 0
57638|1500006|t | 0
(4 rows)
id|value
---------------------------------------------------------------------
123456789| 1
(1 row)
starting permutation: s1-insert s1-begin s1-blocking-shard-split s2-blocking-shard-split s1-commit s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-insert:
-- Id '123456789' maps to shard xxxxx.
SELECT get_shard_id_for_distribution_column('to_split_table', 123456789);
INSERT INTO to_split_table VALUES (123456789, 1);
get_shard_id_for_distribution_column
---------------------------------------------------------------------
1500002
(1 row)
step s1-begin:
BEGIN;
-- the tests are written with the logic where single shard SELECTs
-- do not to open transaction blocks
SET citus.select_opens_transaction_block TO false;
step s1-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500001,
ARRAY['-1073741824'],
ARRAY[1, 2],
'block_writes');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s2-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500002,
ARRAY['1073741824'],
ARRAY[1, 2],
'block_writes');
<waiting ...>
step s1-commit:
COMMIT;
step s2-blocking-shard-split: <... completed>
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500003|t | 0
57637|1500005|t | 1
57638|1500004|t | 0
57638|1500006|t | 0
(4 rows)
id|value
---------------------------------------------------------------------
123456789| 1
(1 row)
starting permutation: s1-load-cache s1-begin s1-select s2-begin s2-blocking-shard-split s1-ddl s2-commit s1-commit s2-print-cluster s2-print-index-count
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-load-cache:
-- Indirect way to load cache.
TRUNCATE to_split_table;
step s1-begin:
BEGIN;
-- the tests are written with the logic where single shard SELECTs
-- do not to open transaction blocks
SET citus.select_opens_transaction_block TO false;
step s1-select:
SELECT count(*) FROM to_split_table WHERE id = 123456789;
count
---------------------------------------------------------------------
0
(1 row)
step s2-begin:
BEGIN;
step s2-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500002,
ARRAY['1073741824'],
ARRAY[1, 2],
'block_writes');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-ddl:
CREATE INDEX test_table_index ON to_split_table(id);
<waiting ...>
step s2-commit:
COMMIT;
step s1-ddl: <... completed>
step s1-commit:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 0
57637|1500003|t | 0
57638|1500004|t | 0
(3 rows)
id|value
---------------------------------------------------------------------
(0 rows)
step s2-print-index-count:
SELECT
nodeport, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from pg_indexes WHERE tablename = ''%s''')
ORDER BY
nodeport;
nodeport|success|result
---------------------------------------------------------------------
57637|t | 1
57637|t | 1
57638|t | 1
(3 rows)
starting permutation: s1-begin s1-select s2-begin s2-blocking-shard-split s1-ddl s2-commit s1-commit s2-print-cluster s2-print-index-count
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
-- the tests are written with the logic where single shard SELECTs
-- do not to open transaction blocks
SET citus.select_opens_transaction_block TO false;
step s1-select:
SELECT count(*) FROM to_split_table WHERE id = 123456789;
count
---------------------------------------------------------------------
0
(1 row)
step s2-begin:
BEGIN;
step s2-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500002,
ARRAY['1073741824'],
ARRAY[1, 2],
'block_writes');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-ddl:
CREATE INDEX test_table_index ON to_split_table(id);
<waiting ...>
step s2-commit:
COMMIT;
step s1-ddl: <... completed>
step s1-commit:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500001|t | 0
57637|1500003|t | 0
57638|1500004|t | 0
(3 rows)
id|value
---------------------------------------------------------------------
(0 rows)
step s2-print-index-count:
SELECT
nodeport, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from pg_indexes WHERE tablename = ''%s''')
ORDER BY
nodeport;
nodeport|success|result
---------------------------------------------------------------------
57637|t | 1
57637|t | 1
57638|t | 1
(3 rows)

View File

@ -0,0 +1,301 @@
Parsed test spec with 2 sessions
starting permutation: s2-add-fkey s1-begin s2-begin s2-blocking-shard-split s1-delete s2-commit s1-commit s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s2-add-fkey:
ALTER TABLE table_to_split ADD CONSTRAINT fkey_const FOREIGN KEY(value) REFERENCES reference_table(id);
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500002,
ARRAY['-1073741824'],
ARRAY[1, 2],
'block_writes');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-delete:
DELETE FROM reference_table WHERE id = 5;
<waiting ...>
step s2-commit:
COMMIT;
step s1-delete: <... completed>
step s1-commit:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('table_to_split', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM table_to_split ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500004|t | 0
57638|1500003|t | 0
57638|1500005|t | 0
(3 rows)
id|value
---------------------------------------------------------------------
(0 rows)
starting permutation: s2-add-fkey s1-begin s2-begin s2-blocking-shard-split s1-update s2-commit s1-commit s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s2-add-fkey:
ALTER TABLE table_to_split ADD CONSTRAINT fkey_const FOREIGN KEY(value) REFERENCES reference_table(id);
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500002,
ARRAY['-1073741824'],
ARRAY[1, 2],
'block_writes');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-update:
UPDATE reference_table SET value = 5 WHERE id = 5;
<waiting ...>
step s2-commit:
COMMIT;
step s1-update: <... completed>
step s1-commit:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('table_to_split', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM table_to_split ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500004|t | 0
57638|1500003|t | 0
57638|1500005|t | 0
(3 rows)
id|value
---------------------------------------------------------------------
(0 rows)
starting permutation: s2-add-fkey s1-begin s2-begin s2-blocking-shard-split s1-insert s2-commit s1-commit s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s2-add-fkey:
ALTER TABLE table_to_split ADD CONSTRAINT fkey_const FOREIGN KEY(value) REFERENCES reference_table(id);
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500002,
ARRAY['-1073741824'],
ARRAY[1, 2],
'block_writes');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-insert:
INSERT INTO reference_table VALUES (5, 10);
<waiting ...>
step s2-commit:
COMMIT;
step s1-insert: <... completed>
step s1-commit:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('table_to_split', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM table_to_split ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500004|t | 0
57638|1500003|t | 0
57638|1500005|t | 0
(3 rows)
id|value
---------------------------------------------------------------------
(0 rows)
starting permutation: s2-add-fkey s1-begin s2-begin s2-blocking-shard-split s1-copy s2-commit s1-commit s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s2-add-fkey:
ALTER TABLE table_to_split ADD CONSTRAINT fkey_const FOREIGN KEY(value) REFERENCES reference_table(id);
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500002,
ARRAY['-1073741824'],
ARRAY[1, 2],
'block_writes');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-copy:
COPY reference_table FROM PROGRAM 'echo "1,1\n2,2\n3,3\n4,4\n5,5"' WITH CSV;
<waiting ...>
step s2-commit:
COMMIT;
step s1-copy: <... completed>
step s1-commit:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('table_to_split', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM table_to_split ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500004|t | 0
57638|1500003|t | 0
57638|1500005|t | 0
(3 rows)
id|value
---------------------------------------------------------------------
(0 rows)
starting permutation: s2-add-fkey s1-begin s2-begin s2-blocking-shard-split s1-ddl s2-commit s1-commit s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s2-add-fkey:
ALTER TABLE table_to_split ADD CONSTRAINT fkey_const FOREIGN KEY(value) REFERENCES reference_table(id);
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-blocking-shard-split:
SELECT pg_catalog.citus_split_shard_by_split_points(
1500002,
ARRAY['-1073741824'],
ARRAY[1, 2],
'block_writes');
citus_split_shard_by_split_points
---------------------------------------------------------------------
(1 row)
step s1-ddl:
CREATE INDEX reference_table_index ON reference_table(id);
<waiting ...>
step s2-commit:
COMMIT;
step s1-ddl: <... completed>
step s1-commit:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('table_to_split', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM table_to_split ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500004|t | 0
57638|1500003|t | 0
57638|1500005|t | 0
(3 rows)
id|value
---------------------------------------------------------------------
(0 rows)

View File

@ -1080,7 +1080,7 @@ SELECT * FROM multi_extension.print_extension_changes();
-- 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 |
@ -1103,7 +1103,10 @@ SELECT * FROM multi_extension.print_extension_changes();
table columnar.chunk_group |
table columnar.options |
table columnar.stripe |
(21 rows)
| function citus_split_shard_by_split_points(bigint,text[],integer[],citus.shard_transfer_mode) void
| function worker_split_copy(bigint,split_copy_info[]) void
| type split_copy_info
(24 rows)
DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff;
-- show running version

View File

@ -107,6 +107,7 @@ ORDER BY 1;
function citus_shard_indexes_on_worker()
function citus_shard_sizes()
function citus_shards_on_worker()
function citus_split_shard_by_split_points(bigint,text[],integer[],citus.shard_transfer_mode)
function citus_stat_activity()
function citus_stat_statements()
function citus_stat_statements_reset()
@ -233,6 +234,7 @@ ORDER BY 1;
function worker_partitioned_table_size(regclass)
function worker_record_sequence_dependency(regclass,regclass,name)
function worker_save_query_explain_analyze(text,jsonb)
function worker_split_copy(bigint,split_copy_info[])
schema citus
schema citus_internal
sequence pg_dist_colocationid_seq
@ -256,6 +258,7 @@ ORDER BY 1;
type citus.shard_transfer_mode
type citus_copy_format
type noderole
type split_copy_info
view citus_dist_stat_activity
view citus_lock_waits
view citus_schema.citus_tables
@ -266,5 +269,5 @@ ORDER BY 1;
view citus_stat_statements
view pg_dist_shard_placement
view time_partitions
(250 rows)
(253 rows)

View File

@ -0,0 +1,227 @@
CREATE SCHEMA worker_shard_binary_copy_test;
SET search_path TO worker_shard_binary_copy_test;
SET citus.shard_count TO 1;
SET citus.shard_replication_factor TO 1;
SET citus.next_shard_id TO 81060000;
-- BEGIN: Create distributed table and insert data.
CREATE TABLE worker_shard_binary_copy_test.shard_to_split_copy (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
SELECT create_distributed_table('shard_to_split_copy', 'l_orderkey');
create_distributed_table
---------------------------------------------------------------------
(1 row)
\COPY shard_to_split_copy FROM STDIN WITH DELIMITER '|'
-- END: Create distributed table and insert data.
-- BEGIN: Switch to Worker1, Create target shards in worker for local 2-way split copy.
\c - - - :worker_1_port
CREATE TABLE worker_shard_binary_copy_test.shard_to_split_copy_81060015 (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
CREATE TABLE worker_shard_binary_copy_test.shard_to_split_copy_81060016 (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
-- End: Switch to Worker1, Create target shards in worker for local 2-way split copy.
-- BEGIN: Switch to Worker2, Create target shards in worker for remote 2-way split copy.
\c - - - :worker_2_port
CREATE TABLE worker_shard_binary_copy_test.shard_to_split_copy_81060015 (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
CREATE TABLE worker_shard_binary_copy_test.shard_to_split_copy_81060016 (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
-- End: Switch to Worker2, Create target shards in worker for remote 2-way split copy.
-- BEGIN: List row count for source shard and targets shard in Worker1.
\c - - - :worker_1_port
SELECT COUNT(*) FROM worker_shard_binary_copy_test.shard_to_split_copy_81060000;
count
---------------------------------------------------------------------
22
(1 row)
SELECT COUNT(*) FROM worker_shard_binary_copy_test.shard_to_split_copy_81060015;
count
---------------------------------------------------------------------
0
(1 row)
SELECT COUNT(*) FROM worker_shard_binary_copy_test.shard_to_split_copy_81060016;
count
---------------------------------------------------------------------
0
(1 row)
-- END: List row count for source shard and targets shard in Worker1.
-- BEGIN: List row count for target shard in Worker2.
\c - - - :worker_2_port
SELECT COUNT(*) FROM worker_shard_binary_copy_test.shard_to_split_copy_81060015;
count
---------------------------------------------------------------------
0
(1 row)
SELECT COUNT(*) FROM worker_shard_binary_copy_test.shard_to_split_copy_81060016;
count
---------------------------------------------------------------------
0
(1 row)
-- END: List row count for targets shard in Worker2.
-- BEGIN: Set worker_1_node and worker_2_node
\c - - - :worker_1_port
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
-- END: Set worker_1_node and worker_2_node
-- BEGIN: Trigger 2-way local shard split copy.
-- Ensure we will perform binary copy.
SET citus.enable_binary_protocol = TRUE;
SELECT * from worker_split_copy(
81060000, -- source shard id to copy
ARRAY[
-- split copy info for split children 1
ROW(81060015, -- destination shard id
-2147483648, -- split range begin
1073741823, --split range end
:worker_1_node)::pg_catalog.split_copy_info,
-- split copy info for split children 2
ROW(81060016, --destination shard id
1073741824, --split range begin
2147483647, --split range end
:worker_1_node)::pg_catalog.split_copy_info
]
);
worker_split_copy
---------------------------------------------------------------------
(1 row)
-- END: Trigger 2-way local shard split copy.
-- BEGIN: Trigger 2-way remote shard split copy.
SELECT * from worker_split_copy(
81060000, -- source shard id to copy
ARRAY[
-- split copy info for split children 1
ROW(81060015, -- destination shard id
-2147483648, -- split range begin
1073741823, --split range end
:worker_2_node)::pg_catalog.split_copy_info,
-- split copy info for split children 2
ROW(81060016, --destination shard id
1073741824, --split range begin
2147483647, --split range end
:worker_2_node)::pg_catalog.split_copy_info
]
);
worker_split_copy
---------------------------------------------------------------------
(1 row)
-- END: Trigger 2-way remote shard split copy.
-- BEGIN: List updated row count for local targets shard.
SELECT COUNT(*) FROM worker_shard_binary_copy_test.shard_to_split_copy_81060015;
count
---------------------------------------------------------------------
21
(1 row)
SELECT COUNT(*) FROM worker_shard_binary_copy_test.shard_to_split_copy_81060016;
count
---------------------------------------------------------------------
1
(1 row)
-- END: List updated row count for local targets shard.
-- BEGIN: List updated row count for remote targets shard.
\c - - - :worker_2_port
SELECT COUNT(*) FROM worker_shard_binary_copy_test.shard_to_split_copy_81060015;
count
---------------------------------------------------------------------
21
(1 row)
SELECT COUNT(*) FROM worker_shard_binary_copy_test.shard_to_split_copy_81060016;
count
---------------------------------------------------------------------
1
(1 row)
-- END: List updated row count for remote targets shard.
-- BEGIN: CLEANUP.
\c - - - :master_port
SET client_min_messages TO WARNING;
DROP SCHEMA citus_split_shard_by_split_points_local CASCADE;
ERROR: schema "citus_split_shard_by_split_points_local" does not exist
-- END: CLEANUP.

View File

@ -0,0 +1,227 @@
CREATE SCHEMA worker_shard_text_copy_test;
SET search_path TO worker_shard_text_copy_test;
SET citus.shard_count TO 1;
SET citus.shard_replication_factor TO 1;
SET citus.next_shard_id TO 81070000;
-- BEGIN: Create distributed table and insert data.
CREATE TABLE worker_shard_text_copy_test.shard_to_split_copy (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
SELECT create_distributed_table('shard_to_split_copy', 'l_orderkey');
create_distributed_table
---------------------------------------------------------------------
(1 row)
\COPY shard_to_split_copy FROM STDIN WITH DELIMITER '|'
-- END: Create distributed table and insert data.
-- BEGIN: Switch to Worker1, Create target shards in worker for local 2-way split copy.
\c - - - :worker_1_port
CREATE TABLE worker_shard_text_copy_test.shard_to_split_copy_81070015 (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
CREATE TABLE worker_shard_text_copy_test.shard_to_split_copy_81070016 (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
-- End: Switch to Worker1, Create target shards in worker for local 2-way split copy.
-- BEGIN: Switch to Worker2, Create target shards in worker for remote 2-way split copy.
\c - - - :worker_2_port
CREATE TABLE worker_shard_text_copy_test.shard_to_split_copy_81070015 (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
CREATE TABLE worker_shard_text_copy_test.shard_to_split_copy_81070016 (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
-- End: Switch to Worker2, Create target shards in worker for remote 2-way split copy.
-- BEGIN: List row count for source shard and targets shard in Worker1.
\c - - - :worker_1_port
SELECT COUNT(*) FROM worker_shard_text_copy_test.shard_to_split_copy_81070000;
count
---------------------------------------------------------------------
22
(1 row)
SELECT COUNT(*) FROM worker_shard_text_copy_test.shard_to_split_copy_81070015;
count
---------------------------------------------------------------------
0
(1 row)
SELECT COUNT(*) FROM worker_shard_text_copy_test.shard_to_split_copy_81070016;
count
---------------------------------------------------------------------
0
(1 row)
-- END: List row count for source shard and targets shard in Worker1.
-- BEGIN: List row count for target shard in Worker2.
\c - - - :worker_2_port
SELECT COUNT(*) FROM worker_shard_text_copy_test.shard_to_split_copy_81070015;
count
---------------------------------------------------------------------
0
(1 row)
SELECT COUNT(*) FROM worker_shard_text_copy_test.shard_to_split_copy_81070016;
count
---------------------------------------------------------------------
0
(1 row)
-- END: List row count for targets shard in Worker2.
-- BEGIN: Set worker_1_node and worker_2_node
\c - - - :worker_1_port
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
-- END: Set worker_1_node and worker_2_node
-- BEGIN: Trigger 2-way local shard split copy.
-- Ensure we will perform text copy.
SET citus.enable_binary_protocol = FALSE;
SELECT * from worker_split_copy(
81070000, -- source shard id to copy
ARRAY[
-- split copy info for split children 1
ROW(81070015, -- destination shard id
-2147483648, -- split range begin
1073741823, --split range end
:worker_1_node)::pg_catalog.split_copy_info,
-- split copy info for split children 2
ROW(81070016, --destination shard id
1073741824, --split range begin
2147483647, --split range end
:worker_1_node)::pg_catalog.split_copy_info
]
);
worker_split_copy
---------------------------------------------------------------------
(1 row)
-- END: Trigger 2-way local shard split copy.
-- BEGIN: Trigger 2-way remote shard split copy.
SELECT * from worker_split_copy(
81070000, -- source shard id to copy
ARRAY[
-- split copy info for split children 1
ROW(81070015, -- destination shard id
-2147483648, -- split range begin
1073741823, --split range end
:worker_2_node)::pg_catalog.split_copy_info,
-- split copy info for split children 2
ROW(81070016, --destination shard id
1073741824, --split range begin
2147483647, --split range end
:worker_2_node)::pg_catalog.split_copy_info
]
);
worker_split_copy
---------------------------------------------------------------------
(1 row)
-- END: Trigger 2-way remote shard split copy.
-- BEGIN: List updated row count for local targets shard.
SELECT COUNT(*) FROM worker_shard_text_copy_test.shard_to_split_copy_81070015;
count
---------------------------------------------------------------------
21
(1 row)
SELECT COUNT(*) FROM worker_shard_text_copy_test.shard_to_split_copy_81070016;
count
---------------------------------------------------------------------
1
(1 row)
-- END: List updated row count for local targets shard.
-- BEGIN: List updated row count for remote targets shard.
\c - - - :worker_2_port
SELECT COUNT(*) FROM worker_shard_text_copy_test.shard_to_split_copy_81070015;
count
---------------------------------------------------------------------
21
(1 row)
SELECT COUNT(*) FROM worker_shard_text_copy_test.shard_to_split_copy_81070016;
count
---------------------------------------------------------------------
1
(1 row)
-- END: List updated row count for remote targets shard.
-- BEGIN: CLEANUP.
\c - - - :master_port
SET client_min_messages TO WARNING;
DROP SCHEMA citus_split_shard_by_split_points_local CASCADE;
ERROR: schema "citus_split_shard_by_split_points_local" does not exist
-- END: CLEANUP.

View File

@ -0,0 +1,263 @@
CREATE SCHEMA worker_split_binary_copy_test;
SET search_path TO worker_split_binary_copy_test;
SET citus.shard_count TO 1;
SET citus.shard_replication_factor TO 1;
SET citus.next_shard_id TO 81060000;
-- Remove extra nodes added, otherwise GetLocalNodeId() does not bahave correctly.
SELECT citus_remove_node('localhost', 8887);
citus_remove_node
---------------------------------------------------------------------
(1 row)
SELECT citus_remove_node('localhost', 9995);
citus_remove_node
---------------------------------------------------------------------
(1 row)
SELECT citus_remove_node('localhost', 9992);
citus_remove_node
---------------------------------------------------------------------
(1 row)
SELECT citus_remove_node('localhost', 9998);
citus_remove_node
---------------------------------------------------------------------
(1 row)
SELECT citus_remove_node('localhost', 9997);
citus_remove_node
---------------------------------------------------------------------
(1 row)
SELECT citus_remove_node('localhost', 8888);
citus_remove_node
---------------------------------------------------------------------
(1 row)
-- BEGIN: Create distributed table and insert data.
CREATE TABLE worker_split_binary_copy_test.shard_to_split_copy (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
SELECT create_distributed_table('shard_to_split_copy', 'l_orderkey');
create_distributed_table
---------------------------------------------------------------------
(1 row)
\COPY shard_to_split_copy FROM STDIN WITH DELIMITER '|'
-- END: Create distributed table and insert data.
-- BEGIN: Switch to Worker1, Create target shards in worker for local 2-way split copy.
\c - - - :worker_1_port
CREATE TABLE worker_split_binary_copy_test.shard_to_split_copy_81060015 (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
CREATE TABLE worker_split_binary_copy_test.shard_to_split_copy_81060016 (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
-- End: Switch to Worker1, Create target shards in worker for local 2-way split copy.
-- BEGIN: Switch to Worker2, Create target shards in worker for remote 2-way split copy.
\c - - - :worker_2_port
CREATE TABLE worker_split_binary_copy_test.shard_to_split_copy_81060015 (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
CREATE TABLE worker_split_binary_copy_test.shard_to_split_copy_81060016 (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
-- End: Switch to Worker2, Create target shards in worker for remote 2-way split copy.
-- BEGIN: List row count for source shard and targets shard in Worker1.
\c - - - :worker_1_port
SELECT COUNT(*) FROM worker_split_binary_copy_test.shard_to_split_copy_81060000;
count
---------------------------------------------------------------------
22
(1 row)
SELECT COUNT(*) FROM worker_split_binary_copy_test.shard_to_split_copy_81060015;
count
---------------------------------------------------------------------
0
(1 row)
SELECT COUNT(*) FROM worker_split_binary_copy_test.shard_to_split_copy_81060016;
count
---------------------------------------------------------------------
0
(1 row)
-- END: List row count for source shard and targets shard in Worker1.
-- BEGIN: List row count for target shard in Worker2.
\c - - - :worker_2_port
SELECT COUNT(*) FROM worker_split_binary_copy_test.shard_to_split_copy_81060015;
count
---------------------------------------------------------------------
0
(1 row)
SELECT COUNT(*) FROM worker_split_binary_copy_test.shard_to_split_copy_81060016;
count
---------------------------------------------------------------------
0
(1 row)
-- END: List row count for targets shard in Worker2.
-- BEGIN: Set worker_1_node and worker_2_node
\c - - - :worker_1_port
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
-- END: Set worker_1_node and worker_2_node
-- BEGIN: Trigger 2-way local shard split copy.
-- Ensure we will perform binary copy.
SET citus.enable_binary_protocol = true;
SELECT * from worker_split_copy(
81060000, -- source shard id to copy
ARRAY[
-- split copy info for split children 1
ROW(81060015, -- destination shard id
-2147483648, -- split range begin
1073741823, --split range end
:worker_1_node)::pg_catalog.split_copy_info,
-- split copy info for split children 2
ROW(81060016, --destination shard id
1073741824, --split range begin
2147483647, --split range end
:worker_1_node)::pg_catalog.split_copy_info
]
);
worker_split_copy
---------------------------------------------------------------------
(1 row)
-- END: Trigger 2-way local shard split copy.
-- BEGIN: Trigger 2-way remote shard split copy.
SELECT * from worker_split_copy(
81060000, -- source shard id to copy
ARRAY[
-- split copy info for split children 1
ROW(81060015, -- destination shard id
-2147483648, -- split range begin
1073741823, --split range end
:worker_2_node)::pg_catalog.split_copy_info,
-- split copy info for split children 2
ROW(81060016, --destination shard id
1073741824, --split range begin
2147483647, --split range end
:worker_2_node)::pg_catalog.split_copy_info
]
);
worker_split_copy
---------------------------------------------------------------------
(1 row)
-- END: Trigger 2-way remote shard split copy.
-- BEGIN: List updated row count for local targets shard.
SELECT COUNT(*) FROM worker_split_binary_copy_test.shard_to_split_copy_81060015;
count
---------------------------------------------------------------------
21
(1 row)
SELECT COUNT(*) FROM worker_split_binary_copy_test.shard_to_split_copy_81060016;
count
---------------------------------------------------------------------
1
(1 row)
-- END: List updated row count for local targets shard.
-- BEGIN: List updated row count for remote targets shard.
\c - - - :worker_2_port
SELECT COUNT(*) FROM worker_split_binary_copy_test.shard_to_split_copy_81060015;
count
---------------------------------------------------------------------
21
(1 row)
SELECT COUNT(*) FROM worker_split_binary_copy_test.shard_to_split_copy_81060016;
count
---------------------------------------------------------------------
1
(1 row)
-- END: List updated row count for remote targets shard.
-- BEGIN: CLEANUP.
\c - - - :master_port
SET client_min_messages TO WARNING;
DROP SCHEMA worker_split_binary_copy_test CASCADE;
-- END: CLEANUP.

View File

@ -0,0 +1,142 @@
CREATE SCHEMA worker_split_copy_test;
SET search_path TO worker_split_copy_test;
SET citus.shard_count TO 2;
SET citus.shard_replication_factor TO 1;
SET citus.next_shard_id TO 81070000;
-- BEGIN: Create distributed table and insert data.
CREATE TABLE worker_split_copy_test."test !/ \n _""dist_123_table"(id int primary key, value char);
SELECT create_distributed_table('"test !/ \n _""dist_123_table"', 'id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
INSERT INTO "test !/ \n _""dist_123_table" (id, value) (SELECT g.id, 'N' FROM generate_series(1, 1000) AS g(id));
-- END: Create distributed table and insert data.
-- BEGIN: Switch to Worker1, Create target shards in worker for local 2-way split copy.
\c - - - :worker_1_port
CREATE TABLE worker_split_copy_test."test !/ \n _""dist_123_table_81070015"(id int primary key, value char);
CREATE TABLE worker_split_copy_test."test !/ \n _""dist_123_table_81070016"(id int primary key, value char);
-- End: Switch to Worker1, Create target shards in worker for local 2-way split copy.
-- BEGIN: List row count for source shard and targets shard in Worker1.
\c - - - :worker_1_port
SELECT COUNT(*) FROM worker_split_copy_test."test !/ \n _""dist_123_table_81070000";
count
---------------------------------------------------------------------
510
(1 row)
SELECT COUNT(*) FROM worker_split_copy_test."test !/ \n _""dist_123_table_81070015";
count
---------------------------------------------------------------------
0
(1 row)
SELECT COUNT(*) FROM worker_split_copy_test."test !/ \n _""dist_123_table_81070016";
count
---------------------------------------------------------------------
0
(1 row)
\c - - - :worker_2_port
SELECT COUNT(*) FROM worker_split_copy_test."test !/ \n _""dist_123_table_81070001";
count
---------------------------------------------------------------------
490
(1 row)
-- END: List row count for source shard and targets shard in Worker1.
-- BEGIN: Set worker_1_node and worker_2_node
\c - - - :worker_1_port
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
-- END: Set worker_1_node and worker_2_node
-- BEGIN: Test Negative scenario
SELECT * from worker_split_copy(
101, -- Invalid source shard id.
ARRAY[
-- split copy info for split children 1
ROW(81070015, -- destination shard id
-2147483648, -- split range begin
-1073741824, --split range end
:worker_1_node)::pg_catalog.split_copy_info,
-- split copy info for split children 2
ROW(81070016, --destination shard id
-1073741823, --split range begin
-1, --split range end
:worker_1_node)::pg_catalog.split_copy_info
]
);
ERROR: could not find valid entry for shard xxxxx
SELECT * from worker_split_copy(
81070000, -- source shard id to copy
ARRAY[] -- empty array
);
ERROR: cannot determine type of empty array
HINT: Explicitly cast to the desired type, for example ARRAY[]::integer[].
SELECT * from worker_split_copy(
81070000, -- source shard id to copy
ARRAY[NULL] -- empty array
);
ERROR: function worker_split_copy(integer, text[]) does not exist
HINT: No function matches the given name and argument types. You might need to add explicit type casts.
SELECT * from worker_split_copy(
81070000, -- source shard id to copy
ARRAY[NULL::pg_catalog.split_copy_info]-- empty array
);
ERROR: pg_catalog.split_copy_info array cannot contain null values
SELECT * from worker_split_copy(
81070000, -- source shard id to copy
ARRAY[ROW(NULL)]-- empty array
);
ERROR: function worker_split_copy(integer, record[]) does not exist
HINT: No function matches the given name and argument types. You might need to add explicit type casts.
SELECT * from worker_split_copy(
81070000, -- source shard id to copy
ARRAY[ROW(NULL, NULL, NULL, NULL)::pg_catalog.split_copy_info] -- empty array
);
ERROR: destination_shard_id for pg_catalog.split_copy_info cannot be null.
-- END: Test Negative scenario
-- BEGIN: Trigger 2-way local shard split copy.
-- Ensure we will perform text copy.
SET citus.enable_binary_protocol = false;
SELECT * from worker_split_copy(
81070000, -- source shard id to copy
ARRAY[
-- split copy info for split children 1
ROW(81070015, -- destination shard id
-2147483648, -- split range begin
-1073741824, --split range end
:worker_1_node)::pg_catalog.split_copy_info,
-- split copy info for split children 2
ROW(81070016, --destination shard id
-1073741823, --split range begin
-1, --split range end
:worker_1_node)::pg_catalog.split_copy_info
]
);
worker_split_copy
---------------------------------------------------------------------
(1 row)
-- END: Trigger 2-way local shard split copy.
-- BEGIN: List updated row count for local targets shard.
SELECT COUNT(*) FROM worker_split_copy_test."test !/ \n _""dist_123_table_81070015";
count
---------------------------------------------------------------------
247
(1 row)
SELECT COUNT(*) FROM worker_split_copy_test."test !/ \n _""dist_123_table_81070016";
count
---------------------------------------------------------------------
263
(1 row)
-- END: List updated row count for local targets shard.
-- BEGIN: CLEANUP.
\c - - - :master_port
SET client_min_messages TO WARNING;
DROP SCHEMA worker_split_copy_test CASCADE;
-- END: CLEANUP.

View File

@ -0,0 +1,226 @@
CREATE SCHEMA worker_split_text_copy_test;
SET search_path TO worker_split_text_copy_test;
SET citus.shard_count TO 1;
SET citus.shard_replication_factor TO 1;
SET citus.next_shard_id TO 81070000;
-- BEGIN: Create distributed table and insert data.
CREATE TABLE worker_split_text_copy_test.shard_to_split_copy (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
SELECT create_distributed_table('shard_to_split_copy', 'l_orderkey');
create_distributed_table
---------------------------------------------------------------------
(1 row)
\COPY shard_to_split_copy FROM STDIN WITH DELIMITER '|'
-- END: Create distributed table and insert data.
-- BEGIN: Switch to Worker1, Create target shards in worker for local 2-way split copy.
\c - - - :worker_1_port
CREATE TABLE worker_split_text_copy_test.shard_to_split_copy_81070015 (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
CREATE TABLE worker_split_text_copy_test.shard_to_split_copy_81070016 (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
-- End: Switch to Worker1, Create target shards in worker for local 2-way split copy.
-- BEGIN: Switch to Worker2, Create target shards in worker for remote 2-way split copy.
\c - - - :worker_2_port
CREATE TABLE worker_split_text_copy_test.shard_to_split_copy_81070015 (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
CREATE TABLE worker_split_text_copy_test.shard_to_split_copy_81070016 (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
-- End: Switch to Worker2, Create target shards in worker for remote 2-way split copy.
-- BEGIN: List row count for source shard and targets shard in Worker1.
\c - - - :worker_1_port
SELECT COUNT(*) FROM worker_split_text_copy_test.shard_to_split_copy_81070000;
count
---------------------------------------------------------------------
22
(1 row)
SELECT COUNT(*) FROM worker_split_text_copy_test.shard_to_split_copy_81070015;
count
---------------------------------------------------------------------
0
(1 row)
SELECT COUNT(*) FROM worker_split_text_copy_test.shard_to_split_copy_81070016;
count
---------------------------------------------------------------------
0
(1 row)
-- END: List row count for source shard and targets shard in Worker1.
-- BEGIN: List row count for target shard in Worker2.
\c - - - :worker_2_port
SELECT COUNT(*) FROM worker_split_text_copy_test.shard_to_split_copy_81070015;
count
---------------------------------------------------------------------
0
(1 row)
SELECT COUNT(*) FROM worker_split_text_copy_test.shard_to_split_copy_81070016;
count
---------------------------------------------------------------------
0
(1 row)
-- END: List row count for targets shard in Worker2.
-- BEGIN: Set worker_1_node and worker_2_node
\c - - - :worker_1_port
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
-- END: Set worker_1_node and worker_2_node
-- BEGIN: Trigger 2-way local shard split copy.
-- Ensure we will perform text copy.
SET citus.enable_binary_protocol = false;
SELECT * from worker_split_copy(
81070000, -- source shard id to copy
ARRAY[
-- split copy info for split children 1
ROW(81070015, -- destination shard id
-2147483648, -- split range begin
1073741823, --split range end
:worker_1_node)::pg_catalog.split_copy_info,
-- split copy info for split children 2
ROW(81070016, --destination shard id
1073741824, --split range begin
2147483647, --split range end
:worker_1_node)::pg_catalog.split_copy_info
]
);
worker_split_copy
---------------------------------------------------------------------
(1 row)
-- END: Trigger 2-way local shard split copy.
-- BEGIN: Trigger 2-way remote shard split copy.
SELECT * from worker_split_copy(
81070000, -- source shard id to copy
ARRAY[
-- split copy info for split children 1
ROW(81070015, -- destination shard id
-2147483648, -- split range begin
1073741823, --split range end
:worker_2_node)::pg_catalog.split_copy_info,
-- split copy info for split children 2
ROW(81070016, --destination shard id
1073741824, --split range begin
2147483647, --split range end
:worker_2_node)::pg_catalog.split_copy_info
]
);
worker_split_copy
---------------------------------------------------------------------
(1 row)
-- END: Trigger 2-way remote shard split copy.
-- BEGIN: List updated row count for local targets shard.
SELECT COUNT(*) FROM worker_split_text_copy_test.shard_to_split_copy_81070015;
count
---------------------------------------------------------------------
21
(1 row)
SELECT COUNT(*) FROM worker_split_text_copy_test.shard_to_split_copy_81070016;
count
---------------------------------------------------------------------
1
(1 row)
-- END: List updated row count for local targets shard.
-- BEGIN: List updated row count for remote targets shard.
\c - - - :worker_2_port
SELECT COUNT(*) FROM worker_split_text_copy_test.shard_to_split_copy_81070015;
count
---------------------------------------------------------------------
21
(1 row)
SELECT COUNT(*) FROM worker_split_text_copy_test.shard_to_split_copy_81070016;
count
---------------------------------------------------------------------
1
(1 row)
-- END: List updated row count for remote targets shard.
-- BEGIN: CLEANUP.
\c - - - :master_port
SET client_min_messages TO WARNING;
DROP SCHEMA worker_split_text_copy_test CASCADE;
-- END: CLEANUP.

View File

@ -0,0 +1,146 @@
setup
{
SET citus.shard_count to 2;
SET citus.shard_replication_factor to 1;
SELECT setval('pg_dist_shardid_seq', 1500000);
CREATE TABLE to_split_table (id int, value int);
SELECT create_distributed_table('to_split_table', 'id');
}
teardown
{
DROP TABLE to_split_table;
}
session "s1"
step "s1-begin"
{
BEGIN;
-- the tests are written with the logic where single shard SELECTs
-- do not to open transaction blocks
SET citus.select_opens_transaction_block TO false;
}
// cache all placements
step "s1-load-cache"
{
-- Indirect way to load cache.
TRUNCATE to_split_table;
}
step "s1-insert"
{
-- Id '123456789' maps to shard 1500002.
SELECT get_shard_id_for_distribution_column('to_split_table', 123456789);
INSERT INTO to_split_table VALUES (123456789, 1);
}
step "s1-update"
{
UPDATE to_split_table SET value = 111 WHERE id = 123456789;
}
step "s1-delete"
{
DELETE FROM to_split_table WHERE id = 123456789;
}
step "s1-select"
{
SELECT count(*) FROM to_split_table WHERE id = 123456789;
}
step "s1-ddl"
{
CREATE INDEX test_table_index ON to_split_table(id);
}
step "s1-copy"
{
COPY to_split_table FROM PROGRAM 'echo "1,1\n2,2\n3,3\n4,4\n5,5"' WITH CSV;
}
step "s1-blocking-shard-split"
{
SELECT pg_catalog.citus_split_shard_by_split_points(
1500001,
ARRAY['-1073741824'],
ARRAY[1, 2],
'block_writes');
}
step "s1-commit"
{
COMMIT;
}
session "s2"
step "s2-begin"
{
BEGIN;
}
step "s2-blocking-shard-split"
{
SELECT pg_catalog.citus_split_shard_by_split_points(
1500002,
ARRAY['1073741824'],
ARRAY[1, 2],
'block_writes');
}
step "s2-commit"
{
COMMIT;
}
step "s2-print-cluster"
{
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM to_split_table ORDER BY id, value;
}
step "s2-print-index-count"
{
SELECT
nodeport, success, result
FROM
run_command_on_placements('to_split_table', 'select count(*) from pg_indexes WHERE tablename = ''%s''')
ORDER BY
nodeport;
}
// Run shard split while concurrently performing DML and index creation
// We expect DML,Copy to fail because the shard they are waiting for is destroyed.
permutation "s1-load-cache" "s1-insert" "s1-begin" "s1-select" "s2-begin" "s2-blocking-shard-split" "s1-update" "s2-commit" "s1-commit" "s2-print-cluster"
permutation "s1-load-cache" "s1-insert" "s1-begin" "s1-select" "s2-begin" "s2-blocking-shard-split" "s1-delete" "s2-commit" "s1-commit" "s2-print-cluster"
permutation "s1-load-cache" "s1-begin" "s1-select" "s2-begin" "s2-blocking-shard-split" "s1-insert" "s2-commit" "s1-commit" "s2-print-cluster"
permutation "s1-load-cache" "s1-begin" "s1-select" "s2-begin" "s2-blocking-shard-split" "s1-copy" "s2-commit" "s1-commit" "s2-print-cluster"
// The same tests without loading the cache at first
permutation "s1-insert" "s1-begin" "s1-select" "s2-begin" "s2-blocking-shard-split" "s1-update" "s2-commit" "s1-commit" "s2-print-cluster"
permutation "s1-insert" "s1-begin" "s1-select" "s2-begin" "s2-blocking-shard-split" "s1-delete" "s2-commit" "s1-commit" "s2-print-cluster"
permutation "s1-begin" "s1-select" "s2-begin" "s2-blocking-shard-split" "s1-insert" "s2-commit" "s1-commit" "s2-print-cluster"
permutation "s1-begin" "s1-select" "s2-begin" "s2-blocking-shard-split" "s1-copy" "s2-commit" "s1-commit" "s2-print-cluster"
// Concurrent shard split blocks on different shards of the same table (or any colocated table)
permutation "s1-load-cache" "s1-insert" "s1-begin" "s1-blocking-shard-split" "s2-blocking-shard-split" "s1-commit" "s2-print-cluster"
// The same test above without loading the cache at first
permutation "s1-insert" "s1-begin" "s1-blocking-shard-split" "s2-blocking-shard-split" "s1-commit" "s2-print-cluster"
// Concurrent DDL blocks on different shards of the same table (or any colocated table)
permutation "s1-load-cache" "s1-begin" "s1-select" "s2-begin" "s2-blocking-shard-split" "s1-ddl" "s2-commit" "s1-commit" "s2-print-cluster" "s2-print-index-count"
// The same tests without loading the cache at first
permutation "s1-begin" "s1-select" "s2-begin" "s2-blocking-shard-split" "s1-ddl" "s2-commit" "s1-commit" "s2-print-cluster" "s2-print-index-count"

View File

@ -0,0 +1,104 @@
setup
{
SELECT setval('pg_dist_shardid_seq', 1500000);
SET citus.shard_count to 2;
SET citus.shard_replication_factor to 1;
CREATE TABLE reference_table (id int PRIMARY KEY, value int);
SELECT create_reference_table('reference_table');
CREATE TABLE table_to_split (id int, value int);
SELECT create_distributed_table('table_to_split', 'id');
}
teardown
{
DROP TABLE table_to_split CASCADE;
DROP TABLE reference_table CASCADE;
}
session "s1"
step "s1-begin"
{
BEGIN;
}
step "s1-insert"
{
INSERT INTO reference_table VALUES (5, 10);
}
step "s1-update"
{
UPDATE reference_table SET value = 5 WHERE id = 5;
}
step "s1-delete"
{
DELETE FROM reference_table WHERE id = 5;
}
step "s1-ddl"
{
CREATE INDEX reference_table_index ON reference_table(id);
}
step "s1-copy"
{
COPY reference_table FROM PROGRAM 'echo "1,1\n2,2\n3,3\n4,4\n5,5"' WITH CSV;
}
step "s1-commit"
{
COMMIT;
}
session "s2"
step "s2-begin"
{
BEGIN;
}
step "s2-blocking-shard-split"
{
SELECT pg_catalog.citus_split_shard_by_split_points(
1500002,
ARRAY['-1073741824'],
ARRAY[1, 2],
'block_writes');
}
step "s2-add-fkey"
{
ALTER TABLE table_to_split ADD CONSTRAINT fkey_const FOREIGN KEY(value) REFERENCES reference_table(id);
}
step "s2-commit"
{
COMMIT;
}
step "s2-print-cluster"
{
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('table_to_split', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM table_to_split ORDER BY id, value;
}
// Run shard split while concurrently performing an DML and index creation on the
// reference table which the distributed table have a foreign key to.
// All modifications should block on shard split.
permutation "s2-add-fkey" "s1-begin" "s2-begin" "s2-blocking-shard-split" "s1-delete" "s2-commit" "s1-commit" "s2-print-cluster"
permutation "s2-add-fkey" "s1-begin" "s2-begin" "s2-blocking-shard-split" "s1-update" "s2-commit" "s1-commit" "s2-print-cluster"
permutation "s2-add-fkey" "s1-begin" "s2-begin" "s2-blocking-shard-split" "s1-insert" "s2-commit" "s1-commit" "s2-print-cluster"
permutation "s2-add-fkey" "s1-begin" "s2-begin" "s2-blocking-shard-split" "s1-copy" "s2-commit" "s1-commit" "s2-print-cluster"
permutation "s2-add-fkey" "s1-begin" "s2-begin" "s2-blocking-shard-split" "s1-ddl" "s2-commit" "s1-commit" "s2-print-cluster"

View File

@ -0,0 +1,15 @@
# Split Shard tests.
# Include tests from 'minimal_schedule' for setup.
test: multi_test_helpers multi_test_helpers_superuser columnar_test_helpers
test: multi_cluster_management
test: multi_test_catalog_views
test: tablespace
# Helpers for foreign key catalogs.
test: foreign_key_to_reference_table
# Split tests go here.
test: worker_split_copy_test
test: worker_split_binary_copy_test
test: worker_split_text_copy_test
test: citus_split_shard_by_split_points_negative
test: citus_split_shard_by_split_points
test: citus_split_shard_by_split_points_failure

View File

@ -0,0 +1,240 @@
/*
Citus Shard Split Test.The test is model similar to 'shard_move_constraints'.
Here is a high level overview of test plan:
1. Create a table 'sensors' (ShardCount = 2) to be split. Add indexes and statistics on this table.
2. Create two other tables: 'reference_table' and 'colocated_dist_table', co-located with sensors.
3. Create Foreign key constraints between the two co-located distributed tables.
4. Load data into the three tables.
5. Move one of the shards for 'sensors' to test ShardMove -> Split.
6. Trigger Split on both shards of 'sensors'. This will also split co-located tables.
7. Move one of the split shard to test Split -> ShardMove.
8. Split an already split shard second time on a different schema.
*/
CREATE SCHEMA "citus_split_test_schema";
CREATE ROLE test_split_role WITH LOGIN;
GRANT USAGE, CREATE ON SCHEMA "citus_split_test_schema" TO test_split_role;
SET ROLE test_split_role;
SET search_path TO "citus_split_test_schema";
SET citus.next_shard_id TO 8981000;
SET citus.next_placement_id TO 8610000;
SET citus.shard_count TO 2;
SET citus.shard_replication_factor TO 1;
-- BEGIN: Create table to split, along with other co-located tables. Add indexes, statistics etc.
CREATE TABLE sensors(
measureid integer,
eventdatetime date,
measure_data jsonb,
meaure_quantity decimal(15, 2),
measure_status char(1),
measure_comment varchar(44),
PRIMARY KEY (measureid, eventdatetime, measure_data));
CREATE INDEX index_on_sensors ON sensors(lower(measureid::text));
ALTER INDEX index_on_sensors ALTER COLUMN 1 SET STATISTICS 1000;
CREATE INDEX hash_index_on_sensors ON sensors USING HASH((measure_data->'IsFailed'));
CREATE INDEX index_with_include_on_sensors ON sensors ((measure_data->'IsFailed')) INCLUDE (measure_data, eventdatetime, measure_status);
CREATE STATISTICS stats_on_sensors (dependencies) ON measureid, eventdatetime FROM sensors;
SELECT create_distributed_table('sensors', 'measureid', colocate_with:='none');
-- END: Create table to split, along with other co-located tables. Add indexes, statistics etc.
-- BEGIN: Create co-located distributed and reference tables.
CREATE TABLE reference_table (measureid integer PRIMARY KEY);
SELECT create_reference_table('reference_table');
CREATE TABLE colocated_dist_table (measureid integer PRIMARY KEY);
CLUSTER colocated_dist_table USING colocated_dist_table_pkey;
SELECT create_distributed_table('colocated_dist_table', 'measureid', colocate_with:='sensors');
CREATE TABLE table_with_index_rep_identity(key int NOT NULL);
CREATE UNIQUE INDEX uqx ON table_with_index_rep_identity(key);
ALTER TABLE table_with_index_rep_identity REPLICA IDENTITY USING INDEX uqx;
CLUSTER table_with_index_rep_identity USING uqx;
SELECT create_distributed_table('table_with_index_rep_identity', 'key', colocate_with:='sensors');
-- END: Create co-located distributed and reference tables.
-- BEGIN : Create Foreign key constraints.
ALTER TABLE sensors ADD CONSTRAINT fkey_table_to_dist FOREIGN KEY (measureid) REFERENCES colocated_dist_table(measureid);
-- END : Create Foreign key constraints.
-- BEGIN : Load data into tables.
INSERT INTO reference_table SELECT i FROM generate_series(0,1000)i;
INSERT INTO colocated_dist_table SELECT i FROM generate_series(0,1000)i;
INSERT INTO sensors SELECT i, '2020-01-05', '{}', 11011.10, 'A', 'I <3 Citus' FROM generate_series(0,1000)i;
SELECT COUNT(*) FROM sensors;
SELECT COUNT(*) FROM reference_table;
SELECT COUNT(*) FROM colocated_dist_table;
-- END: Load data into tables.
-- BEGIN : Display current state.
SELECT shard.shardid, logicalrelid, shardminvalue, shardmaxvalue, nodename, nodeport
FROM pg_dist_shard AS shard
INNER JOIN pg_dist_placement placement ON shard.shardid = placement.shardid
INNER JOIN pg_dist_node node ON placement.groupid = node.groupid
INNER JOIN pg_catalog.pg_class cls ON shard.logicalrelid = cls.oid
WHERE node.noderole = 'primary' AND (logicalrelid = 'sensors'::regclass OR logicalrelid = 'colocated_dist_table'::regclass OR logicalrelid = 'table_with_index_rep_identity'::regclass)
ORDER BY logicalrelid, shardminvalue::BIGINT;
\c - - - :worker_1_port
SET search_path TO "citus_split_test_schema", public, pg_catalog;
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like 'sensors_%'
ORDER BY 1, 2;
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'sensors_%' ORDER BY 1,2;
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'table_with_index_rep_identity_%' ORDER BY 1,2;
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema')
)
ORDER BY stxname ASC;
\c - - - :worker_2_port
SET search_path TO "citus_split_test_schema", public, pg_catalog;
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like 'sensors_%'
ORDER BY 1, 2;
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'sensors_%' ORDER BY 1,2;
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'table_with_index_rep_identity_%' ORDER BY 1,2;
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema')
)
ORDER BY stxname ASC;
-- END : Display current state
-- BEGIN : Move one shard before we split it.
\c - postgres - :master_port
SET ROLE test_split_role;
SET search_path TO "citus_split_test_schema";
SET citus.next_shard_id TO 8981007;
SET citus.defer_drop_after_shard_move TO OFF;
SELECT citus_move_shard_placement(8981000, 'localhost', :worker_1_port, 'localhost', :worker_2_port, shard_transfer_mode:='force_logical');
-- END : Move one shard before we split it.
-- BEGIN : Set node id variables
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
-- END : Set node id variables
-- BEGIN : Split two shards : One with move and One without move.
-- Perform 2 way split
SELECT pg_catalog.citus_split_shard_by_split_points(
8981000,
ARRAY['-1073741824'],
ARRAY[:worker_1_node, :worker_2_node],
'block_writes');
-- Perform 3 way split
SELECT pg_catalog.citus_split_shard_by_split_points(
8981001,
ARRAY['536870911', '1610612735'],
ARRAY[:worker_1_node, :worker_1_node, :worker_2_node],
'block_writes');
-- END : Split two shards : One with move and One without move.
-- BEGIN : Move a shard post split.
SELECT citus_move_shard_placement(8981007, 'localhost', :worker_1_port, 'localhost', :worker_2_port, shard_transfer_mode:='block_writes');
-- END : Move a shard post split.
-- BEGIN : Display current state.
SELECT shard.shardid, logicalrelid, shardminvalue, shardmaxvalue, nodename, nodeport
FROM pg_dist_shard AS shard
INNER JOIN pg_dist_placement placement ON shard.shardid = placement.shardid
INNER JOIN pg_dist_node node ON placement.groupid = node.groupid
INNER JOIN pg_catalog.pg_class cls ON shard.logicalrelid = cls.oid
WHERE node.noderole = 'primary' AND (logicalrelid = 'sensors'::regclass OR logicalrelid = 'colocated_dist_table'::regclass OR logicalrelid = 'table_with_index_rep_identity'::regclass)
ORDER BY logicalrelid, shardminvalue::BIGINT;
\c - - - :worker_1_port
SET search_path TO "citus_split_test_schema", public, pg_catalog;
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like 'sensors_%'
ORDER BY 1, 2;
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'sensors_%' ORDER BY 1,2;
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'table_with_index_rep_identity_%' ORDER BY 1,2;
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema')
)
ORDER BY stxname ASC;
\c - - - :worker_2_port
SET search_path TO "citus_split_test_schema", public, pg_catalog;
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname, fk."Constraint", fk."Definition"
FROM pg_catalog.pg_class tbl
JOIN public.table_fkeys fk on tbl.oid = fk.relid
WHERE tbl.relname like 'sensors_%'
ORDER BY 1, 2;
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'sensors_%' ORDER BY 1,2;
SELECT tablename, indexdef FROM pg_indexes WHERE tablename like 'table_with_index_rep_identity_%' ORDER BY 1,2;
SELECT stxname FROM pg_statistic_ext
WHERE stxnamespace IN (
SELECT oid
FROM pg_namespace
WHERE nspname IN ('citus_split_test_schema')
)
ORDER BY stxname ASC;
-- END : Display current state
-- BEGIN: Should be able to change/drop constraints
\c - postgres - :master_port
SET ROLE test_split_role;
SET search_path TO "citus_split_test_schema";
ALTER INDEX index_on_sensors RENAME TO index_on_sensors_renamed;
ALTER INDEX index_on_sensors_renamed ALTER COLUMN 1 SET STATISTICS 200;
DROP STATISTICS stats_on_sensors;
DROP INDEX index_on_sensors_renamed;
ALTER TABLE sensors DROP CONSTRAINT fkey_table_to_dist;
-- END: Should be able to change/drop constraints
-- BEGIN: Split second time on another schema
SET search_path TO public;
SET citus.next_shard_id TO 8981031;
SELECT pg_catalog.citus_split_shard_by_split_points(
8981007,
ARRAY['-2100000000'],
ARRAY[:worker_1_node, :worker_2_node],
'block_writes');
SET search_path TO "citus_split_test_schema";
SELECT shard.shardid, logicalrelid, shardminvalue, shardmaxvalue, nodename, nodeport
FROM pg_dist_shard AS shard
INNER JOIN pg_dist_placement placement ON shard.shardid = placement.shardid
INNER JOIN pg_dist_node node ON placement.groupid = node.groupid
INNER JOIN pg_catalog.pg_class cls ON shard.logicalrelid = cls.oid
WHERE node.noderole = 'primary' AND (logicalrelid = 'sensors'::regclass OR logicalrelid = 'colocated_dist_table'::regclass OR logicalrelid = 'table_with_index_rep_identity'::regclass)
ORDER BY logicalrelid, shardminvalue::BIGINT;
-- END: Split second time on another schema
-- BEGIN: Validate Data Count
SELECT COUNT(*) FROM sensors;
SELECT COUNT(*) FROM reference_table;
SELECT COUNT(*) FROM colocated_dist_table;
-- END: Validate Data Count
--BEGIN : Cleanup
\c - postgres - :master_port
DROP SCHEMA "citus_split_test_schema" CASCADE;
--END : Cleanup

View File

@ -0,0 +1,80 @@
CREATE SCHEMA "citus_split_failure_test_schema";
SET search_path TO "citus_split_failure_test_schema";
SET citus.shard_count TO 1;
SET citus.next_shard_id TO 890000;
SET citus.shard_replication_factor TO 1;
-- BEGIN: Create table to split
CREATE TABLE sensors(
measureid integer,
eventdatetime date);
CREATE TABLE sensors_colocated(
measureid integer,
eventdatetime2 date);
SELECT create_distributed_table('sensors', 'measureid');
SELECT create_distributed_table('sensors_colocated', 'measureid', colocate_with:='sensors');
-- END: Create table to split
-- BEGIN : Switch to worker and create split shards already so workflow fails.
\c - - - :worker_1_port
SET search_path TO "citus_split_failure_test_schema";
-- Don't create sensors_8981001, workflow will create and clean it.
-- Create rest of the shards so that the workflow fails, but will not clean them.
CREATE TABLE sensors_8981002(
measureid integer,
eventdatetime date);
CREATE TABLE sensors_colocated_8981003(
measureid integer,
eventdatetime date);
CREATE TABLE sensors_colocated_8981004(
measureid integer,
eventdatetime date);
-- A random table which should not be deleted.
CREATE TABLE sensors_nodelete(
measureid integer,
eventdatetime date);
-- List tables in worker.
SET search_path TO "citus_split_failure_test_schema";
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname
FROM pg_catalog.pg_class tbl
WHERE tbl.relname like 'sensors%'
ORDER BY 1;
-- END : Switch to worker and create split shards already so workflow fails.
-- BEGIN : Set node id variables
\c - postgres - :master_port
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
-- END : Set node id variables
-- BEGIN : Split Shard, which is expected to fail.
SET citus.next_shard_id TO 8981001;
SELECT pg_catalog.citus_split_shard_by_split_points(
890000,
ARRAY['-1073741824'],
ARRAY[:worker_1_node, :worker_1_node],
'block_writes');
-- BEGIN : Split Shard, which is expected to fail.
-- BEGIN : Ensure tables were cleaned from worker
\c - - - :worker_1_port
SET search_path TO "citus_split_failure_test_schema";
SET citus.show_shards_for_app_name_prefixes = '*';
SELECT tbl.relname
FROM pg_catalog.pg_class tbl
WHERE tbl.relname like 'sensors%'
ORDER BY 1;
-- END : Ensure tables were cleaned from worker
--BEGIN : Cleanup
\c - postgres - :master_port
DROP SCHEMA "citus_split_failure_test_schema" CASCADE;
--END : Cleanup

View File

@ -0,0 +1,145 @@
-- Negative test cases for citus_split_shard_by_split_points UDF.
CREATE SCHEMA citus_split_shard_by_split_points_negative;
SET search_path TO citus_split_shard_by_split_points_negative;
SET citus.shard_count TO 4;
SET citus.shard_replication_factor TO 1;
SET citus.next_shard_id TO 60761300;
CREATE TABLE range_paritioned_table_to_split(rid bigserial PRIMARY KEY, value char);
SELECT create_distributed_table('range_paritioned_table_to_split', 'rid', 'range');
-- Shards are not created automatically for range distributed table.
SELECT master_create_empty_shard('range_paritioned_table_to_split');
SET citus.next_shard_id TO 49761300;
CREATE TABLE table_to_split (id bigserial PRIMARY KEY, value char);
-- Shard1 | -2147483648 | -1073741825
-- Shard2 | -1073741824 | -1
-- Shard3 | 0 | 1073741823
-- Shard4 | 1073741824 | 2147483647
SELECT create_distributed_table('table_to_split','id');
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
-- UDF fails for any other shard_transfer_mode other than block_writes.
SELECT citus_split_shard_by_split_points(
49761302,
ARRAY['50'],
ARRAY[101, 201],
'auto');
SELECT citus_split_shard_by_split_points(
49761302,
ARRAY['50'],
ARRAY[101, 201],
'force_logical');
SELECT citus_split_shard_by_split_points(
49761302,
ARRAY['50'],
ARRAY[101, 201],
'gibberish');
-- UDF fails for range partitioned tables.
SELECT citus_split_shard_by_split_points(
60761300,
ARRAY['-1073741826'],
ARRAY[:worker_1_node, :worker_2_node]);
-- UDF fails if number of placement node list does not exceed split points by one.
-- Example: One split point defines two way split (2 worker nodes needed).
SELECT citus_split_shard_by_split_points(
49761300,
-- 2 split points defined making it a 3 way split but we only specify 2 placement lists.
ARRAY['-1073741826', '-107374182'],
ARRAY[:worker_1_node, :worker_2_node]); -- 2 worker nodes.
-- UDF fails if split ranges specified are not within the shard id to split.
SELECT citus_split_shard_by_split_points(
49761300, -- Shard range is from (-2147483648, -1073741825)
ARRAY['0'], -- The range we specified is 0 which is not in the range.
ARRAY[:worker_1_node, :worker_2_node]);
-- UDF fails if split points are not strictly increasing.
SELECT citus_split_shard_by_split_points(
49761302,
ARRAY['50', '35'],
ARRAY[:worker_1_node, :worker_2_node, :worker_1_node]);
SELECT citus_split_shard_by_split_points(
49761302,
ARRAY['50', '50'],
ARRAY[:worker_1_node, :worker_2_node, :worker_1_node]);
-- UDF fails if nodeIds are < 1 or Invalid.
SELECT citus_split_shard_by_split_points(
49761302,
ARRAY['50'],
ARRAY[0, :worker_2_node]);
SELECT citus_split_shard_by_split_points(
49761302,
ARRAY['50'],
ARRAY[101, 201]);
-- UDF fails if split point specified is equal to the max value in the range.
-- Example: ShardId 81060002 range is from (-2147483648, -1073741825)
-- '-1073741825' as split point is invalid.
-- '-1073741826' is valid and will split to: (-2147483648, -1073741826) and (-1073741825, -1073741825)
SELECT citus_split_shard_by_split_points(
49761300, -- Shard range is from (-2147483648, -1073741825)
ARRAY['-1073741825'], -- Split point equals shard's max value.
ARRAY[:worker_1_node, :worker_2_node]);
-- UDF fails if resulting shard count from split greater than MAX_SHARD_COUNT (64000)
-- 64000 split point definee 64000+1 way split (64001 worker nodes needed).
WITH shard_ranges AS (SELECT ((-2147483648 + indx))::text as split_points, :worker_1_node as node_ids FROM generate_series(1,64000) indx )
SELECT citus_split_shard_by_split_points(
49761300,
array_agg(split_points),
array_agg(node_ids) || :worker_1_node) --placement node list should exceed split points by one.
FROM shard_ranges;
-- UDF fails where source shard cannot be split further i.e min and max range is equal.
-- Create a Shard where range cannot be split further
SELECT isolate_tenant_to_new_shard('table_to_split', 1);
SELECT citus_split_shard_by_split_points(
49761305,
ARRAY['-1073741826'],
ARRAY[:worker_1_node, :worker_2_node]);
-- Create distributed table with replication factor > 1
SET citus.shard_replication_factor TO 2;
SET citus.next_shard_id TO 51261400;
CREATE TABLE table_to_split_replication_factor_2 (id bigserial PRIMARY KEY, value char);
SELECT create_distributed_table('table_to_split_replication_factor_2','id');
-- UDF fails for replication factor > 1
SELECT citus_split_shard_by_split_points(
51261400,
ARRAY['-1073741826'],
ARRAY[:worker_1_node, :worker_2_node]);
-- Create distributed table with columnar type.
SET citus.next_shard_id TO 51271400;
CREATE TABLE table_to_split_columnar (id bigserial PRIMARY KEY, value char) USING columnar;
SELECT create_distributed_table('table_to_split_columnar','id');
-- UDF fails for columnar table.
SELECT citus_split_shard_by_split_points(
51271400,
ARRAY['-1073741826'],
ARRAY[:worker_1_node, :worker_2_node]);
-- Create distributed table which is partitioned.
SET citus.next_shard_id TO 51271900;
CREATE TABLE table_to_split_partitioned(id integer, dt date) PARTITION BY RANGE(dt);
SELECT create_distributed_table('table_to_split_partitioned','id');
-- UDF fails for partitioned table.
SELECT citus_split_shard_by_split_points(
51271900,
ARRAY['-1073741826'],
ARRAY[:worker_1_node, :worker_2_node]);

View File

@ -0,0 +1,211 @@
CREATE SCHEMA worker_split_binary_copy_test;
SET search_path TO worker_split_binary_copy_test;
SET citus.shard_count TO 1;
SET citus.shard_replication_factor TO 1;
SET citus.next_shard_id TO 81060000;
-- Remove extra nodes added, otherwise GetLocalNodeId() does not bahave correctly.
SELECT citus_remove_node('localhost', 8887);
SELECT citus_remove_node('localhost', 9995);
SELECT citus_remove_node('localhost', 9992);
SELECT citus_remove_node('localhost', 9998);
SELECT citus_remove_node('localhost', 9997);
SELECT citus_remove_node('localhost', 8888);
-- BEGIN: Create distributed table and insert data.
CREATE TABLE worker_split_binary_copy_test.shard_to_split_copy (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
SELECT create_distributed_table('shard_to_split_copy', 'l_orderkey');
\COPY shard_to_split_copy FROM STDIN WITH DELIMITER '|'
99|87114|4639|1|10|11011.10|0.02|0.01|A|F|1994-05-18|1994-06-03|1994-05-23|COLLECT COD|RAIL|kages. requ
99|123766|3767|2|5|8948.80|0.02|0.07|R|F|1994-05-06|1994-05-28|1994-05-20|TAKE BACK RETURN|RAIL|ests cajole fluffily waters. blithe
99|134082|1622|3|42|46875.36|0.02|0.02|A|F|1994-04-19|1994-05-18|1994-04-20|NONE|RAIL|kages are fluffily furiously ir
99|108338|849|4|36|48467.88|0.09|0.02|A|F|1994-07-04|1994-04-17|1994-07-30|DELIVER IN PERSON|AIR|slyly. slyly e
100|62029|2030|1|28|27748.56|0.04|0.05|N|O|1998-05-08|1998-05-13|1998-06-07|COLLECT COD|TRUCK|sts haggle. slowl
100|115979|8491|2|22|43889.34|0.00|0.07|N|O|1998-06-24|1998-04-12|1998-06-29|DELIVER IN PERSON|SHIP|nto beans alongside of the fi
100|46150|8655|3|46|50422.90|0.03|0.04|N|O|1998-05-02|1998-04-10|1998-05-22|TAKE BACK RETURN|SHIP|ular accounts. even
100|38024|3031|4|14|13468.28|0.06|0.03|N|O|1998-05-22|1998-05-01|1998-06-03|COLLECT COD|MAIL|y. furiously ironic ideas gr
100|53439|955|5|37|51519.91|0.05|0.00|N|O|1998-03-06|1998-04-16|1998-03-31|TAKE BACK RETURN|TRUCK|nd the quickly s
101|118282|5816|1|49|63713.72|0.10|0.00|N|O|1996-06-21|1996-05-27|1996-06-29|DELIVER IN PERSON|REG AIR|ts
101|163334|883|2|36|50303.88|0.00|0.01|N|O|1996-05-19|1996-05-01|1996-06-04|DELIVER IN PERSON|AIR|tes. blithely pending dolphins x-ray f
101|138418|5958|3|12|17476.92|0.06|0.02|N|O|1996-03-29|1996-04-20|1996-04-12|COLLECT COD|MAIL|. quickly regular
102|88914|3931|1|37|70407.67|0.06|0.00|N|O|1997-07-24|1997-08-02|1997-08-07|TAKE BACK RETURN|SHIP|ully across the ideas. final deposit
102|169238|6787|2|34|44445.82|0.03|0.08|N|O|1997-08-09|1997-07-28|1997-08-26|TAKE BACK RETURN|SHIP|eposits cajole across
102|182321|4840|3|25|35083.00|0.01|0.01|N|O|1997-07-31|1997-07-24|1997-08-17|NONE|RAIL|bits. ironic accoun
102|61158|8677|4|15|16787.25|0.07|0.07|N|O|1997-06-02|1997-07-13|1997-06-04|DELIVER IN PERSON|SHIP|final packages. carefully even excu
103|194658|2216|1|6|10515.90|0.03|0.05|N|O|1996-10-11|1996-07-25|1996-10-28|NONE|FOB|cajole. carefully ex
103|10426|2928|2|37|49447.54|0.02|0.07|N|O|1996-09-17|1996-07-27|1996-09-20|TAKE BACK RETURN|MAIL|ies. quickly ironic requests use blithely
103|28431|8432|3|23|31266.89|0.01|0.04|N|O|1996-09-11|1996-09-18|1996-09-26|NONE|FOB|ironic accou
103|29022|4027|4|32|30432.64|0.01|0.07|N|O|1996-07-30|1996-08-06|1996-08-04|NONE|RAIL|kages doze. special, regular deposit
-1995148554|112942|2943|1|9|17594.46|0.04|0.04|N|O|1996-08-03|1996-05-31|1996-08-04|DELIVER IN PERSON|TRUCK|c realms print carefully car
-1686493264|15110|113|5|2|2050.22|0.03|0.08|R|F|1994-04-26|1994-03-15|1994-05-15|TAKE BACK RETURN|MAIL|e final, regular requests. carefully
\.
-- END: Create distributed table and insert data.
-- BEGIN: Switch to Worker1, Create target shards in worker for local 2-way split copy.
\c - - - :worker_1_port
CREATE TABLE worker_split_binary_copy_test.shard_to_split_copy_81060015 (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
CREATE TABLE worker_split_binary_copy_test.shard_to_split_copy_81060016 (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
-- End: Switch to Worker1, Create target shards in worker for local 2-way split copy.
-- BEGIN: Switch to Worker2, Create target shards in worker for remote 2-way split copy.
\c - - - :worker_2_port
CREATE TABLE worker_split_binary_copy_test.shard_to_split_copy_81060015 (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
CREATE TABLE worker_split_binary_copy_test.shard_to_split_copy_81060016 (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
-- End: Switch to Worker2, Create target shards in worker for remote 2-way split copy.
-- BEGIN: List row count for source shard and targets shard in Worker1.
\c - - - :worker_1_port
SELECT COUNT(*) FROM worker_split_binary_copy_test.shard_to_split_copy_81060000;
SELECT COUNT(*) FROM worker_split_binary_copy_test.shard_to_split_copy_81060015;
SELECT COUNT(*) FROM worker_split_binary_copy_test.shard_to_split_copy_81060016;
-- END: List row count for source shard and targets shard in Worker1.
-- BEGIN: List row count for target shard in Worker2.
\c - - - :worker_2_port
SELECT COUNT(*) FROM worker_split_binary_copy_test.shard_to_split_copy_81060015;
SELECT COUNT(*) FROM worker_split_binary_copy_test.shard_to_split_copy_81060016;
-- END: List row count for targets shard in Worker2.
-- BEGIN: Set worker_1_node and worker_2_node
\c - - - :worker_1_port
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
-- END: Set worker_1_node and worker_2_node
-- BEGIN: Trigger 2-way local shard split copy.
-- Ensure we will perform binary copy.
SET citus.enable_binary_protocol = true;
SELECT * from worker_split_copy(
81060000, -- source shard id to copy
ARRAY[
-- split copy info for split children 1
ROW(81060015, -- destination shard id
-2147483648, -- split range begin
1073741823, --split range end
:worker_1_node)::pg_catalog.split_copy_info,
-- split copy info for split children 2
ROW(81060016, --destination shard id
1073741824, --split range begin
2147483647, --split range end
:worker_1_node)::pg_catalog.split_copy_info
]
);
-- END: Trigger 2-way local shard split copy.
-- BEGIN: Trigger 2-way remote shard split copy.
SELECT * from worker_split_copy(
81060000, -- source shard id to copy
ARRAY[
-- split copy info for split children 1
ROW(81060015, -- destination shard id
-2147483648, -- split range begin
1073741823, --split range end
:worker_2_node)::pg_catalog.split_copy_info,
-- split copy info for split children 2
ROW(81060016, --destination shard id
1073741824, --split range begin
2147483647, --split range end
:worker_2_node)::pg_catalog.split_copy_info
]
);
-- END: Trigger 2-way remote shard split copy.
-- BEGIN: List updated row count for local targets shard.
SELECT COUNT(*) FROM worker_split_binary_copy_test.shard_to_split_copy_81060015;
SELECT COUNT(*) FROM worker_split_binary_copy_test.shard_to_split_copy_81060016;
-- END: List updated row count for local targets shard.
-- BEGIN: List updated row count for remote targets shard.
\c - - - :worker_2_port
SELECT COUNT(*) FROM worker_split_binary_copy_test.shard_to_split_copy_81060015;
SELECT COUNT(*) FROM worker_split_binary_copy_test.shard_to_split_copy_81060016;
-- END: List updated row count for remote targets shard.
-- BEGIN: CLEANUP.
\c - - - :master_port
SET client_min_messages TO WARNING;
DROP SCHEMA worker_split_binary_copy_test CASCADE;
-- END: CLEANUP.

View File

@ -0,0 +1,110 @@
CREATE SCHEMA worker_split_copy_test;
SET search_path TO worker_split_copy_test;
SET citus.shard_count TO 2;
SET citus.shard_replication_factor TO 1;
SET citus.next_shard_id TO 81070000;
-- BEGIN: Create distributed table and insert data.
CREATE TABLE worker_split_copy_test."test !/ \n _""dist_123_table"(id int primary key, value char);
SELECT create_distributed_table('"test !/ \n _""dist_123_table"', 'id');
INSERT INTO "test !/ \n _""dist_123_table" (id, value) (SELECT g.id, 'N' FROM generate_series(1, 1000) AS g(id));
-- END: Create distributed table and insert data.
-- BEGIN: Switch to Worker1, Create target shards in worker for local 2-way split copy.
\c - - - :worker_1_port
CREATE TABLE worker_split_copy_test."test !/ \n _""dist_123_table_81070015"(id int primary key, value char);
CREATE TABLE worker_split_copy_test."test !/ \n _""dist_123_table_81070016"(id int primary key, value char);
-- End: Switch to Worker1, Create target shards in worker for local 2-way split copy.
-- BEGIN: List row count for source shard and targets shard in Worker1.
\c - - - :worker_1_port
SELECT COUNT(*) FROM worker_split_copy_test."test !/ \n _""dist_123_table_81070000";
SELECT COUNT(*) FROM worker_split_copy_test."test !/ \n _""dist_123_table_81070015";
SELECT COUNT(*) FROM worker_split_copy_test."test !/ \n _""dist_123_table_81070016";
\c - - - :worker_2_port
SELECT COUNT(*) FROM worker_split_copy_test."test !/ \n _""dist_123_table_81070001";
-- END: List row count for source shard and targets shard in Worker1.
-- BEGIN: Set worker_1_node and worker_2_node
\c - - - :worker_1_port
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
-- END: Set worker_1_node and worker_2_node
-- BEGIN: Test Negative scenario
SELECT * from worker_split_copy(
101, -- Invalid source shard id.
ARRAY[
-- split copy info for split children 1
ROW(81070015, -- destination shard id
-2147483648, -- split range begin
-1073741824, --split range end
:worker_1_node)::pg_catalog.split_copy_info,
-- split copy info for split children 2
ROW(81070016, --destination shard id
-1073741823, --split range begin
-1, --split range end
:worker_1_node)::pg_catalog.split_copy_info
]
);
SELECT * from worker_split_copy(
81070000, -- source shard id to copy
ARRAY[] -- empty array
);
SELECT * from worker_split_copy(
81070000, -- source shard id to copy
ARRAY[NULL] -- empty array
);
SELECT * from worker_split_copy(
81070000, -- source shard id to copy
ARRAY[NULL::pg_catalog.split_copy_info]-- empty array
);
SELECT * from worker_split_copy(
81070000, -- source shard id to copy
ARRAY[ROW(NULL)]-- empty array
);
SELECT * from worker_split_copy(
81070000, -- source shard id to copy
ARRAY[ROW(NULL, NULL, NULL, NULL)::pg_catalog.split_copy_info] -- empty array
);
-- END: Test Negative scenario
-- BEGIN: Trigger 2-way local shard split copy.
-- Ensure we will perform text copy.
SET citus.enable_binary_protocol = false;
SELECT * from worker_split_copy(
81070000, -- source shard id to copy
ARRAY[
-- split copy info for split children 1
ROW(81070015, -- destination shard id
-2147483648, -- split range begin
-1073741824, --split range end
:worker_1_node)::pg_catalog.split_copy_info,
-- split copy info for split children 2
ROW(81070016, --destination shard id
-1073741823, --split range begin
-1, --split range end
:worker_1_node)::pg_catalog.split_copy_info
]
);
-- END: Trigger 2-way local shard split copy.
-- BEGIN: List updated row count for local targets shard.
SELECT COUNT(*) FROM worker_split_copy_test."test !/ \n _""dist_123_table_81070015";
SELECT COUNT(*) FROM worker_split_copy_test."test !/ \n _""dist_123_table_81070016";
-- END: List updated row count for local targets shard.
-- BEGIN: CLEANUP.
\c - - - :master_port
SET client_min_messages TO WARNING;
DROP SCHEMA worker_split_copy_test CASCADE;
-- END: CLEANUP.

View File

@ -0,0 +1,203 @@
CREATE SCHEMA worker_split_text_copy_test;
SET search_path TO worker_split_text_copy_test;
SET citus.shard_count TO 1;
SET citus.shard_replication_factor TO 1;
SET citus.next_shard_id TO 81070000;
-- BEGIN: Create distributed table and insert data.
CREATE TABLE worker_split_text_copy_test.shard_to_split_copy (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
SELECT create_distributed_table('shard_to_split_copy', 'l_orderkey');
\COPY shard_to_split_copy FROM STDIN WITH DELIMITER '|'
99|87114|4639|1|10|11011.10|0.02|0.01|A|F|1994-05-18|1994-06-03|1994-05-23|COLLECT COD|RAIL|kages. requ
99|123766|3767|2|5|8948.80|0.02|0.07|R|F|1994-05-06|1994-05-28|1994-05-20|TAKE BACK RETURN|RAIL|ests cajole fluffily waters. blithe
99|134082|1622|3|42|46875.36|0.02|0.02|A|F|1994-04-19|1994-05-18|1994-04-20|NONE|RAIL|kages are fluffily furiously ir
99|108338|849|4|36|48467.88|0.09|0.02|A|F|1994-07-04|1994-04-17|1994-07-30|DELIVER IN PERSON|AIR|slyly. slyly e
100|62029|2030|1|28|27748.56|0.04|0.05|N|O|1998-05-08|1998-05-13|1998-06-07|COLLECT COD|TRUCK|sts haggle. slowl
100|115979|8491|2|22|43889.34|0.00|0.07|N|O|1998-06-24|1998-04-12|1998-06-29|DELIVER IN PERSON|SHIP|nto beans alongside of the fi
100|46150|8655|3|46|50422.90|0.03|0.04|N|O|1998-05-02|1998-04-10|1998-05-22|TAKE BACK RETURN|SHIP|ular accounts. even
100|38024|3031|4|14|13468.28|0.06|0.03|N|O|1998-05-22|1998-05-01|1998-06-03|COLLECT COD|MAIL|y. furiously ironic ideas gr
100|53439|955|5|37|51519.91|0.05|0.00|N|O|1998-03-06|1998-04-16|1998-03-31|TAKE BACK RETURN|TRUCK|nd the quickly s
101|118282|5816|1|49|63713.72|0.10|0.00|N|O|1996-06-21|1996-05-27|1996-06-29|DELIVER IN PERSON|REG AIR|ts
101|163334|883|2|36|50303.88|0.00|0.01|N|O|1996-05-19|1996-05-01|1996-06-04|DELIVER IN PERSON|AIR|tes. blithely pending dolphins x-ray f
101|138418|5958|3|12|17476.92|0.06|0.02|N|O|1996-03-29|1996-04-20|1996-04-12|COLLECT COD|MAIL|. quickly regular
102|88914|3931|1|37|70407.67|0.06|0.00|N|O|1997-07-24|1997-08-02|1997-08-07|TAKE BACK RETURN|SHIP|ully across the ideas. final deposit
102|169238|6787|2|34|44445.82|0.03|0.08|N|O|1997-08-09|1997-07-28|1997-08-26|TAKE BACK RETURN|SHIP|eposits cajole across
102|182321|4840|3|25|35083.00|0.01|0.01|N|O|1997-07-31|1997-07-24|1997-08-17|NONE|RAIL|bits. ironic accoun
102|61158|8677|4|15|16787.25|0.07|0.07|N|O|1997-06-02|1997-07-13|1997-06-04|DELIVER IN PERSON|SHIP|final packages. carefully even excu
103|194658|2216|1|6|10515.90|0.03|0.05|N|O|1996-10-11|1996-07-25|1996-10-28|NONE|FOB|cajole. carefully ex
103|10426|2928|2|37|49447.54|0.02|0.07|N|O|1996-09-17|1996-07-27|1996-09-20|TAKE BACK RETURN|MAIL|ies. quickly ironic requests use blithely
103|28431|8432|3|23|31266.89|0.01|0.04|N|O|1996-09-11|1996-09-18|1996-09-26|NONE|FOB|ironic accou
103|29022|4027|4|32|30432.64|0.01|0.07|N|O|1996-07-30|1996-08-06|1996-08-04|NONE|RAIL|kages doze. special, regular deposit
-1995148554|112942|2943|1|9|17594.46|0.04|0.04|N|O|1996-08-03|1996-05-31|1996-08-04|DELIVER IN PERSON|TRUCK|c realms print carefully car
-1686493264|15110|113|5|2|2050.22|0.03|0.08|R|F|1994-04-26|1994-03-15|1994-05-15|TAKE BACK RETURN|MAIL|e final, regular requests. carefully
\.
-- END: Create distributed table and insert data.
-- BEGIN: Switch to Worker1, Create target shards in worker for local 2-way split copy.
\c - - - :worker_1_port
CREATE TABLE worker_split_text_copy_test.shard_to_split_copy_81070015 (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
CREATE TABLE worker_split_text_copy_test.shard_to_split_copy_81070016 (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
-- End: Switch to Worker1, Create target shards in worker for local 2-way split copy.
-- BEGIN: Switch to Worker2, Create target shards in worker for remote 2-way split copy.
\c - - - :worker_2_port
CREATE TABLE worker_split_text_copy_test.shard_to_split_copy_81070015 (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
CREATE TABLE worker_split_text_copy_test.shard_to_split_copy_81070016 (
l_orderkey bigint not null,
l_partkey integer not null,
l_suppkey integer not null,
l_linenumber integer not null,
l_quantity decimal(15, 2) not null,
l_extendedprice decimal(15, 2) not null,
l_discount decimal(15, 2) not null,
l_tax decimal(15, 2) not null,
l_returnflag char(1) not null,
l_linestatus char(1) not null,
l_shipdate date not null,
l_commitdate date not null,
l_receiptdate date not null,
l_shipinstruct char(25) not null,
l_shipmode char(10) not null,
l_comment varchar(44) not null);
-- End: Switch to Worker2, Create target shards in worker for remote 2-way split copy.
-- BEGIN: List row count for source shard and targets shard in Worker1.
\c - - - :worker_1_port
SELECT COUNT(*) FROM worker_split_text_copy_test.shard_to_split_copy_81070000;
SELECT COUNT(*) FROM worker_split_text_copy_test.shard_to_split_copy_81070015;
SELECT COUNT(*) FROM worker_split_text_copy_test.shard_to_split_copy_81070016;
-- END: List row count for source shard and targets shard in Worker1.
-- BEGIN: List row count for target shard in Worker2.
\c - - - :worker_2_port
SELECT COUNT(*) FROM worker_split_text_copy_test.shard_to_split_copy_81070015;
SELECT COUNT(*) FROM worker_split_text_copy_test.shard_to_split_copy_81070016;
-- END: List row count for targets shard in Worker2.
-- BEGIN: Set worker_1_node and worker_2_node
\c - - - :worker_1_port
SELECT nodeid AS worker_1_node FROM pg_dist_node WHERE nodeport=:worker_1_port \gset
SELECT nodeid AS worker_2_node FROM pg_dist_node WHERE nodeport=:worker_2_port \gset
-- END: Set worker_1_node and worker_2_node
-- BEGIN: Trigger 2-way local shard split copy.
-- Ensure we will perform text copy.
SET citus.enable_binary_protocol = false;
SELECT * from worker_split_copy(
81070000, -- source shard id to copy
ARRAY[
-- split copy info for split children 1
ROW(81070015, -- destination shard id
-2147483648, -- split range begin
1073741823, --split range end
:worker_1_node)::pg_catalog.split_copy_info,
-- split copy info for split children 2
ROW(81070016, --destination shard id
1073741824, --split range begin
2147483647, --split range end
:worker_1_node)::pg_catalog.split_copy_info
]
);
-- END: Trigger 2-way local shard split copy.
-- BEGIN: Trigger 2-way remote shard split copy.
SELECT * from worker_split_copy(
81070000, -- source shard id to copy
ARRAY[
-- split copy info for split children 1
ROW(81070015, -- destination shard id
-2147483648, -- split range begin
1073741823, --split range end
:worker_2_node)::pg_catalog.split_copy_info,
-- split copy info for split children 2
ROW(81070016, --destination shard id
1073741824, --split range begin
2147483647, --split range end
:worker_2_node)::pg_catalog.split_copy_info
]
);
-- END: Trigger 2-way remote shard split copy.
-- BEGIN: List updated row count for local targets shard.
SELECT COUNT(*) FROM worker_split_text_copy_test.shard_to_split_copy_81070015;
SELECT COUNT(*) FROM worker_split_text_copy_test.shard_to_split_copy_81070016;
-- END: List updated row count for local targets shard.
-- BEGIN: List updated row count for remote targets shard.
\c - - - :worker_2_port
SELECT COUNT(*) FROM worker_split_text_copy_test.shard_to_split_copy_81070015;
SELECT COUNT(*) FROM worker_split_text_copy_test.shard_to_split_copy_81070016;
-- END: List updated row count for remote targets shard.
-- BEGIN: CLEANUP.
\c - - - :master_port
SET client_min_messages TO WARNING;
DROP SCHEMA worker_split_text_copy_test CASCADE;
-- END: CLEANUP.