mirror of https://github.com/citusdata/citus.git
Merge branch 'main' into baby_step_pg_15
commit
90b1afe31e
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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;
|
||||
}
|
|
@ -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
|
@ -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;
|
||||
}
|
|
@ -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, ©InfoDatum, &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;
|
||||
}
|
|
@ -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"
|
||||
|
|
|
@ -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,
|
||||
|
|
14
src/backend/distributed/sql/udfs/citus_split_shard_by_split_points/11.1-1.sql
generated
Normal file
14
src/backend/distributed/sql/udfs/citus_split_shard_by_split_points/11.1-1.sql
generated
Normal 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.';
|
|
@ -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.';
|
|
@ -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';
|
|
@ -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';
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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 */
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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_ */
|
|
@ -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 */
|
||||
|
|
|
@ -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_ */
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
|
@ -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
|
|
@ -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.
|
|
@ -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)
|
||||
|
|
@ -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)
|
||||
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
|
||||
|
|
|
@ -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.
|
|
@ -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.
|
|
@ -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.
|
|
@ -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.
|
|
@ -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.
|
|
@ -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"
|
|
@ -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"
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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]);
|
|
@ -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.
|
|
@ -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.
|
|
@ -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.
|
Loading…
Reference in New Issue