Merge remote-tracking branch 'upstream/main' into issue/6694

issue/6694
Gokhan Gulbiz 2023-03-29 10:28:33 +03:00
commit c9a3fbb503
No known key found for this signature in database
GPG Key ID: 608EF06B6BD1B45B
70 changed files with 3443 additions and 118 deletions

View File

@ -455,6 +455,10 @@ jobs:
pg_major: << parameters.pg_major >>
- configure
- enable_core
- run:
name: 'Install DBI.pm'
command: |
apt-get update && apt-get install libdbi-perl && apt-get install libdbd-pg-perl
- run:
name: 'Run Test'
command: |
@ -889,6 +893,10 @@ workflows:
<<: *tap-test-citus-15
name: 'test-15_tap-columnar-freezing'
suite: columnar_freezing
- tap-test-citus:
<<: *tap-test-citus-15
name: 'test-15_tap-cdc'
suite: cdc
- test-arbitrary-configs:
name: 'test-13_check-arbitrary-configs'

View File

@ -18,7 +18,7 @@ generated_downgrade_sql_files += $(patsubst %,$(citus_abs_srcdir)/build/sql/%,$(
DATA_built = $(generated_sql_files)
# directories with source files
SUBDIRS = . commands connection ddl deparser executor metadata operations planner progress relay safeclib shardsplit test transaction utils worker clock
SUBDIRS = . commands cdc connection ddl deparser executor metadata operations planner progress relay safeclib shardsplit test transaction utils worker clock
# enterprise modules
SUBDIRS += replication

View File

@ -0,0 +1,388 @@
/*-------------------------------------------------------------------------
*
* cdc_decoder.c
* CDC Decoder plugin for Citus
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "common/hashfn.h"
#include "utils/typcache.h"
#include "utils/lsyscache.h"
#include "catalog/pg_namespace.h"
#include "distributed/cdc_decoder.h"
#include "distributed/relay_utility.h"
#include "distributed/worker_protocol.h"
#include "distributed/metadata_cache.h"
static LogicalDecodeChangeCB ouputPluginChangeCB;
static bool replication_origin_filter_cb(LogicalDecodingContext *ctx, RepOriginId
origin_id);
static void TranslateChangesIfSchemaChanged(Relation relation, Relation targetRelation,
ReorderBufferChange *change);
static void TranslateAndPublishRelationForCDC(LogicalDecodingContext *ctx,
ReorderBufferTXN *txn,
Relation relation,
ReorderBufferChange *change, Oid shardId,
Oid targetRelationid);
typedef struct
{
uint64 shardId;
Oid distributedTableId;
bool isReferenceTable;
bool isNull;
} ShardIdHashEntry;
static HTAB *shardToDistributedTableMap = NULL;
/*
* InitShardToDistributedTableMap initializes the hash table that is used to
* translate the changes in the shard table to the changes in the distributed table.
*/
static void
InitShardToDistributedTableMap()
{
HASHCTL info;
memset(&info, 0, sizeof(info));
info.keysize = sizeof(uint64);
info.entrysize = sizeof(ShardIdHashEntry);
info.hash = tag_hash;
info.hcxt = CurrentMemoryContext;
int hashFlags = (HASH_ELEM | HASH_CONTEXT | HASH_FUNCTION);
shardToDistributedTableMap = hash_create("CDC Decoder translation hash table", 1024,
&info, hashFlags);
}
/*
* AddShardIdToHashTable adds the shardId to the hash table.
*/
static Oid
AddShardIdToHashTable(Oid shardId, ShardIdHashEntry *entry)
{
entry->shardId = shardId;
entry->distributedTableId = LookupShardRelationFromCatalog(shardId, true);
entry->isReferenceTable = PartitionMethodViaCatalog(entry->distributedTableId) == 'n';
return entry->distributedTableId;
}
static Oid
LookupDistributedTableIdForShardId(Oid shardId, bool *isReferenceTable)
{
bool found;
Oid distributedTableId = InvalidOid;
ShardIdHashEntry *entry = (ShardIdHashEntry *) hash_search(shardToDistributedTableMap,
&shardId,
HASH_FIND | HASH_ENTER,
&found);
if (found)
{
distributedTableId = entry->distributedTableId;
}
else
{
distributedTableId = AddShardIdToHashTable(shardId, entry);
}
*isReferenceTable = entry->isReferenceTable;
return distributedTableId;
}
/*
* InitCDCDecoder is called by from the shard split decoder plugin's init function.
* It sets the call back function for filtering out changes originated from other nodes.
* It also sets the call back function for processing the changes in ouputPluginChangeCB.
* This function is common for both CDC and shard split decoder plugins.
*/
void
InitCDCDecoder(OutputPluginCallbacks *cb, LogicalDecodeChangeCB changeCB)
{
elog(LOG, "Initializing CDC decoder");
cb->filter_by_origin_cb = replication_origin_filter_cb;
ouputPluginChangeCB = changeCB;
/* Initialize the hash table used for mapping shard to shell tables. */
InitShardToDistributedTableMap();
}
/*
* replication_origin_filter_cb call back function filters out publication of changes
* originated from any other node other than the current node. This is
* identified by the "origin_id" of the changes. The origin_id is set to
* a non-zero value in the origin node as part of WAL replication for internal
* operations like shard split/moves/create_distributed_table etc.
*/
static bool
replication_origin_filter_cb(LogicalDecodingContext *ctx, RepOriginId origin_id)
{
return (origin_id != InvalidRepOriginId);
}
/*
* This function is responsible for translating the changes in the shard table to
* the changes in the shell table and publishing the changes as a change to the
* distributed table so that CDD clients are not aware of the shard tables. It also
* handles schema changes to the distributed table.
*/
static void
TranslateAndPublishRelationForCDC(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
Relation relation, ReorderBufferChange *change, Oid
shardId, Oid targetRelationid)
{
/* Get the distributed table's relation for this shard.*/
Relation targetRelation = RelationIdGetRelation(targetRelationid);
/*
* Check if there has been a schema change (such as a dropped column), by comparing
* the number of attributes in the shard table and the shell table.
*/
TranslateChangesIfSchemaChanged(relation, targetRelation, change);
/*
* Publish the change to the shard table as the change in the distributed table,
* so that the CDC client can see the change in the distributed table,
* instead of the shard table, by calling the pgoutput's callback function.
*/
ouputPluginChangeCB(ctx, txn, targetRelation, change);
RelationClose(targetRelation);
}
/*
* PublishChangesIfCdcSlot checks if the current slot is a CDC slot. If so, it publishes
* the changes as the change for the distributed table instead of shard.
* If not, it returns false. It also skips the Citus metadata tables.
*/
void
PublishDistributedTableChanges(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
Relation relation, ReorderBufferChange *change)
{
char *shardRelationName = RelationGetRelationName(relation);
/* Skip publishing CDC changes for any system relations in pg_catalog*/
if (relation->rd_rel->relnamespace == PG_CATALOG_NAMESPACE)
{
return;
}
/* Check if the relation is a distributed table by checking for shard name. */
uint64 shardId = ExtractShardIdFromTableName(shardRelationName, true);
/* If this relation is not distributed, call the pgoutput's callback and return. */
if (shardId == INVALID_SHARD_ID)
{
ouputPluginChangeCB(ctx, txn, relation, change);
return;
}
bool isReferenceTable = false;
Oid distRelationId = LookupDistributedTableIdForShardId(shardId, &isReferenceTable);
if (distRelationId == InvalidOid)
{
ouputPluginChangeCB(ctx, txn, relation, change);
return;
}
/* Publish changes for reference table only from the coordinator node. */
if (isReferenceTable && !IsCoordinator())
{
return;
}
/* translate and publish from shard relation to distributed table relation for CDC. */
TranslateAndPublishRelationForCDC(ctx, txn, relation, change, shardId,
distRelationId);
}
/*
* GetTupleForTargetSchemaForCdc returns a heap tuple with the data from sourceRelationTuple
* to match the schema in targetRelDesc. Either or both source and target relations may have
* dropped columns. This function handles it by adding NULL values for dropped columns in
* target relation and skipping dropped columns in source relation. It returns a heap tuple
* adjusted to the current schema of the target relation.
*/
static HeapTuple
GetTupleForTargetSchemaForCdc(HeapTuple sourceRelationTuple,
TupleDesc sourceRelDesc,
TupleDesc targetRelDesc)
{
/* Allocate memory for sourceValues and sourceNulls arrays. */
Datum *sourceValues = (Datum *) palloc0(sourceRelDesc->natts * sizeof(Datum));
bool *sourceNulls = (bool *) palloc0(sourceRelDesc->natts * sizeof(bool));
/* Deform the source tuple to sourceValues and sourceNulls arrays. */
heap_deform_tuple(sourceRelationTuple, sourceRelDesc, sourceValues,
sourceNulls);
/* This is the next field to Read in the source relation */
uint32 sourceIndex = 0;
uint32 targetIndex = 0;
/* Allocate memory for sourceValues and sourceNulls arrays. */
Datum *targetValues = (Datum *) palloc0(targetRelDesc->natts * sizeof(Datum));
bool *targetNulls = (bool *) palloc0(targetRelDesc->natts * sizeof(bool));
/* Loop through all source and target attributes one by one and handle any dropped attributes.*/
while (targetIndex < targetRelDesc->natts)
{
/* If this target attribute has been dropped, add a NULL attribute in targetValues and continue.*/
if (TupleDescAttr(targetRelDesc, targetIndex)->attisdropped)
{
Datum nullDatum = (Datum) 0;
targetValues[targetIndex] = nullDatum;
targetNulls[targetIndex] = true;
targetIndex++;
}
/* If this source attribute has been dropped, just skip this source attribute.*/
else if (TupleDescAttr(sourceRelDesc, sourceIndex)->attisdropped)
{
sourceIndex++;
continue;
}
/* If both source and target attributes are not dropped, add the attribute field to targetValues. */
else if (sourceIndex < sourceRelDesc->natts)
{
targetValues[targetIndex] = sourceValues[sourceIndex];
targetNulls[targetIndex] = sourceNulls[sourceIndex];
sourceIndex++;
targetIndex++;
}
else
{
/* If there are no more source fields, add a NULL field in targetValues. */
Datum nullDatum = (Datum) 0;
targetValues[targetIndex] = nullDatum;
targetNulls[targetIndex] = true;
targetIndex++;
}
}
/* Form a new tuple from the target values created by the above loop. */
HeapTuple targetRelationTuple = heap_form_tuple(targetRelDesc, targetValues,
targetNulls);
return targetRelationTuple;
}
/* HasSchemaChanged function returns if there any schema changes between source and target relations.*/
static bool
HasSchemaChanged(TupleDesc sourceRelationDesc, TupleDesc targetRelationDesc)
{
bool hasSchemaChanged = (sourceRelationDesc->natts != targetRelationDesc->natts);
if (hasSchemaChanged)
{
return true;
}
for (uint32 i = 0; i < sourceRelationDesc->natts; i++)
{
if (TupleDescAttr(sourceRelationDesc, i)->attisdropped ||
TupleDescAttr(targetRelationDesc, i)->attisdropped)
{
hasSchemaChanged = true;
break;
}
}
return hasSchemaChanged;
}
/*
* TranslateChangesIfSchemaChanged translates the tuples ReorderBufferChange
* if there is a schema change between source and target relations.
*/
static void
TranslateChangesIfSchemaChanged(Relation sourceRelation, Relation targetRelation,
ReorderBufferChange *change)
{
TupleDesc sourceRelationDesc = RelationGetDescr(sourceRelation);
TupleDesc targetRelationDesc = RelationGetDescr(targetRelation);
/* if there are no changes between source and target relations, return. */
if (!HasSchemaChanged(sourceRelationDesc, targetRelationDesc))
{
return;
}
/* Check the ReorderBufferChange's action type and handle them accordingly.*/
switch (change->action)
{
case REORDER_BUFFER_CHANGE_INSERT:
{
/* For insert action, only new tuple should always be translated*/
HeapTuple sourceRelationNewTuple = &(change->data.tp.newtuple->tuple);
HeapTuple targetRelationNewTuple = GetTupleForTargetSchemaForCdc(
sourceRelationNewTuple, sourceRelationDesc, targetRelationDesc);
change->data.tp.newtuple->tuple = *targetRelationNewTuple;
break;
}
/*
* For update changes both old and new tuples need to be translated for target relation
* if the REPLICA IDENTITY is set to FULL. Otherwise, only the new tuple needs to be
* translated for target relation.
*/
case REORDER_BUFFER_CHANGE_UPDATE:
{
/* For update action, new tuple should always be translated*/
/* Get the new tuple from the ReorderBufferChange, and translate it to target relation. */
HeapTuple sourceRelationNewTuple = &(change->data.tp.newtuple->tuple);
HeapTuple targetRelationNewTuple = GetTupleForTargetSchemaForCdc(
sourceRelationNewTuple, sourceRelationDesc, targetRelationDesc);
change->data.tp.newtuple->tuple = *targetRelationNewTuple;
/*
* Format oldtuple according to the target relation. If the column values of replica
* identiy change, then the old tuple is non-null and needs to be formatted according
* to the target relation schema.
*/
if (change->data.tp.oldtuple != NULL)
{
HeapTuple sourceRelationOldTuple = &(change->data.tp.oldtuple->tuple);
HeapTuple targetRelationOldTuple = GetTupleForTargetSchemaForCdc(
sourceRelationOldTuple,
sourceRelationDesc,
targetRelationDesc);
change->data.tp.oldtuple->tuple = *targetRelationOldTuple;
}
break;
}
case REORDER_BUFFER_CHANGE_DELETE:
{
/* For delete action, only old tuple should be translated*/
HeapTuple sourceRelationOldTuple = &(change->data.tp.oldtuple->tuple);
HeapTuple targetRelationOldTuple = GetTupleForTargetSchemaForCdc(
sourceRelationOldTuple,
sourceRelationDesc,
targetRelationDesc);
change->data.tp.oldtuple->tuple = *targetRelationOldTuple;
break;
}
default:
{
/* Do nothing for other action types. */
break;
}
}
}

View File

@ -55,6 +55,7 @@
#include "distributed/multi_partitioning_utils.h"
#include "distributed/reference_table_utils.h"
#include "distributed/relation_access_tracking.h"
#include "distributed/replication_origin_session_utils.h"
#include "distributed/shared_library_init.h"
#include "distributed/shard_utils.h"
#include "distributed/worker_protocol.h"
@ -406,7 +407,10 @@ UndistributeTable(TableConversionParameters *params)
params->shardCountIsNull = true;
TableConversionState *con = CreateTableConversion(params);
return ConvertTable(con);
SetupReplicationOriginLocalSession();
TableConversionReturn *conv = ConvertTable(con);
ResetReplicationOriginLocalSession();
return conv;
}

View File

@ -2408,12 +2408,12 @@ CopyLocalDataIntoShards(Oid distributedRelationId)
EState *estate = CreateExecutorState();
ExprContext *econtext = GetPerTupleExprContext(estate);
econtext->ecxt_scantuple = slot;
const bool nonPublishableData = false;
DestReceiver *copyDest =
(DestReceiver *) CreateCitusCopyDestReceiver(distributedRelationId,
columnNameList,
partitionColumnIndex,
estate, NULL);
estate, NULL, nonPublishableData);
/* initialise state for writing to shards, we'll open connections on demand */
copyDest->rStartup(copyDest, 0, tupleDescriptor);

View File

@ -36,6 +36,7 @@
#include "distributed/local_multi_copy.h"
#include "distributed/shard_utils.h"
#include "distributed/version_compat.h"
#include "distributed/replication_origin_session_utils.h"
/* managed via GUC, default is 512 kB */
int LocalCopyFlushThresholdByte = 512 * 1024;
@ -46,7 +47,7 @@ static void AddSlotToBuffer(TupleTableSlot *slot, CitusCopyDestReceiver *copyDes
static bool ShouldAddBinaryHeaders(StringInfo buffer, bool isBinary);
static bool ShouldSendCopyNow(StringInfo buffer);
static void DoLocalCopy(StringInfo buffer, Oid relationId, int64 shardId,
CopyStmt *copyStatement, bool isEndOfCopy);
CopyStmt *copyStatement, bool isEndOfCopy, bool isPublishable);
static int ReadFromLocalBufferCallback(void *outBuf, int minRead, int maxRead);
@ -94,7 +95,7 @@ WriteTupleToLocalShard(TupleTableSlot *slot, CitusCopyDestReceiver *copyDest, in
bool isEndOfCopy = false;
DoLocalCopy(localCopyOutState->fe_msgbuf, copyDest->distributedRelationId,
shardId,
copyDest->copyStatement, isEndOfCopy);
copyDest->copyStatement, isEndOfCopy, copyDest->isPublishable);
resetStringInfo(localCopyOutState->fe_msgbuf);
}
}
@ -133,7 +134,7 @@ FinishLocalCopyToShard(CitusCopyDestReceiver *copyDest, int64 shardId,
}
bool isEndOfCopy = true;
DoLocalCopy(localCopyOutState->fe_msgbuf, copyDest->distributedRelationId, shardId,
copyDest->copyStatement, isEndOfCopy);
copyDest->copyStatement, isEndOfCopy, copyDest->isPublishable);
}
@ -197,7 +198,7 @@ ShouldSendCopyNow(StringInfo buffer)
*/
static void
DoLocalCopy(StringInfo buffer, Oid relationId, int64 shardId, CopyStmt *copyStatement,
bool isEndOfCopy)
bool isEndOfCopy, bool isPublishable)
{
/*
* Set the buffer as a global variable to allow ReadFromLocalBufferCallback
@ -205,6 +206,10 @@ DoLocalCopy(StringInfo buffer, Oid relationId, int64 shardId, CopyStmt *copyStat
* ReadFromLocalBufferCallback.
*/
LocalCopyBuffer = buffer;
if (!isPublishable)
{
SetupReplicationOriginLocalSession();
}
Oid shardOid = GetTableLocalShardOid(relationId, shardId);
Relation shard = table_open(shardOid, RowExclusiveLock);
@ -219,6 +224,10 @@ DoLocalCopy(StringInfo buffer, Oid relationId, int64 shardId, CopyStmt *copyStat
EndCopyFrom(cstate);
table_close(shard, NoLock);
if (!isPublishable)
{
ResetReplicationOriginLocalSession();
}
free_parsestate(pState);
}

View File

@ -85,6 +85,7 @@
#include "distributed/relation_access_tracking.h"
#include "distributed/remote_commands.h"
#include "distributed/remote_transaction.h"
#include "distributed/replication_origin_session_utils.h"
#include "distributed/resource_lock.h"
#include "distributed/shard_pruning.h"
#include "distributed/shared_connection_stats.h"
@ -270,7 +271,8 @@ static CopyConnectionState * GetConnectionState(HTAB *connectionStateHash,
static CopyShardState * GetShardState(uint64 shardId, HTAB *shardStateHash,
HTAB *connectionStateHash,
bool *found, bool shouldUseLocalCopy, CopyOutState
copyOutState, bool isColocatedIntermediateResult);
copyOutState, bool isColocatedIntermediateResult,
bool isPublishable);
static MultiConnection * CopyGetPlacementConnection(HTAB *connectionStateHash,
ShardPlacement *placement,
bool colocatedIntermediateResult);
@ -285,7 +287,8 @@ static void InitializeCopyShardState(CopyShardState *shardState,
uint64 shardId,
bool canUseLocalCopy,
CopyOutState copyOutState,
bool colocatedIntermediateResult);
bool colocatedIntermediateResult, bool
isPublishable);
static void StartPlacementStateCopyCommand(CopyPlacementState *placementState,
CopyStmt *copyStatement,
CopyOutState copyOutState);
@ -492,9 +495,11 @@ CopyToExistingShards(CopyStmt *copyStatement, QueryCompletion *completionTag)
ExprContext *executorExpressionContext = GetPerTupleExprContext(executorState);
/* set up the destination for the COPY */
const bool publishableData = true;
CitusCopyDestReceiver *copyDest = CreateCitusCopyDestReceiver(tableId, columnNameList,
partitionColumnIndex,
executorState, NULL);
executorState, NULL,
publishableData);
/* if the user specified an explicit append-to_shard option, write to it */
uint64 appendShardId = ProcessAppendToShardOption(tableId, copyStatement);
@ -1934,7 +1939,7 @@ CopyFlushOutput(CopyOutState cstate, char *start, char *pointer)
CitusCopyDestReceiver *
CreateCitusCopyDestReceiver(Oid tableId, List *columnNameList, int partitionColumnIndex,
EState *executorState,
char *intermediateResultIdPrefix)
char *intermediateResultIdPrefix, bool isPublishable)
{
CitusCopyDestReceiver *copyDest = (CitusCopyDestReceiver *) palloc0(
sizeof(CitusCopyDestReceiver));
@ -1953,6 +1958,7 @@ CreateCitusCopyDestReceiver(Oid tableId, List *columnNameList, int partitionColu
copyDest->executorState = executorState;
copyDest->colocatedIntermediateResultIdPrefix = intermediateResultIdPrefix;
copyDest->memoryContext = CurrentMemoryContext;
copyDest->isPublishable = isPublishable;
return copyDest;
}
@ -2318,7 +2324,9 @@ CitusSendTupleToPlacements(TupleTableSlot *slot, CitusCopyDestReceiver *copyDest
&cachedShardStateFound,
copyDest->shouldUseLocalCopy,
copyDest->copyOutState,
isColocatedIntermediateResult);
isColocatedIntermediateResult,
copyDest->isPublishable);
if (!cachedShardStateFound)
{
firstTupleInShard = true;
@ -2751,6 +2759,11 @@ ShutdownCopyConnectionState(CopyConnectionState *connectionState,
if (activePlacementState != NULL)
{
EndPlacementStateCopyCommand(activePlacementState, copyOutState);
if (!copyDest->isPublishable)
{
ResetReplicationOriginRemoteSession(
activePlacementState->connectionState->connection);
}
}
dlist_foreach(iter, &connectionState->bufferedPlacementList)
@ -2764,6 +2777,10 @@ ShutdownCopyConnectionState(CopyConnectionState *connectionState,
SendCopyDataToPlacement(placementState->data, shardId,
connectionState->connection);
EndPlacementStateCopyCommand(placementState, copyOutState);
if (!copyDest->isPublishable)
{
ResetReplicationOriginRemoteSession(connectionState->connection);
}
}
}
@ -3436,7 +3453,7 @@ static CopyShardState *
GetShardState(uint64 shardId, HTAB *shardStateHash,
HTAB *connectionStateHash, bool *found, bool
shouldUseLocalCopy, CopyOutState copyOutState,
bool isColocatedIntermediateResult)
bool isColocatedIntermediateResult, bool isPublishable)
{
CopyShardState *shardState = (CopyShardState *) hash_search(shardStateHash, &shardId,
HASH_ENTER, found);
@ -3444,7 +3461,8 @@ GetShardState(uint64 shardId, HTAB *shardStateHash,
{
InitializeCopyShardState(shardState, connectionStateHash,
shardId, shouldUseLocalCopy,
copyOutState, isColocatedIntermediateResult);
copyOutState, isColocatedIntermediateResult,
isPublishable);
}
return shardState;
@ -3461,7 +3479,8 @@ InitializeCopyShardState(CopyShardState *shardState,
HTAB *connectionStateHash, uint64 shardId,
bool shouldUseLocalCopy,
CopyOutState copyOutState,
bool colocatedIntermediateResult)
bool colocatedIntermediateResult,
bool isPublishable)
{
ListCell *placementCell = NULL;
int failedPlacementCount = 0;
@ -3532,6 +3551,11 @@ InitializeCopyShardState(CopyShardState *shardState,
RemoteTransactionBeginIfNecessary(connection);
}
if (!isPublishable)
{
SetupReplicationOriginRemoteSession(connection);
}
CopyPlacementState *placementState = palloc0(sizeof(CopyPlacementState));
placementState->shardState = shardState;
placementState->data = makeStringInfo();

View File

@ -1484,6 +1484,7 @@ AfterXactHostConnectionHandling(ConnectionHashEntry *entry, bool isCommit)
* - Current cached connections is already at MaxCachedConnectionsPerWorker
* - Connection is forced to close at the end of transaction
* - Connection is not in OK state
* - Connection has a replication origin setup
* - A transaction is still in progress (usually because we are cancelling a distributed transaction)
* - A connection reached its maximum lifetime
*/
@ -1503,6 +1504,7 @@ ShouldShutdownConnection(MultiConnection *connection, const int cachedConnection
PQstatus(connection->pgConn) != CONNECTION_OK ||
!RemoteTransactionIdle(connection) ||
connection->requiresReplication ||
connection->isReplicationOriginSessionSetup ||
(MaxCachedConnectionLifetime >= 0 &&
MillisecondsToTimeout(connection->connectionEstablishmentStart,
MaxCachedConnectionLifetime) <= 0);

View File

@ -573,6 +573,47 @@ SendRemoteCommand(MultiConnection *connection, const char *command)
}
/*
* ExecuteRemoteCommandAndCheckResult executes the given command in the remote node and
* checks if the result is equal to the expected result. If the result is equal to the
* expected result, the function returns true, otherwise it returns false.
*/
bool
ExecuteRemoteCommandAndCheckResult(MultiConnection *connection, char *command,
char *expected)
{
if (!SendRemoteCommand(connection, command))
{
/* if we cannot connect, we warn and report false */
ReportConnectionError(connection, WARNING);
return false;
}
bool raiseInterrupts = true;
PGresult *queryResult = GetRemoteCommandResult(connection, raiseInterrupts);
/* if remote node throws an error, we also throw an error */
if (!IsResponseOK(queryResult))
{
ReportResultError(connection, queryResult, ERROR);
}
StringInfo queryResultString = makeStringInfo();
/* Evaluate the queryResult and store it into the queryResultString */
bool success = EvaluateSingleQueryResult(connection, queryResult, queryResultString);
bool result = false;
if (success && strcmp(queryResultString->data, expected) == 0)
{
result = true;
}
PQclear(queryResult);
ForgetResults(connection);
return result;
}
/*
* ReadFirstColumnAsText reads the first column of result tuples from the given
* PGresult struct and returns them in a StringInfo list.

View File

@ -409,11 +409,13 @@ ExecutePlanIntoColocatedIntermediateResults(Oid targetRelationId,
columnNameList);
/* set up a DestReceiver that copies into the intermediate table */
const bool publishableData = true;
CitusCopyDestReceiver *copyDest = CreateCitusCopyDestReceiver(targetRelationId,
columnNameList,
partitionColumnIndex,
executorState,
intermediateResultIdPrefix);
intermediateResultIdPrefix,
publishableData);
ExecutePlanIntoDestReceiver(selectPlan, paramListInfo, (DestReceiver *) copyDest);
@ -443,10 +445,12 @@ ExecutePlanIntoRelation(Oid targetRelationId, List *insertTargetList,
columnNameList);
/* set up a DestReceiver that copies into the distributed table */
const bool publishableData = true;
CitusCopyDestReceiver *copyDest = CreateCitusCopyDestReceiver(targetRelationId,
columnNameList,
partitionColumnIndex,
executorState, NULL);
executorState, NULL,
publishableData);
ExecutePlanIntoDestReceiver(selectPlan, paramListInfo, (DestReceiver *) copyDest);

View File

@ -24,6 +24,7 @@
#include "distributed/relation_utils.h"
#include "distributed/version_compat.h"
#include "distributed/local_executor.h"
#include "distributed/replication_origin_session_utils.h"
/*
* LocalCopyBuffer is used in copy callback to return the copied rows.
@ -80,6 +81,7 @@ static void LocalCopyToShard(ShardCopyDestReceiver *copyDest, CopyOutState
localCopyOutState);
static void ConnectToRemoteAndStartCopy(ShardCopyDestReceiver *copyDest);
static bool
CanUseLocalCopy(uint32_t destinationNodeId)
{
@ -103,6 +105,12 @@ ConnectToRemoteAndStartCopy(ShardCopyDestReceiver *copyDest)
NULL /* database (current) */);
ClaimConnectionExclusively(copyDest->connection);
RemoteTransactionBeginIfNecessary(copyDest->connection);
SetupReplicationOriginRemoteSession(copyDest->connection);
StringInfo copyStatement = ConstructShardCopyStatement(
copyDest->destinationShardFullyQualifiedName,
copyDest->copyOutState->binary,
@ -185,6 +193,8 @@ ShardCopyDestReceiverReceive(TupleTableSlot *slot, DestReceiver *dest)
CopyOutState copyOutState = copyDest->copyOutState;
if (copyDest->useLocalCopy)
{
/* Setup replication origin session for local copy*/
WriteLocalTuple(slot, copyDest);
if (copyOutState->fe_msgbuf->len > LocalCopyFlushThresholdByte)
{
@ -260,6 +270,11 @@ ShardCopyDestReceiverStartup(DestReceiver *dest, int operation, TupleDesc
copyDest->columnOutputFunctions = ColumnOutputFunctions(inputTupleDescriptor,
copyOutState->binary);
copyDest->copyOutState = copyOutState;
if (copyDest->useLocalCopy)
{
/* Setup replication origin session for local copy*/
SetupReplicationOriginLocalSession();
}
}
@ -318,6 +333,9 @@ ShardCopyDestReceiverShutdown(DestReceiver *dest)
PQclear(result);
ForgetResults(copyDest->connection);
ResetReplicationOriginRemoteSession(copyDest->connection);
CloseConnection(copyDest->connection);
}
}
@ -330,6 +348,10 @@ static void
ShardCopyDestReceiverDestroy(DestReceiver *dest)
{
ShardCopyDestReceiver *copyDest = (ShardCopyDestReceiver *) dest;
if (copyDest->useLocalCopy)
{
ResetReplicationOriginLocalSession();
}
if (copyDest->copyOutState)
{

View File

@ -89,7 +89,6 @@ int PlannerLevel = 0;
static bool ListContainsDistributedTableRTE(List *rangeTableList,
bool *maybeHasForeignDistributedTable);
static bool IsUpdateOrDeleteOrMerge(Query *query);
static PlannedStmt * CreateDistributedPlannedStmt(
DistributedPlanningContext *planContext);
static PlannedStmt * InlineCtesAndCreateDistributedPlannedStmt(uint64 planId,
@ -609,18 +608,6 @@ IsMultiTaskPlan(DistributedPlan *distributedPlan)
}
/*
* IsUpdateOrDelete returns true if the query performs an update or delete.
*/
bool
IsUpdateOrDeleteOrMerge(Query *query)
{
return query->commandType == CMD_UPDATE ||
query->commandType == CMD_DELETE ||
query->commandType == CMD_MERGE;
}
/*
* PlanFastPathDistributedStmt creates a distributed planned statement using
* the FastPathPlanner.
@ -791,7 +778,7 @@ CreateDistributedPlannedStmt(DistributedPlanningContext *planContext)
* if it is planned as a multi shard modify query.
*/
if ((distributedPlan->planningError ||
(IsUpdateOrDeleteOrMerge(planContext->originalQuery) && IsMultiTaskPlan(
(UpdateOrDeleteOrMergeQuery(planContext->originalQuery) && IsMultiTaskPlan(
distributedPlan))) &&
hasUnresolvedParams)
{

View File

@ -63,11 +63,43 @@ DistributedPlan *
CreateMergePlan(Query *originalQuery, Query *query,
PlannerRestrictionContext *plannerRestrictionContext)
{
/*
* For now, this is a place holder until we isolate the merge
* planning into it's own code-path.
*/
return CreateModifyPlan(originalQuery, query, plannerRestrictionContext);
DistributedPlan *distributedPlan = CitusMakeNode(DistributedPlan);
bool multiShardQuery = false;
Assert(originalQuery->commandType == CMD_MERGE);
distributedPlan->modLevel = RowModifyLevelForQuery(query);
distributedPlan->planningError = MergeQuerySupported(originalQuery,
multiShardQuery,
plannerRestrictionContext);
if (distributedPlan->planningError != NULL)
{
return distributedPlan;
}
Job *job = RouterJob(originalQuery, plannerRestrictionContext,
&distributedPlan->planningError);
if (distributedPlan->planningError != NULL)
{
return distributedPlan;
}
ereport(DEBUG1, (errmsg("Creating MERGE router plan")));
distributedPlan->workerJob = job;
distributedPlan->combineQuery = NULL;
/* MERGE doesn't support RETURNING clause */
distributedPlan->expectResults = false;
distributedPlan->targetRelationId = ResultRelationOidForQuery(query);
distributedPlan->fastPathRouterPlan =
plannerRestrictionContext->fastPathRestrictionContext->fastPathRouterQuery;
return distributedPlan;
}
@ -89,12 +121,6 @@ MergeQuerySupported(Query *originalQuery, bool multiShardQuery,
#else
/* For non-MERGE commands it's a no-op */
if (!IsMergeQuery(originalQuery))
{
return NULL;
}
/*
* TODO: For now, we are adding an exception where any volatile or stable
* functions are not allowed in the MERGE query, but this will become too
@ -596,12 +622,6 @@ MergeQualAndTargetListFunctionsSupported(Oid resultRelationId, FromExpr *joinTre
{
TargetEntry *targetEntry = (TargetEntry *) lfirst(targetEntryCell);
/* skip resjunk entries: UPDATE adds some for ctid, etc. */
if (targetEntry->resjunk)
{
continue;
}
bool targetEntryDistributionColumn = false;
AttrNumber targetColumnAttrNumber = InvalidAttrNumber;

View File

@ -4573,7 +4573,8 @@ RowModifyLevelForQuery(Query *query)
}
if (commandType == CMD_UPDATE ||
commandType == CMD_DELETE)
commandType == CMD_DELETE ||
commandType == CMD_MERGE)
{
return ROW_MODIFY_NONCOMMUTATIVE;
}

View File

@ -114,6 +114,7 @@ typedef struct WalkerState
} WalkerState;
bool EnableRouterExecution = true;
bool EnableNonColocatedRouterQueryPushdown = false;
/* planner functions forward declarations */
@ -140,9 +141,6 @@ static void ErrorIfNoShardsExist(CitusTableCacheEntry *cacheEntry);
static DeferredErrorMessage * DeferErrorIfModifyView(Query *queryTree);
static Job * CreateJob(Query *query);
static Task * CreateTask(TaskType taskType);
static Job * RouterJob(Query *originalQuery,
PlannerRestrictionContext *plannerRestrictionContext,
DeferredErrorMessage **planningError);
static bool RelationPrunesToMultipleShards(List *relationShardList);
static void NormalizeMultiRowInsertTargetList(Query *query);
static void AppendNextDummyColReference(Alias *expendedReferenceNames);
@ -910,14 +908,10 @@ ModifyQuerySupported(Query *queryTree, Query *originalQuery, bool multiShardQuer
PlannerRestrictionContext *plannerRestrictionContext)
{
Oid distributedTableId = InvalidOid;
DeferredErrorMessage *error = MergeQuerySupported(originalQuery, multiShardQuery,
plannerRestrictionContext);
if (error)
{
return error;
}
error = ModifyPartialQuerySupported(queryTree, multiShardQuery, &distributedTableId);
DeferredErrorMessage *error =
ModifyPartialQuerySupported(queryTree, multiShardQuery,
&distributedTableId);
if (error)
{
return error;
@ -982,17 +976,10 @@ ModifyQuerySupported(Query *queryTree, Query *originalQuery, bool multiShardQuer
}
else if (rangeTableEntry->relkind == RELKIND_MATVIEW)
{
if (IsMergeAllowedOnRelation(originalQuery, rangeTableEntry))
{
continue;
}
else
{
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
"materialized views in "
"modify queries are not supported",
NULL, NULL);
}
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
"materialized views in "
"modify queries are not supported",
NULL, NULL);
}
/* for other kinds of relations, check if it's distributed */
else
@ -1087,7 +1074,7 @@ ModifyQuerySupported(Query *queryTree, Query *originalQuery, bool multiShardQuer
}
}
if (commandType != CMD_INSERT && commandType != CMD_MERGE)
if (commandType != CMD_INSERT)
{
DeferredErrorMessage *errorMessage = NULL;
@ -1825,7 +1812,7 @@ ExtractFirstCitusTableId(Query *query)
* RouterJob builds a Job to represent a single shard select/update/delete and
* multiple shard update/delete queries.
*/
static Job *
Job *
RouterJob(Query *originalQuery, PlannerRestrictionContext *plannerRestrictionContext,
DeferredErrorMessage **planningError)
{
@ -2349,9 +2336,20 @@ PlanRouterQuery(Query *originalQuery,
}
Assert(UpdateOrDeleteOrMergeQuery(originalQuery));
planningError = ModifyQuerySupported(originalQuery, originalQuery,
isMultiShardQuery,
plannerRestrictionContext);
if (IsMergeQuery(originalQuery))
{
planningError = MergeQuerySupported(originalQuery,
isMultiShardQuery,
plannerRestrictionContext);
}
else
{
planningError = ModifyQuerySupported(originalQuery, originalQuery,
isMultiShardQuery,
plannerRestrictionContext);
}
if (planningError != NULL)
{
return planningError;
@ -3618,6 +3616,8 @@ DeferErrorIfUnsupportedRouterPlannableSelectQuery(Query *query)
bool hasDistributedTable = false;
bool hasReferenceTable = false;
List *distributedRelationList = NIL;
ExtractRangeTableRelationWalker((Node *) query, &rangeTableRelationList);
foreach(rangeTableRelationCell, rangeTableRelationList)
{
@ -3655,6 +3655,8 @@ DeferErrorIfUnsupportedRouterPlannableSelectQuery(Query *query)
if (IsCitusTableType(distributedTableId, DISTRIBUTED_TABLE))
{
hasDistributedTable = true;
distributedRelationList = lappend_oid(distributedRelationList,
distributedTableId);
}
/*
@ -3709,6 +3711,15 @@ DeferErrorIfUnsupportedRouterPlannableSelectQuery(Query *query)
NULL, NULL);
}
if (!EnableNonColocatedRouterQueryPushdown &&
!AllDistributedRelationsInListColocated(distributedRelationList))
{
return DeferredError(ERRCODE_FEATURE_NOT_SUPPORTED,
"router planner does not support queries that "
"reference non-colocated distributed tables",
NULL, NULL);
}
#if PG_VERSION_NUM >= PG_VERSION_14
DeferredErrorMessage *CTEWithSearchClauseError =
ErrorIfQueryHasCTEWithSearchClause(query);

View File

@ -154,7 +154,6 @@ static Var * PartitionKeyForRTEIdentityInQuery(Query *query, int targetRTEIndex,
static bool AllDistributedRelationsInRestrictionContextColocated(
RelationRestrictionContext *
restrictionContext);
static bool AllDistributedRelationsInListColocated(List *relationList);
static bool IsNotSafeRestrictionToRecursivelyPlan(Node *node);
static JoinRestrictionContext * FilterJoinRestrictionContext(
JoinRestrictionContext *joinRestrictionContext, Relids
@ -1964,7 +1963,7 @@ AllDistributedRelationsInRTEListColocated(List *rangeTableEntryList)
* AllDistributedRelationsInListColocated determines whether all of the
* distributed relations in the given list are co-located.
*/
static bool
bool
AllDistributedRelationsInListColocated(List *relationList)
{
int initialColocationId = INVALID_COLOCATION_ID;

View File

@ -8,21 +8,27 @@
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "distributed/cdc_decoder.h"
#include "distributed/shardinterval_utils.h"
#include "distributed/shardsplit_shared_memory.h"
#include "distributed/worker_shard_visibility.h"
#include "distributed/worker_protocol.h"
#include "distributed/listutils.h"
#include "distributed/metadata/distobject.h"
#include "replication/logical.h"
#include "utils/typcache.h"
#include "utils/lsyscache.h"
#include "catalog/pg_namespace.h"
extern void _PG_output_plugin_init(OutputPluginCallbacks *cb);
static LogicalDecodeChangeCB pgoutputChangeCB;
static LogicalDecodeChangeCB ouputPluginChangeCB;
static HTAB *SourceToDestinationShardMap = NULL;
/* Plugin callback */
static void split_change_cb(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
Relation relation, ReorderBufferChange *change);
static void shard_split_and_cdc_change_cb(LogicalDecodingContext *ctx,
ReorderBufferTXN *txn,
Relation relation, ReorderBufferChange *change);
/* Helper methods */
static int32_t GetHashValueForIncomingTuple(Relation sourceShardRelation,
@ -38,6 +44,22 @@ static HeapTuple GetTupleForTargetSchema(HeapTuple sourceRelationTuple,
TupleDesc sourceTupleDesc,
TupleDesc targetTupleDesc);
inline static bool IsShardSplitSlot(char *replicationSlotName);
#define CITUS_SHARD_SLOT_PREFIX "citus_shard_"
#define CITUS_SHARD_SLOT_PREFIX_SIZE (sizeof(CITUS_SHARD_SLOT_PREFIX) - 1)
/* build time macro for base decoder plugin name for CDC and Shard Split. */
#ifndef CDC_SHARD_SPLIT_BASE_DECODER_PLUGIN_NAME
#define CDC_SHARD_SPLIT_BASE_DECODER_PLUGIN_NAME "pgoutput"
#endif
/* build time macro for base decoder plugin's initialization function name for CDC and Shard Split. */
#ifndef CDC_SHARD_SPLIT_BASE_DECODER_PLUGIN_INIT_FUNCTION_NAME
#define CDC_SHARD_SPLIT_BASE_DECODER_PLUGIN_INIT_FUNCTION_NAME "_PG_output_plugin_init"
#endif
/*
* Postgres uses 'pgoutput' as default plugin for logical replication.
* We want to reuse Postgres pgoutput's functionality as much as possible.
@ -47,9 +69,10 @@ void
_PG_output_plugin_init(OutputPluginCallbacks *cb)
{
LogicalOutputPluginInit plugin_init =
(LogicalOutputPluginInit) (void *) load_external_function("pgoutput",
"_PG_output_plugin_init",
false, NULL);
(LogicalOutputPluginInit) (void *)
load_external_function(CDC_SHARD_SPLIT_BASE_DECODER_PLUGIN_NAME,
CDC_SHARD_SPLIT_BASE_DECODER_PLUGIN_INIT_FUNCTION_NAME,
false, NULL);
if (plugin_init == NULL)
{
@ -60,25 +83,61 @@ _PG_output_plugin_init(OutputPluginCallbacks *cb)
plugin_init(cb);
/* actual pgoutput callback will be called with the appropriate destination shard */
pgoutputChangeCB = cb->change_cb;
cb->change_cb = split_change_cb;
ouputPluginChangeCB = cb->change_cb;
cb->change_cb = shard_split_and_cdc_change_cb;
InitCDCDecoder(cb, ouputPluginChangeCB);
}
/*
* split_change function emits the incoming tuple change
* Check if the replication slot is for Shard split by checking for prefix.
*/
inline static
bool
IsShardSplitSlot(char *replicationSlotName)
{
return strncmp(replicationSlotName, CITUS_SHARD_SLOT_PREFIX,
CITUS_SHARD_SLOT_PREFIX_SIZE) == 0;
}
/*
* shard_split_and_cdc_change_cb function emits the incoming tuple change
* to the appropriate destination shard.
*/
static void
split_change_cb(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
Relation relation, ReorderBufferChange *change)
shard_split_and_cdc_change_cb(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
Relation relation, ReorderBufferChange *change)
{
/*
* If Citus has not been loaded yet, pass the changes
* through to the undrelying decoder plugin.
*/
if (!CitusHasBeenLoaded())
{
ouputPluginChangeCB(ctx, txn, relation, change);
return;
}
/* check if the relation is publishable.*/
if (!is_publishable_relation(relation))
{
return;
}
char *replicationSlotName = ctx->slot->data.name.data;
if (replicationSlotName == NULL)
{
elog(ERROR, "Replication slot name is NULL!");
return;
}
/* check for the internal shard split names, if not, assume the slot is for CDC. */
if (!IsShardSplitSlot(replicationSlotName))
{
PublishDistributedTableChanges(ctx, txn, relation, change);
return;
}
/*
* Initialize SourceToDestinationShardMap if not already initialized.
@ -198,7 +257,7 @@ split_change_cb(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
}
}
pgoutputChangeCB(ctx, txn, targetRelation, change);
ouputPluginChangeCB(ctx, txn, targetRelation, change);
RelationClose(targetRelation);
}

View File

@ -74,6 +74,7 @@
#include "distributed/recursive_planning.h"
#include "distributed/reference_table_utils.h"
#include "distributed/relation_access_tracking.h"
#include "distributed/replication_origin_session_utils.h"
#include "distributed/run_from_same_connection.h"
#include "distributed/shard_cleaner.h"
#include "distributed/shard_transfer.h"
@ -1132,6 +1133,16 @@ RegisterCitusConfigVariables(void)
GUC_STANDARD,
NULL, NULL, NULL);
DefineCustomBoolVariable(
"citus.enable_change_data_capture",
gettext_noop("Enables using replication origin tracking for change data capture"),
NULL,
&EnableChangeDataCapture,
false,
PGC_USERSET,
GUC_STANDARD,
NULL, NULL, NULL);
DefineCustomBoolVariable(
"citus.enable_cluster_clock",
gettext_noop("When users explicitly call UDF citus_get_transaction_clock() "
@ -1268,6 +1279,26 @@ RegisterCitusConfigVariables(void)
GUC_NO_SHOW_ALL,
NULL, NULL, NULL);
DefineCustomBoolVariable(
"citus.enable_non_colocated_router_query_pushdown",
gettext_noop("Enables router planner for the queries that reference "
"non-colocated distributed tables."),
gettext_noop("Normally, router planner planner is only enabled for "
"the queries that reference colocated distributed tables "
"because it is not guaranteed to have the target shards "
"always on the same node, e.g., after rebalancing the "
"shards. For this reason, while enabling this flag allows "
"some degree of optimization for the queries that reference "
"non-colocated distributed tables, it is not guaranteed "
"that the same query will work after rebalancing the shards "
"or altering the shard count of one of those distributed "
"tables."),
&EnableNonColocatedRouterQueryPushdown,
true,
PGC_USERSET,
GUC_NO_SHOW_ALL,
NULL, NULL, NULL);
DefineCustomBoolVariable(
"citus.enable_repartition_joins",
gettext_noop("Allows Citus to repartition data between nodes."),
@ -2406,7 +2437,6 @@ RegisterCitusConfigVariables(void)
GUC_STANDARD,
NULL, NULL, NULL);
/* warn about config items in the citus namespace that are not registered above */
EmitWarningsOnPlaceholders("citus");

View File

@ -1,3 +1,3 @@
-- citus--11.2-1--11.3-1
#include "udfs/repl_origin_helper/11.3-1.sql"
#include "udfs/worker_modify_identity_columns/11.3-1.sql"

View File

@ -1,2 +1,5 @@
-- citus--11.3-1--11.2-1
DROP FUNCTION pg_catalog.citus_internal_start_replication_origin_tracking();
DROP FUNCTION pg_catalog.citus_internal_stop_replication_origin_tracking();
DROP FUNCTION pg_catalog.citus_internal_is_replication_origin_tracking_active();
DROP FUNCTION IF EXISTS pg_catalog.worker_modify_identity_columns(regclass);

View File

@ -0,0 +1,20 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_start_replication_origin_tracking()
RETURNS void
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$citus_internal_start_replication_origin_tracking$$;
COMMENT ON FUNCTION pg_catalog.citus_internal_start_replication_origin_tracking()
IS 'To start replication origin tracking for skipping publishing of duplicated events during internal data movements for CDC';
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_stop_replication_origin_tracking()
RETURNS void
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$citus_internal_stop_replication_origin_tracking$$;
COMMENT ON FUNCTION pg_catalog.citus_internal_stop_replication_origin_tracking()
IS 'To stop replication origin tracking for skipping publishing of duplicated events during internal data movements for CDC';
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_is_replication_origin_tracking_active()
RETURNS boolean
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$citus_internal_is_replication_origin_tracking_active$$;
COMMENT ON FUNCTION pg_catalog.citus_internal_is_replication_origin_tracking_active()
IS 'To check if replication origin tracking is active for skipping publishing of duplicated events during internal data movements for CDC';

View File

@ -0,0 +1,20 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_start_replication_origin_tracking()
RETURNS void
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$citus_internal_start_replication_origin_tracking$$;
COMMENT ON FUNCTION pg_catalog.citus_internal_start_replication_origin_tracking()
IS 'To start replication origin tracking for skipping publishing of duplicated events during internal data movements for CDC';
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_stop_replication_origin_tracking()
RETURNS void
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$citus_internal_stop_replication_origin_tracking$$;
COMMENT ON FUNCTION pg_catalog.citus_internal_stop_replication_origin_tracking()
IS 'To stop replication origin tracking for skipping publishing of duplicated events during internal data movements for CDC';
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_is_replication_origin_tracking_active()
RETURNS boolean
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$citus_internal_is_replication_origin_tracking_active$$;
COMMENT ON FUNCTION pg_catalog.citus_internal_is_replication_origin_tracking_active()
IS 'To check if replication origin tracking is active for skipping publishing of duplicated events during internal data movements for CDC';

View File

@ -34,6 +34,7 @@
#include "distributed/multi_logical_replication.h"
#include "distributed/multi_explain.h"
#include "distributed/repartition_join_execution.h"
#include "distributed/replication_origin_session_utils.h"
#include "distributed/transaction_management.h"
#include "distributed/placement_connection.h"
#include "distributed/relation_access_tracking.h"
@ -391,6 +392,9 @@ CoordinatedTransactionCallback(XactEvent event, void *arg)
ResetGlobalVariables();
ResetRelationAccessHash();
/* Reset any local replication origin session since transaction has been aborted.*/
ResetReplicationOriginLocalSession();
/* empty the CitusXactCallbackContext to ensure we're not leaking memory */
MemoryContextReset(CitusXactCallbackContext);
@ -715,6 +719,8 @@ CoordinatedSubTransactionCallback(SubXactEvent event, SubTransactionId subId,
SetCreateCitusTransactionLevel(0);
}
/* Reset any local replication origin session since subtransaction has been aborted.*/
ResetReplicationOriginLocalSession();
MemoryContextSwitchTo(previousContext);
break;

View File

@ -0,0 +1,239 @@
/*-------------------------------------------------------------------------
*
* replication_origin_session_utils.c
* Functions for managing replication origin session.
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "distributed/replication_origin_session_utils.h"
#include "distributed/remote_commands.h"
#include "distributed/metadata_cache.h"
#include "utils/builtins.h"
#include "miscadmin.h"
static bool IsRemoteReplicationOriginSessionSetup(MultiConnection *connection);
static void SetupMemoryContextResetReplicationOriginHandler(void);
static void SetupReplicationOriginSessionHelper(bool isContexResetSetupNeeded);
static inline bool IsLocalReplicationOriginSessionActive(void);
PG_FUNCTION_INFO_V1(citus_internal_start_replication_origin_tracking);
PG_FUNCTION_INFO_V1(citus_internal_stop_replication_origin_tracking);
PG_FUNCTION_INFO_V1(citus_internal_is_replication_origin_tracking_active);
/*
* This variable is used to remember the replication origin id of the current session
* before resetting it to DoNotReplicateId in SetupReplicationOriginLocalSession.
*/
static RepOriginId OriginalOriginId = InvalidRepOriginId;
/*
* Setting that controls whether replication origin tracking is enabled
*/
bool EnableChangeDataCapture = false;
/* citus_internal_start_replication_origin_tracking starts a new replication origin session
* in the local node. This function is used to avoid publishing the WAL records to the
* replication slot by setting replication origin to DoNotReplicateId in WAL records.
* It remembers the previous replication origin for the current session which will be
* used to reset the replication origin to the previous value when the session ends.
*/
Datum
citus_internal_start_replication_origin_tracking(PG_FUNCTION_ARGS)
{
if (!EnableChangeDataCapture)
{
PG_RETURN_VOID();
}
SetupReplicationOriginSessionHelper(false);
PG_RETURN_VOID();
}
/* citus_internal_stop_replication_origin_tracking ends the current replication origin session
* in the local node. This function is used to reset the replication origin to the
* earlier value of replication origin.
*/
Datum
citus_internal_stop_replication_origin_tracking(PG_FUNCTION_ARGS)
{
ResetReplicationOriginLocalSession();
PG_RETURN_VOID();
}
/* citus_internal_is_replication_origin_tracking_active checks if the current replication origin
* session is active in the local node.
*/
Datum
citus_internal_is_replication_origin_tracking_active(PG_FUNCTION_ARGS)
{
bool result = IsLocalReplicationOriginSessionActive();
PG_RETURN_BOOL(result);
}
/* IsLocalReplicationOriginSessionActive checks if the current replication origin
* session is active in the local node.
*/
inline bool
IsLocalReplicationOriginSessionActive(void)
{
return (replorigin_session_origin == DoNotReplicateId);
}
/*
* SetupMemoryContextResetReplicationOriginHandler registers a callback function
* that resets the replication origin session in case of any error for the current
* memory context.
*/
static void
SetupMemoryContextResetReplicationOriginHandler()
{
MemoryContextCallback *replicationOriginResetCallback = palloc0(
sizeof(MemoryContextCallback));
replicationOriginResetCallback->func =
ResetReplicationOriginLocalSessionCallbackHandler;
replicationOriginResetCallback->arg = NULL;
MemoryContextRegisterResetCallback(CurrentMemoryContext,
replicationOriginResetCallback);
}
/*
* SetupReplicationOriginSessionHelper sets up a new replication origin session in a
* local session. It takes an argument isContexResetSetupNeeded to decide whether
* to register a callback function that resets the replication origin session in case
* of any error for the current memory context.
*/
static void
SetupReplicationOriginSessionHelper(bool isContexResetSetupNeeded)
{
if (!EnableChangeDataCapture)
{
return;
}
OriginalOriginId = replorigin_session_origin;
replorigin_session_origin = DoNotReplicateId;
if (isContexResetSetupNeeded)
{
SetupMemoryContextResetReplicationOriginHandler();
}
}
/*
* SetupReplicationOriginLocalSession sets up a new replication origin session in a
* local session.
*/
void
SetupReplicationOriginLocalSession()
{
SetupReplicationOriginSessionHelper(true);
}
/*
* ResetReplicationOriginLocalSession resets the replication origin session in a
* local node.
*/
void
ResetReplicationOriginLocalSession(void)
{
if (replorigin_session_origin != DoNotReplicateId)
{
return;
}
replorigin_session_origin = OriginalOriginId;
}
/*
* ResetReplicationOriginLocalSessionCallbackHandler is a callback function that
* resets the replication origin session in a local node. This is used to register
* with MemoryContextRegisterResetCallback to reset the replication origin session
* in case of any error for the given memory context.
*/
void
ResetReplicationOriginLocalSessionCallbackHandler(void *arg)
{
ResetReplicationOriginLocalSession();
}
/*
* SetupReplicationOriginRemoteSession sets up a new replication origin session in a
* remote session. The identifier is used to create a unique replication origin name
* for the session in the remote node.
*/
void
SetupReplicationOriginRemoteSession(MultiConnection *connection)
{
if (!EnableChangeDataCapture)
{
return;
}
if (connection != NULL && !IsRemoteReplicationOriginSessionSetup(connection))
{
StringInfo replicationOriginSessionSetupQuery = makeStringInfo();
appendStringInfo(replicationOriginSessionSetupQuery,
"select pg_catalog.citus_internal_start_replication_origin_tracking();");
ExecuteCriticalRemoteCommand(connection,
replicationOriginSessionSetupQuery->data);
connection->isReplicationOriginSessionSetup = true;
}
}
/*
* ResetReplicationOriginRemoteSession resets the replication origin session in a
* remote node.
*/
void
ResetReplicationOriginRemoteSession(MultiConnection *connection)
{
if (connection != NULL && connection->isReplicationOriginSessionSetup)
{
StringInfo replicationOriginSessionResetQuery = makeStringInfo();
appendStringInfo(replicationOriginSessionResetQuery,
"select pg_catalog.citus_internal_stop_replication_origin_tracking();");
ExecuteCriticalRemoteCommand(connection,
replicationOriginSessionResetQuery->data);
connection->isReplicationOriginSessionSetup = false;
}
}
/*
* IsRemoteReplicationOriginSessionSetup checks if the replication origin is setup
* already in the remote session by calliing the UDF
* citus_internal_is_replication_origin_tracking_active(). This is also remembered
* in the connection object to avoid calling the UDF again next time.
*/
static bool
IsRemoteReplicationOriginSessionSetup(MultiConnection *connection)
{
if (connection->isReplicationOriginSessionSetup)
{
return true;
}
StringInfo isReplicationOriginSessionSetupQuery = makeStringInfo();
appendStringInfo(isReplicationOriginSessionSetupQuery,
"SELECT pg_catalog.citus_internal_is_replication_origin_tracking_active()");
bool result =
ExecuteRemoteCommandAndCheckResult(connection,
isReplicationOriginSessionSetupQuery->data,
"t");
connection->isReplicationOriginSessionSetup = result;
return result;
}

View File

@ -0,0 +1,27 @@
/*-------------------------------------------------------------------------
*
* cdc_decoder..h
* Utility functions and declerations for cdc decoder.
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#ifndef CITUS_CDC_DECODER_H
#define CITUS_CDC_DECODER_H
#include "postgres.h"
#include "fmgr.h"
#include "replication/logical.h"
void PublishDistributedTableChanges(LogicalDecodingContext *ctx, ReorderBufferTXN *txn,
Relation relation, ReorderBufferChange *change);
void InitCDCDecoder(OutputPluginCallbacks *cb, LogicalDecodeChangeCB changeCB);
/* used in the replication_origin_filter_cb function. */
#define InvalidRepOriginId 0
#endif /* CITUS_CDC_DECODER_H */

View File

@ -133,6 +133,12 @@ typedef struct CitusCopyDestReceiver
/* if true, should copy to local placements in the current session */
bool shouldUseLocalCopy;
/*
* if true, the data from this dest receiver should be published for CDC clients.
* This is set tot false for internal transfers like shard split/move/rebalance etc.
*/
bool isPublishable;
/*
* Copy into colocated intermediate result. When this is set, the
* COPY assumes there are hypothetical colocated shards to the
@ -161,7 +167,8 @@ extern CitusCopyDestReceiver * CreateCitusCopyDestReceiver(Oid relationId,
List *columnNameList,
int partitionColumnIndex,
EState *executorState,
char *intermediateResultPrefix);
char *intermediateResultPrefix,
bool isPublishable);
extern FmgrInfo * ColumnOutputFunctions(TupleDesc rowDescriptor, bool binaryFormat);
extern bool CanUseBinaryCopyFormat(TupleDesc tupleDescription);
extern bool CanUseBinaryCopyFormatForTargetList(List *targetEntryList);

View File

@ -173,6 +173,9 @@ typedef struct MultiConnection
/* is the connection currently in use, and shouldn't be used by anything else */
bool claimedExclusively;
/* is the replication origin session has already been setup for this connection. */
bool isReplicationOriginSessionSetup;
/*
* Should be used to access/modify metadata. See REQUIRE_METADATA_CONNECTION for
* the details.

View File

@ -28,6 +28,8 @@
extern bool EnableRouterExecution;
extern bool EnableFastPathRouterPlanner;
extern bool EnableNonColocatedRouterQueryPushdown;
extern DistributedPlan * CreateRouterPlan(Query *originalQuery, Query *query,
PlannerRestrictionContext *
plannerRestrictionContext);
@ -112,5 +114,8 @@ extern bool TargetEntryChangesValue(TargetEntry *targetEntry, Var *column,
extern bool MasterIrreducibleExpression(Node *expression, bool *varArgument,
bool *badCoalesce);
extern bool HasDangerousJoinUsing(List *rtableList, Node *jtnode);
extern Job * RouterJob(Query *originalQuery,
PlannerRestrictionContext *plannerRestrictionContext,
DeferredErrorMessage **planningError);
#endif /* MULTI_ROUTER_PLANNER_H */

View File

@ -55,4 +55,5 @@ extern RelationRestrictionContext * FilterRelationRestrictionContext(
Relids
queryRteIdentities);
extern bool AllDistributedRelationsInRTEListColocated(List *rangeTableEntryList);
extern bool AllDistributedRelationsInListColocated(List *relationList);
#endif /* RELATION_RESTRICTION_EQUIVALENCE_H */

View File

@ -48,6 +48,8 @@ extern void ExecuteCriticalRemoteCommand(MultiConnection *connection,
const char *command);
extern void ExecuteRemoteCommandInConnectionList(List *nodeConnectionList,
const char *command);
extern bool ExecuteRemoteCommandAndCheckResult(MultiConnection *connection,
char *command, char *expected);
extern int ExecuteOptionalRemoteCommand(MultiConnection *connection,
const char *command,
PGresult **result);

View File

@ -0,0 +1,31 @@
/*-------------------------------------------------------------------------
*
* replication_origin_utils.h
* Utilities related to replication origin.
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#ifndef REPLICATION_ORIGIN_SESSION_UTILS_H
#define REPLICATION_ORIGIN_SESSION_UTILS_H
#include "postgres.h"
#include "replication/origin.h"
#include "distributed/connection_management.h"
extern void InitializeReplicationOriginSessionUtils(void);
extern void SetupReplicationOriginRemoteSession(MultiConnection *connection);
extern void ResetReplicationOriginRemoteSession(MultiConnection *connection);
extern void SetupReplicationOriginLocalSession(void);
extern void ResetReplicationOriginLocalSession(void);
extern void ResetReplicationOriginLocalSessionCallbackHandler(void *arg);
extern bool EnableChangeDataCapture;
#endif /* REPLICATION_ORIGIN_SESSION_UTILS_H */

45
src/test/cdc/Makefile Normal file
View File

@ -0,0 +1,45 @@
#-------------------------------------------------------------------------
#
# Makefile for src/test/cdc
#
# Test that CDC publication works correctly.
#
#-------------------------------------------------------------------------
subdir = src/test/cdc
top_builddir = ../../..
include $(top_builddir)/Makefile.global
pg_version = $(shell $(PG_CONFIG) --version 2>/dev/null)
pg_whole_version = $(shell echo "$(pg_version)"| sed -e 's/^PostgreSQL \([0-9]*\)\(\.[0-9]*\)\{0,1\}\(.*\)/\1\2/')
pg_major_version = $(shell echo "$(pg_whole_version)"| sed -e 's/^\([0-9]\{2\}\)\(.*\)/\1/')
export pg_major_version
test_path = t/*.pl
# copied from pgxs/Makefile.global to use postgres' abs build dir for pg_regress
ifeq ($(enable_tap_tests),yes)
define citus_prove_installcheck
rm -rf '$(CURDIR)'/tmp_check
$(MKDIR_P) '$(CURDIR)'/tmp_check
cd $(srcdir) && \
TESTDIR='$(CURDIR)' \
PATH="$(bindir):$$PATH" \
PGPORT='6$(DEF_PGPORT)' \
top_builddir='$(CURDIR)/$(top_builddir)' \
PG_REGRESS='$(pgxsdir)/src/test/regress/pg_regress' \
TEMP_CONFIG='$(CURDIR)'/postgresql.conf \
$(PROVE) $(PG_PROVE_FLAGS) $(PROVE_FLAGS) $(if $(PROVE_TESTS),$(PROVE_TESTS),$(test_path))
endef
else
citus_prove_installcheck = @echo "TAP tests not enabled when postgres was compiled"
endif
installcheck:
$(citus_prove_installcheck)
clean distclean maintainer-clean:
rm -rf tmp_check

View File

@ -0,0 +1,2 @@
shared_preload_libraries=citus
shared_preload_libraries='citus'

View File

@ -0,0 +1,98 @@
# Basic CDC test for create_distributed_table
use strict;
use warnings;
use Test::More;
use lib './t';
use cdctestlib;
# Initialize co-ordinator node
my $select_stmt = qq(SELECT * FROM sensors ORDER BY measureid, eventdatetime, measure_data;);
my $result = 0;
### Create the citus cluster with coordinator and two worker nodes
our ($node_coordinator, @workers) = create_citus_cluster(2,"localhost",57636);
our $node_cdc_client = create_node('cdc_client', 0, "localhost", 57639);
# Create the sensors table and ndexes.
my $initial_schema = "
CREATE TABLE sensors(
measureid integer,
eventdatetime timestamptz,
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;";
$node_coordinator->safe_psql('postgres',$initial_schema);
$node_cdc_client->safe_psql('postgres',$initial_schema);
create_cdc_publication_and_slots_for_coordinator($node_coordinator,'sensors');
connect_cdc_client_to_coordinator_publication($node_coordinator, $node_cdc_client);
wait_for_cdc_client_to_catch_up_with_coordinator($node_coordinator);
create_cdc_replication_slots_for_workers(\@workers);
# Distribut the sensors table to worker nodes.
$node_coordinator->safe_psql('postgres',"SELECT create_distributed_table('sensors', 'measureid');");
create_cdc_publication_for_workers(\@workers,'sensors');
connect_cdc_client_to_workers_publication(\@workers, $node_cdc_client);
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
# Insert some data to the sensors table in the coordinator node.
$node_coordinator->safe_psql('postgres',"
INSERT INTO sensors
SELECT i, '2020-01-05', '{}', 11011.10, 'A', 'I <3 Citus'
FROM generate_series(0,10)i;");
# Wait for the data changes to be replicated to the cdc client node.
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC basic test - distributed table insert data');
# Update some data in the sensors table in the coordinator node.
$node_coordinator->safe_psql('postgres',"
UPDATE sensors
SET
eventdatetime=NOW(),
measure_data = jsonb_set(measure_data, '{val}', measureid::text::jsonb , TRUE),
measure_status = CASE
WHEN measureid % 2 = 0
THEN 'y'
ELSE 'n'
END,
measure_comment= 'Comment:' || measureid::text;");
# Wait for the data changes to be replicated to the cdc client node.
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
# Compare the data in the coordinator and cdc client nodes.
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC basic test - distributed table update data');
# Delete some data from the sensors table in the coordinator node.
$node_coordinator->safe_psql('postgres',"
DELETE FROM sensors
WHERE (measureid % 2) = 0;");
# Wait for the data changes to be replicated to the cdc client node.
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
# Compare the data in the coordinator and cdc client nodes.
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC basic test - distributed table delete data');
drop_cdc_client_subscriptions($node_cdc_client,\@workers);
done_testing();

View File

@ -0,0 +1,100 @@
# CDC test for create_distributed_table_concurrently
use strict;
use warnings;
use Test::More;
use lib './t';
use cdctestlib;
# Initialize co-ordinator node
my $select_stmt = qq(SELECT * FROM sensors ORDER BY measureid, eventdatetime, measure_data;);
my $result = 0;
### Create the citus cluster with coordinator and two worker nodes
our ($node_coordinator, @workers) = create_citus_cluster(2,"localhost",57636);
our $node_cdc_client = create_node('cdc_client', 0, "localhost", 57639);
# Creeate the sensors table and ndexes.
my $initial_schema = "
CREATE TABLE sensors(
measureid integer,
eventdatetime timestamptz,
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;";
$node_coordinator->safe_psql('postgres',$initial_schema);
$node_cdc_client->safe_psql('postgres',$initial_schema);
create_cdc_publication_and_slots_for_coordinator($node_coordinator,'sensors');
connect_cdc_client_to_coordinator_publication($node_coordinator, $node_cdc_client);
wait_for_cdc_client_to_catch_up_with_coordinator($node_coordinator);
create_cdc_replication_slots_for_workers(\@workers);
# Distribut the sensors table to worker nodes.
$node_coordinator->safe_psql('postgres',"SELECT create_distributed_table_concurrently('sensors', 'measureid');");
create_cdc_publication_for_workers(\@workers,'sensors');
connect_cdc_client_to_workers_publication(\@workers, $node_cdc_client);
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
# Insert some data to the sensors table in the coordinator node.
$node_coordinator->safe_psql('postgres',"
INSERT INTO sensors
SELECT i, '2020-01-05', '{}', 11011.10, 'A', 'I <3 Citus'
FROM generate_series(0,10)i;");
# Wait for the data changes to be replicated to the cdc client node.
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC test - create_distributed_table_concurrently insert data');
# Update some data in the sensors table in the coordinator node.
$node_coordinator->safe_psql('postgres',"
UPDATE sensors
SET
eventdatetime=NOW(),
measure_data = jsonb_set(measure_data, '{val}', measureid::text::jsonb , TRUE),
measure_status = CASE
WHEN measureid % 2 = 0
THEN 'y'
ELSE 'n'
END,
measure_comment= 'Comment:' || measureid::text;");
# Wait for the data changes to be replicated to the cdc client node.
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
# Compare the data in the coordinator and cdc client nodes.
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC test - create_distributed_table_concurrently update data');
# Delete some data from the sensors table in the coordinator node.
$node_coordinator->safe_psql('postgres',"
DELETE FROM sensors
WHERE (measureid % 2) = 0;");
# Wait for the data changes to be replicated to the cdc client node.
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
# Compare the data in the coordinator and cdc client nodes.
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC test - create_distributed_table_concurrently delete data');
drop_cdc_client_subscriptions($node_cdc_client,\@workers);
done_testing();

View File

@ -0,0 +1,83 @@
# CDC test for inserts during create distributed table concurrently
use strict;
use warnings;
use Test::More;
use lib './t';
use cdctestlib;
use threads;
# Initialize co-ordinator node
our $select_stmt = qq(SELECT * FROM sensors ORDER BY measureid, eventdatetime, measure_data;);
my $add_local_meta_data_stmt = qq(SELECT citus_add_local_table_to_metadata('sensors'););
my $result = 0;
### Create the citus cluster with coordinator and two worker nodes
our ($node_coordinator, @workers) = create_citus_cluster(2,"localhost",57636);
our $node_cdc_client = create_node('cdc_client', 0, "localhost", 57639, "");
# Creeate the sensors table and ndexes.
my $initial_schema = "
CREATE TABLE sensors(
measureid integer,
eventdatetime timestamptz,
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;";
$node_coordinator->safe_psql('postgres',$initial_schema);
$node_coordinator->safe_psql('postgres',$add_local_meta_data_stmt);
$node_cdc_client->safe_psql('postgres',$initial_schema);
create_cdc_publication_and_replication_slots_for_citus_cluster($node_coordinator,\@workers,'sensors');
connect_cdc_client_to_citus_cluster_publications($node_coordinator,\@workers,$node_cdc_client);
#insert data into the sensors table in the coordinator node before distributing the table.
$node_coordinator->safe_psql('postgres',"
INSERT INTO sensors
SELECT i, '2020-01-05', '{}', 11011.10, 'A', 'I <3 Citus'
FROM generate_series(0,10)i;");
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator,\@workers);
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC create_distributed_table - insert data');
sub create_distributed_table_thread() {
$node_coordinator->safe_psql('postgres',"SELECT create_distributed_table_concurrently('sensors', 'measureid');");
}
sub insert_data_into_distributed_table_thread() {
# Insert some data to the sensors table in the coordinator node.
$node_coordinator->safe_psql('postgres',"
INSERT INTO sensors
SELECT i, '2020-01-05', '{}', 11011.10, 'A', 'I <3 Citus'
FROM generate_series(-10,-1)i;");
}
# Create the distributed table concurrently in a separate thread.
my $thr_create = threads->create(\&create_distributed_table_thread);
my $thr_insert = threads->create(\&insert_data_into_distributed_table_thread);
$thr_create->join();
$thr_insert->join();
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator,\@workers);
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC create_distributed_table - insert data');
drop_cdc_client_subscriptions($node_cdc_client,\@workers);
done_testing();

View File

@ -0,0 +1,94 @@
# Schema change CDC test for Citus
use strict;
use warnings;
use Test::More;
use lib './t';
use cdctestlib;
# Initialize co-ordinator node
my $select_stmt = qq(SELECT * FROM sensors ORDER BY measureid, eventdatetime, measure_data;);
my $result = 0;
### Create the citus cluster with coordinator and two worker nodes
our ($node_coordinator, @workers) = create_citus_cluster(2,"localhost",57636);
our $node_cdc_client = create_node('cdc_client', 0, "localhost", 57639);
print("coordinator port: " . $node_coordinator->port() . "\n");
print("worker0 port:" . $workers[0]->port() . "\n");
print("worker1 port:" . $workers[1]->port() . "\n");
print("cdc_client port:" .$node_cdc_client->port() . "\n");
# Creeate the sensors table and ndexes.
my $initial_schema = "
CREATE TABLE sensors(
measureid integer,
eventdatetime timestamptz,
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;";
$node_coordinator->safe_psql('postgres',$initial_schema);
$node_cdc_client->safe_psql('postgres',$initial_schema);
create_cdc_publication_and_slots_for_coordinator($node_coordinator,'sensors');
connect_cdc_client_to_coordinator_publication($node_coordinator, $node_cdc_client);
wait_for_cdc_client_to_catch_up_with_coordinator($node_coordinator);
create_cdc_replication_slots_for_workers(\@workers);
#insert data into the sensors table in the coordinator node before distributing the table.
$node_coordinator->safe_psql('postgres',"
INSERT INTO sensors
SELECT i, '2020-01-05', '{}', 11011.10, 'A', 'I <3 Citus'
FROM generate_series(0,100)i;");
$node_coordinator->safe_psql('postgres',"SET citus.shard_count = 2; SELECT create_distributed_table_concurrently('sensors', 'measureid');");
create_cdc_publication_for_workers(\@workers,'sensors');
connect_cdc_client_to_workers_publication(\@workers, $node_cdc_client);
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC create_distributed_table - schema change before move');
my $shard_to_move = $node_coordinator->safe_psql('postgres',
"SELECT shardid FROM citus_shards ORDER BY shardid LIMIT 1;");
my $host1 = $node_coordinator->safe_psql('postgres',
"SELECT nodename FROM citus_shards ORDER BY shardid LIMIT 1;");
my $port1 = $node_coordinator->safe_psql('postgres',
"SELECT nodeport FROM citus_shards ORDER BY shardid LIMIT 1;");
my $shard_last = $node_coordinator->safe_psql('postgres',
"SELECT shardid FROM citus_shards ORDER BY shardid DESC LIMIT 1;");
my $host2 = $node_coordinator->safe_psql('postgres',
"SELECT nodename FROM citus_shards ORDER BY shardid DESC LIMIT 1;");
my $port2 = $node_coordinator->safe_psql('postgres',
"SELECT nodeport FROM citus_shards ORDER BY shardid DESC LIMIT 1;");
my $move_params = "select citus_move_shard_placement($shard_to_move,'$host1',$port1,'$host2',$port2,'force_logical');";
print("move_params: $move_params\n");
$node_coordinator->safe_psql('postgres',$move_params);
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator,\@workers);
#wait_for_cdc_client_to_catch_up_with_workers(\@workers);
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC create_distributed_table - schema change and move shard');
drop_cdc_client_subscriptions($node_cdc_client,\@workers);
done_testing();

View File

@ -0,0 +1,52 @@
# Basic CDC test for create_distributed_table
use strict;
use warnings;
use Test::More;
use lib './t';
use cdctestlib;
# Initialize co-ordinator node
my $select_stmt = qq(SELECT * FROM sensors ORDER BY measureid, eventdatetime, measure_data;);
my $result = 0;
my $ref_select_stmt = qq(SELECT * FROM reference_table ORDER BY measureid;);
### Create the citus cluster with coordinator and two worker nodes
our ($node_coordinator, @workers) = create_citus_cluster(2,"localhost",57636);
our $node_cdc_client = create_node('cdc_client', 0, "localhost", 57639);
$node_coordinator->safe_psql('postgres',"CREATE TABLE reference_table(measureid integer PRIMARY KEY);");
$node_cdc_client->safe_psql('postgres',"CREATE TABLE reference_table(measureid integer PRIMARY KEY);");
create_cdc_publication_and_slots_for_coordinator($node_coordinator,'reference_table');
connect_cdc_client_to_coordinator_publication($node_coordinator, $node_cdc_client);
wait_for_cdc_client_to_catch_up_with_coordinator($node_coordinator);
# Create the reference table in the coordinator and cdc client nodes.
$node_coordinator->safe_psql('postgres',"SELECT create_reference_table('reference_table');");
create_cdc_replication_slots_for_workers(\@workers);
connect_cdc_client_to_workers_publication(\@workers, $node_cdc_client);
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$ref_select_stmt);
is($result, 1, 'CDC reference taable test 1');
# Insert data to the reference table in the coordinator node.
$node_coordinator->safe_psql('postgres',"INSERT INTO reference_table SELECT i FROM generate_series(0,100)i;");
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$ref_select_stmt);
is($result, 1, 'CDC reference taable test 2');
$node_coordinator->safe_psql('postgres',"INSERT INTO reference_table SELECT i FROM generate_series(101,200)i;");
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$ref_select_stmt);
is($result, 1, 'CDC reference taable test 3');
drop_cdc_client_subscriptions($node_cdc_client,\@workers);
done_testing();

View File

@ -0,0 +1,127 @@
# Schema change CDC test for Citus
use strict;
use warnings;
use Test::More;
use lib './t';
use cdctestlib;
# Initialize co-ordinator node
my $select_stmt = qq(SELECT * FROM sensors ORDER BY measureid, eventdatetime, measure_data;);
my $select_stmt_after_drop = qq(SELECT measureid, eventdatetime, measure_data, measure_status, measure_comment FROM sensors ORDER BY measureid, eventdatetime, measure_data;);
my $result = 0;
### Create the citus cluster with coordinator and two worker nodes
our ($node_coordinator, @workers) = create_citus_cluster(2,"localhost",57636);
our $node_cdc_client = create_node('cdc_client', 0, "localhost", 57639);
print("coordinator port: " . $node_coordinator->port() . "\n");
print("worker0 port:" . $workers[0]->port() . "\n");
print("worker1 port:" . $workers[1]->port() . "\n");
print("cdc_client port:" .$node_cdc_client->port() . "\n");
# Creeate the sensors table and ndexes.
my $initial_schema = "
CREATE TABLE sensors(
measureid integer,
eventdatetime timestamptz,
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;";
$node_coordinator->safe_psql('postgres',$initial_schema);
$node_cdc_client->safe_psql('postgres',$initial_schema);
create_cdc_publication_and_slots_for_coordinator($node_coordinator,'sensors');
connect_cdc_client_to_coordinator_publication($node_coordinator, $node_cdc_client);
wait_for_cdc_client_to_catch_up_with_coordinator($node_coordinator);
#insert data into the sensors table in the coordinator node before distributing the table.
$node_coordinator->safe_psql('postgres',"
INSERT INTO sensors
SELECT i, '2020-01-05', '{}', 11011.10, 'A', 'I <3 Citus'
FROM generate_series(0,100)i;");
$node_coordinator->safe_psql('postgres',"SET citus.shard_count = 2; SELECT create_distributed_table_concurrently('sensors', 'measureid');");
#connect_cdc_client_to_coordinator_publication($node_coordinator, $node_cdc_client);
create_cdc_publication_and_slots_for_workers(\@workers,'sensors');
connect_cdc_client_to_workers_publication(\@workers, $node_cdc_client);
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC create_distributed_table - schema change before move');
$node_coordinator->safe_psql('postgres',"ALTER TABLE sensors DROP COLUMN meaure_quantity;");
my $shard_to_move = $node_coordinator->safe_psql('postgres',
"SELECT shardid FROM citus_shards ORDER BY shardid LIMIT 1;");
my $host1 = $node_coordinator->safe_psql('postgres',
"SELECT nodename FROM citus_shards ORDER BY shardid LIMIT 1;");
my $port1 = $node_coordinator->safe_psql('postgres',
"SELECT nodeport FROM citus_shards ORDER BY shardid LIMIT 1;");
my $shard_last = $node_coordinator->safe_psql('postgres',
"SELECT shardid FROM citus_shards ORDER BY shardid DESC LIMIT 1;");
my $host2 = $node_coordinator->safe_psql('postgres',
"SELECT nodename FROM citus_shards ORDER BY shardid DESC LIMIT 1;");
my $port2 = $node_coordinator->safe_psql('postgres',
"SELECT nodeport FROM citus_shards ORDER BY shardid DESC LIMIT 1;");
my $move_params = "select citus_move_shard_placement($shard_to_move,'$host1',$port1,'$host2',$port2,'force_logical');";
print("move_params: $move_params\n");
$node_coordinator->safe_psql('postgres',$move_params);
$node_coordinator->safe_psql('postgres',"
INSERT INTO sensors
SELECT i, '2020-01-05', '{}', 'A', 'I <3 Citus'
FROM generate_series(-10,-1)i;");
$node_cdc_client->safe_psql('postgres',"ALTER TABLE sensors DROP COLUMN meaure_quantity;");
wait_for_cdc_client_to_catch_up_with_workers(\@workers);
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC create_distributed_table - schema change and move shard');
# Update some data in the sensors table to check the schema change handling logic in CDC decoder.
$node_coordinator->safe_psql('postgres',"
UPDATE sensors
SET
measure_status = CASE
WHEN measureid % 2 = 0
THEN 'y'
ELSE 'n'
END;");
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator,\@workers);
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC create_distributed_table - update data after schema change');
# Update some data in the sensors table to check the schema change handling logic in CDC decoder.
$node_coordinator->safe_psql('postgres',"
DELETE FROM sensors
WHERE
measure_status = 'n';");
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator,\@workers);
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC create_distributed_table - delete data after schem change');
drop_cdc_client_subscriptions($node_cdc_client,\@workers);
done_testing();

View File

@ -0,0 +1,111 @@
# Basic CDC test for create_distributed_table
use strict;
use warnings;
use Test::More;
use lib './t';
use cdctestlib;
# Initialize co-ordinator node
my $select_stmt = qq(SELECT * FROM sensors ORDER BY measureid, eventdatetime, measure_data;);
my $result = 0;
### Create the citus cluster with coordinator and two worker nodes
our ($node_coordinator, @workers) = create_citus_cluster(2,"localhost",57636);
my $command = "SELECT citus_set_node_property('localhost', 57636, 'shouldhaveshards', true);";
$node_coordinator->safe_psql('postgres',$command);
our $node_cdc_client = create_node('cdc_client', 0, "localhost", 57639);
# Create the sensors table and ndexes.
my $initial_schema = "
CREATE TABLE sensors(
measureid integer,
eventdatetime timestamptz,
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;";
$node_coordinator->safe_psql('postgres',$initial_schema);
$node_cdc_client->safe_psql('postgres',$initial_schema);
create_cdc_publication_and_slots_for_coordinator($node_coordinator,'sensors');
connect_cdc_client_to_coordinator_publication($node_coordinator, $node_cdc_client);
wait_for_cdc_client_to_catch_up_with_coordinator($node_coordinator);
create_cdc_replication_slots_for_workers(\@workers);
# Distribut the sensors table to worker nodes.
$node_coordinator->safe_psql('postgres',"SELECT create_distributed_table('sensors', 'measureid');");
create_cdc_publication_for_workers(\@workers,'sensors');
connect_cdc_client_to_workers_publication(\@workers, $node_cdc_client);
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
# Insert some data to the sensors table in the coordinator node.
$node_coordinator->safe_psql('postgres',"
INSERT INTO sensors
SELECT i, '2020-01-05', '{}', 11011.10, 'A', 'I <3 Citus'
FROM generate_series(0,10)i;");
# Wait for the data changes to be replicated to the cdc client node.
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC basic test - distributed table insert data');
# Update some data in the sensors table in the coordinator node.
$node_coordinator->safe_psql('postgres',"
UPDATE sensors
SET
eventdatetime=NOW(),
measure_data = jsonb_set(measure_data, '{val}', measureid::text::jsonb , TRUE),
measure_status = CASE
WHEN measureid % 2 = 0
THEN 'y'
ELSE 'n'
END,
measure_comment= 'Comment:' || measureid::text;");
# Wait for the data changes to be replicated to the cdc client node.
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
# Compare the data in the coordinator and cdc client nodes.
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC basic test - distributed table update data');
# Delete some data from the sensors table in the coordinator node.
$node_coordinator->safe_psql('postgres',"
DELETE FROM sensors
WHERE (measureid % 2) = 0;");
# Wait for the data changes to be replicated to the cdc client node.
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
# Compare the data in the coordinator and cdc client nodes.
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC basic test - distributed table delete data');
$node_coordinator->safe_psql('postgres',"
SELECT undistribute_table('sensors',cascade_via_foreign_keys=>true);");
# Wait for the data changes to be replicated to the cdc client node.
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
# Compare the data in the coordinator and cdc client nodes.
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC basic test - distributed table delete data');
drop_cdc_client_subscriptions($node_cdc_client,\@workers);
done_testing();

View File

@ -0,0 +1,84 @@
# Basic CDC test for create_distributed_table
use strict;
use warnings;
use Test::More;
use lib './t';
use cdctestlib;
# Initialize co-ordinator node
my $select_stmt = qq(SELECT * FROM sensors ORDER BY measureid, eventdatetime, measure_data;);
my $result = 0;
my $citus_config = "
citus.shard_count = 2
citus.shard_replication_factor = 1
";
### Create the citus cluster with coordinator and two worker nodes
our ($node_coordinator, @workers) = create_citus_cluster(1,"localhost",57636, $citus_config);
my $command = "SELECT citus_set_node_property('localhost', 57636, 'shouldhaveshards', true);";
$node_coordinator->safe_psql('postgres',$command);
our $node_cdc_client = create_node('cdc_client', 0, "localhost", 57639);
# Create the sensors table and ndexes.
my $initial_schema = "
CREATE TABLE sensors(
measureid integer,
eventdatetime timestamptz,
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;";
$node_coordinator->safe_psql('postgres',$initial_schema);
$node_cdc_client->safe_psql('postgres',$initial_schema);
create_cdc_publication_and_slots_for_coordinator($node_coordinator,'sensors');
connect_cdc_client_to_coordinator_publication($node_coordinator, $node_cdc_client);
wait_for_cdc_client_to_catch_up_with_coordinator($node_coordinator);
create_cdc_replication_slots_for_workers(\@workers);
# Distribut the sensors table to worker nodes.
$node_coordinator->safe_psql('postgres',"SELECT create_distributed_table('sensors', 'measureid');");
create_cdc_publication_for_workers(\@workers,'sensors');
connect_cdc_client_to_workers_publication(\@workers, $node_cdc_client);
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
# Compare the data in the coordinator and cdc client nodes.
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC split test - distributed table create data');
# Insert some data to the sensors table in the coordinator node.
$node_coordinator->safe_psql('postgres',"
INSERT INTO sensors
SELECT i, '2020-01-05', '{}', 11011.10, 'A', 'I <3 Citus'
FROM generate_series(-100,100)i;");
# Compare the data in the coordinator and cdc client nodes.
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC split test - distributed table insert data');
# Wait for the data changes to be replicated to the cdc client node.
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
$node_coordinator->safe_psql('postgres',"
SELECT citus_split_shard_by_split_points(102008,ARRAY['-50'],ARRAY[1,2], 'block_writes');");
# Compare the data in the coordinator and cdc client nodes.
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC split test - distributed table split data');
drop_cdc_client_subscriptions($node_cdc_client,\@workers);
done_testing();

View File

@ -0,0 +1,84 @@
# Basic CDC test for create_distributed_table
use strict;
use warnings;
use Test::More;
use lib './t';
use cdctestlib;
# Initialize co-ordinator node
my $select_stmt = qq(SELECT * FROM sensors ORDER BY measureid, eventdatetime, measure_data;);
my $result = 0;
my $citus_config = "
citus.shard_count = 2
citus.shard_replication_factor = 1
";
### Create the citus cluster with coordinator and two worker nodes
our ($node_coordinator, @workers) = create_citus_cluster(1,"localhost",57636, $citus_config);
my $command = "SELECT citus_set_node_property('localhost', 57636, 'shouldhaveshards', true);";
$node_coordinator->safe_psql('postgres',$command);
our $node_cdc_client = create_node('cdc_client', 0, "localhost", 57639);
# Create the sensors table and ndexes.
my $initial_schema = "
CREATE TABLE sensors(
measureid integer,
eventdatetime timestamptz,
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;";
$node_coordinator->safe_psql('postgres',$initial_schema);
$node_cdc_client->safe_psql('postgres',$initial_schema);
create_cdc_publication_and_slots_for_coordinator($node_coordinator,'sensors');
connect_cdc_client_to_coordinator_publication($node_coordinator, $node_cdc_client);
wait_for_cdc_client_to_catch_up_with_coordinator($node_coordinator);
create_cdc_replication_slots_for_workers(\@workers);
# Distribut the sensors table to worker nodes.
$node_coordinator->safe_psql('postgres',"SELECT create_distributed_table('sensors', 'measureid');");
create_cdc_publication_for_workers(\@workers,'sensors');
connect_cdc_client_to_workers_publication(\@workers, $node_cdc_client);
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
# Compare the data in the coordinator and cdc client nodes.
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC split test - distributed table create data');
# Insert some data to the sensors table in the coordinator node.
$node_coordinator->safe_psql('postgres',"
INSERT INTO sensors
SELECT i, '2020-01-05', '{}', 11011.10, 'A', 'I <3 Citus'
FROM generate_series(-100,100)i;");
# Compare the data in the coordinator and cdc client nodes.
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC split test - distributed table insert data');
# Wait for the data changes to be replicated to the cdc client node.
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
$node_coordinator->safe_psql('postgres',"
SELECT citus_split_shard_by_split_points(102008,ARRAY['-50'],ARRAY[1,2], 'force_logical');");
# Compare the data in the coordinator and cdc client nodes.
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC split test - distributed table split data');
drop_cdc_client_subscriptions($node_cdc_client,\@workers);
done_testing();

View File

@ -0,0 +1,107 @@
# Basic CDC test for create_distributed_table
use strict;
use warnings;
use Test::More;
use lib './t';
use cdctestlib;
use threads;
# Initialize co-ordinator node
my $select_stmt = qq(SELECT * FROM sensors ORDER BY measureid, eventdatetime, measure_data;);
my $result = 0;
my $citus_config = "
citus.shard_count = 2
citus.shard_replication_factor = 1
";
### Create the citus cluster with coordinator and two worker nodes
our ($node_coordinator, @workers) = create_citus_cluster(1,"localhost",57636, $citus_config);
my $command = "SELECT citus_set_node_property('localhost', 57636, 'shouldhaveshards', true);";
$node_coordinator->safe_psql('postgres',$command);
our $node_cdc_client = create_node('cdc_client', 0, "localhost", 57639);
# Create the sensors table and ndexes.
my $initial_schema = "
CREATE TABLE sensors(
measureid integer,
eventdatetime timestamptz,
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;";
$node_coordinator->safe_psql('postgres',$initial_schema);
$node_cdc_client->safe_psql('postgres',$initial_schema);
create_cdc_publication_and_slots_for_coordinator($node_coordinator,'sensors');
connect_cdc_client_to_coordinator_publication($node_coordinator, $node_cdc_client);
wait_for_cdc_client_to_catch_up_with_coordinator($node_coordinator);
create_cdc_replication_slots_for_workers(\@workers);
# Distribut the sensors table to worker nodes.
$node_coordinator->safe_psql('postgres',"SELECT create_distributed_table('sensors', 'measureid');");
create_cdc_publication_for_workers(\@workers,'sensors');
connect_cdc_client_to_workers_publication(\@workers, $node_cdc_client);
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
# Compare the data in the coordinator and cdc client nodes.
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC split test - distributed table create data');
# Insert some data to the sensors table in the coordinator node.
$node_coordinator->safe_psql('postgres',"
INSERT INTO sensors
SELECT i, '2020-01-05', '{}', 11011.10, 'A', 'I <3 Citus'
FROM generate_series(-100,100)i;");
# Compare the data in the coordinator and cdc client nodes.
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC split test - distributed table insert data');
sub insert_data_into_distributed_table_thread() {
# Insert some data to the sensors table in the coordinator node.
$node_coordinator->safe_psql('postgres',"
INSERT INTO sensors
SELECT i, '2020-01-05', '{}', 11011.10, 'A', 'I <3 Citus'
FROM generate_series(101,200)i;");
}
sub split_distributed_table_thread() {
$node_coordinator->safe_psql('postgres',"
SELECT citus_split_shard_by_split_points(102008,ARRAY['-50'],ARRAY[1,2], 'force_logical');");
}
# Create the distributed table concurrently in a separate thread.
my $thr_create = threads->create(\&split_distributed_table_thread);
# Insert some data to the sensors table in the coordinator node while the table is being distributed.
my $thr_insert = threads->create(\&insert_data_into_distributed_table_thread);
# Wait for the threads to finish.
$thr_create->join();
$thr_insert->join();
# Wait for the data changes to be replicated to the cdc client node.
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
# Compare the data in the coordinator and cdc client nodes.
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC split test - distributed table split data');
drop_cdc_client_subscriptions($node_cdc_client,\@workers);
done_testing();

View File

@ -0,0 +1,95 @@
# Basic CDC test for create_distributed_table
use strict;
use warnings;
use Test::More;
use lib './t';
use cdctestlib;
# Initialize co-ordinator node
my $select_stmt = qq(SELECT * FROM sensors ORDER BY measureid, eventdatetime, measure_data;);
my $add_local_meta_data_stmt = qq(SELECT citus_add_local_table_to_metadata('sensors'););
my $result = 0;
my $citus_config = "
citus.shard_count = 2
citus.shard_replication_factor = 1
";
### Create the citus cluster with coordinator and two worker nodes
our ($node_coordinator, @workers) = create_citus_cluster(1,"localhost",57636, $citus_config);
my $command = "SELECT citus_set_node_property('localhost', 57636, 'shouldhaveshards', true);";
$node_coordinator->safe_psql('postgres',$command);
our $node_cdc_client = create_node('cdc_client', 0, "localhost", 57639);
# Create the sensors table and ndexes.
my $initial_schema = "
CREATE TABLE sensors(
measureid integer,
eventdatetime timestamptz,
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;";
$node_coordinator->safe_psql('postgres',$initial_schema);
$node_coordinator->safe_psql('postgres',$add_local_meta_data_stmt);
$node_cdc_client->safe_psql('postgres',$initial_schema);
create_cdc_publication_and_replication_slots_for_citus_cluster($node_coordinator,\@workers,'sensors');
connect_cdc_client_to_citus_cluster_publications($node_coordinator,\@workers,$node_cdc_client);
# Distribut the sensors table to worker nodes.
$node_coordinator->safe_psql('postgres',"SELECT create_distributed_table_concurrently('sensors', 'measureid');");
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator,\@workers);
# Compare the data in the coordinator and cdc client nodes.
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC split test - distributed table create data');
# Insert some data to the sensors table in the coordinator node.
$node_coordinator->safe_psql('postgres',"
SELECT alter_distributed_table('sensors', shard_count:=6, cascade_to_colocated:=true);");
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
# Compare the data in the coordinator and cdc client nodes.
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC split test - alter distributed table ');
#$node_cdc_client->safe_psql("postgres","alter subscription cdc_subscription refresh publication;");
$node_cdc_client->safe_psql("postgres","alter subscription cdc_subscription_1 refresh publication;");
#Drop the CDC client subscription and recreate them , since the
#alter_distributed_table has changed the Oid of the distributed table.
#So the CDC client has to create Oid to table mappings again for
#CDC to work again.
drop_cdc_client_subscriptions($node_cdc_client,\@workers);
create_cdc_publication_and_replication_slots_for_citus_cluster($node_coordinator,\@workers,'sensors');
connect_cdc_client_to_citus_cluster_publications($node_coordinator,\@workers,$node_cdc_client);
# Insert some data to the sensors table in the coordinator node.
$node_coordinator->safe_psql('postgres',"
INSERT INTO sensors
SELECT i, '2020-01-05', '{}', 11011.10, 'A', 'I <3 Citus'
FROM generate_series(0,10)i;");
# Wait for the data changes to be replicated to the cdc client node.
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC basic test - distributed table insert data');
drop_cdc_client_subscriptions($node_cdc_client,\@workers);
done_testing();

View File

@ -0,0 +1,88 @@
# Basic CDC test for create_distributed_table
use strict;
use warnings;
use Test::More;
use lib './t';
use cdctestlib;
# Initialize co-ordinator node
my $select_stmt = qq(SELECT * FROM sensors ORDER BY measureid, eventdatetime, measure_data;);
my $result = 0;
### Create the citus cluster with coordinator and two worker nodes
our ($node_coordinator, @workers) = create_citus_cluster(2,"localhost",57636);
our $node_cdc_client = create_node('cdc_client', 0, "localhost", 57639);
# Create the sensors table and ndexes.
my $initial_schema = "
CREATE TABLE sensors(
measureid integer,
eventdatetime timestamptz,
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;";
$node_coordinator->safe_psql('postgres',$initial_schema);
$node_cdc_client->safe_psql('postgres',$initial_schema);
create_cdc_publication_and_slots_for_coordinator($node_coordinator,'sensors');
connect_cdc_client_to_coordinator_publication($node_coordinator, $node_cdc_client);
wait_for_cdc_client_to_catch_up_with_coordinator($node_coordinator);
create_cdc_replication_slots_for_workers(\@workers);
# Distribut the sensors table to worker nodes.
$node_coordinator->safe_psql('postgres',"SELECT create_distributed_table('sensors', 'measureid');");
create_cdc_publication_for_workers(\@workers,'sensors');
connect_cdc_client_to_workers_publication(\@workers, $node_cdc_client);
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC restart test - distributed table creation');
# Insert some data to the sensors table in the coordinator node.
$node_coordinator->safe_psql('postgres',"
INSERT INTO sensors
SELECT i, '2020-01-05', '{}', 11011.10, 'A', 'I <3 Citus'
FROM generate_series(0,10)i;");
# Wait for the data changes to be replicated to the cdc client node.
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC restart test - distributed table insert data');
print("stopping worker 0");
$workers[0]->stop();
print("starting worker 0 againg..");
$workers[0]->start();
wait_for_cdc_client_to_catch_up_with_citus_cluster($node_coordinator, \@workers);
# Insert some data to the sensors table in the coordinator node.
$node_coordinator->safe_psql('postgres',"
INSERT INTO sensors
SELECT i, '2020-01-05', '{}', 11011.10, 'A', 'I <3 Citus'
FROM generate_series(11,20)i;");
$result = compare_tables_in_different_nodes($node_coordinator,$node_cdc_client,'postgres',$select_stmt);
is($result, 1, 'CDC restart test - distributed table after restart');
drop_cdc_client_subscriptions($node_cdc_client,\@workers);
done_testing();

View File

@ -0,0 +1,355 @@
use strict;
use warnings;
my $pg_major_version = int($ENV{'pg_major_version'});
print("working with PG major version : $pg_major_version\n");
if ($pg_major_version >= 15) {
eval "use PostgreSQL::Test::Cluster";
eval "use PostgreSQL::Test::Utils";
} else {
eval "use PostgresNode";
}
#use PostgresNode;
use DBI;
our $NODE_TYPE_COORDINATOR = 1;
our $NODE_TYPE_WORKER = 2;
our $NODE_TYPE_CDC_CLIENT = 3;
sub compare_tables_in_different_nodes
{
my $result = 1;
my ($node1, $node2, $dbname, $stmt) = @_;
# Connect to the first database node
my $dbh1 = DBI->connect("dbi:Pg:" . $node1->connstr($dbname));
# Connect to the second database node
my $dbh2 = DBI->connect("dbi:Pg:" . $node2->connstr($dbname));
# Define the SQL query for the first database node
my $sth1 = $dbh1->prepare($stmt);
$sth1->execute();
# Define the SQL query for the second database node
my $sth2 = $dbh2->prepare($stmt);
$sth2->execute();
# Get the field names for the table
my @field_names = @{$sth2->{NAME}};
#$sth1->dump_results();
#$sth2->dump_results();
our @row1, our @row2;
# Use a cursor to iterate over the first database node's data
while (1) {
@row1 = $sth1->fetchrow_array();
@row2 = $sth2->fetchrow_array();
#print("row1: @row1\n");
#print("row2: @row2\n");
# Use a cursor to iterate over the second database node's data
if (@row1 and @row2) {
#print("row1: @row1\n");
#print("row2: @row2\n");
my $field_count_row1 = scalar @row1;
my $field_count_row2 = scalar @row2;
if ($field_count_row1 != $field_count_row2) {
print "Field count mismatch: $field_count_row1 != $field_count_row2 \n";
print "First row: @row1\n";
#print "Second row: @row2\n";
for (my $i = 0; $i < scalar @row2; $i++) {
print("Field $i, field name: $field_names[$i], value: $row2[$i] \n");
}
$result = 0;
last;
}
# Compare the data in each field in each row of the two nodes
for (my $i = 0; $i < scalar @row1; $i++) {
if ($row1[$i] ne $row2[$i]) {
print "Data mismatch in field '$field_names[$i]'\n";
print "$row1[$i] != $row2[$i]\n";
print "First row: @row1\n";
print "Second row: @row2\n";
$result = 0;
last;
}
}
} elsif (@row1 and !@row2) {
print "First node has more rows than the second node\n";
$result = 0;
last;
} elsif (!@row1 and @row2) {
print "Second node has more rows than the first node\n";
$result = 0;
last;
} else {
last;
}
}
$sth1->finish();
$sth2->finish();
$dbh1->disconnect();
$dbh2->disconnect();
return $result;
}
sub create_node {
my ($name,$node_type,$host, $port, $config) = @_;
if (!defined($config)) {
$config = ""
}
our $node;
if ($pg_major_version >= 15) {
$PostgreSQL::Test::Cluster::use_unix_sockets = 0;
$PostgreSQL::Test::Cluster::use_tcp = 1;
$PostgreSQL::Test::Cluster::test_pghost = 'localhost';
my %params = ( "port" => $port, "host" => "localhost");
$node = PostgreSQL::Test::Cluster->new($name, %params);
} else {
$PostgresNode::use_tcp = 1;
$PostgresNode::test_pghost = '127.0.0.1';
my %params = ( "port" => $port, "host" => "localhost");
$node = get_new_node($name, %params);
}
print("node's port:" . $node->port . "\n");
$port += 1;
my $citus_config_options = "
max_connections = 100
max_wal_senders = 100
max_replication_slots = 100
citus.enable_change_data_capture = on
log_statement = 'all'
citus.override_table_visibility = off";
if ($config ne "") {
$citus_config_options = $citus_config_options . $config
}
my $client_config_options = "
max_connections = 100
max_wal_senders = 100
max_replication_slots = 100
";
$node->init(allows_streaming => 'logical');
if ($node_type == $NODE_TYPE_COORDINATOR || $node_type == $NODE_TYPE_WORKER) {
$node->append_conf("postgresql.conf",$citus_config_options);
} else {
$node->append_conf("postgresql.conf",$citus_config_options);
}
$node->start();
if ($node_type == $NODE_TYPE_COORDINATOR || $node_type == $NODE_TYPE_WORKER) {
$node->safe_psql('postgres', "CREATE EXTENSION citus;");
my $value = $node->safe_psql('postgres', "SHOW citus.enable_change_data_capture;");
print("citus.enable_change_data_capture value is $value\n")
}
return $node;
}
# Create a Citus cluster with the given number of workers
sub create_citus_cluster {
my ($no_workers,$host,$port,$citus_config) = @_;
my @workers = ();
my $node_coordinator;
print("citus_config :", $citus_config);
if ($citus_config ne "") {
$node_coordinator = create_node('coordinator', $NODE_TYPE_COORDINATOR,$host, $port, $citus_config);
} else {
$node_coordinator = create_node('coordinator', $NODE_TYPE_COORDINATOR,$host, $port);
}
my $coord_host = $node_coordinator->host();
my $coord_port = $node_coordinator->port();
$node_coordinator->safe_psql('postgres',"SELECT pg_catalog.citus_set_coordinator_host('$coord_host', $coord_port);");
for (my $i = 0; $i < $no_workers; $i++) {
$port = $port + 1;
my $node_worker;
if ($citus_config ne "") {
$node_worker = create_node("worker$i", $NODE_TYPE_WORKER,"localhost", $port, $citus_config);
} else {
$node_worker = create_node("worker$i", $NODE_TYPE_WORKER,"localhost", $port);
}
my $node_worker_host = $node_worker->host();
my $node_worker_port = $node_worker->port();
$node_coordinator->safe_psql('postgres',"SELECT pg_catalog.citus_add_node('$node_worker_host', $node_worker_port);");
push @workers, $node_worker;
}
return $node_coordinator, @workers;
}
sub create_cdc_publication_and_replication_slots_for_citus_cluster {
my $node_coordinator = $_[0];
my $workersref = $_[1];
my $table_names = $_[2];
create_cdc_publication_and_slots_for_coordinator($node_coordinator, $table_names);
create_cdc_publication_and_slots_for_workers($workersref, $table_names);
}
sub create_cdc_publication_and_slots_for_coordinator {
my $node_coordinator = $_[0];
my $table_names = $_[1];
print("node node_coordinator connstr: \n" . $node_coordinator->connstr());
my $pub = $node_coordinator->safe_psql('postgres',"SELECT * FROM pg_publication WHERE pubname = 'cdc_publication';");
if ($pub ne "") {
$node_coordinator->safe_psql('postgres',"DROP PUBLICATION IF EXISTS cdc_publication;");
}
$node_coordinator->safe_psql('postgres',"CREATE PUBLICATION cdc_publication FOR TABLE $table_names;");
$node_coordinator->safe_psql('postgres',"SELECT pg_catalog.pg_create_logical_replication_slot('cdc_replication_slot','citus',false,false)");
}
sub create_cdc_publication_and_slots_for_workers {
my $workersref = $_[0];
my $table_names = $_[1];
create_cdc_publication_for_workers($workersref, $table_names);
create_cdc_replication_slots_for_workers($workersref);
}
sub create_cdc_publication_for_workers {
my $workersref = $_[0];
my $table_names = $_[1];
for (@$workersref) {
my $pub = $_->safe_psql('postgres',"SELECT * FROM pg_publication WHERE pubname = 'cdc_publication';");
if ($pub ne "") {
$_->safe_psql('postgres',"DROP PUBLICATION IF EXISTS cdc_publication;");
}
if ($table_names eq "all") {
$_->safe_psql('postgres',"CREATE PUBLICATION cdc_publication FOR ALL TABLES;");
} else {
$_->safe_psql('postgres',"CREATE PUBLICATION cdc_publication FOR TABLE $table_names;");
}
}
}
sub create_cdc_replication_slots_for_workers {
my $workersref = $_[0];
for (@$workersref) {
my $slot = $_->safe_psql('postgres',"select * from pg_replication_slots where slot_name = 'cdc_replication_slot';");
if ($slot ne "") {
$_->safe_psql('postgres',"SELECT pg_catalog.pg_drop_replication_slot('cdc_replication_slot');");
}
$_->safe_psql('postgres',"SELECT pg_catalog.pg_create_logical_replication_slot('cdc_replication_slot','citus',false,true)");
}
}
sub connect_cdc_client_to_citus_cluster_publications {
my $node_coordinator = $_[0];
my $workersref = $_[1];
my $node_cdc_client = $_[2];
my $num_args = scalar(@_);
if ($num_args > 3) {
my $copy_arg = $_[3];
connect_cdc_client_to_coordinator_publication($node_coordinator,$node_cdc_client, $copy_arg);
} else {
connect_cdc_client_to_coordinator_publication($node_coordinator,$node_cdc_client);
}
connect_cdc_client_to_workers_publication($workersref, $node_cdc_client);
}
sub connect_cdc_client_to_coordinator_publication {
my $node_coordinator = $_[0];
my $node_cdc_client = $_[1];
my $num_args = scalar(@_);
my $copy_data = "";
if ($num_args > 2) {
my $copy_arg = $_[2];
$copy_data = 'copy_data='. $copy_arg;
} else {
$copy_data = 'copy_data=false';
}
my $conn_str = $node_coordinator->connstr() . " dbname=postgres";
my $subscription = 'cdc_subscription';
print "creating subscription $subscription for coordinator: $conn_str\n";
$node_cdc_client->safe_psql('postgres',"
CREATE SUBSCRIPTION $subscription
CONNECTION '$conn_str'
PUBLICATION cdc_publication
WITH (
create_slot=false,
enabled=true,
slot_name=cdc_replication_slot,"
. $copy_data. ");"
);
}
sub connect_cdc_client_to_workers_publication {
my $workersref = $_[0];
my $node_cdc_client = $_[1];
my $i = 1;
for (@$workersref) {
my $conn_str = $_->connstr() . " dbname=postgres";
my $subscription = 'cdc_subscription_' . $i;
print "creating subscription $subscription for node$i: $conn_str\n";
my $subscription_stmt = "CREATE SUBSCRIPTION $subscription
CONNECTION '$conn_str'
PUBLICATION cdc_publication
WITH (
create_slot=false,
enabled=true,
slot_name=cdc_replication_slot,
copy_data=false);
";
$node_cdc_client->safe_psql('postgres',$subscription_stmt);
$i++;
}
}
sub wait_for_cdc_client_to_catch_up_with_citus_cluster {
my $node_coordinator = $_[0];
my ($workersref) = $_[1];
my $subscription = 'cdc_subscription';
print "coordinator: waiting for cdc client subscription $subscription to catch up\n";
$node_coordinator->wait_for_catchup($subscription);
wait_for_cdc_client_to_catch_up_with_workers($workersref);
}
sub wait_for_cdc_client_to_catch_up_with_coordinator {
my $node_coordinator = $_[0];
my $subscription = 'cdc_subscription';
print "coordinator: waiting for cdc client subscription $subscription to catch up\n";
$node_coordinator->wait_for_catchup($subscription);
}
sub wait_for_cdc_client_to_catch_up_with_workers {
my ($workersref) = $_[0];
my $i = 1;
for (@$workersref) {
my $subscription = 'cdc_subscription_' . $i;
print "node$i: waiting for cdc client subscription $subscription to catch up\n";
$_->wait_for_catchup($subscription);
$i++;
}
}
sub drop_cdc_client_subscriptions {
my $node = $_[0];
my ($workersref) = $_[1];
$node->safe_psql('postgres',"drop subscription cdc_subscription");
my $i = 1;
for (@$workersref) {
my $subscription = 'cdc_subscription_' . $i;
$node->safe_psql('postgres',"drop subscription " . $subscription);
$i++;
}
}

View File

@ -1,4 +1,5 @@
#!/usr/bin/env python3
from __future__ import annotations
import argparse
import os

View File

@ -0,0 +1,147 @@
Parsed test spec with 2 sessions
starting permutation: s1-begin s1-upd-ins s2-result s1-commit s2-result
step s1-begin: BEGIN;
step s1-upd-ins: MERGE INTO prept USING preps ON prept.t1 = preps.s1
WHEN MATCHED THEN UPDATE SET t2 = t2 + 1
WHEN NOT MATCHED THEN INSERT VALUES(s1, s2);
step s2-result: SELECT * FROM prept ORDER BY 1;
t1|t2
---------------------------------------------------------------------
100| 0
(1 row)
step s1-commit: COMMIT;
step s2-result: SELECT * FROM prept ORDER BY 1;
t1|t2
---------------------------------------------------------------------
100| 1
200| 0
(2 rows)
starting permutation: s1-begin s1-upd-ins s2-begin s2-upd-del s1-commit s2-commit s2-result
step s1-begin: BEGIN;
step s1-upd-ins: MERGE INTO prept USING preps ON prept.t1 = preps.s1
WHEN MATCHED THEN UPDATE SET t2 = t2 + 1
WHEN NOT MATCHED THEN INSERT VALUES(s1, s2);
step s2-begin: BEGIN;
step s2-upd-del: MERGE INTO prept USING preps ON prept.t1 = preps.s1
WHEN MATCHED AND prept.t2 = 0 THEN DELETE
WHEN MATCHED THEN UPDATE SET t2 = t2 + 1;
<waiting ...>
step s1-commit: COMMIT;
step s2-upd-del: <... completed>
step s2-commit: COMMIT;
step s2-result: SELECT * FROM prept ORDER BY 1;
t1|t2
---------------------------------------------------------------------
100| 2
(1 row)
starting permutation: s2-begin s2-upd-del s1-begin s1-upd-ins s2-commit s1-commit s2-result
step s2-begin: BEGIN;
step s2-upd-del: MERGE INTO prept USING preps ON prept.t1 = preps.s1
WHEN MATCHED AND prept.t2 = 0 THEN DELETE
WHEN MATCHED THEN UPDATE SET t2 = t2 + 1;
step s1-begin: BEGIN;
step s1-upd-ins: MERGE INTO prept USING preps ON prept.t1 = preps.s1
WHEN MATCHED THEN UPDATE SET t2 = t2 + 1
WHEN NOT MATCHED THEN INSERT VALUES(s1, s2);
<waiting ...>
step s2-commit: COMMIT;
step s1-upd-ins: <... completed>
step s1-commit: COMMIT;
step s2-result: SELECT * FROM prept ORDER BY 1;
t1|t2
---------------------------------------------------------------------
100| 0
200| 0
(2 rows)
starting permutation: s1-begin s1-upd-ins s2-begin s2-upd s1-commit s2-commit s2-result
step s1-begin: BEGIN;
step s1-upd-ins: MERGE INTO prept USING preps ON prept.t1 = preps.s1
WHEN MATCHED THEN UPDATE SET t2 = t2 + 1
WHEN NOT MATCHED THEN INSERT VALUES(s1, s2);
step s2-begin: BEGIN;
step s2-upd: MERGE INTO prept USING preps ON prept.t1 = preps.s1
WHEN MATCHED THEN UPDATE SET t2 = t2 + 1;
<waiting ...>
step s1-commit: COMMIT;
step s2-upd: <... completed>
step s2-commit: COMMIT;
step s2-result: SELECT * FROM prept ORDER BY 1;
t1|t2
---------------------------------------------------------------------
100| 2
200| 1
(2 rows)
starting permutation: s2-begin s2-ins s1-begin s1-del s2-upd s2-result s2-commit s1-commit s2-result
step s2-begin: BEGIN;
step s2-ins: MERGE INTO prept USING preps ON prept.t1 = preps.s1
WHEN NOT MATCHED THEN INSERT VALUES(s1, s2);
step s1-begin: BEGIN;
step s1-del: MERGE INTO prept USING preps ON prept.t1 = preps.s1
WHEN MATCHED THEN DELETE;
<waiting ...>
step s2-upd: MERGE INTO prept USING preps ON prept.t1 = preps.s1
WHEN MATCHED THEN UPDATE SET t2 = t2 + 1;
step s2-result: SELECT * FROM prept ORDER BY 1;
t1|t2
---------------------------------------------------------------------
100| 1
200| 1
(2 rows)
step s2-commit: COMMIT;
step s1-del: <... completed>
step s1-commit: COMMIT;
step s2-result: SELECT * FROM prept ORDER BY 1;
t1|t2
---------------------------------------------------------------------
(0 rows)
starting permutation: s1-begin s1-del-ins s2-begin s2-upd s1-result s1-ins s1-commit s2-upd s2-commit s2-result
step s1-begin: BEGIN;
step s1-del-ins: MERGE INTO prept USING preps ON prept.t1 = preps.s1
WHEN MATCHED THEN DELETE
WHEN NOT MATCHED THEN INSERT VALUES(s1, s2);
step s2-begin: BEGIN;
step s2-upd: MERGE INTO prept USING preps ON prept.t1 = preps.s1
WHEN MATCHED THEN UPDATE SET t2 = t2 + 1;
<waiting ...>
step s1-result: SELECT * FROM prept ORDER BY 1;
t1|t2
---------------------------------------------------------------------
200| 0
(1 row)
step s1-ins: MERGE INTO prept USING preps ON prept.t1 = preps.s1
WHEN NOT MATCHED THEN INSERT VALUES(s1, s2);
step s1-commit: COMMIT;
step s2-upd: <... completed>
step s2-upd: MERGE INTO prept USING preps ON prept.t1 = preps.s1
WHEN MATCHED THEN UPDATE SET t2 = t2 + 1;
step s2-commit: COMMIT;
step s2-result: SELECT * FROM prept ORDER BY 1;
t1|t2
---------------------------------------------------------------------
100| 2
200| 2
(2 rows)

View File

@ -0,0 +1,5 @@
Parsed test spec with 2 sessions
starting permutation: s1-begin s1-upd-ins s2-result s1-commit s2-result
setup failed: ERROR: MERGE is not supported on PG versions below 15
CONTEXT: PL/pgSQL function inline_code_block line XX at RAISE

View File

@ -0,0 +1,26 @@
Parsed test spec with 2 sessions
starting permutation: s1-begin s1-upd-ins s2-begin s2-update s1-commit s2-commit s1-result s2-result
step s1-begin: BEGIN;
step s1-upd-ins: MERGE INTO prept USING preps ON prept.t1 = preps.s1
WHEN MATCHED THEN UPDATE SET t2 = t2 + 1
WHEN NOT MATCHED THEN INSERT VALUES(s1, s2);
step s2-begin: BEGIN;
step s2-update: UPDATE preps SET s2 = s2 + 1; <waiting ...>
step s1-commit: COMMIT;
step s2-update: <... completed>
step s2-commit: COMMIT;
step s1-result: SELECT * FROM preps ORDER BY 1;
s1|s2
---------------------------------------------------------------------
100| 1
200| 1
(2 rows)
step s2-result: SELECT * FROM prept ORDER BY 1;
t1|t2
---------------------------------------------------------------------
100| 1
200| 0
(2 rows)

View File

@ -0,0 +1,5 @@
Parsed test spec with 2 sessions
starting permutation: s1-begin s1-upd-ins s2-begin s2-update s1-commit s2-commit s1-result s2-result
setup failed: ERROR: MERGE is not supported on PG versions below 15
CONTEXT: PL/pgSQL function inline_code_block line XX at RAISE

View File

@ -1072,6 +1072,7 @@ WHEN MATCHED THEN
UPDATE SET value = vl_source.value, id = vl_target.id + 1
WHEN NOT MATCHED THEN
INSERT VALUES(vl_source.ID, vl_source.value);
DEBUG: Creating MERGE router plan
DEBUG: <Deparsed MERGE query: MERGE INTO merge_schema.vl_target_xxxxxxx vl_target USING (SELECT vl.id, vl.value FROM (VALUES (100,'source1'::text), (200,'source2'::text)) vl(id, value)) vl_source ON (vl_source.id OPERATOR(pg_catalog.=) vl_target.id) WHEN MATCHED THEN UPDATE SET id = (vl_target.id OPERATOR(pg_catalog.+) 1), value = (vl_source.value COLLATE "default") WHEN NOT MATCHED THEN INSERT (id, value) VALUES (vl_source.id, (vl_source.value COLLATE "default"))>
RESET client_min_messages;
SELECT * INTO vl_local FROM vl_target ORDER BY 1 ;
@ -1125,6 +1126,7 @@ WHEN MATCHED THEN
DO NOTHING
WHEN NOT MATCHED THEN
INSERT VALUES(rs_source.id);
DEBUG: Creating MERGE router plan
DEBUG: <Deparsed MERGE query: MERGE INTO merge_schema.rs_target_xxxxxxx rs_target USING (SELECT id.id FROM merge_schema.f_immutable(99) id(id) WHERE (id.id OPERATOR(pg_catalog.=) ANY (SELECT 99))) rs_source ON (rs_source.id OPERATOR(pg_catalog.=) rs_target.id) WHEN MATCHED THEN DO NOTHING WHEN NOT MATCHED THEN INSERT (id) VALUES (rs_source.id)>
RESET client_min_messages;
SELECT * INTO rs_local FROM rs_target ORDER BY 1 ;
@ -1255,6 +1257,7 @@ WHEN MATCHED THEN
DO NOTHING
WHEN NOT MATCHED THEN
INSERT VALUES(fn_source.id, fn_source.source);
DEBUG: Creating MERGE router plan
DEBUG: <Deparsed MERGE query: MERGE INTO merge_schema.fn_target_xxxxxxx fn_target USING (SELECT dist_table.id, dist_table.source FROM merge_schema.dist_table_xxxxxxx dist_table) fn_source ON (fn_source.id OPERATOR(pg_catalog.=) fn_target.id) WHEN MATCHED THEN DO NOTHING WHEN NOT MATCHED THEN INSERT (id, data) VALUES (fn_source.id, fn_source.source)>
RESET client_min_messages;
SELECT * INTO fn_local FROM fn_target ORDER BY 1 ;
@ -1327,6 +1330,7 @@ MERGE INTO ft_target
DELETE
WHEN NOT MATCHED THEN
INSERT (id, user_val) VALUES (foreign_table.id, foreign_table.user_val);
DEBUG: Creating MERGE router plan
DEBUG: <Deparsed MERGE query: MERGE INTO merge_schema.ft_target USING merge_schema.foreign_table_xxxxxxx foreign_table ON (foreign_table.id OPERATOR(pg_catalog.=) ft_target.id) WHEN MATCHED THEN DELETE WHEN NOT MATCHED THEN INSERT (id, user_val) VALUES (foreign_table.id, (foreign_table.user_val COLLATE "default"))>
RESET client_min_messages;
SELECT * FROM ft_target;
@ -1557,7 +1561,9 @@ DEBUG: <Deparsed MERGE query: MERGE INTO merge_schema.target_cj_xxxxxxx target
DEBUG: <Deparsed MERGE query: MERGE INTO merge_schema.target_cj_xxxxxxx target USING (SELECT source_cj1.sid1, source_cj1.src1, source_cj1.val1 FROM merge_schema.source_cj1_xxxxxxx source_cj1) sub ON ((target.tid OPERATOR(pg_catalog.=) sub.sid1) AND (target.tid OPERATOR(pg_catalog.=) $1)) WHEN MATCHED THEN UPDATE SET val = sub.val1 WHEN NOT MATCHED THEN DO NOTHING >
DEBUG: <Deparsed MERGE query: MERGE INTO merge_schema.target_cj_xxxxxxx target USING (SELECT source_cj1.sid1, source_cj1.src1, source_cj1.val1 FROM merge_schema.source_cj1_xxxxxxx source_cj1) sub ON ((target.tid OPERATOR(pg_catalog.=) sub.sid1) AND (target.tid OPERATOR(pg_catalog.=) $1)) WHEN MATCHED THEN UPDATE SET val = sub.val1 WHEN NOT MATCHED THEN DO NOTHING >
DEBUG: <Deparsed MERGE query: MERGE INTO merge_schema.target_cj_xxxxxxx target USING (SELECT source_cj1.sid1, source_cj1.src1, source_cj1.val1 FROM merge_schema.source_cj1_xxxxxxx source_cj1) sub ON ((target.tid OPERATOR(pg_catalog.=) sub.sid1) AND (target.tid OPERATOR(pg_catalog.=) $1)) WHEN MATCHED THEN UPDATE SET val = sub.val1 WHEN NOT MATCHED THEN DO NOTHING >
DEBUG: Creating MERGE router plan
DEBUG: <Deparsed MERGE query: MERGE INTO merge_schema.target_cj_xxxxxxx target USING (SELECT source_cj1.sid1, source_cj1.src1, source_cj1.val1 FROM merge_schema.source_cj1_xxxxxxx source_cj1) sub ON ((target.tid OPERATOR(pg_catalog.=) sub.sid1) AND (target.tid OPERATOR(pg_catalog.=) $1)) WHEN MATCHED THEN UPDATE SET val = sub.val1 WHEN NOT MATCHED THEN DO NOTHING >
DEBUG: Creating MERGE router plan
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing MERGE INTO merge_schema.target_cj_xxxxxxx target USING (SELECT source_cj1.sid1, source_cj1.src1, source_cj1.val1 FROM merge_schema.source_cj1_xxxxxxx source_cj1) sub ON ((target.tid OPERATOR(pg_catalog.=) sub.sid1) AND (target.tid OPERATOR(pg_catalog.=) $1)) WHEN MATCHED THEN UPDATE SET val = sub.val1 WHEN NOT MATCHED THEN DO NOTHING
@ -2414,6 +2420,11 @@ SELECT * FROM target_set ORDER BY 1, 2;
-- Error and Unsupported scenarios
--
MERGE INTO target_set
USING source_set AS foo ON target_set.t1 = foo.s1
WHEN MATCHED THEN
UPDATE SET ctid = '(0,100)';
ERROR: cannot assign to system column "ctid"
MERGE INTO target_set
USING (SELECT s1,s2 FROM source_set UNION SELECT s2,s1 FROM source_set) AS foo ON target_set.t1 = foo.s1
WHEN MATCHED THEN
UPDATE SET t2 = t2 + 1;

View File

@ -1360,10 +1360,13 @@ SELECT * FROM multi_extension.print_extension_changes();
-- Snapshot of state at 11.3-1
ALTER EXTENSION citus UPDATE TO '11.3-1';
SELECT * FROM multi_extension.print_extension_changes();
previous_object | current_object
previous_object | current_object
---------------------------------------------------------------------
| function citus_internal_is_replication_origin_tracking_active() boolean
| function citus_internal_start_replication_origin_tracking() void
| function citus_internal_stop_replication_origin_tracking() void
| function worker_modify_identity_columns(regclass) void
(1 row)
(4 rows)
DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff;
-- show running version

View File

@ -129,12 +129,25 @@ BEGIN;
INSERT INTO researchers VALUES (8, 5, 'Douglas Engelbart');
INSERT INTO labs VALUES (5, 'Los Alamos');
COMMIT;
SET citus.enable_non_colocated_router_query_pushdown TO ON;
SELECT * FROM researchers, labs WHERE labs.id = researchers.lab_id AND researchers.lab_id = 5;
id | lab_id | name | id | name
---------------------------------------------------------------------
8 | 5 | Douglas Engelbart | 5 | Los Alamos
(1 row)
SET citus.enable_non_colocated_router_query_pushdown TO OFF;
-- fails because researchers and labs are not colocated
SELECT * FROM researchers, labs WHERE labs.id = researchers.lab_id AND researchers.lab_id = 5;
ERROR: complex joins are only supported when all distributed tables are joined on their distribution columns with equal operator
-- works thanks to "OFFSET 0" trick
SELECT * FROM (SELECT * FROM researchers OFFSET 0) researchers, labs WHERE labs.id = researchers.lab_id AND researchers.lab_id = 5;
id | lab_id | name | id | name
---------------------------------------------------------------------
8 | 5 | Douglas Engelbart | 5 | Los Alamos
(1 row)
RESET citus.enable_non_colocated_router_query_pushdown;
-- and the other way around is also allowed
BEGIN;
INSERT INTO labs VALUES (6, 'Bell Labs');

View File

@ -129,12 +129,25 @@ BEGIN;
INSERT INTO researchers_mx VALUES (8, 5, 'Douglas Engelbart');
INSERT INTO labs_mx VALUES (5, 'Los Alamos');
COMMIT;
SELECT * FROM researchers_mx, labs_mx WHERE labs_mx.id = researchers_mx.lab_id and researchers_mx.lab_id = 5;;
SET citus.enable_non_colocated_router_query_pushdown TO ON;
SELECT * FROM researchers_mx, labs_mx WHERE labs_mx.id = researchers_mx.lab_id and researchers_mx.lab_id = 5 ORDER BY 1,2,3,4,5;
id | lab_id | name | id | name
---------------------------------------------------------------------
8 | 5 | Douglas Engelbart | 5 | Los Alamos
(1 row)
SET citus.enable_non_colocated_router_query_pushdown TO OFF;
-- fails because researchers and labs are not colocated
SELECT * FROM researchers_mx, labs_mx WHERE labs_mx.id = researchers_mx.lab_id and researchers_mx.lab_id = 5;
ERROR: complex joins are only supported when all distributed tables are joined on their distribution columns with equal operator
-- works thanks to "OFFSET 0" trick
SELECT * FROM (SELECT * FROM researchers_mx OFFSET 0) researchers_mx, labs_mx WHERE labs_mx.id = researchers_mx.lab_id and researchers_mx.lab_id = 5 ORDER BY 1,2,3,4,5;
id | lab_id | name | id | name
---------------------------------------------------------------------
8 | 5 | Douglas Engelbart | 5 | Los Alamos
(1 row)
RESET citus.enable_non_colocated_router_query_pushdown;
-- and the other way around is also allowed
BEGIN;
SET LOCAL citus.enable_local_execution TO off;
@ -148,7 +161,8 @@ BEGIN;
INSERT INTO researchers_mx VALUES (8, 5, 'Douglas Engelbart');
INSERT INTO labs_mx VALUES (5, 'Los Alamos');
COMMIT;
SELECT * FROM researchers_mx, labs_mx WHERE labs_mx.id = researchers_mx.lab_id and researchers_mx.lab_id = 5;
SET citus.enable_non_colocated_router_query_pushdown TO ON;
SELECT * FROM researchers_mx, labs_mx WHERE labs_mx.id = researchers_mx.lab_id and researchers_mx.lab_id = 5 ORDER BY 1,2,3,4,5;
id | lab_id | name | id | name
---------------------------------------------------------------------
8 | 5 | Douglas Engelbart | 5 | Los Alamos
@ -157,6 +171,21 @@ SELECT * FROM researchers_mx, labs_mx WHERE labs_mx.id = researchers_mx.lab_id a
8 | 5 | Douglas Engelbart | 5 | Los Alamos
(4 rows)
SET citus.enable_non_colocated_router_query_pushdown TO OFF;
-- fails because researchers and labs are not colocated
SELECT * FROM researchers_mx, labs_mx WHERE labs_mx.id = researchers_mx.lab_id and researchers_mx.lab_id = 5;
ERROR: complex joins are only supported when all distributed tables are joined on their distribution columns with equal operator
-- works thanks to "OFFSET 0" trick
SELECT * FROM (SELECT * FROM researchers_mx OFFSET 0) researchers_mx, labs_mx WHERE labs_mx.id = researchers_mx.lab_id and researchers_mx.lab_id = 5 ORDER BY 1,2,3,4,5;
id | lab_id | name | id | name
---------------------------------------------------------------------
8 | 5 | Douglas Engelbart | 5 | Los Alamos
8 | 5 | Douglas Engelbart | 5 | Los Alamos
8 | 5 | Douglas Engelbart | 5 | Los Alamos
8 | 5 | Douglas Engelbart | 5 | Los Alamos
(4 rows)
RESET citus.enable_non_colocated_router_query_pushdown;
-- and the other way around is also allowed
BEGIN;
SET LOCAL citus.enable_local_execution TO off;

View File

@ -586,11 +586,13 @@ DEBUG: query has a single distribution column value: 10
(3 rows)
-- following join is router plannable since the same worker
-- has both shards
-- has both shards when citus.enable_non_colocated_router_query_pushdown
-- is enabled
SET citus.enable_non_colocated_router_query_pushdown TO ON;
SELECT a.author_id as first_author, b.word_count as second_word_count
FROM articles_hash_mx a, articles_single_shard_hash_mx b
WHERE a.author_id = 10 and a.author_id = b.author_id
LIMIT 3;
ORDER by 1,2 LIMIT 3;
DEBUG: Creating router plan
DEBUG: query has a single distribution column value: 10
first_author | second_word_count
@ -600,6 +602,45 @@ DEBUG: query has a single distribution column value: 10
10 | 19519
(3 rows)
SET citus.enable_non_colocated_router_query_pushdown TO OFF;
-- but this is not the case otherwise
SELECT a.author_id as first_author, b.word_count as second_word_count
FROM articles_hash_mx a, articles_single_shard_hash_mx b
WHERE a.author_id = 10 and a.author_id = b.author_id
ORDER by 1,2 LIMIT 3;
DEBUG: router planner does not support queries that reference non-colocated distributed tables
DEBUG: push down of limit count: 3
DEBUG: join prunable for task partitionId 0 and 1
DEBUG: join prunable for task partitionId 0 and 2
DEBUG: join prunable for task partitionId 0 and 3
DEBUG: join prunable for task partitionId 1 and 0
DEBUG: join prunable for task partitionId 1 and 2
DEBUG: join prunable for task partitionId 1 and 3
DEBUG: join prunable for task partitionId 2 and 0
DEBUG: join prunable for task partitionId 2 and 1
DEBUG: join prunable for task partitionId 2 and 3
DEBUG: join prunable for task partitionId 3 and 0
DEBUG: join prunable for task partitionId 3 and 1
DEBUG: join prunable for task partitionId 3 and 2
DEBUG: pruning merge fetch taskId 1
DETAIL: Creating dependency on merge taskId 2
DEBUG: pruning merge fetch taskId 2
DETAIL: Creating dependency on merge taskId 2
DEBUG: pruning merge fetch taskId 4
DETAIL: Creating dependency on merge taskId 4
DEBUG: pruning merge fetch taskId 5
DETAIL: Creating dependency on merge taskId 4
DEBUG: pruning merge fetch taskId 7
DETAIL: Creating dependency on merge taskId 6
DEBUG: pruning merge fetch taskId 8
DETAIL: Creating dependency on merge taskId 6
DEBUG: pruning merge fetch taskId 10
DETAIL: Creating dependency on merge taskId 8
DEBUG: pruning merge fetch taskId 11
DETAIL: Creating dependency on merge taskId 8
ERROR: the query contains a join that requires repartitioning
HINT: Set citus.enable_repartition_joins to on to enable repartitioning
RESET citus.enable_non_colocated_router_query_pushdown;
-- following join is not router plannable since there are no
-- workers containing both shards, but will work through recursive
-- planning

View File

@ -767,11 +767,13 @@ DEBUG: query has a single distribution column value: 10
(3 rows)
-- following join is router plannable since the same worker
-- has both shards
-- has both shards when citus.enable_non_colocated_router_query_pushdown
-- is enabled
SET citus.enable_non_colocated_router_query_pushdown TO ON;
SELECT a.author_id as first_author, b.word_count as second_word_count
FROM articles_hash a, articles_single_shard_hash b
WHERE a.author_id = 10 and a.author_id = b.author_id
LIMIT 3;
ORDER BY 1,2 LIMIT 3;
DEBUG: Creating router plan
DEBUG: query has a single distribution column value: 10
first_author | second_word_count
@ -781,6 +783,45 @@ DEBUG: query has a single distribution column value: 10
10 | 19519
(3 rows)
SET citus.enable_non_colocated_router_query_pushdown TO OFF;
-- but this is not the case otherwise
SELECT a.author_id as first_author, b.word_count as second_word_count
FROM articles_hash a, articles_single_shard_hash b
WHERE a.author_id = 10 and a.author_id = b.author_id
ORDER BY 1,2 LIMIT 3;
DEBUG: router planner does not support queries that reference non-colocated distributed tables
DEBUG: push down of limit count: 3
DEBUG: join prunable for task partitionId 0 and 1
DEBUG: join prunable for task partitionId 0 and 2
DEBUG: join prunable for task partitionId 0 and 3
DEBUG: join prunable for task partitionId 1 and 0
DEBUG: join prunable for task partitionId 1 and 2
DEBUG: join prunable for task partitionId 1 and 3
DEBUG: join prunable for task partitionId 2 and 0
DEBUG: join prunable for task partitionId 2 and 1
DEBUG: join prunable for task partitionId 2 and 3
DEBUG: join prunable for task partitionId 3 and 0
DEBUG: join prunable for task partitionId 3 and 1
DEBUG: join prunable for task partitionId 3 and 2
DEBUG: pruning merge fetch taskId 1
DETAIL: Creating dependency on merge taskId 2
DEBUG: pruning merge fetch taskId 2
DETAIL: Creating dependency on merge taskId 2
DEBUG: pruning merge fetch taskId 4
DETAIL: Creating dependency on merge taskId 4
DEBUG: pruning merge fetch taskId 5
DETAIL: Creating dependency on merge taskId 4
DEBUG: pruning merge fetch taskId 7
DETAIL: Creating dependency on merge taskId 6
DEBUG: pruning merge fetch taskId 8
DETAIL: Creating dependency on merge taskId 6
DEBUG: pruning merge fetch taskId 10
DETAIL: Creating dependency on merge taskId 8
DEBUG: pruning merge fetch taskId 11
DETAIL: Creating dependency on merge taskId 8
ERROR: the query contains a join that requires repartitioning
HINT: Set citus.enable_repartition_joins to on to enable repartitioning
RESET citus.enable_non_colocated_router_query_pushdown;
-- following join is not router plannable since there are no
-- workers containing both shards, but will work through recursive
-- planning
@ -1420,6 +1461,11 @@ DEBUG: Creating router plan
---------------------------------------------------------------------
(0 rows)
-- Even if the where clause contains "false", the query is not router
-- plannable when citus.enable_non_colocated_router_query_pushdown
-- is disabled. This is because, the tables are not colocated.
SET citus.enable_non_colocated_router_query_pushdown TO ON;
-- the same query, router plannable
SELECT a.author_id as first_author, b.word_count as second_word_count
FROM articles_hash a, articles_single_shard_hash b
WHERE a.author_id = 10 and a.author_id = b.author_id and false;
@ -1428,6 +1474,17 @@ DEBUG: Creating router plan
---------------------------------------------------------------------
(0 rows)
SET citus.enable_non_colocated_router_query_pushdown TO OFF;
-- the same query, _not_ router plannable
SELECT a.author_id as first_author, b.word_count as second_word_count
FROM articles_hash a, articles_single_shard_hash b
WHERE a.author_id = 10 and a.author_id = b.author_id and false;
DEBUG: router planner does not support queries that reference non-colocated distributed tables
first_author | second_word_count
---------------------------------------------------------------------
(0 rows)
RESET citus.enable_non_colocated_router_query_pushdown;
SELECT *
FROM articles_hash
WHERE null;
@ -1900,15 +1957,54 @@ DEBUG: Creating router plan
-- join between hash and range partition tables are router plannable
-- only if both tables pruned down to single shard and co-located on the same
-- node.
-- router plannable
SET citus.enable_non_colocated_router_query_pushdown TO ON;
-- router plannable when citus.enable_non_colocated_router_query_pushdown is on
SELECT * FROM articles_hash ar join authors_range au on (ar.author_id = au.id)
WHERE ar.author_id = 2;
WHERE ar.author_id = 2 ORDER BY 1,2,3,4,5,6;
DEBUG: Creating router plan
DEBUG: query has a single distribution column value: 2
id | author_id | title | word_count | name | id
---------------------------------------------------------------------
(0 rows)
SET citus.enable_non_colocated_router_query_pushdown TO OFF;
-- not router plannable otherwise
SELECT * FROM articles_hash ar join authors_range au on (ar.author_id = au.id)
WHERE ar.author_id = 2 ORDER BY 1,2,3,4,5,6;
DEBUG: router planner does not support queries that reference non-colocated distributed tables
DEBUG: join prunable for task partitionId 0 and 1
DEBUG: join prunable for task partitionId 0 and 2
DEBUG: join prunable for task partitionId 0 and 3
DEBUG: join prunable for task partitionId 1 and 0
DEBUG: join prunable for task partitionId 1 and 2
DEBUG: join prunable for task partitionId 1 and 3
DEBUG: join prunable for task partitionId 2 and 0
DEBUG: join prunable for task partitionId 2 and 1
DEBUG: join prunable for task partitionId 2 and 3
DEBUG: join prunable for task partitionId 3 and 0
DEBUG: join prunable for task partitionId 3 and 1
DEBUG: join prunable for task partitionId 3 and 2
DEBUG: pruning merge fetch taskId 1
DETAIL: Creating dependency on merge taskId 2
DEBUG: pruning merge fetch taskId 2
DETAIL: Creating dependency on merge taskId 5
DEBUG: pruning merge fetch taskId 4
DETAIL: Creating dependency on merge taskId 4
DEBUG: pruning merge fetch taskId 5
DETAIL: Creating dependency on merge taskId 10
DEBUG: pruning merge fetch taskId 7
DETAIL: Creating dependency on merge taskId 6
DEBUG: pruning merge fetch taskId 8
DETAIL: Creating dependency on merge taskId 15
DEBUG: pruning merge fetch taskId 10
DETAIL: Creating dependency on merge taskId 8
DEBUG: pruning merge fetch taskId 11
DETAIL: Creating dependency on merge taskId 20
id | author_id | title | word_count | name | id
---------------------------------------------------------------------
(0 rows)
RESET citus.enable_non_colocated_router_query_pushdown;
-- not router plannable
SELECT * FROM articles_hash ar join authors_range au on (ar.author_id = au.id)
WHERE ar.author_id = 3;
@ -2476,5 +2572,55 @@ DROP USER router_user;
\c - - - :master_port
DROP OWNED BY router_user;
DROP USER router_user;
SET search_path TO multi_router_planner;
SET citus.next_shard_id TO 850000;
SET citus.shard_replication_factor TO 1;
CREATE TABLE single_shard_dist(a int, b int);
SELECT create_distributed_table('single_shard_dist', 'a', shard_count=>1);
create_distributed_table
---------------------------------------------------------------------
(1 row)
SET citus.shard_replication_factor TO 2;
CREATE TABLE table_with_four_shards(a int, b int);
SELECT create_distributed_table('table_with_four_shards', 'a', shard_count=>4);
create_distributed_table
---------------------------------------------------------------------
(1 row)
SET client_min_messages TO DEBUG2;
-- Make sure that router rejects planning this query because
-- the target shards are not placed on the same node when
-- citus.enable_non_colocated_router_query_pushdown is disabled.
-- Otherwise, it throws a somewhat meaningless error but we assume
-- that the user is aware of the setting.
SET citus.enable_non_colocated_router_query_pushdown TO ON;
WITH cte AS (
DELETE FROM table_with_four_shards WHERE a = 1 RETURNING *
)
SELECT * FROM single_shard_dist WHERE b IN (SELECT b FROM cte);
DEBUG: Creating router plan
DEBUG: query has a single distribution column value: 1
ERROR: relation "multi_router_planner.single_shard_dist_850000" does not exist
CONTEXT: while executing command on localhost:xxxxx
SET citus.enable_non_colocated_router_query_pushdown TO OFF;
WITH cte AS (
DELETE FROM table_with_four_shards WHERE a = 1 RETURNING *
)
SELECT * FROM single_shard_dist WHERE b IN (SELECT b FROM cte);
DEBUG: router planner does not support queries that reference non-colocated distributed tables
DEBUG: generating subplan XXX_1 for CTE cte: DELETE FROM multi_router_planner.table_with_four_shards WHERE (a OPERATOR(pg_catalog.=) 1) RETURNING a, b
DEBUG: Distributed planning for a fast-path router query
DEBUG: Creating router plan
DEBUG: query has a single distribution column value: 1
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT a, b FROM multi_router_planner.single_shard_dist WHERE (b OPERATOR(pg_catalog.=) ANY (SELECT cte.b FROM (SELECT intermediate_result.a, intermediate_result.b FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(a integer, b integer)) cte))
DEBUG: Creating router plan
a | b
---------------------------------------------------------------------
(0 rows)
RESET citus.enable_non_colocated_router_query_pushdown;
SET client_min_messages TO WARNING;
DROP SCHEMA multi_router_planner CASCADE;

View File

@ -481,12 +481,13 @@ DEBUG: query has a single distribution column value: 10
10 | 6363
(3 rows)
-- now show that JOINs with multiple tables are not router executable
-- they are executed by real-time executor
-- Not router plannable when citus.enable_non_colocated_router_query_pushdown
-- is disabled.
SET citus.enable_non_colocated_router_query_pushdown TO ON;
SELECT a.author_id as first_author, b.word_count as second_word_count
FROM articles a, articles_single_shard b
WHERE a.author_id = 10 and a.author_id = b.author_id
LIMIT 3;
ORDER BY 1,2 LIMIT 3;
DEBUG: Creating router plan
DEBUG: query has a single distribution column value: 10
first_author | second_word_count
@ -496,6 +497,88 @@ DEBUG: query has a single distribution column value: 10
10 | 19519
(3 rows)
SET citus.enable_non_colocated_router_query_pushdown TO OFF;
SELECT a.author_id as first_author, b.word_count as second_word_count
FROM articles a, articles_single_shard b
WHERE a.author_id = 10 and a.author_id = b.author_id
ORDER BY 1,2 LIMIT 3;
DEBUG: router planner does not support queries that reference non-colocated distributed tables
DEBUG: push down of limit count: 3
DEBUG: join prunable for task partitionId 0 and 1
DEBUG: join prunable for task partitionId 0 and 2
DEBUG: join prunable for task partitionId 0 and 3
DEBUG: join prunable for task partitionId 1 and 0
DEBUG: join prunable for task partitionId 1 and 2
DEBUG: join prunable for task partitionId 1 and 3
DEBUG: join prunable for task partitionId 2 and 0
DEBUG: join prunable for task partitionId 2 and 1
DEBUG: join prunable for task partitionId 2 and 3
DEBUG: join prunable for task partitionId 3 and 0
DEBUG: join prunable for task partitionId 3 and 1
DEBUG: join prunable for task partitionId 3 and 2
DEBUG: pruning merge fetch taskId 1
DETAIL: Creating dependency on merge taskId 2
DEBUG: pruning merge fetch taskId 2
DETAIL: Creating dependency on merge taskId 2
DEBUG: pruning merge fetch taskId 4
DETAIL: Creating dependency on merge taskId 4
DEBUG: pruning merge fetch taskId 5
DETAIL: Creating dependency on merge taskId 4
DEBUG: pruning merge fetch taskId 7
DETAIL: Creating dependency on merge taskId 6
DEBUG: pruning merge fetch taskId 8
DETAIL: Creating dependency on merge taskId 6
DEBUG: pruning merge fetch taskId 10
DETAIL: Creating dependency on merge taskId 8
DEBUG: pruning merge fetch taskId 11
DETAIL: Creating dependency on merge taskId 8
ERROR: the query contains a join that requires repartitioning
HINT: Set citus.enable_repartition_joins to on to enable repartitioning
-- but they can be executed via repartition join planner
SET citus.enable_repartition_joins TO ON;
SELECT a.author_id as first_author, b.word_count as second_word_count
FROM articles a, articles_single_shard b
WHERE a.author_id = 10 and a.author_id = b.author_id
ORDER BY 1,2 LIMIT 3;
DEBUG: router planner does not support queries that reference non-colocated distributed tables
DEBUG: push down of limit count: 3
DEBUG: join prunable for task partitionId 0 and 1
DEBUG: join prunable for task partitionId 0 and 2
DEBUG: join prunable for task partitionId 0 and 3
DEBUG: join prunable for task partitionId 1 and 0
DEBUG: join prunable for task partitionId 1 and 2
DEBUG: join prunable for task partitionId 1 and 3
DEBUG: join prunable for task partitionId 2 and 0
DEBUG: join prunable for task partitionId 2 and 1
DEBUG: join prunable for task partitionId 2 and 3
DEBUG: join prunable for task partitionId 3 and 0
DEBUG: join prunable for task partitionId 3 and 1
DEBUG: join prunable for task partitionId 3 and 2
DEBUG: pruning merge fetch taskId 1
DETAIL: Creating dependency on merge taskId 2
DEBUG: pruning merge fetch taskId 2
DETAIL: Creating dependency on merge taskId 2
DEBUG: pruning merge fetch taskId 4
DETAIL: Creating dependency on merge taskId 4
DEBUG: pruning merge fetch taskId 5
DETAIL: Creating dependency on merge taskId 4
DEBUG: pruning merge fetch taskId 7
DETAIL: Creating dependency on merge taskId 6
DEBUG: pruning merge fetch taskId 8
DETAIL: Creating dependency on merge taskId 6
DEBUG: pruning merge fetch taskId 10
DETAIL: Creating dependency on merge taskId 8
DEBUG: pruning merge fetch taskId 11
DETAIL: Creating dependency on merge taskId 8
first_author | second_word_count
---------------------------------------------------------------------
10 | 19519
10 | 19519
10 | 19519
(3 rows)
RESET citus.enable_repartition_joins;
RESET citus.enable_non_colocated_router_query_pushdown;
-- do not create the master query for LIMIT on a single shard SELECT
SELECT *
FROM articles

View File

@ -1900,6 +1900,7 @@ MERGE INTO pa_target t
UPDATE SET balance = balance + delta, val = val || ' updated by merge'
WHEN NOT MATCHED THEN
INSERT VALUES ('2017-01-15', sid, delta, 'inserted by merge');
DEBUG: Creating MERGE router plan
DEBUG: <Deparsed MERGE query: MERGE INTO pgmerge_schema.pa_target t USING (SELECT pa_source.sid, pa_source.delta FROM pgmerge_schema.pa_source_xxxxxxx pa_source WHERE (pa_source.sid OPERATOR(pg_catalog.<) 10)) s ON (t.tid OPERATOR(pg_catalog.=) s.sid) WHEN MATCHED THEN UPDATE SET balance = (t.balance OPERATOR(pg_catalog.+) s.delta), val = (t.val OPERATOR(pg_catalog.||) ' updated by merge'::text) WHEN NOT MATCHED THEN INSERT (logts, tid, balance, val) VALUES ('Sun Jan 15 00:00:00 2017'::timestamp without time zone, s.sid, s.delta, 'inserted by merge'::text)>
--INSERT VALUES (slogts::timestamp, sid, delta, 'inserted by merge');
SELECT * FROM pa_target ORDER BY tid;
@ -2093,6 +2094,7 @@ WHEN MATCHED THEN UPDATE
WHEN NOT MATCHED THEN INSERT
(city_id, logdate, peaktemp, unitsales)
VALUES (city_id, logdate, peaktemp, unitsales);
DEBUG: Creating MERGE router plan
DEBUG: <Deparsed MERGE query: MERGE INTO pgmerge_schema.measurement m USING pgmerge_schema.new_measurement_xxxxxxx nm ON ((m.city_id OPERATOR(pg_catalog.=) nm.city_id) AND (m.logdate OPERATOR(pg_catalog.=) nm.logdate)) WHEN MATCHED AND (nm.peaktemp IS NULL) THEN DELETE WHEN MATCHED THEN UPDATE SET peaktemp = GREATEST(m.peaktemp, nm.peaktemp), unitsales = (m.unitsales OPERATOR(pg_catalog.+) COALESCE(nm.unitsales, 0)) WHEN NOT MATCHED THEN INSERT (city_id, logdate, peaktemp, unitsales) VALUES (nm.city_id, nm.logdate, nm.peaktemp, nm.unitsales)>
RESET client_min_messages;
SELECT tableoid::regclass, * FROM measurement ORDER BY city_id, logdate;

View File

@ -74,7 +74,10 @@ ORDER BY 1;
function citus_internal_delete_partition_metadata(regclass)
function citus_internal_delete_shard_metadata(bigint)
function citus_internal_global_blocked_processes()
function citus_internal_is_replication_origin_tracking_active()
function citus_internal_local_blocked_processes()
function citus_internal_start_replication_origin_tracking()
function citus_internal_stop_replication_origin_tracking()
function citus_internal_update_placement_metadata(bigint,integer,integer)
function citus_internal_update_relation_colocation(oid,integer)
function citus_is_clock_after(cluster_clock,cluster_clock)
@ -319,5 +322,5 @@ ORDER BY 1;
view citus_stat_statements
view pg_dist_shard_placement
view time_partitions
(311 rows)
(314 rows)

View File

@ -107,4 +107,10 @@ test: isolation_multiuser_locking
test: isolation_acquire_distributed_locks
test: isolation_concurrent_move_create_table
# MERGE
test: isolation_merge
test: isolation_merge_replicated
# Note: Always keep this test at the end
test: isolation_check_mx

View File

@ -90,6 +90,7 @@ my $workerCount = 2;
my $serversAreShutdown = "TRUE";
my $usingWindows = 0;
my $mitmPid = 0;
my $workerCount = 2;
if ($Config{osname} eq "MSWin32")
{
@ -487,6 +488,7 @@ push(@pgOptions, "citus.explain_analyze_sort_method='taskId'");
push(@pgOptions, "citus.enable_manual_changes_to_shards=on");
push(@pgOptions, "citus.allow_unsafe_locks_from_workers=on");
push(@pgOptions, "citus.stat_statements_track = 'all'");
push(@pgOptions, "citus.enable_change_data_capture=on");
# Some tests look at shards in pg_class, make sure we can usually see them:
push(@pgOptions, "citus.show_shards_for_app_name_prefixes='pg_regress'");

View File

@ -0,0 +1,92 @@
//
// How we organize this isolation test spec, is explained at README.md file in this directory.
//
// create distributed tables to test behavior of MERGE in concurrent operations
setup
{
DO
$do$
DECLARE ver int;
BEGIN
SELECT substring(version(), '\d+')::int into ver;
IF (ver < 15)
THEN
RAISE EXCEPTION 'MERGE is not supported on PG versions below 15';
END IF;
END
$do$;
SET citus.shard_replication_factor TO 1;
SELECT 1 FROM master_add_node('localhost', 57637);
SELECT 1 FROM master_add_node('localhost', 57638);
CREATE TABLE prept(t1 int, t2 int);
CREATE TABLE preps(s1 int, s2 int);
SELECT create_distributed_table('prept', 't1'), create_distributed_table('preps', 's1');
INSERT INTO prept VALUES(100, 0);
INSERT INTO preps VALUES(100, 0);
INSERT INTO preps VALUES(200, 0);
}
// drop distributed tables
teardown
{
DROP TABLE IF EXISTS prept CASCADE;
DROP TABLE IF EXISTS preps CASCADE;
}
// session 1
session "s1"
step "s1-begin" { BEGIN; }
step "s1-upd-ins" { MERGE INTO prept USING preps ON prept.t1 = preps.s1
WHEN MATCHED THEN UPDATE SET t2 = t2 + 1
WHEN NOT MATCHED THEN INSERT VALUES(s1, s2);
}
step "s1-del-ins" { MERGE INTO prept USING preps ON prept.t1 = preps.s1
WHEN MATCHED THEN DELETE
WHEN NOT MATCHED THEN INSERT VALUES(s1, s2);
}
step "s1-del" { MERGE INTO prept USING preps ON prept.t1 = preps.s1
WHEN MATCHED THEN DELETE;
}
step "s1-ins" { MERGE INTO prept USING preps ON prept.t1 = preps.s1
WHEN NOT MATCHED THEN INSERT VALUES(s1, s2);
}
step "s1-commit" { COMMIT; }
step "s1-result" { SELECT * FROM prept ORDER BY 1; }
// session 2
session "s2"
step "s2-begin" { BEGIN; }
step "s2-upd-del" { MERGE INTO prept USING preps ON prept.t1 = preps.s1
WHEN MATCHED AND prept.t2 = 0 THEN DELETE
WHEN MATCHED THEN UPDATE SET t2 = t2 + 1;
}
step "s2-upd" { MERGE INTO prept USING preps ON prept.t1 = preps.s1
WHEN MATCHED THEN UPDATE SET t2 = t2 + 1;
}
step "s2-ins" { MERGE INTO prept USING preps ON prept.t1 = preps.s1
WHEN NOT MATCHED THEN INSERT VALUES(s1, s2);
}
step "s2-commit" { COMMIT; }
step "s2-result" { SELECT * FROM prept ORDER BY 1; }
// permutations - MERGE vs MERGE
permutation "s1-begin" "s1-upd-ins" "s2-result" "s1-commit" "s2-result"
permutation "s1-begin" "s1-upd-ins" "s2-begin" "s2-upd-del" "s1-commit" "s2-commit" "s2-result"
permutation "s2-begin" "s2-upd-del" "s1-begin" "s1-upd-ins" "s2-commit" "s1-commit" "s2-result"
permutation "s1-begin" "s1-upd-ins" "s2-begin" "s2-upd" "s1-commit" "s2-commit" "s2-result"
permutation "s2-begin" "s2-ins" "s1-begin" "s1-del" "s2-upd" "s2-result" "s2-commit" "s1-commit" "s2-result"
permutation "s1-begin" "s1-del-ins" "s2-begin" "s2-upd" "s1-result" "s1-ins" "s1-commit" "s2-upd" "s2-commit" "s2-result"

View File

@ -0,0 +1,58 @@
//
// How we organize this isolation test spec, is explained at README.md file in this directory.
//
// create distributed tables to test behavior of MERGE in concurrent operations
setup
{
DO
$do$
DECLARE ver int;
BEGIN
SELECT substring(version(), '\d+')::int into ver;
IF (ver < 15)
THEN
RAISE EXCEPTION 'MERGE is not supported on PG versions below 15';
END IF;
END
$do$;
SET citus.shard_replication_factor TO 2;
SELECT 1 FROM master_add_node('localhost', 57637);
SELECT 1 FROM master_add_node('localhost', 57638);
CREATE TABLE prept(t1 int, t2 int);
CREATE TABLE preps(s1 int, s2 int);
SELECT create_distributed_table('prept', 't1'), create_distributed_table('preps', 's1');
INSERT INTO prept VALUES(100, 0);
INSERT INTO preps VALUES(100, 0);
INSERT INTO preps VALUES(200, 0);
}
// drop distributed tables
teardown
{
DROP TABLE IF EXISTS prept CASCADE;
DROP TABLE IF EXISTS preps CASCADE;
}
// session 1
session "s1"
step "s1-begin" { BEGIN; }
step "s1-upd-ins" { MERGE INTO prept USING preps ON prept.t1 = preps.s1
WHEN MATCHED THEN UPDATE SET t2 = t2 + 1
WHEN NOT MATCHED THEN INSERT VALUES(s1, s2); }
step "s1-result" { SELECT * FROM preps ORDER BY 1; }
step "s1-commit" { COMMIT; }
// session 2
session "s2"
step "s2-begin" { BEGIN; }
step "s2-update" { UPDATE preps SET s2 = s2 + 1; }
step "s2-commit" { COMMIT; }
step "s2-result" { SELECT * FROM prept ORDER BY 1; }
// permutations - MERGE vs UPDATE (on source)
permutation "s1-begin" "s1-upd-ins" "s2-begin" "s2-update" "s1-commit" "s2-commit" "s1-result" "s2-result"

View File

@ -1540,6 +1540,11 @@ SELECT * FROM target_set ORDER BY 1, 2;
-- Error and Unsupported scenarios
--
MERGE INTO target_set
USING source_set AS foo ON target_set.t1 = foo.s1
WHEN MATCHED THEN
UPDATE SET ctid = '(0,100)';
MERGE INTO target_set
USING (SELECT s1,s2 FROM source_set UNION SELECT s2,s1 FROM source_set) AS foo ON target_set.t1 = foo.s1
WHEN MATCHED THEN

View File

@ -117,8 +117,20 @@ INSERT INTO researchers VALUES (8, 5, 'Douglas Engelbart');
INSERT INTO labs VALUES (5, 'Los Alamos');
COMMIT;
SET citus.enable_non_colocated_router_query_pushdown TO ON;
SELECT * FROM researchers, labs WHERE labs.id = researchers.lab_id AND researchers.lab_id = 5;
SET citus.enable_non_colocated_router_query_pushdown TO OFF;
-- fails because researchers and labs are not colocated
SELECT * FROM researchers, labs WHERE labs.id = researchers.lab_id AND researchers.lab_id = 5;
-- works thanks to "OFFSET 0" trick
SELECT * FROM (SELECT * FROM researchers OFFSET 0) researchers, labs WHERE labs.id = researchers.lab_id AND researchers.lab_id = 5;
RESET citus.enable_non_colocated_router_query_pushdown;
-- and the other way around is also allowed
BEGIN;
INSERT INTO labs VALUES (6, 'Bell Labs');

View File

@ -116,7 +116,19 @@ INSERT INTO researchers_mx VALUES (8, 5, 'Douglas Engelbart');
INSERT INTO labs_mx VALUES (5, 'Los Alamos');
COMMIT;
SELECT * FROM researchers_mx, labs_mx WHERE labs_mx.id = researchers_mx.lab_id and researchers_mx.lab_id = 5;;
SET citus.enable_non_colocated_router_query_pushdown TO ON;
SELECT * FROM researchers_mx, labs_mx WHERE labs_mx.id = researchers_mx.lab_id and researchers_mx.lab_id = 5 ORDER BY 1,2,3,4,5;
SET citus.enable_non_colocated_router_query_pushdown TO OFF;
-- fails because researchers and labs are not colocated
SELECT * FROM researchers_mx, labs_mx WHERE labs_mx.id = researchers_mx.lab_id and researchers_mx.lab_id = 5;
-- works thanks to "OFFSET 0" trick
SELECT * FROM (SELECT * FROM researchers_mx OFFSET 0) researchers_mx, labs_mx WHERE labs_mx.id = researchers_mx.lab_id and researchers_mx.lab_id = 5 ORDER BY 1,2,3,4,5;
RESET citus.enable_non_colocated_router_query_pushdown;
-- and the other way around is also allowed
BEGIN;
@ -133,8 +145,20 @@ INSERT INTO researchers_mx VALUES (8, 5, 'Douglas Engelbart');
INSERT INTO labs_mx VALUES (5, 'Los Alamos');
COMMIT;
SET citus.enable_non_colocated_router_query_pushdown TO ON;
SELECT * FROM researchers_mx, labs_mx WHERE labs_mx.id = researchers_mx.lab_id and researchers_mx.lab_id = 5 ORDER BY 1,2,3,4,5;
SET citus.enable_non_colocated_router_query_pushdown TO OFF;
-- fails because researchers and labs are not colocated
SELECT * FROM researchers_mx, labs_mx WHERE labs_mx.id = researchers_mx.lab_id and researchers_mx.lab_id = 5;
-- works thanks to "OFFSET 0" trick
SELECT * FROM (SELECT * FROM researchers_mx OFFSET 0) researchers_mx, labs_mx WHERE labs_mx.id = researchers_mx.lab_id and researchers_mx.lab_id = 5 ORDER BY 1,2,3,4,5;
RESET citus.enable_non_colocated_router_query_pushdown;
-- and the other way around is also allowed
BEGIN;
SET LOCAL citus.enable_local_execution TO off;

View File

@ -275,11 +275,25 @@ SELECT a.author_id as first_author, b.word_count as second_word_count
LIMIT 3;
-- following join is router plannable since the same worker
-- has both shards
-- has both shards when citus.enable_non_colocated_router_query_pushdown
-- is enabled
SET citus.enable_non_colocated_router_query_pushdown TO ON;
SELECT a.author_id as first_author, b.word_count as second_word_count
FROM articles_hash_mx a, articles_single_shard_hash_mx b
WHERE a.author_id = 10 and a.author_id = b.author_id
LIMIT 3;
ORDER by 1,2 LIMIT 3;
SET citus.enable_non_colocated_router_query_pushdown TO OFF;
-- but this is not the case otherwise
SELECT a.author_id as first_author, b.word_count as second_word_count
FROM articles_hash_mx a, articles_single_shard_hash_mx b
WHERE a.author_id = 10 and a.author_id = b.author_id
ORDER by 1,2 LIMIT 3;
RESET citus.enable_non_colocated_router_query_pushdown;
-- following join is not router plannable since there are no
-- workers containing both shards, but will work through recursive

View File

@ -384,11 +384,26 @@ SELECT a.author_id as first_author, b.word_count as second_word_count
LIMIT 3;
-- following join is router plannable since the same worker
-- has both shards
-- has both shards when citus.enable_non_colocated_router_query_pushdown
-- is enabled
SET citus.enable_non_colocated_router_query_pushdown TO ON;
SELECT a.author_id as first_author, b.word_count as second_word_count
FROM articles_hash a, articles_single_shard_hash b
WHERE a.author_id = 10 and a.author_id = b.author_id
LIMIT 3;
ORDER BY 1,2 LIMIT 3;
SET citus.enable_non_colocated_router_query_pushdown TO OFF;
-- but this is not the case otherwise
SELECT a.author_id as first_author, b.word_count as second_word_count
FROM articles_hash a, articles_single_shard_hash b
WHERE a.author_id = 10 and a.author_id = b.author_id
ORDER BY 1,2 LIMIT 3;
RESET citus.enable_non_colocated_router_query_pushdown;
-- following join is not router plannable since there are no
-- workers containing both shards, but will work through recursive
@ -649,10 +664,26 @@ SELECT *
FROM articles_hash
WHERE author_id = 1 and 1=0;
-- Even if the where clause contains "false", the query is not router
-- plannable when citus.enable_non_colocated_router_query_pushdown
-- is disabled. This is because, the tables are not colocated.
SET citus.enable_non_colocated_router_query_pushdown TO ON;
-- the same query, router plannable
SELECT a.author_id as first_author, b.word_count as second_word_count
FROM articles_hash a, articles_single_shard_hash b
WHERE a.author_id = 10 and a.author_id = b.author_id and false;
SET citus.enable_non_colocated_router_query_pushdown TO OFF;
-- the same query, _not_ router plannable
SELECT a.author_id as first_author, b.word_count as second_word_count
FROM articles_hash a, articles_single_shard_hash b
WHERE a.author_id = 10 and a.author_id = b.author_id and false;
RESET citus.enable_non_colocated_router_query_pushdown;
SELECT *
FROM articles_hash
WHERE null;
@ -903,9 +934,20 @@ SELECT * FROM articles_range ar join authors_range au on (ar.id = au.id)
-- join between hash and range partition tables are router plannable
-- only if both tables pruned down to single shard and co-located on the same
-- node.
-- router plannable
SET citus.enable_non_colocated_router_query_pushdown TO ON;
-- router plannable when citus.enable_non_colocated_router_query_pushdown is on
SELECT * FROM articles_hash ar join authors_range au on (ar.author_id = au.id)
WHERE ar.author_id = 2;
WHERE ar.author_id = 2 ORDER BY 1,2,3,4,5,6;
SET citus.enable_non_colocated_router_query_pushdown TO OFF;
-- not router plannable otherwise
SELECT * FROM articles_hash ar join authors_range au on (ar.author_id = au.id)
WHERE ar.author_id = 2 ORDER BY 1,2,3,4,5,6;
RESET citus.enable_non_colocated_router_query_pushdown;
-- not router plannable
SELECT * FROM articles_hash ar join authors_range au on (ar.author_id = au.id)
@ -1213,5 +1255,40 @@ DROP USER router_user;
DROP OWNED BY router_user;
DROP USER router_user;
SET search_path TO multi_router_planner;
SET citus.next_shard_id TO 850000;
SET citus.shard_replication_factor TO 1;
CREATE TABLE single_shard_dist(a int, b int);
SELECT create_distributed_table('single_shard_dist', 'a', shard_count=>1);
SET citus.shard_replication_factor TO 2;
CREATE TABLE table_with_four_shards(a int, b int);
SELECT create_distributed_table('table_with_four_shards', 'a', shard_count=>4);
SET client_min_messages TO DEBUG2;
-- Make sure that router rejects planning this query because
-- the target shards are not placed on the same node when
-- citus.enable_non_colocated_router_query_pushdown is disabled.
-- Otherwise, it throws a somewhat meaningless error but we assume
-- that the user is aware of the setting.
SET citus.enable_non_colocated_router_query_pushdown TO ON;
WITH cte AS (
DELETE FROM table_with_four_shards WHERE a = 1 RETURNING *
)
SELECT * FROM single_shard_dist WHERE b IN (SELECT b FROM cte);
SET citus.enable_non_colocated_router_query_pushdown TO OFF;
WITH cte AS (
DELETE FROM table_with_four_shards WHERE a = 1 RETURNING *
)
SELECT * FROM single_shard_dist WHERE b IN (SELECT b FROM cte);
RESET citus.enable_non_colocated_router_query_pushdown;
SET client_min_messages TO WARNING;
DROP SCHEMA multi_router_planner CASCADE;

View File

@ -248,12 +248,34 @@ SELECT a.author_id as first_author, b.word_count as second_word_count
WHERE a.author_id = 10 and a.author_id = b.author_id
LIMIT 3;
-- now show that JOINs with multiple tables are not router executable
-- they are executed by real-time executor
-- Not router plannable when citus.enable_non_colocated_router_query_pushdown
-- is disabled.
SET citus.enable_non_colocated_router_query_pushdown TO ON;
SELECT a.author_id as first_author, b.word_count as second_word_count
FROM articles a, articles_single_shard b
WHERE a.author_id = 10 and a.author_id = b.author_id
LIMIT 3;
ORDER BY 1,2 LIMIT 3;
SET citus.enable_non_colocated_router_query_pushdown TO OFF;
SELECT a.author_id as first_author, b.word_count as second_word_count
FROM articles a, articles_single_shard b
WHERE a.author_id = 10 and a.author_id = b.author_id
ORDER BY 1,2 LIMIT 3;
-- but they can be executed via repartition join planner
SET citus.enable_repartition_joins TO ON;
SELECT a.author_id as first_author, b.word_count as second_word_count
FROM articles a, articles_single_shard b
WHERE a.author_id = 10 and a.author_id = b.author_id
ORDER BY 1,2 LIMIT 3;
RESET citus.enable_repartition_joins;
RESET citus.enable_non_colocated_router_query_pushdown;
-- do not create the master query for LIMIT on a single shard SELECT
SELECT *