Merge branch 'main' into niupre/TestDeferredDropAndCleanup

niupre/TestDeferredDropAndCleanup
Nitish Upreti 2022-09-06 10:46:31 -07:00
commit 2d5984cabd
77 changed files with 3969 additions and 518 deletions

View File

@ -22,6 +22,14 @@ parameters:
style_checker_tools_version: style_checker_tools_version:
type: string type: string
default: '0.8.18' default: '0.8.18'
flaky_test:
type: string
default: ''
flaky_test_make:
type: string
default: check-minimal
jobs: jobs:
build: build:
description: Build the citus extension description: Build the citus extension
@ -529,9 +537,116 @@ jobs:
name: install dependencies and run ch_benchmark tests name: install dependencies and run ch_benchmark tests
no_output_timeout: 20m no_output_timeout: 20m
test-flakyness:
description: Runs a test multiple times to see if it's flaky
parallelism: 32
parameters:
pg_major:
description: 'postgres major version'
type: integer
image:
description: 'docker image to use as for the tests'
type: string
default: citus/exttester
image_tag:
description: 'docker image tag to use'
type: string
make:
description: 'make target'
type: string
default: check-minimal
test:
description: 'the test that should be run multiple times'
type: string
runs:
description: 'number of times that the test should be run in total'
type: integer
default: 1600
docker:
- image: '<< parameters.image >>:<< parameters.image_tag >><< pipeline.parameters.image_suffix >>'
working_directory: /home/circleci/project
resource_class: small
steps:
- checkout
- attach_workspace:
at: .
- run:
name: 'Install Extension'
command: |
tar xfv "${CIRCLE_WORKING_DIRECTORY}/install-${PG_MAJOR}.tar" --directory /
- run:
name: 'Configure'
command: |
chown -R circleci .
gosu circleci ./configure --without-pg-version-check
- run:
name: 'Enable core dumps'
command: |
ulimit -c unlimited
- run:
name: 'Run minimal tests'
command: |
gosu circleci make -C src/test/regress << parameters.make >> EXTRA_TESTS="$(for i in $(seq << parameters.runs >> | circleci tests split); do echo -n '<< parameters.test >> ' ; done)"
no_output_timeout: 2m
- run:
name: 'Regressions'
command: |
if [ -f "src/test/regress/regression.diffs" ]; then
cat src/test/regress/regression.diffs
exit 1
fi
when: on_fail
- run:
name: 'Copy coredumps'
command: |
mkdir -p /tmp/core_dumps
if ls core.* 1> /dev/null 2>&1; then
cp core.* /tmp/core_dumps
fi
when: on_fail
- store_artifacts:
name: 'Save regressions'
path: src/test/regress/regression.diffs
- store_artifacts:
name: 'Save mitmproxy output (failure test specific)'
path: src/test/regress/proxy.output
- store_artifacts:
name: 'Save results'
path: src/test/regress/results/
- store_artifacts:
name: 'Save core dumps'
path: /tmp/core_dumps
- store_artifacts:
name: 'Save coordinator log'
path: src/test/regress/tmp_check/master/log
- store_artifacts:
name: 'Save worker1 log'
path: src/test/regress/tmp_check/worker.57637/log
- store_artifacts:
name: 'Save worker2 log'
path: src/test/regress/tmp_check/worker.57638/log
workflows: workflows:
version: 2 version: 2
flaky_test_debugging:
when: << pipeline.parameters.flaky_test >>
jobs:
- build:
name: build-flaky-15
pg_major: 15
image_tag: '<< pipeline.parameters.pg15_version >>'
- test-flakyness:
name: 'test-15_flaky'
pg_major: 15
image_tag: '<< pipeline.parameters.pg15_version >>'
requires: [build-flaky-15]
make: '<< pipeline.parameters.flaky_test_make >>'
test: '<< pipeline.parameters.flaky_test >>'
build_and_test: build_and_test:
when:
not: << pipeline.parameters.flaky_test >>
jobs: jobs:
- build: - build:
name: build-13 name: build-13

View File

@ -1303,6 +1303,12 @@ AddColumnarScanPath(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte,
cpath->methods = &ColumnarScanPathMethods; cpath->methods = &ColumnarScanPathMethods;
#if (PG_VERSION_NUM >= PG_VERSION_15)
/* necessary to avoid extra Result node in PG15 */
cpath->flags = CUSTOMPATH_SUPPORT_PROJECTION;
#endif
/* /*
* populate generic path information * populate generic path information
*/ */
@ -1545,6 +1551,12 @@ ColumnarScanPath_PlanCustomPath(PlannerInfo *root,
cscan->scan.plan.targetlist = list_copy(tlist); cscan->scan.plan.targetlist = list_copy(tlist);
cscan->scan.scanrelid = best_path->path.parent->relid; cscan->scan.scanrelid = best_path->path.parent->relid;
#if (PG_VERSION_NUM >= 150000)
/* necessary to avoid extra Result node in PG15 */
cscan->flags = CUSTOMPATH_SUPPORT_PROJECTION;
#endif
return (Plan *) cscan; return (Plan *) cscan;
} }

View File

@ -139,6 +139,14 @@ static Oid DropFKeysAndUndistributeTable(Oid relationId);
static void DropFKeysRelationInvolvedWithTableType(Oid relationId, int tableTypeFlag); static void DropFKeysRelationInvolvedWithTableType(Oid relationId, int tableTypeFlag);
static void CopyLocalDataIntoShards(Oid relationId); static void CopyLocalDataIntoShards(Oid relationId);
static List * TupleDescColumnNameList(TupleDesc tupleDescriptor); static List * TupleDescColumnNameList(TupleDesc tupleDescriptor);
#if (PG_VERSION_NUM >= PG_VERSION_15)
static bool DistributionColumnUsesNumericColumnNegativeScale(TupleDesc relationDesc,
Var *distributionColumn);
static int numeric_typmod_scale(int32 typmod);
static bool is_valid_numeric_typmod(int32 typmod);
#endif
static bool DistributionColumnUsesGeneratedStoredColumn(TupleDesc relationDesc, static bool DistributionColumnUsesGeneratedStoredColumn(TupleDesc relationDesc,
Var *distributionColumn); Var *distributionColumn);
static bool CanUseExclusiveConnections(Oid relationId, bool localTableEmpty); static bool CanUseExclusiveConnections(Oid relationId, bool localTableEmpty);
@ -1681,6 +1689,20 @@ EnsureRelationCanBeDistributed(Oid relationId, Var *distributionColumn,
"AS (...) STORED."))); "AS (...) STORED.")));
} }
#if (PG_VERSION_NUM >= PG_VERSION_15)
/* verify target relation is not distributed by a column of type numeric with negative scale */
if (distributionMethod != DISTRIBUTE_BY_NONE &&
DistributionColumnUsesNumericColumnNegativeScale(relationDesc,
distributionColumn))
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot distribute relation: %s", relationName),
errdetail("Distribution column must not use numeric type "
"with negative scale")));
}
#endif
/* check for support function needed by specified partition method */ /* check for support function needed by specified partition method */
if (distributionMethod == DISTRIBUTE_BY_HASH) if (distributionMethod == DISTRIBUTE_BY_HASH)
{ {
@ -2401,6 +2423,59 @@ RelationUsesIdentityColumns(TupleDesc relationDesc)
} }
#if (PG_VERSION_NUM >= PG_VERSION_15)
/*
* is_valid_numeric_typmod checks if the typmod value is valid
*
* Because of the offset, valid numeric typmods are at least VARHDRSZ
*
* Copied from PG. See numeric.c for understanding how this works.
*/
static bool
is_valid_numeric_typmod(int32 typmod)
{
return typmod >= (int32) VARHDRSZ;
}
/*
* numeric_typmod_scale extracts the scale from a numeric typmod.
*
* Copied from PG. See numeric.c for understanding how this works.
*
*/
static int
numeric_typmod_scale(int32 typmod)
{
return (((typmod - VARHDRSZ) & 0x7ff) ^ 1024) - 1024;
}
/*
* DistributionColumnUsesNumericColumnNegativeScale returns whether a given relation uses
* numeric data type with negative scale on distribution column
*/
static bool
DistributionColumnUsesNumericColumnNegativeScale(TupleDesc relationDesc,
Var *distributionColumn)
{
Form_pg_attribute attributeForm = TupleDescAttr(relationDesc,
distributionColumn->varattno - 1);
if (attributeForm->atttypid == NUMERICOID &&
is_valid_numeric_typmod(attributeForm->atttypmod) &&
numeric_typmod_scale(attributeForm->atttypmod) < 0)
{
return true;
}
return false;
}
#endif
/* /*
* DistributionColumnUsesGeneratedStoredColumn returns whether a given relation uses * DistributionColumnUsesGeneratedStoredColumn returns whether a given relation uses
* GENERATED ALWAYS AS (...) STORED on distribution column * GENERATED ALWAYS AS (...) STORED on distribution column

View File

@ -1176,7 +1176,7 @@ static DistributeObjectOps View_Rename = {
static DistributeObjectOps Trigger_Rename = { static DistributeObjectOps Trigger_Rename = {
.deparse = NULL, .deparse = NULL,
.qualify = NULL, .qualify = NULL,
.preprocess = NULL, .preprocess = PreprocessAlterTriggerRenameStmt,
.operationType = DIST_OPS_ALTER, .operationType = DIST_OPS_ALTER,
.postprocess = PostprocessAlterTriggerRenameStmt, .postprocess = PostprocessAlterTriggerRenameStmt,
.address = NULL, .address = NULL,

View File

@ -53,6 +53,9 @@ static void ExtractDropStmtTriggerAndRelationName(DropStmt *dropTriggerStmt,
char **relationName); char **relationName);
static void ErrorIfDropStmtDropsMultipleTriggers(DropStmt *dropTriggerStmt); static void ErrorIfDropStmtDropsMultipleTriggers(DropStmt *dropTriggerStmt);
static int16 GetTriggerTypeById(Oid triggerId); static int16 GetTriggerTypeById(Oid triggerId);
#if (PG_VERSION_NUM < PG_VERSION_15)
static void ErrorOutIfCloneTrigger(Oid tgrelid, const char *tgname);
#endif
/* GUC that overrides trigger checks for distributed tables and reference tables */ /* GUC that overrides trigger checks for distributed tables and reference tables */
@ -319,6 +322,40 @@ CreateTriggerEventExtendNames(CreateTrigStmt *createTriggerStmt, char *schemaNam
} }
/*
* PreprocessAlterTriggerRenameStmt is called before a ALTER TRIGGER RENAME
* command has been executed by standard process utility. This function errors
* out if we are trying to rename a child trigger on a partition of a distributed
* table. In PG15, this is not allowed anyway.
*/
List *
PreprocessAlterTriggerRenameStmt(Node *node, const char *queryString,
ProcessUtilityContext processUtilityContext)
{
#if (PG_VERSION_NUM < PG_VERSION_15)
RenameStmt *renameTriggerStmt = castNode(RenameStmt, node);
Assert(renameTriggerStmt->renameType == OBJECT_TRIGGER);
RangeVar *relation = renameTriggerStmt->relation;
bool missingOk = false;
Oid relationId = RangeVarGetRelid(relation, ALTER_TRIGGER_LOCK_MODE, missingOk);
if (!IsCitusTable(relationId))
{
return NIL;
}
EnsureCoordinator();
ErrorOutForTriggerIfNotSupported(relationId);
ErrorOutIfCloneTrigger(relationId, renameTriggerStmt->subname);
#endif
return NIL;
}
/* /*
* PostprocessAlterTriggerRenameStmt is called after a ALTER TRIGGER RENAME * PostprocessAlterTriggerRenameStmt is called after a ALTER TRIGGER RENAME
* command has been executed by standard process utility. This function errors * command has been executed by standard process utility. This function errors
@ -611,6 +648,64 @@ ErrorOutForTriggerIfNotSupported(Oid relationId)
} }
#if (PG_VERSION_NUM < PG_VERSION_15)
/*
* ErrorOutIfCloneTrigger is a helper function to error
* out if we are trying to rename a child trigger on a
* partition of a distributed table.
* A lot of this code is borrowed from PG15 because
* renaming clone triggers isn't allowed in PG15 anymore.
*/
static void
ErrorOutIfCloneTrigger(Oid tgrelid, const char *tgname)
{
HeapTuple tuple;
ScanKeyData key[2];
Relation tgrel = table_open(TriggerRelationId, RowExclusiveLock);
/*
* Search for the trigger to modify.
*/
ScanKeyInit(&key[0],
Anum_pg_trigger_tgrelid,
BTEqualStrategyNumber, F_OIDEQ,
ObjectIdGetDatum(tgrelid));
ScanKeyInit(&key[1],
Anum_pg_trigger_tgname,
BTEqualStrategyNumber, F_NAMEEQ,
CStringGetDatum(tgname));
SysScanDesc tgscan = systable_beginscan(tgrel, TriggerRelidNameIndexId, true,
NULL, 2, key);
if (HeapTupleIsValid(tuple = systable_getnext(tgscan)))
{
Form_pg_trigger trigform = (Form_pg_trigger) GETSTRUCT(tuple);
/*
* If the trigger descends from a trigger on a parent partitioned
* table, reject the rename.
* Appended shard ids to find the trigger on the partition's shards
* are not correct. Hence we would fail to find the trigger on the
* partition's shard.
*/
if (OidIsValid(trigform->tgparentid))
{
ereport(ERROR, (
errmsg(
"cannot rename child triggers on distributed partitions")));
}
}
systable_endscan(tgscan);
table_close(tgrel, RowExclusiveLock);
}
#endif
/* /*
* GetDropTriggerStmtRelation takes a DropStmt for a trigger object and returns * GetDropTriggerStmtRelation takes a DropStmt for a trigger object and returns
* RangeVar for the relation that owns the trigger. * RangeVar for the relation that owns the trigger.

View File

@ -1421,7 +1421,7 @@ DistributedExecutionRequiresRollback(List *taskList)
* Do not check SelectOpensTransactionBlock, always open transaction block * Do not check SelectOpensTransactionBlock, always open transaction block
* if SELECT FOR UPDATE is executed inside a distributed transaction. * if SELECT FOR UPDATE is executed inside a distributed transaction.
*/ */
return IsTransactionBlock(); return IsMultiStatementTransaction();
} }
if (ReadOnlyTask(task->taskType)) if (ReadOnlyTask(task->taskType))

View File

@ -31,6 +31,7 @@
#include "distributed/multi_server_executor.h" #include "distributed/multi_server_executor.h"
#include "distributed/multi_router_planner.h" #include "distributed/multi_router_planner.h"
#include "distributed/query_stats.h" #include "distributed/query_stats.h"
#include "distributed/shard_utils.h"
#include "distributed/subplan_execution.h" #include "distributed/subplan_execution.h"
#include "distributed/worker_log_messages.h" #include "distributed/worker_log_messages.h"
#include "distributed/worker_protocol.h" #include "distributed/worker_protocol.h"
@ -67,6 +68,9 @@ static void CitusEndScan(CustomScanState *node);
static void CitusReScan(CustomScanState *node); static void CitusReScan(CustomScanState *node);
static void SetJobColocationId(Job *job); static void SetJobColocationId(Job *job);
static void EnsureForceDelegationDistributionKey(Job *job); static void EnsureForceDelegationDistributionKey(Job *job);
static void EnsureAnchorShardsInJobExist(Job *job);
static bool AnchorShardsInTaskListExist(List *taskList);
static void TryToRerouteFastPathModifyQuery(Job *job);
/* create custom scan methods for all executors */ /* create custom scan methods for all executors */
@ -406,6 +410,19 @@ CitusBeginModifyScan(CustomScanState *node, EState *estate, int eflags)
/* prevent concurrent placement changes */ /* prevent concurrent placement changes */
AcquireMetadataLocks(workerJob->taskList); AcquireMetadataLocks(workerJob->taskList);
/*
* In case of a split, the shard might no longer be available. In that
* case try to reroute. We can only do this for fast path queries.
*/
if (currentPlan->fastPathRouterPlan &&
!AnchorShardsInTaskListExist(workerJob->taskList))
{
TryToRerouteFastPathModifyQuery(workerJob);
}
/* ensure there is no invalid shard */
EnsureAnchorShardsInJobExist(workerJob);
/* modify tasks are always assigned using first-replica policy */ /* modify tasks are always assigned using first-replica policy */
workerJob->taskList = FirstReplicaAssignTaskList(workerJob->taskList); workerJob->taskList = FirstReplicaAssignTaskList(workerJob->taskList);
} }
@ -440,6 +457,65 @@ CitusBeginModifyScan(CustomScanState *node, EState *estate, int eflags)
} }
/*
* TryToRerouteFastPathModifyQuery tries to reroute non-existent shards in given job if it finds any such shard,
* only for fastpath queries.
*
* Should only be called if the job belongs to a fastpath modify query
*/
static void
TryToRerouteFastPathModifyQuery(Job *job)
{
if (job->jobQuery->commandType == CMD_INSERT)
{
RegenerateTaskListForInsert(job);
}
else
{
RegenerateTaskForFasthPathQuery(job);
RebuildQueryStrings(job);
}
}
/*
* EnsureAnchorShardsInJobExist ensures all shards are valid in job.
* If it finds a non-existent shard in given job, it throws an error.
*/
static void
EnsureAnchorShardsInJobExist(Job *job)
{
if (!AnchorShardsInTaskListExist(job->taskList))
{
ereport(ERROR, (errcode(ERRCODE_T_R_SERIALIZATION_FAILURE),
errmsg("shard for the given value does not exist"),
errdetail(
"A concurrent shard split may have moved the data into a new set of shards."),
errhint("Retry the query.")));
}
}
/*
* AnchorShardsInTaskListExist checks whether all the anchor shards in the task list
* still exist.
*/
static bool
AnchorShardsInTaskListExist(List *taskList)
{
Task *task = NULL;
foreach_ptr(task, taskList)
{
if (!ShardExists(task->anchorShardId))
{
return false;
}
}
return true;
}
/* /*
* ModifyJobNeedsEvaluation checks whether the functions and parameters in the job query * ModifyJobNeedsEvaluation checks whether the functions and parameters in the job query
* need to be evaluated before we can build task query strings. * need to be evaluated before we can build task query strings.

View File

@ -233,7 +233,7 @@ static ScanKeyData DistObjectScanKey[3];
/* local function forward declarations */ /* local function forward declarations */
static HeapTuple PgDistPartitionTupleViaCatalog(Oid relationId); static HeapTuple PgDistPartitionTupleViaCatalog(Oid relationId);
static ShardIdCacheEntry * LookupShardIdCacheEntry(int64 shardId); static ShardIdCacheEntry * LookupShardIdCacheEntry(int64 shardId, bool missingOk);
static CitusTableCacheEntry * BuildCitusTableCacheEntry(Oid relationId); static CitusTableCacheEntry * BuildCitusTableCacheEntry(Oid relationId);
static void BuildCachedShardList(CitusTableCacheEntry *cacheEntry); static void BuildCachedShardList(CitusTableCacheEntry *cacheEntry);
static void PrepareWorkerNodeCache(void); static void PrepareWorkerNodeCache(void);
@ -282,10 +282,11 @@ static Oid LookupEnumValueId(Oid typeId, char *valueName);
static void InvalidateCitusTableCacheEntrySlot(CitusTableCacheEntrySlot *cacheSlot); static void InvalidateCitusTableCacheEntrySlot(CitusTableCacheEntrySlot *cacheSlot);
static void InvalidateDistTableCache(void); static void InvalidateDistTableCache(void);
static void InvalidateDistObjectCache(void); static void InvalidateDistObjectCache(void);
static void InitializeTableCacheEntry(int64 shardId); static bool InitializeTableCacheEntry(int64 shardId, bool missingOk);
static bool IsCitusTableTypeInternal(char partitionMethod, char replicationModel, static bool IsCitusTableTypeInternal(char partitionMethod, char replicationModel,
CitusTableType tableType); CitusTableType tableType);
static bool RefreshTableCacheEntryIfInvalid(ShardIdCacheEntry *shardEntry); static bool RefreshTableCacheEntryIfInvalid(ShardIdCacheEntry *shardEntry, bool
missingOk);
static Oid DistAuthinfoRelationId(void); static Oid DistAuthinfoRelationId(void);
static Oid DistAuthinfoIndexId(void); static Oid DistAuthinfoIndexId(void);
@ -783,7 +784,8 @@ CitusTableList(void)
ShardInterval * ShardInterval *
LoadShardInterval(uint64 shardId) LoadShardInterval(uint64 shardId)
{ {
ShardIdCacheEntry *shardIdEntry = LookupShardIdCacheEntry(shardId); bool missingOk = false;
ShardIdCacheEntry *shardIdEntry = LookupShardIdCacheEntry(shardId, missingOk);
CitusTableCacheEntry *tableEntry = shardIdEntry->tableEntry; CitusTableCacheEntry *tableEntry = shardIdEntry->tableEntry;
int shardIndex = shardIdEntry->shardIndex; int shardIndex = shardIdEntry->shardIndex;
@ -800,13 +802,33 @@ LoadShardInterval(uint64 shardId)
} }
/*
* ShardExists returns whether given shard exists or not. It fails if missingOk is false
* and shard is not found.
*/
bool
ShardExists(uint64 shardId)
{
bool missingOk = true;
ShardIdCacheEntry *shardIdEntry = LookupShardIdCacheEntry(shardId, missingOk);
if (!shardIdEntry)
{
return false;
}
return true;
}
/* /*
* RelationIdOfShard returns the relationId of the given shardId. * RelationIdOfShard returns the relationId of the given shardId.
*/ */
Oid Oid
RelationIdForShard(uint64 shardId) RelationIdForShard(uint64 shardId)
{ {
ShardIdCacheEntry *shardIdEntry = LookupShardIdCacheEntry(shardId); bool missingOk = false;
ShardIdCacheEntry *shardIdEntry = LookupShardIdCacheEntry(shardId, missingOk);
CitusTableCacheEntry *tableEntry = shardIdEntry->tableEntry; CitusTableCacheEntry *tableEntry = shardIdEntry->tableEntry;
return tableEntry->relationId; return tableEntry->relationId;
} }
@ -819,7 +841,8 @@ RelationIdForShard(uint64 shardId)
bool bool
ReferenceTableShardId(uint64 shardId) ReferenceTableShardId(uint64 shardId)
{ {
ShardIdCacheEntry *shardIdEntry = LookupShardIdCacheEntry(shardId); bool missingOk = false;
ShardIdCacheEntry *shardIdEntry = LookupShardIdCacheEntry(shardId, missingOk);
CitusTableCacheEntry *tableEntry = shardIdEntry->tableEntry; CitusTableCacheEntry *tableEntry = shardIdEntry->tableEntry;
return IsCitusTableTypeCacheEntry(tableEntry, REFERENCE_TABLE); return IsCitusTableTypeCacheEntry(tableEntry, REFERENCE_TABLE);
} }
@ -837,7 +860,8 @@ DistributedTableShardId(uint64 shardId)
return false; return false;
} }
ShardIdCacheEntry *shardIdEntry = LookupShardIdCacheEntry(shardId); bool missingOk = false;
ShardIdCacheEntry *shardIdEntry = LookupShardIdCacheEntry(shardId, missingOk);
CitusTableCacheEntry *tableEntry = shardIdEntry->tableEntry; CitusTableCacheEntry *tableEntry = shardIdEntry->tableEntry;
return IsCitusTableTypeCacheEntry(tableEntry, DISTRIBUTED_TABLE); return IsCitusTableTypeCacheEntry(tableEntry, DISTRIBUTED_TABLE);
} }
@ -852,7 +876,8 @@ DistributedTableShardId(uint64 shardId)
GroupShardPlacement * GroupShardPlacement *
LoadGroupShardPlacement(uint64 shardId, uint64 placementId) LoadGroupShardPlacement(uint64 shardId, uint64 placementId)
{ {
ShardIdCacheEntry *shardIdEntry = LookupShardIdCacheEntry(shardId); bool missingOk = false;
ShardIdCacheEntry *shardIdEntry = LookupShardIdCacheEntry(shardId, missingOk);
CitusTableCacheEntry *tableEntry = shardIdEntry->tableEntry; CitusTableCacheEntry *tableEntry = shardIdEntry->tableEntry;
int shardIndex = shardIdEntry->shardIndex; int shardIndex = shardIdEntry->shardIndex;
@ -887,7 +912,8 @@ LoadGroupShardPlacement(uint64 shardId, uint64 placementId)
ShardPlacement * ShardPlacement *
LoadShardPlacement(uint64 shardId, uint64 placementId) LoadShardPlacement(uint64 shardId, uint64 placementId)
{ {
ShardIdCacheEntry *shardIdEntry = LookupShardIdCacheEntry(shardId); bool missingOk = false;
ShardIdCacheEntry *shardIdEntry = LookupShardIdCacheEntry(shardId, missingOk);
CitusTableCacheEntry *tableEntry = shardIdEntry->tableEntry; CitusTableCacheEntry *tableEntry = shardIdEntry->tableEntry;
int shardIndex = shardIdEntry->shardIndex; int shardIndex = shardIdEntry->shardIndex;
GroupShardPlacement *groupPlacement = LoadGroupShardPlacement(shardId, placementId); GroupShardPlacement *groupPlacement = LoadGroupShardPlacement(shardId, placementId);
@ -910,7 +936,8 @@ ShardPlacementOnGroupIncludingOrphanedPlacements(int32 groupId, uint64 shardId)
{ {
ShardPlacement *placementOnNode = NULL; ShardPlacement *placementOnNode = NULL;
ShardIdCacheEntry *shardIdEntry = LookupShardIdCacheEntry(shardId); bool missingOk = false;
ShardIdCacheEntry *shardIdEntry = LookupShardIdCacheEntry(shardId, missingOk);
CitusTableCacheEntry *tableEntry = shardIdEntry->tableEntry; CitusTableCacheEntry *tableEntry = shardIdEntry->tableEntry;
int shardIndex = shardIdEntry->shardIndex; int shardIndex = shardIdEntry->shardIndex;
GroupShardPlacement *placementArray = GroupShardPlacement *placementArray =
@ -1130,7 +1157,8 @@ ShardPlacementListIncludingOrphanedPlacements(uint64 shardId)
{ {
List *placementList = NIL; List *placementList = NIL;
ShardIdCacheEntry *shardIdEntry = LookupShardIdCacheEntry(shardId); bool missingOk = false;
ShardIdCacheEntry *shardIdEntry = LookupShardIdCacheEntry(shardId, missingOk);
CitusTableCacheEntry *tableEntry = shardIdEntry->tableEntry; CitusTableCacheEntry *tableEntry = shardIdEntry->tableEntry;
int shardIndex = shardIdEntry->shardIndex; int shardIndex = shardIdEntry->shardIndex;
@ -1170,15 +1198,24 @@ ShardPlacementListIncludingOrphanedPlacements(uint64 shardId)
* build the cache entry. Afterwards we know that the shard has to be in the * build the cache entry. Afterwards we know that the shard has to be in the
* cache if it exists. If the shard does *not* exist, this function errors * cache if it exists. If the shard does *not* exist, this function errors
* (because LookupShardRelationFromCatalog errors out). * (because LookupShardRelationFromCatalog errors out).
*
* If missingOk is true and the shard cannot be found, the function returns false.
*/ */
static void static bool
InitializeTableCacheEntry(int64 shardId) InitializeTableCacheEntry(int64 shardId, bool missingOk)
{ {
bool missingOk = false;
Oid relationId = LookupShardRelationFromCatalog(shardId, missingOk); Oid relationId = LookupShardRelationFromCatalog(shardId, missingOk);
if (!OidIsValid(relationId))
{
Assert(missingOk);
return false;
}
/* trigger building the cache for the shard id */ /* trigger building the cache for the shard id */
GetCitusTableCacheEntry(relationId); /* lgtm[cpp/return-value-ignored] */ GetCitusTableCacheEntry(relationId); /* lgtm[cpp/return-value-ignored] */
return true;
} }
@ -1188,7 +1225,7 @@ InitializeTableCacheEntry(int64 shardId)
* entry in the cache and false if it didn't. * entry in the cache and false if it didn't.
*/ */
static bool static bool
RefreshTableCacheEntryIfInvalid(ShardIdCacheEntry *shardEntry) RefreshTableCacheEntryIfInvalid(ShardIdCacheEntry *shardEntry, bool missingOk)
{ {
/* /*
* We might have some concurrent metadata changes. In order to get the changes, * We might have some concurrent metadata changes. In order to get the changes,
@ -1200,7 +1237,8 @@ RefreshTableCacheEntryIfInvalid(ShardIdCacheEntry *shardEntry)
return false; return false;
} }
Oid oldRelationId = shardEntry->tableEntry->relationId; Oid oldRelationId = shardEntry->tableEntry->relationId;
Oid currentRelationId = LookupShardRelationFromCatalog(shardEntry->shardId, false); Oid currentRelationId = LookupShardRelationFromCatalog(shardEntry->shardId,
missingOk);
/* /*
* The relation OID to which the shard belongs could have changed, * The relation OID to which the shard belongs could have changed,
@ -1215,11 +1253,12 @@ RefreshTableCacheEntryIfInvalid(ShardIdCacheEntry *shardEntry)
/* /*
* LookupShardCacheEntry returns the cache entry belonging to a shard, or * LookupShardCacheEntry returns the cache entry belonging to a shard.
* errors out if that shard is unknown. * It errors out if that shard is unknown and missingOk is false. Else,
* it will return a NULL cache entry.
*/ */
static ShardIdCacheEntry * static ShardIdCacheEntry *
LookupShardIdCacheEntry(int64 shardId) LookupShardIdCacheEntry(int64 shardId, bool missingOk)
{ {
bool foundInCache = false; bool foundInCache = false;
bool recheck = false; bool recheck = false;
@ -1233,12 +1272,16 @@ LookupShardIdCacheEntry(int64 shardId)
if (!foundInCache) if (!foundInCache)
{ {
InitializeTableCacheEntry(shardId); if (!InitializeTableCacheEntry(shardId, missingOk))
{
return NULL;
}
recheck = true; recheck = true;
} }
else else
{ {
recheck = RefreshTableCacheEntryIfInvalid(shardEntry); recheck = RefreshTableCacheEntryIfInvalid(shardEntry, missingOk);
} }
/* /*
@ -1252,7 +1295,8 @@ LookupShardIdCacheEntry(int64 shardId)
if (!foundInCache) if (!foundInCache)
{ {
ereport(ERROR, (errmsg("could not find valid entry for shard " int eflag = (missingOk) ? DEBUG1 : ERROR;
ereport(eflag, (errmsg("could not find valid entry for shard "
UINT64_FORMAT, shardId))); UINT64_FORMAT, shardId)));
} }
} }
@ -4277,6 +4321,9 @@ InvalidateCitusTableCacheEntrySlot(CitusTableCacheEntrySlot *cacheSlot)
{ {
/* reload the metadata */ /* reload the metadata */
cacheSlot->citusTableMetadata->isValid = false; cacheSlot->citusTableMetadata->isValid = false;
/* clean up ShardIdCacheHash */
RemoveStaleShardIdCacheEntries(cacheSlot->citusTableMetadata);
} }
} }
@ -4677,37 +4724,6 @@ LookupShardRelationFromCatalog(int64 shardId, bool missingOk)
} }
/*
* ShardExists returns whether the given shard ID exists in pg_dist_shard.
*/
bool
ShardExists(int64 shardId)
{
ScanKeyData scanKey[1];
int scanKeyCount = 1;
Relation pgDistShard = table_open(DistShardRelationId(), AccessShareLock);
bool shardExists = false;
ScanKeyInit(&scanKey[0], Anum_pg_dist_shard_shardid,
BTEqualStrategyNumber, F_INT8EQ, Int64GetDatum(shardId));
SysScanDesc scanDescriptor = systable_beginscan(pgDistShard,
DistShardShardidIndexId(), true,
NULL, scanKeyCount, scanKey);
HeapTuple heapTuple = systable_getnext(scanDescriptor);
if (HeapTupleIsValid(heapTuple))
{
shardExists = true;
}
systable_endscan(scanDescriptor);
table_close(pgDistShard, NoLock);
return shardExists;
}
/* /*
* GetPartitionTypeInputInfo populates output parameters with the interval type * GetPartitionTypeInputInfo populates output parameters with the interval type
* identifier and modifier for the specified partition key/method combination. * identifier and modifier for the specified partition key/method combination.

View File

@ -2677,9 +2677,16 @@ DeleteNodeRow(char *nodeName, int32 nodePort)
/* /*
* simple_heap_delete() expects that the caller has at least an * simple_heap_delete() expects that the caller has at least an
* AccessShareLock on replica identity index. * AccessShareLock on primary key index.
*
* XXX: This does not seem required, do we really need to acquire this lock?
* Postgres doesn't acquire such locks on indexes before deleting catalog tuples.
* Linking here the reasons we added this lock acquirement:
* https://github.com/citusdata/citus/pull/2851#discussion_r306569462
* https://github.com/citusdata/citus/pull/2855#discussion_r313628554
* https://github.com/citusdata/citus/issues/1890
*/ */
Relation replicaIndex = index_open(RelationGetReplicaIndex(pgDistNode), Relation replicaIndex = index_open(RelationGetPrimaryKeyIndex(pgDistNode),
AccessShareLock); AccessShareLock);
ScanKeyInit(&scanKey[0], Anum_pg_dist_node_nodename, ScanKeyInit(&scanKey[0], Anum_pg_dist_node_nodename,

View File

@ -21,6 +21,7 @@
#include "catalog/pg_class.h" #include "catalog/pg_class.h"
#include "catalog/pg_enum.h" #include "catalog/pg_enum.h"
#include "distributed/adaptive_executor.h" #include "distributed/adaptive_executor.h"
#include "distributed/backend_data.h"
#include "distributed/citus_ruleutils.h" #include "distributed/citus_ruleutils.h"
#include "distributed/colocation_utils.h" #include "distributed/colocation_utils.h"
#include "distributed/commands.h" #include "distributed/commands.h"
@ -397,15 +398,28 @@ citus_move_shard_placement(PG_FUNCTION_ARGS)
targetNodeName, targetNodePort); targetNodeName, targetNodePort);
/*
* We want to be able to track progress of shard moves using
* get_rebalancer_progress. If this move is initiated by the rebalancer,
* then the rebalancer call has already set up the shared memory that is
* used to do that. But if citus_move_shard_placement is called directly by
* the user (or through any other mechanism), then the shared memory is not
* set up yet. In that case we do it here.
*/
if (!IsRebalancerInternalBackend())
{
WorkerNode *sourceNode = FindWorkerNode(sourceNodeName, sourceNodePort); WorkerNode *sourceNode = FindWorkerNode(sourceNodeName, sourceNodePort);
WorkerNode *targetNode = FindWorkerNode(targetNodeName, targetNodePort); WorkerNode *targetNode = FindWorkerNode(targetNodeName, targetNodePort);
PlacementUpdateEvent *placementUpdateEvent = palloc0(sizeof(PlacementUpdateEvent)); PlacementUpdateEvent *placementUpdateEvent = palloc0(
sizeof(PlacementUpdateEvent));
placementUpdateEvent->updateType = PLACEMENT_UPDATE_MOVE; placementUpdateEvent->updateType = PLACEMENT_UPDATE_MOVE;
placementUpdateEvent->shardId = shardId; placementUpdateEvent->shardId = shardId;
placementUpdateEvent->sourceNode = sourceNode; placementUpdateEvent->sourceNode = sourceNode;
placementUpdateEvent->targetNode = targetNode; placementUpdateEvent->targetNode = targetNode;
SetupRebalanceMonitor(list_make1(placementUpdateEvent), relationId); SetupRebalanceMonitor(list_make1(placementUpdateEvent), relationId,
REBALANCE_PROGRESS_MOVING);
}
/* /*
* At this point of the shard moves, we don't need to block the writes to * At this point of the shard moves, we don't need to block the writes to

View File

@ -201,6 +201,8 @@ static int PlacementsHashCompare(const void *lhsKey, const void *rhsKey, Size ke
static uint32 PlacementsHashHashCode(const void *key, Size keySize); static uint32 PlacementsHashHashCode(const void *key, Size keySize);
static bool WorkerNodeListContains(List *workerNodeList, const char *workerName, static bool WorkerNodeListContains(List *workerNodeList, const char *workerName,
uint32 workerPort); uint32 workerPort);
static void UpdateColocatedShardPlacementProgress(uint64 shardId, char *sourceName,
int sourcePort, uint64 progress);
static bool IsPlacementOnWorkerNode(ShardPlacement *placement, WorkerNode *workerNode); static bool IsPlacementOnWorkerNode(ShardPlacement *placement, WorkerNode *workerNode);
static NodeFillState * FindFillStateForPlacement(RebalanceState *state, static NodeFillState * FindFillStateForPlacement(RebalanceState *state,
ShardPlacement *placement); ShardPlacement *placement);
@ -258,6 +260,7 @@ PG_FUNCTION_INFO_V1(pg_dist_rebalance_strategy_enterprise_check);
bool RunningUnderIsolationTest = false; bool RunningUnderIsolationTest = false;
int MaxRebalancerLoggedIgnoredMoves = 5; int MaxRebalancerLoggedIgnoredMoves = 5;
#ifdef USE_ASSERT_CHECKING #ifdef USE_ASSERT_CHECKING
/* /*
@ -763,7 +766,9 @@ ExecutePlacementUpdates(List *placementUpdateList, Oid shardReplicationModeOid,
* dsm handle so that it can be used for updating the progress and cleaning things up. * dsm handle so that it can be used for updating the progress and cleaning things up.
*/ */
void void
SetupRebalanceMonitor(List *placementUpdateList, Oid relationId) SetupRebalanceMonitor(List *placementUpdateList,
Oid relationId,
uint64 initialProgressState)
{ {
List *colocatedUpdateList = GetColocatedRebalanceSteps(placementUpdateList); List *colocatedUpdateList = GetColocatedRebalanceSteps(placementUpdateList);
ListCell *colocatedUpdateCell = NULL; ListCell *colocatedUpdateCell = NULL;
@ -787,7 +792,7 @@ SetupRebalanceMonitor(List *placementUpdateList, Oid relationId)
event->shardId = colocatedUpdate->shardId; event->shardId = colocatedUpdate->shardId;
event->sourcePort = colocatedUpdate->sourceNode->workerPort; event->sourcePort = colocatedUpdate->sourceNode->workerPort;
event->targetPort = colocatedUpdate->targetNode->workerPort; event->targetPort = colocatedUpdate->targetNode->workerPort;
pg_atomic_init_u64(&event->progress, REBALANCE_PROGRESS_MOVING); pg_atomic_init_u64(&event->progress, initialProgressState);
eventIndex++; eventIndex++;
} }
@ -1186,34 +1191,63 @@ BuildShardSizesHash(ProgressMonitorData *monitor, HTAB *shardStatistics)
PlacementUpdateEventProgress *step = placementUpdateEvents + eventIndex; PlacementUpdateEventProgress *step = placementUpdateEvents + eventIndex;
uint64 shardId = step->shardId; uint64 shardId = step->shardId;
uint64 shardSize = 0;
uint64 backupShardSize = 0;
uint64 progress = pg_atomic_read_u64(&step->progress);
uint64 shardSize = WorkerShardSize(shardStatistics, step->sourceName, uint64 sourceSize = WorkerShardSize(shardStatistics, step->sourceName,
step->sourcePort, shardId); step->sourcePort, shardId);
uint64 targetSize = WorkerShardSize(shardStatistics, step->targetName,
step->targetPort, shardId);
if (progress == REBALANCE_PROGRESS_WAITING ||
progress == REBALANCE_PROGRESS_MOVING)
{
/*
* If we are not done with the move, the correct shard size is the
* size on the source.
*/
shardSize = sourceSize;
backupShardSize = targetSize;
}
else if (progress == REBALANCE_PROGRESS_MOVED)
{
/*
* If we are done with the move, the correct shard size is the size
* on the target
*/
shardSize = targetSize;
backupShardSize = sourceSize;
}
if (shardSize == 0) if (shardSize == 0)
{ {
/* if (backupShardSize == 0)
* It's possible that we are reading the sizes after the move has
* already fininshed. This means that the shards on the source
* might have already been deleted. In that case we instead report
* the size on the target as the shard size, since that is now the
* only existing shard.
*/
shardSize = WorkerShardSize(shardStatistics, step->targetName,
step->targetPort, shardId);
if (shardSize == 0)
{ {
/* /*
* We don't have any useful shard size. This can happen when a * We don't have any useful shard size. This can happen when a
* shard is moved multiple times and it is not present on * shard is moved multiple times and it is not present on
* either of these nodes. Probably the shard is on a worker * either of these nodes. Probably the shard is on a worker
* related to the next move. In the weird case that this shard * related to another event. In the weird case that this shard
* is on the nodes and actually is size 0, we will have no * is on the nodes and actually is size 0, we will have no
* entry in the hashmap. When fetching from it we always * entry in the hashmap. When fetching from it we always
* default to 0 if no entry is found, so that's fine. * default to 0 if no entry is found, so that's fine.
*/ */
continue; continue;
} }
/*
* Because of the way we fetch shard sizes they are from a slightly
* earlier moment than the progress state we just read from shared
* memory. Usually this is no problem, but there exist some race
* conditions where this matters. For example, for very quick moves
* it is possible that even though a step is now reported as MOVED,
* when we read the shard sizes the move had not even started yet.
* This in turn can mean that the target size is 0 while the source
* size is not. We try to handle such rare edge cases by falling
* back on the other shard size if that one is not 0.
*/
shardSize = backupShardSize;
} }
@ -1427,6 +1461,15 @@ GetMovedShardIdsByWorker(PlacementUpdateEventProgress *steps, int stepCount,
AddToWorkerShardIdSet(shardsByWorker, step->sourceName, step->sourcePort, AddToWorkerShardIdSet(shardsByWorker, step->sourceName, step->sourcePort,
step->shardId); step->shardId);
if (pg_atomic_read_u64(&step->progress) == REBALANCE_PROGRESS_WAITING)
{
/*
* shard move has not started so we don't need target stats for
* this shard
*/
continue;
}
AddToWorkerShardIdSet(shardsByWorker, step->targetName, step->targetPort, AddToWorkerShardIdSet(shardsByWorker, step->targetName, step->targetPort,
step->shardId); step->shardId);
} }
@ -1559,6 +1602,8 @@ RebalanceTableShards(RebalanceOptions *options, Oid shardReplicationModeOid)
* This uses the first relationId from the list, it's only used for display * This uses the first relationId from the list, it's only used for display
* purposes so it does not really matter which to show * purposes so it does not really matter which to show
*/ */
SetupRebalanceMonitor(placementUpdateList, linitial_oid(options->relationIdList),
REBALANCE_PROGRESS_WAITING);
ExecutePlacementUpdates(placementUpdateList, shardReplicationModeOid, "Moving"); ExecutePlacementUpdates(placementUpdateList, shardReplicationModeOid, "Moving");
FinalizeCurrentProgressMonitor(); FinalizeCurrentProgressMonitor();
} }
@ -1635,11 +1680,21 @@ UpdateShardPlacement(PlacementUpdateEvent *placementUpdateEvent,
errmsg("only moving or copying shards is supported"))); errmsg("only moving or copying shards is supported")));
} }
UpdateColocatedShardPlacementProgress(shardId,
sourceNode->workerName,
sourceNode->workerPort,
REBALANCE_PROGRESS_MOVING);
/* /*
* In case of failure, we throw an error such that rebalance_table_shards * In case of failure, we throw an error such that rebalance_table_shards
* fails early. * fails early.
*/ */
ExecuteRebalancerCommandInSeparateTransaction(placementUpdateCommand->data); ExecuteRebalancerCommandInSeparateTransaction(placementUpdateCommand->data);
UpdateColocatedShardPlacementProgress(shardId,
sourceNode->workerName,
sourceNode->workerPort,
REBALANCE_PROGRESS_MOVED);
} }
@ -2700,6 +2755,51 @@ WorkerNodeListContains(List *workerNodeList, const char *workerName, uint32 work
} }
/*
* UpdateColocatedShardPlacementProgress updates the progress of the given placement,
* along with its colocated placements, to the given state.
*/
static void
UpdateColocatedShardPlacementProgress(uint64 shardId, char *sourceName, int sourcePort,
uint64 progress)
{
ProgressMonitorData *header = GetCurrentProgressMonitor();
if (header != NULL)
{
PlacementUpdateEventProgress *steps = ProgressMonitorSteps(header);
ListCell *colocatedShardIntervalCell = NULL;
ShardInterval *shardInterval = LoadShardInterval(shardId);
List *colocatedShardIntervalList = ColocatedShardIntervalList(shardInterval);
for (int moveIndex = 0; moveIndex < header->stepCount; moveIndex++)
{
PlacementUpdateEventProgress *step = steps + moveIndex;
uint64 currentShardId = step->shardId;
bool colocatedShard = false;
foreach(colocatedShardIntervalCell, colocatedShardIntervalList)
{
ShardInterval *candidateShard = lfirst(colocatedShardIntervalCell);
if (candidateShard->shardId == currentShardId)
{
colocatedShard = true;
break;
}
}
if (colocatedShard &&
strcmp(step->sourceName, sourceName) == 0 &&
step->sourcePort == sourcePort)
{
pg_atomic_write_u64(&step->progress, progress);
}
}
}
}
/* /*
* pg_dist_rebalance_strategy_enterprise_check is a now removed function, but * pg_dist_rebalance_strategy_enterprise_check is a now removed function, but
* to avoid issues during upgrades a C stub is kept. * to avoid issues during upgrades a C stub is kept.

View File

@ -503,6 +503,8 @@ SplitShard(SplitMode splitMode,
List *workersForPlacementList = GetWorkerNodesFromWorkerIds(nodeIdsForPlacementList); List *workersForPlacementList = GetWorkerNodesFromWorkerIds(nodeIdsForPlacementList);
ErrorIfNotAllNodesHaveReferenceTableReplicas(workersForPlacementList);
List *sourceColocatedShardIntervalList = NIL; List *sourceColocatedShardIntervalList = NIL;
if (colocatedShardIntervalList == NIL) if (colocatedShardIntervalList == NIL)
{ {
@ -522,7 +524,6 @@ SplitShard(SplitMode splitMode,
if (splitMode == BLOCKING_SPLIT) if (splitMode == BLOCKING_SPLIT)
{ {
EnsureReferenceTablesExistOnAllNodesExtended(TRANSFER_MODE_BLOCK_WRITES);
BlockingShardSplit( BlockingShardSplit(
splitOperation, splitOperation,
splitWorkflowId, splitWorkflowId,

View File

@ -30,7 +30,6 @@ static List * RemoteScanTargetList(List *workerTargetList);
static PlannedStmt * BuildSelectStatementViaStdPlanner(Query *combineQuery, static PlannedStmt * BuildSelectStatementViaStdPlanner(Query *combineQuery,
List *remoteScanTargetList, List *remoteScanTargetList,
CustomScan *remoteScan); CustomScan *remoteScan);
static bool FindCitusExtradataContainerRTE(Node *node, RangeTblEntry **result);
static Plan * CitusCustomScanPathPlan(PlannerInfo *root, RelOptInfo *rel, static Plan * CitusCustomScanPathPlan(PlannerInfo *root, RelOptInfo *rel,
struct CustomPath *best_path, List *tlist, struct CustomPath *best_path, List *tlist,
@ -136,6 +135,12 @@ CreateCitusCustomScanPath(PlannerInfo *root, RelOptInfo *relOptInfo,
path->custom_path.path.pathtarget = relOptInfo->reltarget; path->custom_path.path.pathtarget = relOptInfo->reltarget;
path->custom_path.path.parent = relOptInfo; path->custom_path.path.parent = relOptInfo;
#if (PG_VERSION_NUM >= PG_VERSION_15)
/* necessary to avoid extra Result node in PG15 */
path->custom_path.flags = CUSTOMPATH_SUPPORT_PROJECTION;
#endif
/* /*
* The 100k rows we put on the cost of the path is kind of arbitrary and could be * The 100k rows we put on the cost of the path is kind of arbitrary and could be
* improved in accuracy to produce better plans. * improved in accuracy to produce better plans.
@ -317,7 +322,7 @@ BuildSelectStatementViaStdPlanner(Query *combineQuery, List *remoteScanTargetLis
* Finds the rangetable entry in the query that refers to the citus_extradata_container * Finds the rangetable entry in the query that refers to the citus_extradata_container
* and stores the pointer in result. * and stores the pointer in result.
*/ */
static bool bool
FindCitusExtradataContainerRTE(Node *node, RangeTblEntry **result) FindCitusExtradataContainerRTE(Node *node, RangeTblEntry **result)
{ {
if (node == NULL) if (node == NULL)

View File

@ -17,6 +17,7 @@
#include "catalog/pg_constraint.h" #include "catalog/pg_constraint.h"
#include "distributed/citus_nodefuncs.h" #include "distributed/citus_nodefuncs.h"
#include "distributed/citus_ruleutils.h" #include "distributed/citus_ruleutils.h"
#include "distributed/combine_query_planner.h"
#include "distributed/deparse_shard_query.h" #include "distributed/deparse_shard_query.h"
#include "distributed/insert_select_planner.h" #include "distributed/insert_select_planner.h"
#include "distributed/listutils.h" #include "distributed/listutils.h"
@ -79,12 +80,13 @@ RebuildQueryStrings(Job *workerJob)
if (UpdateOrDeleteQuery(query)) if (UpdateOrDeleteQuery(query))
{ {
List *relationShardList = task->relationShardList;
/* /*
* For UPDATE and DELETE queries, we may have subqueries and joins, so * For UPDATE and DELETE queries, we may have subqueries and joins, so
* we use relation shard list to update shard names and call * we use relation shard list to update shard names and call
* pg_get_query_def() directly. * pg_get_query_def() directly.
*/ */
List *relationShardList = task->relationShardList;
UpdateRelationToShardNames((Node *) query, relationShardList); UpdateRelationToShardNames((Node *) query, relationShardList);
} }
else if (query->commandType == CMD_INSERT && task->modifyWithSubquery) else if (query->commandType == CMD_INSERT && task->modifyWithSubquery)
@ -229,7 +231,16 @@ UpdateRelationToShardNames(Node *node, List *relationShardList)
RangeTblEntry *newRte = (RangeTblEntry *) node; RangeTblEntry *newRte = (RangeTblEntry *) node;
if (newRte->rtekind != RTE_RELATION) if (newRte->rtekind == RTE_FUNCTION)
{
newRte = NULL;
if (!FindCitusExtradataContainerRTE(node, &newRte))
{
/* only update function rtes containing citus_extradata_container */
return false;
}
}
else if (newRte->rtekind != RTE_RELATION)
{ {
return false; return false;
} }

View File

@ -559,9 +559,26 @@ int
GetRTEIdentity(RangeTblEntry *rte) GetRTEIdentity(RangeTblEntry *rte)
{ {
Assert(rte->rtekind == RTE_RELATION); Assert(rte->rtekind == RTE_RELATION);
Assert(rte->values_lists != NIL);
/*
* Since SQL functions might be in-lined by standard_planner,
* we might miss assigning an RTE identity for RangeTblEntries
* related to SQL functions. We already have checks in other
* places to throw an error for SQL functions but they are not
* sufficient due to function in-lining; so here we capture such
* cases and throw an error here.
*/
if (list_length(rte->values_lists) != 2)
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot perform distributed planning on this "
"query because parameterized queries for SQL "
"functions referencing distributed tables are "
"not supported"),
errhint("Consider using PL/pgSQL functions instead.")));
}
Assert(IsA(rte->values_lists, IntList)); Assert(IsA(rte->values_lists, IntList));
Assert(list_length(rte->values_lists) == 2);
return linitial_int(rte->values_lists); return linitial_int(rte->values_lists);
} }
@ -1369,7 +1386,14 @@ FinalizePlan(PlannedStmt *localPlan, DistributedPlan *distributedPlan)
Node *distributedPlanData = (Node *) distributedPlan; Node *distributedPlanData = (Node *) distributedPlan;
customScan->custom_private = list_make1(distributedPlanData); customScan->custom_private = list_make1(distributedPlanData);
#if (PG_VERSION_NUM >= PG_VERSION_15)
/* necessary to avoid extra Result node in PG15 */
customScan->flags = CUSTOMPATH_SUPPORT_BACKWARD_SCAN | CUSTOMPATH_SUPPORT_PROJECTION;
#else
customScan->flags = CUSTOMPATH_SUPPORT_BACKWARD_SCAN; customScan->flags = CUSTOMPATH_SUPPORT_BACKWARD_SCAN;
#endif
/* /*
* Fast path queries cannot have any subplans by definition, so skip * Fast path queries cannot have any subplans by definition, so skip

View File

@ -2677,9 +2677,6 @@ TargetShardIntervalForFastPathQuery(Query *query, bool *isMultiShardQuery,
} }
/* we're only expecting single shard from a single table */ /* we're only expecting single shard from a single table */
Node *distKey PG_USED_FOR_ASSERTS_ONLY = NULL;
Assert(FastPathRouterQuery(query, &distKey) || !EnableFastPathRouterPlanner);
if (list_length(prunedShardIntervalList) > 1) if (list_length(prunedShardIntervalList) > 1)
{ {
*isMultiShardQuery = true; *isMultiShardQuery = true;

View File

@ -109,11 +109,17 @@ GetCurrentProgressMonitor(void)
/* /*
* FinalizeCurrentProgressMonitor releases the dynamic memory segment of the current * FinalizeCurrentProgressMonitor releases the dynamic memory segment of the current
* progress monitoring data structure and removes the process from * progress monitoring data structure and removes the process from
* pg_stat_get_progress_info() output. * pg_stat_get_progress_info() output. If there's no such dynamic memory
* segment this is a no-op.
*/ */
void void
FinalizeCurrentProgressMonitor(void) FinalizeCurrentProgressMonitor(void)
{ {
if (currentProgressDSMHandle == DSM_HANDLE_INVALID)
{
return;
}
dsm_segment *dsmSegment = dsm_find_mapping(currentProgressDSMHandle); dsm_segment *dsmSegment = dsm_find_mapping(currentProgressDSMHandle);
if (dsmSegment != NULL) if (dsmSegment != NULL)

View File

@ -2125,6 +2125,20 @@ RegisterCitusConfigVariables(void)
ShowShardsForAppNamePrefixesAssignHook, ShowShardsForAppNamePrefixesAssignHook,
NULL); NULL);
DefineCustomBoolVariable(
"citus.skip_advisory_lock_permission_checks",
gettext_noop("Postgres would normally enforce some "
"ownership checks while acquiring locks. "
"When this setting is 'off', Citus skips"
"ownership checks on internal advisory "
"locks."),
NULL,
&SkipAdvisoryLockPermissionChecks,
false,
GUC_SUPERUSER_ONLY,
GUC_NO_SHOW_ALL,
NULL, NULL, NULL);
DefineCustomBoolVariable( DefineCustomBoolVariable(
"citus.skip_jsonb_validation_in_copy", "citus.skip_jsonb_validation_in_copy",
gettext_noop("Skip validation of JSONB columns on the coordinator during COPY " gettext_noop("Skip validation of JSONB columns on the coordinator during COPY "
@ -2692,10 +2706,17 @@ StatisticsCollectionGucCheckHook(bool *newval, void **extra, GucSource source)
static void static void
CitusAuthHook(Port *port, int status) CitusAuthHook(Port *port, int status)
{ {
uint64 gpid = ExtractGlobalPID(port->application_name); /*
* We determine the backend type here because other calls in this hook rely
* on it, both IsExternalClientBackend and InitializeBackendData. These
* calls would normally initialize its value based on the application_name
* global, but this global is not set yet at this point in the connection
* initialization. So here we determine it based on the value from Port.
*/
DetermineCitusBackendType(port->application_name);
/* external connections to not have a GPID immediately */ /* external connections to not have a GPID immediately */
if (gpid == INVALID_CITUS_INTERNAL_BACKEND_GPID) if (IsExternalClientBackend())
{ {
/* /*
* We raise the shared connection counter pre-emptively. As a result, we may * We raise the shared connection counter pre-emptively. As a result, we may
@ -2747,7 +2768,8 @@ CitusAuthHook(Port *port, int status)
* replication connection. A replication connection backend will never call * replication connection. A replication connection backend will never call
* StartupCitusBackend, which normally sets up the global PID. * StartupCitusBackend, which normally sets up the global PID.
*/ */
InitializeBackendData(gpid); InitializeBackendData(port->application_name);
/* let other authentication hooks to kick in first */ /* let other authentication hooks to kick in first */
if (original_client_auth_hook) if (original_client_auth_hook)

View File

@ -105,9 +105,6 @@ static BackendData *MyBackendData = NULL;
static CitusBackendType CurrentBackendType = CITUS_BACKEND_NOT_ASSIGNED; static CitusBackendType CurrentBackendType = CITUS_BACKEND_NOT_ASSIGNED;
static void DetermineCitusBackendType(void);
PG_FUNCTION_INFO_V1(assign_distributed_transaction_id); PG_FUNCTION_INFO_V1(assign_distributed_transaction_id);
PG_FUNCTION_INFO_V1(get_current_transaction_id); PG_FUNCTION_INFO_V1(get_current_transaction_id);
PG_FUNCTION_INFO_V1(get_global_active_transactions); PG_FUNCTION_INFO_V1(get_global_active_transactions);
@ -681,7 +678,7 @@ TotalProcCount(void)
* initialized this backend can be correctly shown in citus_lock_waits. * initialized this backend can be correctly shown in citus_lock_waits.
*/ */
void void
InitializeBackendData(uint64 globalPID) InitializeBackendData(const char *applicationName)
{ {
if (MyBackendData != NULL) if (MyBackendData != NULL)
{ {
@ -693,6 +690,8 @@ InitializeBackendData(uint64 globalPID)
return; return;
} }
uint64 gpid = ExtractGlobalPID(applicationName);
MyBackendData = &backendManagementShmemData->backends[MyProc->pgprocno]; MyBackendData = &backendManagementShmemData->backends[MyProc->pgprocno];
Assert(MyBackendData); Assert(MyBackendData);
@ -704,20 +703,8 @@ InitializeBackendData(uint64 globalPID)
UnSetGlobalPID(); UnSetGlobalPID();
SpinLockAcquire(&MyBackendData->mutex); SpinLockAcquire(&MyBackendData->mutex);
MyBackendData->distributedCommandOriginator = IsExternalClientBackend();
/* MyBackendData->globalPID = gpid;
* Use the given globalPID to initialize
*/
if (globalPID == INVALID_CITUS_INTERNAL_BACKEND_GPID)
{
MyBackendData->distributedCommandOriginator =
true;
}
else
{
MyBackendData->globalPID = globalPID;
MyBackendData->distributedCommandOriginator = false;
}
SpinLockRelease(&MyBackendData->mutex); SpinLockRelease(&MyBackendData->mutex);
/* /*
@ -1045,7 +1032,7 @@ citus_pid_for_gpid(PG_FUNCTION_ARGS)
* if the application name is not compatible with Citus' application names returns 0. * if the application name is not compatible with Citus' application names returns 0.
*/ */
uint64 uint64
ExtractGlobalPID(char *applicationName) ExtractGlobalPID(const char *applicationName)
{ {
/* does application name exist */ /* does application name exist */
if (!applicationName) if (!applicationName)
@ -1064,16 +1051,16 @@ ExtractGlobalPID(char *applicationName)
return INVALID_CITUS_INTERNAL_BACKEND_GPID; return INVALID_CITUS_INTERNAL_BACKEND_GPID;
} }
/* are the remaining characters of the application name numbers */
uint64 numberOfRemainingChars = strlen(applicationNameCopy) - prefixLength;
if (numberOfRemainingChars <= 0 ||
!strisdigit_s(applicationNameCopy + prefixLength, numberOfRemainingChars))
{
return INVALID_CITUS_INTERNAL_BACKEND_GPID;
}
char *globalPIDString = &applicationNameCopy[prefixLength]; char *globalPIDString = &applicationNameCopy[prefixLength];
uint64 globalPID = strtoul(globalPIDString, NULL, 10); uint64 globalPID = strtoul(globalPIDString, NULL, 10);
if (globalPID == 0)
{
/*
* INVALID_CITUS_INTERNAL_BACKEND_GPID is 0, but just to be explicit
* about how we handle strtoul errors.
*/
return INVALID_CITUS_INTERNAL_BACKEND_GPID;
}
return globalPID; return globalPID;
} }
@ -1371,7 +1358,7 @@ IsRebalancerInternalBackend(void)
{ {
if (CurrentBackendType == CITUS_BACKEND_NOT_ASSIGNED) if (CurrentBackendType == CITUS_BACKEND_NOT_ASSIGNED)
{ {
DetermineCitusBackendType(); DetermineCitusBackendType(application_name);
} }
return CurrentBackendType == CITUS_REBALANCER_BACKEND; return CurrentBackendType == CITUS_REBALANCER_BACKEND;
@ -1387,7 +1374,7 @@ IsCitusInternalBackend(void)
{ {
if (CurrentBackendType == CITUS_BACKEND_NOT_ASSIGNED) if (CurrentBackendType == CITUS_BACKEND_NOT_ASSIGNED)
{ {
DetermineCitusBackendType(); DetermineCitusBackendType(application_name);
} }
return CurrentBackendType == CITUS_INTERNAL_BACKEND; return CurrentBackendType == CITUS_INTERNAL_BACKEND;
@ -1403,29 +1390,41 @@ IsCitusRunCommandBackend(void)
{ {
if (CurrentBackendType == CITUS_BACKEND_NOT_ASSIGNED) if (CurrentBackendType == CITUS_BACKEND_NOT_ASSIGNED)
{ {
DetermineCitusBackendType(); DetermineCitusBackendType(application_name);
} }
return CurrentBackendType == CITUS_RUN_COMMAND_BACKEND; return CurrentBackendType == CITUS_RUN_COMMAND_BACKEND;
} }
bool
IsExternalClientBackend(void)
{
if (CurrentBackendType == CITUS_BACKEND_NOT_ASSIGNED)
{
DetermineCitusBackendType(application_name);
}
return CurrentBackendType == EXTERNAL_CLIENT_BACKEND;
}
/* /*
* DetermineCitusBackendType determines the type of backend based on the application_name. * DetermineCitusBackendType determines the type of backend based on the application_name.
*/ */
static void void
DetermineCitusBackendType(void) DetermineCitusBackendType(const char *applicationName)
{ {
if (ExtractGlobalPID(application_name) != INVALID_CITUS_INTERNAL_BACKEND_GPID) if (ExtractGlobalPID(applicationName) != INVALID_CITUS_INTERNAL_BACKEND_GPID)
{ {
CurrentBackendType = CITUS_INTERNAL_BACKEND; CurrentBackendType = CITUS_INTERNAL_BACKEND;
} }
else if (application_name && strcmp(application_name, CITUS_REBALANCER_NAME) == 0) else if (applicationName && strcmp(applicationName, CITUS_REBALANCER_NAME) == 0)
{ {
CurrentBackendType = CITUS_REBALANCER_BACKEND; CurrentBackendType = CITUS_REBALANCER_BACKEND;
} }
else if (application_name && else if (applicationName &&
strcmp(application_name, CITUS_RUN_COMMAND_APPLICATION_NAME) == 0) strcmp(applicationName, CITUS_RUN_COMMAND_APPLICATION_NAME) == 0)
{ {
CurrentBackendType = CITUS_RUN_COMMAND_BACKEND; CurrentBackendType = CITUS_RUN_COMMAND_BACKEND;
} }

View File

@ -1303,10 +1303,17 @@ DeleteColocationGroupLocally(uint32 colocationId)
{ {
/* /*
* simple_heap_delete() expects that the caller has at least an * simple_heap_delete() expects that the caller has at least an
* AccessShareLock on replica identity index. * AccessShareLock on primary key index.
*
* XXX: This does not seem required, do we really need to acquire this lock?
* Postgres doesn't acquire such locks on indexes before deleting catalog tuples.
* Linking here the reasons we added this lock acquirement:
* https://github.com/citusdata/citus/pull/2851#discussion_r306569462
* https://github.com/citusdata/citus/pull/2855#discussion_r313628554
* https://github.com/citusdata/citus/issues/1890
*/ */
Relation replicaIndex = Relation replicaIndex =
index_open(RelationGetReplicaIndex(pgDistColocation), index_open(RelationGetPrimaryKeyIndex(pgDistColocation),
AccessShareLock); AccessShareLock);
simple_heap_delete(pgDistColocation, &(heapTuple->t_self)); simple_heap_delete(pgDistColocation, &(heapTuple->t_self));

View File

@ -50,6 +50,7 @@ static void ReplicateReferenceTableShardToNode(ShardInterval *shardInterval,
int nodePort); int nodePort);
static bool AnyRelationsModifiedInTransaction(List *relationIdList); static bool AnyRelationsModifiedInTransaction(List *relationIdList);
static List * ReplicatedMetadataSyncedDistributedTableList(void); static List * ReplicatedMetadataSyncedDistributedTableList(void);
static bool NodeHasAllReferenceTableReplicas(WorkerNode *workerNode);
/* exports for SQL callable functions */ /* exports for SQL callable functions */
PG_FUNCTION_INFO_V1(upgrade_to_reference_table); PG_FUNCTION_INFO_V1(upgrade_to_reference_table);
@ -688,3 +689,81 @@ ReplicateAllReferenceTablesToNode(WorkerNode *workerNode)
} }
} }
} }
/*
* ErrorIfNotAllNodesHaveReferenceTableReplicas throws an error when one of the
* nodes in the list does not have reference table replicas.
*/
void
ErrorIfNotAllNodesHaveReferenceTableReplicas(List *workerNodeList)
{
WorkerNode *workerNode = NULL;
foreach_ptr(workerNode, workerNodeList)
{
if (!NodeHasAllReferenceTableReplicas(workerNode))
{
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("reference tables have not been replicated to "
"node %s:%d yet",
workerNode->workerName,
workerNode->workerPort),
errdetail("Reference tables are lazily replicated after "
"adding a node, but must exist before shards can "
"be created on that node."),
errhint("Run SELECT replicate_reference_tables(); to "
"ensure reference tables exist on all nodes.")));
}
}
}
/*
* NodeHasAllReferenceTablesReplicas returns whether the given worker node has reference
* table replicas. If there are no reference tables the function returns true.
*
* This function does not do any locking, so the situation could change immediately after,
* though we can only ever transition from false to true, so only "false" could be the
* incorrect answer.
*
* In the case where the function returns true because no reference tables exist
* on the node, a reference table could be created immediately after. However, the
* creation logic guarantees that this reference table will be created on all the
* nodes, so our answer was correct.
*/
static bool
NodeHasAllReferenceTableReplicas(WorkerNode *workerNode)
{
List *referenceTableIdList = CitusTableTypeIdList(REFERENCE_TABLE);
if (list_length(referenceTableIdList) == 0)
{
/* no reference tables exist */
return true;
}
Oid referenceTableId = linitial_oid(referenceTableIdList);
List *shardIntervalList = LoadShardIntervalList(referenceTableId);
if (list_length(shardIntervalList) != 1)
{
/* check for corrupt metadata */
ereport(ERROR, (errmsg("reference table \"%s\" can only have 1 shard",
get_rel_name(referenceTableId))));
}
ShardInterval *shardInterval = (ShardInterval *) linitial(shardIntervalList);
List *shardPlacementList = ActiveShardPlacementList(shardInterval->shardId);
ShardPlacement *placement = NULL;
foreach_ptr(placement, shardPlacementList)
{
if (placement->groupId == workerNode->groupId)
{
/* our worker has a reference table placement */
return true;
}
}
return false;
}

View File

@ -109,6 +109,8 @@ PG_FUNCTION_INFO_V1(lock_relation_if_exists);
/* Config variable managed via guc.c */ /* Config variable managed via guc.c */
bool EnableAcquiringUnsafeLockFromWorkers = false; bool EnableAcquiringUnsafeLockFromWorkers = false;
bool SkipAdvisoryLockPermissionChecks = false;
/* /*
* lock_shard_metadata allows the shard distribution metadata to be locked * lock_shard_metadata allows the shard distribution metadata to be locked
@ -248,7 +250,10 @@ lock_shard_resources(PG_FUNCTION_ARGS)
continue; continue;
} }
if (!SkipAdvisoryLockPermissionChecks)
{
EnsureTablePermissions(relationId, aclMask); EnsureTablePermissions(relationId, aclMask);
}
LockShardResource(shardId, lockMode); LockShardResource(shardId, lockMode);
} }

View File

@ -16,7 +16,10 @@
#include "utils/fmgrprotos.h" #include "utils/fmgrprotos.h"
#include "utils/lsyscache.h" #include "utils/lsyscache.h"
#include "distributed/coordinator_protocol.h" #include "distributed/coordinator_protocol.h"
#include "distributed/listutils.h"
#include "distributed/log_utils.h"
#include "distributed/metadata_utility.h" #include "distributed/metadata_utility.h"
#include "distributed/multi_physical_planner.h"
#include "distributed/relay_utility.h" #include "distributed/relay_utility.h"
#include "distributed/shard_utils.h" #include "distributed/shard_utils.h"

View File

@ -50,7 +50,7 @@ extern void BackendManagementShmemInit(void);
extern size_t BackendManagementShmemSize(void); extern size_t BackendManagementShmemSize(void);
extern void InitializeBackendManagement(void); extern void InitializeBackendManagement(void);
extern int TotalProcCount(void); extern int TotalProcCount(void);
extern void InitializeBackendData(uint64 globalPID); extern void InitializeBackendData(const char *applicationName);
extern void LockBackendSharedMemory(LWLockMode lockMode); extern void LockBackendSharedMemory(LWLockMode lockMode);
extern void UnlockBackendSharedMemory(void); extern void UnlockBackendSharedMemory(void);
extern void UnSetDistributedTransactionId(void); extern void UnSetDistributedTransactionId(void);
@ -62,7 +62,7 @@ extern void SetBackendDataGlobalPID(uint64 globalPID);
extern uint64 GetGlobalPID(void); extern uint64 GetGlobalPID(void);
extern void SetBackendDataDistributedCommandOriginator(bool extern void SetBackendDataDistributedCommandOriginator(bool
distributedCommandOriginator); distributedCommandOriginator);
extern uint64 ExtractGlobalPID(char *applicationName); extern uint64 ExtractGlobalPID(const char *applicationName);
extern int ExtractNodeIdFromGlobalPID(uint64 globalPID, bool missingOk); extern int ExtractNodeIdFromGlobalPID(uint64 globalPID, bool missingOk);
extern int ExtractProcessIdFromGlobalPID(uint64 globalPID); extern int ExtractProcessIdFromGlobalPID(uint64 globalPID);
extern void GetBackendDataForProc(PGPROC *proc, BackendData *result); extern void GetBackendDataForProc(PGPROC *proc, BackendData *result);
@ -74,9 +74,11 @@ extern LocalTransactionId GetMyProcLocalTransactionId(void);
extern int GetExternalClientBackendCount(void); extern int GetExternalClientBackendCount(void);
extern uint32 IncrementExternalClientBackendCounter(void); extern uint32 IncrementExternalClientBackendCounter(void);
extern void DecrementExternalClientBackendCounter(void); extern void DecrementExternalClientBackendCounter(void);
extern void DetermineCitusBackendType(const char *applicationName);
extern bool IsCitusInternalBackend(void); extern bool IsCitusInternalBackend(void);
extern bool IsRebalancerInternalBackend(void); extern bool IsRebalancerInternalBackend(void);
extern bool IsCitusRunCommandBackend(void); extern bool IsCitusRunCommandBackend(void);
extern bool IsExternalClientBackend(void);
extern void ResetCitusBackendType(void); extern void ResetCitusBackendType(void);
#define INVALID_CITUS_INTERNAL_BACKEND_GPID 0 #define INVALID_CITUS_INTERNAL_BACKEND_GPID 0

View File

@ -27,6 +27,7 @@ extern Path * CreateCitusCustomScanPath(PlannerInfo *root, RelOptInfo *relOptInf
CustomScan *remoteScan); CustomScan *remoteScan);
extern PlannedStmt * PlanCombineQuery(struct DistributedPlan *distributedPlan, extern PlannedStmt * PlanCombineQuery(struct DistributedPlan *distributedPlan,
struct CustomScan *dataScan); struct CustomScan *dataScan);
extern bool FindCitusExtradataContainerRTE(Node *node, RangeTblEntry **result);
extern bool ReplaceCitusExtraDataContainer; extern bool ReplaceCitusExtraDataContainer;
extern CustomScan *ReplaceCitusExtraDataContainerWithCustomScan; extern CustomScan *ReplaceCitusExtraDataContainerWithCustomScan;

View File

@ -667,6 +667,8 @@ extern List * CreateTriggerStmtObjectAddress(Node *node, bool missingOk, bool
isPostprocess); isPostprocess);
extern void CreateTriggerEventExtendNames(CreateTrigStmt *createTriggerStmt, extern void CreateTriggerEventExtendNames(CreateTrigStmt *createTriggerStmt,
char *schemaName, uint64 shardId); char *schemaName, uint64 shardId);
extern List * PreprocessAlterTriggerRenameStmt(Node *node, const char *queryString,
ProcessUtilityContext processUtilityContext);
extern List * PostprocessAlterTriggerRenameStmt(Node *node, const char *queryString); extern List * PostprocessAlterTriggerRenameStmt(Node *node, const char *queryString);
extern void AlterTriggerRenameEventExtendNames(RenameStmt *renameTriggerStmt, extern void AlterTriggerRenameEventExtendNames(RenameStmt *renameTriggerStmt,
char *schemaName, uint64 shardId); char *schemaName, uint64 shardId);

View File

@ -157,6 +157,7 @@ extern uint32 ColocationIdViaCatalog(Oid relationId);
extern bool IsCitusLocalTableByDistParams(char partitionMethod, char replicationModel); extern bool IsCitusLocalTableByDistParams(char partitionMethod, char replicationModel);
extern List * CitusTableList(void); extern List * CitusTableList(void);
extern ShardInterval * LoadShardInterval(uint64 shardId); extern ShardInterval * LoadShardInterval(uint64 shardId);
extern bool ShardExists(uint64 shardId);
extern Oid RelationIdForShard(uint64 shardId); extern Oid RelationIdForShard(uint64 shardId);
extern bool ReferenceTableShardId(uint64 shardId); extern bool ReferenceTableShardId(uint64 shardId);
extern bool DistributedTableShardId(uint64 shardId); extern bool DistributedTableShardId(uint64 shardId);
@ -174,7 +175,6 @@ extern int32 GetLocalNodeId(void);
extern void CitusTableCacheFlushInvalidatedEntries(void); extern void CitusTableCacheFlushInvalidatedEntries(void);
extern Oid LookupShardRelationFromCatalog(int64 shardId, bool missing_ok); extern Oid LookupShardRelationFromCatalog(int64 shardId, bool missing_ok);
extern List * ShardPlacementListIncludingOrphanedPlacements(uint64 shardId); extern List * ShardPlacementListIncludingOrphanedPlacements(uint64 shardId);
extern bool ShardExists(int64 shardId);
extern void CitusInvalidateRelcacheByRelid(Oid relationId); extern void CitusInvalidateRelcacheByRelid(Oid relationId);
extern void CitusInvalidateRelcacheByShardId(int64 shardId); extern void CitusInvalidateRelcacheByShardId(int64 shardId);
extern void InvalidateForeignKeyGraph(void); extern void InvalidateForeignKeyGraph(void);

View File

@ -26,5 +26,6 @@ extern void DeleteAllReplicatedTablePlacementsFromNodeGroup(int32 groupId,
bool localOnly); bool localOnly);
extern int CompareOids(const void *leftElement, const void *rightElement); extern int CompareOids(const void *leftElement, const void *rightElement);
extern void ReplicateAllReferenceTablesToNode(WorkerNode *workerNode); extern void ReplicateAllReferenceTablesToNode(WorkerNode *workerNode);
extern void ErrorIfNotAllNodesHaveReferenceTableReplicas(List *workerNodeList);
#endif /* REFERENCE_TABLE_UTILS_H_ */ #endif /* REFERENCE_TABLE_UTILS_H_ */

View File

@ -150,6 +150,7 @@ enum DistLockConfigs
DIST_LOCK_NOWAIT = 2 DIST_LOCK_NOWAIT = 2
}; };
/* Lock shard/relation metadata for safe modifications */ /* Lock shard/relation metadata for safe modifications */
extern void LockShardDistributionMetadata(int64 shardId, LOCKMODE lockMode); extern void LockShardDistributionMetadata(int64 shardId, LOCKMODE lockMode);
extern void LockPlacementCleanup(void); extern void LockPlacementCleanup(void);
@ -195,5 +196,6 @@ extern void AcquireDistributedLockOnRelations(List *relationList, LOCKMODE lockM
extern void PreprocessLockStatement(LockStmt *stmt, ProcessUtilityContext context); extern void PreprocessLockStatement(LockStmt *stmt, ProcessUtilityContext context);
extern bool EnableAcquiringUnsafeLockFromWorkers; extern bool EnableAcquiringUnsafeLockFromWorkers;
extern bool SkipAdvisoryLockPermissionChecks;
#endif /* RESOURCE_LOCK_H */ #endif /* RESOURCE_LOCK_H */

View File

@ -73,7 +73,9 @@
/* *INDENT-ON* */ /* *INDENT-ON* */
#define REBALANCE_ACTIVITY_MAGIC_NUMBER 1337 #define REBALANCE_ACTIVITY_MAGIC_NUMBER 1337
#define REBALANCE_PROGRESS_WAITING 0
#define REBALANCE_PROGRESS_MOVING 1 #define REBALANCE_PROGRESS_MOVING 1
#define REBALANCE_PROGRESS_MOVED 2
/* Enumeration that defines different placement update types */ /* Enumeration that defines different placement update types */
typedef enum typedef enum
@ -193,7 +195,9 @@ extern List * ReplicationPlacementUpdates(List *workerNodeList, List *shardPlace
extern void ExecuteRebalancerCommandInSeparateTransaction(char *command); extern void ExecuteRebalancerCommandInSeparateTransaction(char *command);
extern void AcquirePlacementColocationLock(Oid relationId, int lockMode, extern void AcquirePlacementColocationLock(Oid relationId, int lockMode,
const char *operationName); const char *operationName);
extern void SetupRebalanceMonitor(List *placementUpdateList, Oid relationId);
extern void SetupRebalanceMonitor(List *placementUpdateList,
Oid relationId,
uint64 initialProgressState);
#endif /* SHARD_REBALANCER_H */ #endif /* SHARD_REBALANCER_H */

View File

@ -1,4 +1,4 @@
test: upgrade_basic_after upgrade_type_after upgrade_ref2ref_after upgrade_distributed_function_after upgrade_rebalance_strategy_after upgrade_list_citus_objects upgrade_autoconverted_after upgrade_citus_stat_activity upgrade_citus_locks test: upgrade_basic_after upgrade_type_after upgrade_ref2ref_after upgrade_distributed_function_after upgrade_rebalance_strategy_after upgrade_list_citus_objects upgrade_autoconverted_after upgrade_citus_stat_activity upgrade_citus_locks upgrade_distributed_triggers_after
# This attempts dropping citus extension (and rollbacks), so please do # This attempts dropping citus extension (and rollbacks), so please do
# not run in parallel with any other tests. # not run in parallel with any other tests.

View File

@ -8,6 +8,7 @@ test: upgrade_distributed_function_before upgrade_rebalance_strategy_before
test: upgrade_autoconverted_before test: upgrade_autoconverted_before
test: upgrade_citus_stat_activity test: upgrade_citus_stat_activity
test: upgrade_citus_locks test: upgrade_citus_locks
test: upgrade_distributed_triggers_before
# upgrade_columnar_before renames public schema to citus_schema, so let's # upgrade_columnar_before renames public schema to citus_schema, so let's
# run this test as the last one. # run this test as the last one.

View File

@ -78,5 +78,36 @@ $$);
(1 row) (1 row)
END; END;
CREATE OR REPLACE FUNCTION select_for_update()
RETURNS void
AS $$
DECLARE
my int;
BEGIN
SELECT y INTO my FROM test WHERE x = 1 FOR UPDATE;
END;
$$ LANGUAGE plpgsql;
-- so that we can prove that we open a transaction block by logging below:
-- "NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;..."
SET citus.log_remote_commands TO on;
SELECT select_for_update();
NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(xx, xx, 'xxxxxxx');
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
CONTEXT: SQL statement "SELECT y FROM test WHERE x = 1 FOR UPDATE"
PL/pgSQL function select_for_update() line XX at SQL statement
NOTICE: issuing SELECT y FROM adaptive_executor.test_801009000 test WHERE (x OPERATOR(pg_catalog.=) 1) FOR UPDATE OF test
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
CONTEXT: SQL statement "SELECT y FROM test WHERE x = 1 FOR UPDATE"
PL/pgSQL function select_for_update() line XX at SQL statement
NOTICE: issuing COMMIT
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
select_for_update
---------------------------------------------------------------------
(1 row)
SET citus.log_remote_commands TO off;
DROP SCHEMA adaptive_executor CASCADE; DROP SCHEMA adaptive_executor CASCADE;
NOTICE: drop cascades to table test NOTICE: drop cascades to 2 other objects
DETAIL: drop cascades to table test
drop cascades to function select_for_update()

View File

@ -0,0 +1,41 @@
CREATE SCHEMA adv_lock_permission;
SET search_path to adv_lock_permission;
-- do not cache any connections, we change some settings and don't want old ones cached
SET citus.max_cached_conns_per_worker TO 0;
CREATE ROLE user_1 WITH LOGIN;
CREATE TABLE reference_table_1 (A int);
SELECT create_reference_table('reference_table_1');
create_reference_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE reference_table_2 (A int);
SELECT create_reference_table('reference_table_2');
create_reference_table
---------------------------------------------------------------------
(1 row)
GRANT USAGE ON SCHEMA adv_lock_permission TO user_1;
GRANT SELECT ON reference_table_1 TO user_1;
GRANT INSERT, UPDATE ON reference_table_2 TO user_1;
SET ROLE user_1;
-- do not cache any connections, we change some settings and don't want old ones cached
SET citus.max_cached_conns_per_worker TO 0;
SET search_path to adv_lock_permission;
INSERT INTO reference_table_2 SELECT * FROM reference_table_1;
ERROR: permission denied for table reference_table_1
CONTEXT: while executing command on localhost:xxxxx
SET ROLE postgres;
-- do not cache any connections, we change some settings and don't want old ones cached
SET citus.max_cached_conns_per_worker TO 0;
-- change the role so that it can skip permission checks
ALTER ROLE user_1 SET citus.skip_advisory_lock_permission_checks TO on;
SET ROLE user_1;
SET citus.max_cached_conns_per_worker TO 0;
INSERT INTO reference_table_2 SELECT * FROM reference_table_1;
SET ROLE postgres;
SET client_min_messages TO ERROR;
DROP SCHEMA adv_lock_permission CASCADE;
DROP ROLE user_1;

View File

@ -887,22 +887,39 @@ ALTER TABLE loc_tb ADD CONSTRAINT fkey FOREIGN KEY (a) references ref_tb(a);
ERROR: Citus can not handle circular dependencies between distributed objects ERROR: Citus can not handle circular dependencies between distributed objects
-- drop the view&matview with circular dependency -- drop the view&matview with circular dependency
DROP VIEW v103 CASCADE; DROP VIEW v103 CASCADE;
SET client_min_messages TO DEBUG1;
-- now it should successfully add to metadata and create the views on workers -- now it should successfully add to metadata and create the views on workers
ALTER TABLE loc_tb ADD CONSTRAINT fkey FOREIGN KEY (a) references ref_tb(a); ALTER TABLE loc_tb ADD CONSTRAINT fkey FOREIGN KEY (a) references ref_tb(a);
DEBUG: executing "CREATE OR REPLACE VIEW citus_local_tables_mx.v100 (a) AS SELECT loc_tb.a -- verify the views are created on workers
FROM citus_local_tables_mx.loc_tb; ALTER VIEW citus_local_tables_mx.v100 OWNER TO postgres" select run_command_on_workers($$SELECT count(*)=0 from citus_local_tables_mx.v100$$);
DEBUG: "view v100" has dependency to "table loc_tb" that is not in Citus' metadata run_command_on_workers
DEBUG: executing "CREATE OR REPLACE VIEW citus_local_tables_mx.v101 (a) AS SELECT loc_tb.a ---------------------------------------------------------------------
FROM (citus_local_tables_mx.loc_tb (localhost,57637,t,t)
JOIN citus_local_tables_mx.ref_tb USING (a)); ALTER VIEW citus_local_tables_mx.v101 OWNER TO postgres" (localhost,57638,t,t)
DEBUG: "view v101" has dependency to "table loc_tb" that is not in Citus' metadata (2 rows)
DEBUG: executing "CREATE MATERIALIZED VIEW citus_local_tables_mx.matview_101 USING heap AS SELECT loc_tb.a
FROM citus_local_tables_mx.loc_tb;ALTER MATERIALIZED VIEW citus_local_tables_mx.matview_101 OWNER TO postgres" select run_command_on_workers($$SELECT count(*)=0 from citus_local_tables_mx.v101$$);
DEBUG: executing "CREATE OR REPLACE VIEW citus_local_tables_mx.v102 (a) AS SELECT v101.a run_command_on_workers
FROM citus_local_tables_mx.v101; ALTER VIEW citus_local_tables_mx.v102 OWNER TO postgres" ---------------------------------------------------------------------
DEBUG: "view v102" has dependency to "table loc_tb" that is not in Citus' metadata (localhost,57637,t,t)
DEBUG: validating foreign key constraint "fkey_xxxxxxx" (localhost,57638,t,t)
(2 rows)
select run_command_on_workers($$SELECT count(*)=0 from citus_local_tables_mx.v102$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,t)
(localhost,57638,t,t)
(2 rows)
CREATE TABLE loc_tb_2 (a int);
CREATE VIEW v104 AS SELECT * from loc_tb_2;
SET client_min_messages TO DEBUG1;
-- verify the CREATE command for the view is generated correctly
ALTER TABLE loc_tb_2 ADD CONSTRAINT fkey_2 FOREIGN KEY (a) references ref_tb(a);
DEBUG: executing "CREATE OR REPLACE VIEW citus_local_tables_mx.v104 (a) AS SELECT loc_tb_2.a
FROM citus_local_tables_mx.loc_tb_2; ALTER VIEW citus_local_tables_mx.v104 OWNER TO postgres"
DEBUG: "view v104" has dependency to "table loc_tb_2" that is not in Citus' metadata
DEBUG: validating foreign key constraint "fkey_2_1330083"
SET client_min_messages TO WARNING; SET client_min_messages TO WARNING;
-- works fine -- works fine
select run_command_on_workers($$SELECT count(*) from citus_local_tables_mx.v100, citus_local_tables_mx.v101, citus_local_tables_mx.v102$$); select run_command_on_workers($$SELECT count(*) from citus_local_tables_mx.v100, citus_local_tables_mx.v101, citus_local_tables_mx.v102$$);
@ -922,14 +939,14 @@ select run_command_on_workers($$SELECT count(*) from citus_local_tables_mx.v100$
(localhost,57638,f,"ERROR: relation ""citus_local_tables_mx.v100"" does not exist") (localhost,57638,f,"ERROR: relation ""citus_local_tables_mx.v100"" does not exist")
(2 rows) (2 rows)
select run_command_on_workers($$SELECT count(*) from citus_local_tables_mx.v101$$); select run_command_on_workers($$SELECT count(*) from citus_local_tables_mx.v101$$);
run_command_on_workers run_command_on_workers
--------------------------------------------------------------------- ---------------------------------------------------------------------
(localhost,57637,f,"ERROR: relation ""citus_local_tables_mx.v101"" does not exist") (localhost,57637,f,"ERROR: relation ""citus_local_tables_mx.v101"" does not exist")
(localhost,57638,f,"ERROR: relation ""citus_local_tables_mx.v101"" does not exist") (localhost,57638,f,"ERROR: relation ""citus_local_tables_mx.v101"" does not exist")
(2 rows) (2 rows)
select run_command_on_workers($$SELECT count(*) from citus_local_tables_mx.v102$$); select run_command_on_workers($$SELECT count(*) from citus_local_tables_mx.v102$$);
run_command_on_workers run_command_on_workers
--------------------------------------------------------------------- ---------------------------------------------------------------------
(localhost,57637,f,"ERROR: relation ""citus_local_tables_mx.v102"" does not exist") (localhost,57637,f,"ERROR: relation ""citus_local_tables_mx.v102"" does not exist")

View File

@ -264,21 +264,17 @@ EXPLAIN (analyze on, costs off, timing off, summary off)
Columnar Projected Columns: a Columnar Projected Columns: a
(9 rows) (9 rows)
SELECT plan_without_arrows($Q$
EXPLAIN (costs off, timing off, summary off) EXPLAIN (costs off, timing off, summary off)
SELECT y, * FROM another_columnar_table; SELECT y, * FROM another_columnar_table;
$Q$); QUERY PLAN
plan_without_arrows
--------------------------------------------------------------------- ---------------------------------------------------------------------
Custom Scan (ColumnarScan) on another_columnar_table Custom Scan (ColumnarScan) on another_columnar_table
Columnar Projected Columns: x, y Columnar Projected Columns: x, y
(2 rows) (2 rows)
SELECT plan_without_arrows($Q$
EXPLAIN (costs off, timing off, summary off) EXPLAIN (costs off, timing off, summary off)
SELECT *, x FROM another_columnar_table; SELECT *, x FROM another_columnar_table;
$Q$); QUERY PLAN
plan_without_arrows
--------------------------------------------------------------------- ---------------------------------------------------------------------
Custom Scan (ColumnarScan) on another_columnar_table Custom Scan (ColumnarScan) on another_columnar_table
Columnar Projected Columns: x, y Columnar Projected Columns: x, y

View File

@ -958,15 +958,13 @@ SELECT * FROM weird_col_explain;
(7 rows) (7 rows)
\set VERBOSITY terse \set VERBOSITY terse
SELECT public.plan_without_result_lines($Q$
EXPLAIN (COSTS OFF, SUMMARY OFF) EXPLAIN (COSTS OFF, SUMMARY OFF)
SELECT *, "bbbbbbbbbbbbbbbbbbbbbbbbb\!bbbb'bbbbbbbbbbbbbbbbbbbbb''bbbbbbbb" SELECT *, "bbbbbbbbbbbbbbbbbbbbbbbbb\!bbbb'bbbbbbbbbbbbbbbbbbbbb''bbbbbbbb"
FROM weird_col_explain FROM weird_col_explain
WHERE "bbbbbbbbbbbbbbbbbbbbbbbbb\!bbbb'bbbbbbbbbbbbbbbbbbbbb''bbbbbbbb" * 2 > WHERE "bbbbbbbbbbbbbbbbbbbbbbbbb\!bbbb'bbbbbbbbbbbbbbbbbbbbb''bbbbbbbb" * 2 >
"aaaaaaaaaaaa$aaaaaa$$aaaaaaaaaaaaaaaaaaaaaaaaaaaaa'aaaaaaaa'$a'!"; "aaaaaaaaaaaa$aaaaaa$$aaaaaaaaaaaaaaaaaaaaaaaaaaaaa'aaaaaaaa'$a'!";
$Q$);
NOTICE: identifier "aaaaaaaaaaaa$aaaaaa$$aaaaaaaaaaaaaaaaaaaaaaaaaaaaa'aaaaaaaa'$a'!" will be truncated to "aaaaaaaaaaaa$aaaaaa$$aaaaaaaaaaaaaaaaaaaaaaaaaaaaa'aaaaaaaa'$a'" NOTICE: identifier "aaaaaaaaaaaa$aaaaaa$$aaaaaaaaaaaaaaaaaaaaaaaaaaaaa'aaaaaaaa'$a'!" will be truncated to "aaaaaaaaaaaa$aaaaaa$$aaaaaaaaaaaaaaaaaaaaaaaaaaaaa'aaaaaaaa'$a'"
plan_without_result_lines QUERY PLAN
--------------------------------------------------------------------- ---------------------------------------------------------------------
Custom Scan (Citus Adaptive) Custom Scan (Citus Adaptive)
Task Count: 4 Task Count: 4

View File

@ -1261,10 +1261,8 @@ NOTICE: copying the data has completed
(1 row) (1 row)
SELECT public.plan_without_result_lines($Q$
explain (costs off) insert into table_with_sequences select y, x from table_with_sequences; explain (costs off) insert into table_with_sequences select y, x from table_with_sequences;
$Q$); QUERY PLAN
plan_without_result_lines
--------------------------------------------------------------------- ---------------------------------------------------------------------
Custom Scan (Citus INSERT ... SELECT) Custom Scan (Citus INSERT ... SELECT)
INSERT/SELECT method: pull to coordinator INSERT/SELECT method: pull to coordinator
@ -1289,10 +1287,8 @@ NOTICE: copying the data has completed
(1 row) (1 row)
SELECT public.plan_without_result_lines($Q$
explain (costs off) insert into table_with_user_sequences select y, x from table_with_user_sequences; explain (costs off) insert into table_with_user_sequences select y, x from table_with_user_sequences;
$Q$); QUERY PLAN
plan_without_result_lines
--------------------------------------------------------------------- ---------------------------------------------------------------------
Custom Scan (Citus INSERT ... SELECT) Custom Scan (Citus INSERT ... SELECT)
INSERT/SELECT method: pull to coordinator INSERT/SELECT method: pull to coordinator

View File

@ -1261,10 +1261,8 @@ NOTICE: copying the data has completed
(1 row) (1 row)
SELECT public.plan_without_result_lines($Q$
explain (costs off) insert into table_with_sequences select y, x from table_with_sequences; explain (costs off) insert into table_with_sequences select y, x from table_with_sequences;
$Q$); QUERY PLAN
plan_without_result_lines
--------------------------------------------------------------------- ---------------------------------------------------------------------
Custom Scan (Citus INSERT ... SELECT) Custom Scan (Citus INSERT ... SELECT)
INSERT/SELECT method: pull to coordinator INSERT/SELECT method: pull to coordinator
@ -1289,10 +1287,8 @@ NOTICE: copying the data has completed
(1 row) (1 row)
SELECT public.plan_without_result_lines($Q$
explain (costs off) insert into table_with_user_sequences select y, x from table_with_user_sequences; explain (costs off) insert into table_with_user_sequences select y, x from table_with_user_sequences;
$Q$); QUERY PLAN
plan_without_result_lines
--------------------------------------------------------------------- ---------------------------------------------------------------------
Custom Scan (Citus INSERT ... SELECT) Custom Scan (Citus INSERT ... SELECT)
INSERT/SELECT method: pull to coordinator INSERT/SELECT method: pull to coordinator

View File

@ -56,7 +56,6 @@ step s2-commit:
COMMIT; COMMIT;
step s1-update: <... completed> step s1-update: <... completed>
ERROR: could not find valid entry for shard xxxxx
step s1-commit: step s1-commit:
COMMIT; COMMIT;
@ -80,7 +79,7 @@ nodeport|shardid|success|result
id|value id|value
--------------------------------------------------------------------- ---------------------------------------------------------------------
123456789| 1 123456789| 111
(1 row) (1 row)
@ -140,7 +139,6 @@ step s2-commit:
COMMIT; COMMIT;
step s1-delete: <... completed> step s1-delete: <... completed>
ERROR: could not find valid entry for shard xxxxx
step s1-commit: step s1-commit:
COMMIT; COMMIT;
@ -158,14 +156,13 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500001|t | 0 57637|1500001|t | 0
57637|1500003|t | 1 57637|1500003|t | 0
57638|1500004|t | 0 57638|1500004|t | 0
(3 rows) (3 rows)
id|value id|value
--------------------------------------------------------------------- ---------------------------------------------------------------------
123456789| 1 (0 rows)
(1 row)
starting permutation: s1-load-cache s1-begin s1-select s2-begin s2-blocking-shard-split s1-insert s2-commit s1-commit s2-print-cluster starting permutation: s1-load-cache s1-begin s1-select s2-begin s2-blocking-shard-split s1-insert s2-commit s1-commit s2-print-cluster
@ -221,7 +218,6 @@ get_shard_id_for_distribution_column
1500002 1500002
(1 row) (1 row)
ERROR: could not find valid entry for shard xxxxx
step s1-commit: step s1-commit:
COMMIT; COMMIT;
@ -239,13 +235,14 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500001|t | 0 57637|1500001|t | 0
57637|1500003|t | 0 57637|1500003|t | 1
57638|1500004|t | 0 57638|1500004|t | 0
(3 rows) (3 rows)
id|value id|value
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) 123456789| 1
(1 row)
starting permutation: s1-load-cache s1-begin s1-select s2-begin s2-blocking-shard-split s1-copy s2-commit s1-commit s2-print-cluster starting permutation: s1-load-cache s1-begin s1-select s2-begin s2-blocking-shard-split s1-copy s2-commit s1-commit s2-print-cluster
@ -373,7 +370,6 @@ step s2-commit:
COMMIT; COMMIT;
step s1-update: <... completed> step s1-update: <... completed>
ERROR: could not find valid entry for shard xxxxx
step s1-commit: step s1-commit:
COMMIT; COMMIT;
@ -397,7 +393,7 @@ nodeport|shardid|success|result
id|value id|value
--------------------------------------------------------------------- ---------------------------------------------------------------------
123456789| 1 123456789| 111
(1 row) (1 row)
@ -453,7 +449,6 @@ step s2-commit:
COMMIT; COMMIT;
step s1-delete: <... completed> step s1-delete: <... completed>
ERROR: could not find valid entry for shard xxxxx
step s1-commit: step s1-commit:
COMMIT; COMMIT;
@ -471,14 +466,13 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500001|t | 0 57637|1500001|t | 0
57637|1500003|t | 1 57637|1500003|t | 0
57638|1500004|t | 0 57638|1500004|t | 0
(3 rows) (3 rows)
id|value id|value
--------------------------------------------------------------------- ---------------------------------------------------------------------
123456789| 1 (0 rows)
(1 row)
starting permutation: s1-begin s1-select s2-begin s2-blocking-shard-split s1-insert s2-commit s1-commit s2-print-cluster starting permutation: s1-begin s1-select s2-begin s2-blocking-shard-split s1-insert s2-commit s1-commit s2-print-cluster
@ -530,7 +524,6 @@ get_shard_id_for_distribution_column
1500002 1500002
(1 row) (1 row)
ERROR: could not find valid entry for shard xxxxx
step s1-commit: step s1-commit:
COMMIT; COMMIT;
@ -548,13 +541,14 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500001|t | 0 57637|1500001|t | 0
57637|1500003|t | 0 57637|1500003|t | 1
57638|1500004|t | 0 57638|1500004|t | 0
(3 rows) (3 rows)
id|value id|value
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) 123456789| 1
(1 row)
starting permutation: s1-begin s1-select s2-begin s2-blocking-shard-split s1-copy s2-commit s1-commit s2-print-cluster starting permutation: s1-begin s1-select s2-begin s2-blocking-shard-split s1-copy s2-commit s1-commit s2-print-cluster

View File

@ -116,7 +116,7 @@ step s3-view-worker:
query |citus_nodename_for_nodeid|citus_nodeport_for_nodeid|state |wait_event_type|wait_event|usename |datname query |citus_nodename_for_nodeid|citus_nodeport_for_nodeid|state |wait_event_type|wait_event|usename |datname
--------------------------------------------------------------------- ---------------------------------------------------------------------
INSERT INTO public.test_table_1300008 (column1, column2) VALUES (100, 100)|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression INSERT INTO public.test_table_1300003 (column1, column2) VALUES (100, 100)|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression
(1 row) (1 row)
step s2-rollback: step s2-rollback:
@ -180,10 +180,10 @@ step s3-view-worker:
query |citus_nodename_for_nodeid|citus_nodeport_for_nodeid|state |wait_event_type|wait_event|usename |datname query |citus_nodename_for_nodeid|citus_nodeport_for_nodeid|state |wait_event_type|wait_event|usename |datname
--------------------------------------------------------------------- ---------------------------------------------------------------------
SELECT count(*) AS count FROM public.test_table_1300014 test_table WHERE true|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression SELECT count(*) AS count FROM public.test_table_1300004 test_table WHERE true|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression
SELECT count(*) AS count FROM public.test_table_1300013 test_table WHERE true|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression SELECT count(*) AS count FROM public.test_table_1300003 test_table WHERE true|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression
SELECT count(*) AS count FROM public.test_table_1300012 test_table WHERE true|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression SELECT count(*) AS count FROM public.test_table_1300002 test_table WHERE true|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression
SELECT count(*) AS count FROM public.test_table_1300011 test_table WHERE true|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression SELECT count(*) AS count FROM public.test_table_1300001 test_table WHERE true|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression
(4 rows) (4 rows)
step s2-rollback: step s2-rollback:
@ -247,7 +247,7 @@ step s3-view-worker:
query |citus_nodename_for_nodeid|citus_nodeport_for_nodeid|state |wait_event_type|wait_event|usename |datname query |citus_nodename_for_nodeid|citus_nodeport_for_nodeid|state |wait_event_type|wait_event|usename |datname
--------------------------------------------------------------------- ---------------------------------------------------------------------
SELECT count(*) AS count FROM public.test_table_1300017 test_table WHERE (column1 OPERATOR(pg_catalog.=) 55)|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression SELECT count(*) AS count FROM public.test_table_1300002 test_table WHERE (column1 OPERATOR(pg_catalog.=) 55)|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression
(1 row) (1 row)
step s2-rollback: step s2-rollback:

View File

@ -434,7 +434,6 @@ step s1-end:
COMMIT; COMMIT;
step s4-insert: <... completed> step s4-insert: <... completed>
ERROR: could not find valid entry for shard xxxxx
step s4-end: step s4-end:
COMMIT; COMMIT;
@ -452,13 +451,14 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57638|1500002|t | 0 57638|1500002|t | 0
57638|1500003|t | 1 57638|1500003|t | 2
(2 rows) (2 rows)
id|value id|value
--------------------------------------------------------------------- ---------------------------------------------------------------------
900| 1
123456789| 1 123456789| 1
(1 row) (2 rows)
starting permutation: s2-print-cluster s3-acquire-advisory-lock s1-begin s2-begin s1-non-blocking-shard-split s2-insert s2-end s2-print-cluster s3-release-advisory-lock s1-end s2-print-cluster starting permutation: s2-print-cluster s3-acquire-advisory-lock s1-begin s2-begin s1-non-blocking-shard-split s2-insert s2-end s2-print-cluster s3-release-advisory-lock s1-end s2-print-cluster

View File

@ -35,7 +35,9 @@ table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname
--------------------------------------------------------------------- ---------------------------------------------------------------------
colocated1|1500001| 49152|localhost | 57637| 49152|localhost | 57638| 0| 1 colocated1|1500001| 49152|localhost | 57637| 49152|localhost | 57638| 0| 1
colocated2|1500005| 376832|localhost | 57637| 376832|localhost | 57638| 0| 1 colocated2|1500005| 376832|localhost | 57637| 376832|localhost | 57638| 0| 1
(2 rows) colocated1|1500002| 196608|localhost | 57637| 196608|localhost | 57638| 0| 0
colocated2|1500006| 8192|localhost | 57637| 8192|localhost | 57638| 0| 0
(4 rows)
step s2-unlock-1-start: step s2-unlock-1-start:
ROLLBACK; ROLLBACK;
@ -112,9 +114,11 @@ step s7-get-progress:
table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname|targetport|target_shard_size|progress
--------------------------------------------------------------------- ---------------------------------------------------------------------
colocated1|1500001| 73728|localhost | 57637| 0|localhost | 57638| 73728| 2
colocated2|1500005| 401408|localhost | 57637| 0|localhost | 57638| 401408| 2
colocated1|1500002| 196608|localhost | 57637| 196608|localhost | 57638| 0| 1 colocated1|1500002| 196608|localhost | 57637| 196608|localhost | 57638| 0| 1
colocated2|1500006| 8192|localhost | 57637| 8192|localhost | 57638| 0| 1 colocated2|1500006| 8192|localhost | 57637| 8192|localhost | 57638| 0| 1
(2 rows) (4 rows)
step s3-unlock-2-start: step s3-unlock-2-start:
ROLLBACK; ROLLBACK;
@ -205,7 +209,9 @@ table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname
--------------------------------------------------------------------- ---------------------------------------------------------------------
colocated1|1500001| 49152|localhost | 57637| 49152|localhost | 57638| 73728| 1 colocated1|1500001| 49152|localhost | 57637| 49152|localhost | 57638| 73728| 1
colocated2|1500005| 376832|localhost | 57637| 376832|localhost | 57638| 401408| 1 colocated2|1500005| 376832|localhost | 57637| 376832|localhost | 57638| 401408| 1
(2 rows) colocated1|1500002| 196608|localhost | 57637| 196608|localhost | 57638| 0| 0
colocated2|1500006| 8192|localhost | 57637| 8192|localhost | 57638| 0| 0
(4 rows)
step s7-release-lock: step s7-release-lock:
COMMIT; COMMIT;
@ -288,7 +294,9 @@ table_name|shardid|shard_size|sourcename|sourceport|source_shard_size|targetname
--------------------------------------------------------------------- ---------------------------------------------------------------------
colocated1|1500001| 49152|localhost | 57637| 49152|localhost | 57638| 8192| 1 colocated1|1500001| 49152|localhost | 57637| 49152|localhost | 57638| 8192| 1
colocated2|1500005| 376832|localhost | 57637| 376832|localhost | 57638| 8192| 1 colocated2|1500005| 376832|localhost | 57637| 376832|localhost | 57638| 8192| 1
(2 rows) colocated1|1500002| 196608|localhost | 57637| 196608|localhost | 57638| 0| 0
colocated2|1500006| 8192|localhost | 57637| 8192|localhost | 57638| 0| 0
(4 rows)
step s6-release-advisory-lock: step s6-release-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152); SELECT pg_advisory_unlock(44000, 55152);

View File

@ -44,7 +44,6 @@ step s2-commit:
COMMIT; COMMIT;
step s1-update: <... completed> step s1-update: <... completed>
ERROR: could not find valid entry for shard xxxxx
step s1-commit: step s1-commit:
COMMIT; COMMIT;
@ -69,7 +68,7 @@ nodeport|shardid|success|result
id|value id|value
--------------------------------------------------------------------- ---------------------------------------------------------------------
5| 10 5| 5
(1 row) (1 row)
@ -117,7 +116,6 @@ step s2-commit:
COMMIT; COMMIT;
step s1-delete: <... completed> step s1-delete: <... completed>
ERROR: could not find valid entry for shard xxxxx
step s1-commit: step s1-commit:
COMMIT; COMMIT;
@ -135,11 +133,85 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500009|t | 0 57637|1500009|t | 0
57637|1500010|t | 1 57637|1500010|t | 0
57637|1500011|t | 0 57637|1500011|t | 0
57638|1500008|t | 0 57638|1500008|t | 0
(4 rows) (4 rows)
id|value
---------------------------------------------------------------------
(0 rows)
starting permutation: s1-load-cache s1-insert s1-begin s1-select s2-begin s2-isolate-tenant s1-update-complex s2-commit s1-commit s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-load-cache:
TRUNCATE isolation_table;
step s1-insert:
INSERT INTO isolation_table VALUES (5, 10);
step s1-begin:
BEGIN;
-- the tests are written with the logic where single shard SELECTs
-- do not to open transaction blocks
SET citus.select_opens_transaction_block TO false;
step s1-select:
SELECT count(*) FROM isolation_table WHERE id = 5;
count
---------------------------------------------------------------------
1
(1 row)
step s2-begin:
BEGIN;
step s2-isolate-tenant:
SELECT isolate_tenant_to_new_shard('isolation_table', 5, shard_transfer_mode => 'block_writes');
isolate_tenant_to_new_shard
---------------------------------------------------------------------
1500016
(1 row)
step s1-update-complex:
UPDATE isolation_table SET value = 5 WHERE id IN (
SELECT max(id) FROM isolation_table
);
<waiting ...>
step s2-commit:
COMMIT;
step s1-update-complex: <... completed>
ERROR: shard for the given value does not exist
step s1-commit:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('isolation_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM isolation_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500015|t | 0
57637|1500016|t | 1
57637|1500017|t | 0
57638|1500014|t | 0
(4 rows)
id|value id|value
--------------------------------------------------------------------- ---------------------------------------------------------------------
5| 10 5| 10
@ -177,7 +249,7 @@ step s2-isolate-tenant:
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500016 1500022
(1 row) (1 row)
step s1-insert: step s1-insert:
@ -187,7 +259,6 @@ step s2-commit:
COMMIT; COMMIT;
step s1-insert: <... completed> step s1-insert: <... completed>
ERROR: could not find valid entry for shard xxxxx
step s1-commit: step s1-commit:
COMMIT; COMMIT;
@ -204,15 +275,16 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500015|t | 0 57637|1500021|t | 0
57637|1500016|t | 0 57637|1500022|t | 1
57637|1500017|t | 0 57637|1500023|t | 0
57638|1500014|t | 0 57638|1500020|t | 0
(4 rows) (4 rows)
id|value id|value
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) 5| 10
(1 row)
starting permutation: s1-load-cache s1-begin s1-select s2-begin s2-isolate-tenant s1-copy s2-commit s1-commit s2-print-cluster starting permutation: s1-load-cache s1-begin s1-select s2-begin s2-isolate-tenant s1-copy s2-commit s1-commit s2-print-cluster
@ -246,7 +318,7 @@ step s2-isolate-tenant:
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500022 1500028
(1 row) (1 row)
step s1-copy: step s1-copy:
@ -273,10 +345,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500021|t | 0 57637|1500027|t | 0
57637|1500022|t | 0 57637|1500028|t | 0
57637|1500023|t | 0 57637|1500029|t | 0
57638|1500020|t | 0 57638|1500026|t | 0
(4 rows) (4 rows)
id|value id|value
@ -315,7 +387,7 @@ step s2-isolate-tenant:
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500028 1500034
(1 row) (1 row)
step s1-ddl: step s1-ddl:
@ -341,10 +413,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500027|t | 0 57637|1500033|t | 0
57637|1500028|t | 0 57637|1500034|t | 0
57637|1500029|t | 0 57637|1500035|t | 0
57638|1500026|t | 0 57638|1500032|t | 0
(4 rows) (4 rows)
id|value id|value
@ -399,7 +471,7 @@ step s2-isolate-tenant:
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500034 1500040
(1 row) (1 row)
step s1-update: step s1-update:
@ -409,7 +481,6 @@ step s2-commit:
COMMIT; COMMIT;
step s1-update: <... completed> step s1-update: <... completed>
ERROR: could not find valid entry for shard xxxxx
step s1-commit: step s1-commit:
COMMIT; COMMIT;
@ -426,15 +497,15 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500033|t | 0 57637|1500039|t | 0
57637|1500034|t | 1 57637|1500040|t | 1
57637|1500035|t | 0 57637|1500041|t | 0
57638|1500032|t | 0 57638|1500038|t | 0
(4 rows) (4 rows)
id|value id|value
--------------------------------------------------------------------- ---------------------------------------------------------------------
5| 10 5| 5
(1 row) (1 row)
@ -469,7 +540,7 @@ step s2-isolate-tenant:
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500040 1500046
(1 row) (1 row)
step s1-delete: step s1-delete:
@ -479,7 +550,6 @@ step s2-commit:
COMMIT; COMMIT;
step s1-delete: <... completed> step s1-delete: <... completed>
ERROR: could not find valid entry for shard xxxxx
step s1-commit: step s1-commit:
COMMIT; COMMIT;
@ -496,10 +566,81 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500039|t | 0 57637|1500045|t | 0
57637|1500040|t | 1 57637|1500046|t | 0
57637|1500041|t | 0 57637|1500047|t | 0
57638|1500038|t | 0 57638|1500044|t | 0
(4 rows)
id|value
---------------------------------------------------------------------
(0 rows)
starting permutation: s1-insert s1-begin s1-select s2-begin s2-isolate-tenant s1-update-complex s2-commit s1-commit s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-insert:
INSERT INTO isolation_table VALUES (5, 10);
step s1-begin:
BEGIN;
-- the tests are written with the logic where single shard SELECTs
-- do not to open transaction blocks
SET citus.select_opens_transaction_block TO false;
step s1-select:
SELECT count(*) FROM isolation_table WHERE id = 5;
count
---------------------------------------------------------------------
1
(1 row)
step s2-begin:
BEGIN;
step s2-isolate-tenant:
SELECT isolate_tenant_to_new_shard('isolation_table', 5, shard_transfer_mode => 'block_writes');
isolate_tenant_to_new_shard
---------------------------------------------------------------------
1500052
(1 row)
step s1-update-complex:
UPDATE isolation_table SET value = 5 WHERE id IN (
SELECT max(id) FROM isolation_table
);
<waiting ...>
step s2-commit:
COMMIT;
step s1-update-complex: <... completed>
ERROR: shard for the given value does not exist
step s1-commit:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('isolation_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM isolation_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500051|t | 0
57637|1500052|t | 1
57637|1500053|t | 0
57638|1500050|t | 0
(4 rows) (4 rows)
id|value id|value
@ -536,7 +677,7 @@ step s2-isolate-tenant:
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500046 1500058
(1 row) (1 row)
step s1-insert: step s1-insert:
@ -546,7 +687,6 @@ step s2-commit:
COMMIT; COMMIT;
step s1-insert: <... completed> step s1-insert: <... completed>
ERROR: could not find valid entry for shard xxxxx
step s1-commit: step s1-commit:
COMMIT; COMMIT;
@ -563,15 +703,16 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500045|t | 0 57637|1500057|t | 0
57637|1500046|t | 0 57637|1500058|t | 1
57637|1500047|t | 0 57637|1500059|t | 0
57638|1500044|t | 0 57638|1500056|t | 0
(4 rows) (4 rows)
id|value id|value
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) 5| 10
(1 row)
starting permutation: s1-begin s1-select s2-begin s2-isolate-tenant s1-copy s2-commit s1-commit s2-print-cluster starting permutation: s1-begin s1-select s2-begin s2-isolate-tenant s1-copy s2-commit s1-commit s2-print-cluster
@ -602,7 +743,7 @@ step s2-isolate-tenant:
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500052 1500064
(1 row) (1 row)
step s1-copy: step s1-copy:
@ -629,10 +770,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500051|t | 0 57637|1500063|t | 0
57637|1500052|t | 0 57637|1500064|t | 0
57637|1500053|t | 0 57637|1500065|t | 0
57638|1500050|t | 0 57638|1500062|t | 0
(4 rows) (4 rows)
id|value id|value
@ -668,7 +809,7 @@ step s2-isolate-tenant:
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500058 1500070
(1 row) (1 row)
step s1-ddl: step s1-ddl:
@ -694,10 +835,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500057|t | 0 57637|1500069|t | 0
57637|1500058|t | 0 57637|1500070|t | 0
57637|1500059|t | 0 57637|1500071|t | 0
57638|1500056|t | 0 57638|1500068|t | 0
(4 rows) (4 rows)
id|value id|value
@ -744,7 +885,7 @@ step s1-isolate-tenant:
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500064 1500076
(1 row) (1 row)
step s2-isolate-tenant: step s2-isolate-tenant:
@ -767,10 +908,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500061|t | 1 57637|1500073|t | 1
57638|1500063|t | 0 57638|1500075|t | 0
57638|1500064|t | 0 57638|1500076|t | 0
57638|1500065|t | 0 57638|1500077|t | 0
(4 rows) (4 rows)
id|value id|value
@ -799,7 +940,7 @@ step s1-isolate-tenant:
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500070 1500082
(1 row) (1 row)
step s2-isolate-tenant: step s2-isolate-tenant:
@ -822,10 +963,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500067|t | 1 57637|1500079|t | 1
57638|1500069|t | 0 57638|1500081|t | 0
57638|1500070|t | 0 57638|1500082|t | 0
57638|1500071|t | 0 57638|1500083|t | 0
(4 rows) (4 rows)
id|value id|value

View File

@ -59,7 +59,7 @@ t
step s2-isolate-tenant: <... completed> step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500078 1500090
(1 row) (1 row)
step s2-commit: step s2-commit:
@ -78,10 +78,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500077|t | 0 57637|1500089|t | 0
57637|1500078|t | 1 57637|1500090|t | 1
57637|1500079|t | 0 57637|1500091|t | 0
57638|1500074|t | 0 57638|1500086|t | 0
(4 rows) (4 rows)
id|value id|value
@ -149,7 +149,7 @@ t
step s2-isolate-tenant: <... completed> step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500086 1500098
(1 row) (1 row)
step s2-commit: step s2-commit:
@ -168,10 +168,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500085|t | 0 57637|1500097|t | 0
57637|1500086|t | 0 57637|1500098|t | 0
57637|1500087|t | 0 57637|1500099|t | 0
57638|1500082|t | 0 57638|1500094|t | 0
(4 rows) (4 rows)
id|value id|value
@ -179,6 +179,98 @@ id|value
(0 rows) (0 rows)
starting permutation: s1-load-cache s1-insert s3-acquire-advisory-lock s1-begin s1-select s2-begin s2-isolate-tenant s1-update-complex s1-commit s3-release-advisory-lock s2-commit s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-load-cache:
TRUNCATE isolation_table;
TRUNCATE isolation_table2;
step s1-insert:
INSERT INTO isolation_table VALUES (5, 10);
INSERT INTO isolation_table2 VALUES (5, 10);
step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
-- the tests are written with the logic where single shard SELECTs
-- do not to open transaction blocks
SET citus.select_opens_transaction_block TO false;
step s1-select:
SELECT count(*) FROM isolation_table WHERE id = 5;
count
---------------------------------------------------------------------
1
(1 row)
step s2-begin:
BEGIN;
step s2-isolate-tenant:
SELECT isolate_tenant_to_new_shard('isolation_table', 5, shard_transfer_mode => 'force_logical');
<waiting ...>
step s1-update-complex:
UPDATE isolation_table SET value = 5 WHERE id IN (
SELECT max(id) FROM isolation_table
);
step s1-commit:
COMMIT;
step s3-release-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard
---------------------------------------------------------------------
1500106
(1 row)
step s2-commit:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('isolation_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM isolation_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500105|t | 0
57637|1500106|t | 1
57637|1500107|t | 0
57638|1500102|t | 0
(4 rows)
id|value
---------------------------------------------------------------------
5| 5
(1 row)
starting permutation: s1-load-cache s3-acquire-advisory-lock s1-begin s1-select s2-begin s2-isolate-tenant s1-insert s1-commit s3-release-advisory-lock s2-commit s2-print-cluster starting permutation: s1-load-cache s3-acquire-advisory-lock s1-begin s1-select s2-begin s2-isolate-tenant s1-insert s1-commit s3-release-advisory-lock s2-commit s2-print-cluster
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -235,7 +327,7 @@ t
step s2-isolate-tenant: <... completed> step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500094 1500114
(1 row) (1 row)
step s2-commit: step s2-commit:
@ -254,10 +346,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500093|t | 0 57637|1500113|t | 0
57637|1500094|t | 1 57637|1500114|t | 1
57637|1500095|t | 0 57637|1500115|t | 0
57638|1500090|t | 0 57638|1500110|t | 0
(4 rows) (4 rows)
id|value id|value
@ -321,7 +413,7 @@ t
step s2-isolate-tenant: <... completed> step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500102 1500122
(1 row) (1 row)
step s2-commit: step s2-commit:
@ -340,10 +432,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500101|t | 1 57637|1500121|t | 1
57637|1500102|t | 1 57637|1500122|t | 1
57637|1500103|t | 2 57637|1500123|t | 2
57638|1500098|t | 1 57638|1500118|t | 1
(4 rows) (4 rows)
id|value id|value
@ -411,7 +503,7 @@ t
step s2-isolate-tenant: <... completed> step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500110 1500130
(1 row) (1 row)
step s2-commit: step s2-commit:
@ -430,10 +522,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500109|t | 0 57637|1500129|t | 0
57637|1500110|t | 1 57637|1500130|t | 1
57637|1500111|t | 0 57637|1500131|t | 0
57638|1500106|t | 0 57638|1500126|t | 0
(4 rows) (4 rows)
id|value id|value
@ -497,7 +589,7 @@ t
step s2-isolate-tenant: <... completed> step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500118 1500138
(1 row) (1 row)
step s2-commit: step s2-commit:
@ -516,10 +608,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500117|t | 0 57637|1500137|t | 0
57637|1500118|t | 0 57637|1500138|t | 0
57637|1500119|t | 0 57637|1500139|t | 0
57638|1500114|t | 0 57638|1500134|t | 0
(4 rows) (4 rows)
id|value id|value
@ -527,6 +619,94 @@ id|value
(0 rows) (0 rows)
starting permutation: s1-insert s3-acquire-advisory-lock s1-begin s1-select s2-begin s2-isolate-tenant s1-update-complex s1-commit s3-release-advisory-lock s2-commit s2-print-cluster
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-insert:
INSERT INTO isolation_table VALUES (5, 10);
INSERT INTO isolation_table2 VALUES (5, 10);
step s3-acquire-advisory-lock:
SELECT pg_advisory_lock(44000, 55152);
pg_advisory_lock
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
-- the tests are written with the logic where single shard SELECTs
-- do not to open transaction blocks
SET citus.select_opens_transaction_block TO false;
step s1-select:
SELECT count(*) FROM isolation_table WHERE id = 5;
count
---------------------------------------------------------------------
1
(1 row)
step s2-begin:
BEGIN;
step s2-isolate-tenant:
SELECT isolate_tenant_to_new_shard('isolation_table', 5, shard_transfer_mode => 'force_logical');
<waiting ...>
step s1-update-complex:
UPDATE isolation_table SET value = 5 WHERE id IN (
SELECT max(id) FROM isolation_table
);
step s1-commit:
COMMIT;
step s3-release-advisory-lock:
SELECT pg_advisory_unlock(44000, 55152);
pg_advisory_unlock
---------------------------------------------------------------------
t
(1 row)
step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard
---------------------------------------------------------------------
1500146
(1 row)
step s2-commit:
COMMIT;
step s2-print-cluster:
-- row count per shard
SELECT
nodeport, shardid, success, result
FROM
run_command_on_placements('isolation_table', 'select count(*) from %s')
ORDER BY
nodeport, shardid;
-- rows
SELECT id, value FROM isolation_table ORDER BY id, value;
nodeport|shardid|success|result
---------------------------------------------------------------------
57637|1500145|t | 0
57637|1500146|t | 1
57637|1500147|t | 0
57638|1500142|t | 0
(4 rows)
id|value
---------------------------------------------------------------------
5| 5
(1 row)
starting permutation: s3-acquire-advisory-lock s1-begin s1-select s2-begin s2-isolate-tenant s1-insert s1-commit s3-release-advisory-lock s2-commit s2-print-cluster starting permutation: s3-acquire-advisory-lock s1-begin s1-select s2-begin s2-isolate-tenant s1-insert s1-commit s3-release-advisory-lock s2-commit s2-print-cluster
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -579,7 +759,7 @@ t
step s2-isolate-tenant: <... completed> step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500126 1500154
(1 row) (1 row)
step s2-commit: step s2-commit:
@ -598,10 +778,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500125|t | 0 57637|1500153|t | 0
57637|1500126|t | 1 57637|1500154|t | 1
57637|1500127|t | 0 57637|1500155|t | 0
57638|1500122|t | 0 57638|1500150|t | 0
(4 rows) (4 rows)
id|value id|value
@ -661,7 +841,7 @@ t
step s2-isolate-tenant: <... completed> step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500134 1500162
(1 row) (1 row)
step s2-commit: step s2-commit:
@ -680,10 +860,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500133|t | 1 57637|1500161|t | 1
57637|1500134|t | 1 57637|1500162|t | 1
57637|1500135|t | 2 57637|1500163|t | 2
57638|1500130|t | 1 57638|1500158|t | 1
(4 rows) (4 rows)
id|value id|value
@ -736,7 +916,7 @@ t
step s2-isolate-tenant: <... completed> step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500142 1500170
(1 row) (1 row)
step s2-print-cluster: step s2-print-cluster:
@ -752,10 +932,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500141|t | 0 57637|1500169|t | 0
57637|1500142|t | 1 57637|1500170|t | 1
57637|1500143|t | 0 57637|1500171|t | 0
57638|1500138|t | 0 57638|1500166|t | 0
(4 rows) (4 rows)
id|value id|value
@ -804,7 +984,7 @@ t
step s2-isolate-tenant: <... completed> step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500150 1500178
(1 row) (1 row)
step s2-print-cluster: step s2-print-cluster:
@ -820,10 +1000,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500149|t | 0 57637|1500177|t | 0
57637|1500150|t | 1 57637|1500178|t | 1
57637|1500151|t | 0 57637|1500179|t | 0
57638|1500146|t | 0 57638|1500174|t | 0
(4 rows) (4 rows)
id|value id|value
@ -872,7 +1052,7 @@ t
step s2-isolate-tenant: <... completed> step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500158 1500186
(1 row) (1 row)
step s2-print-cluster: step s2-print-cluster:
@ -888,10 +1068,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500157|t | 0 57637|1500185|t | 0
57637|1500158|t | 1 57637|1500186|t | 1
57637|1500159|t | 0 57637|1500187|t | 0
57638|1500154|t | 0 57638|1500182|t | 0
(4 rows) (4 rows)
id|value id|value
@ -943,7 +1123,7 @@ t
step s2-isolate-tenant: <... completed> step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500169 1500197
(1 row) (1 row)
step s2-commit: step s2-commit:
@ -962,10 +1142,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500168|t | 0 57637|1500196|t | 0
57637|1500169|t | 1 57637|1500197|t | 1
57637|1500170|t | 0 57637|1500198|t | 0
57638|1500165|t | 0 57638|1500193|t | 0
(4 rows) (4 rows)
id|value id|value
@ -1004,7 +1184,7 @@ step s1-isolate-tenant-no-same-coloc-blocking:
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500183 1500211
(1 row) (1 row)
step s3-release-advisory-lock: step s3-release-advisory-lock:
@ -1018,7 +1198,7 @@ t
step s2-isolate-tenant: <... completed> step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500180 1500208
(1 row) (1 row)
step s2-print-cluster: step s2-print-cluster:
@ -1034,10 +1214,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500179|t | 0 57637|1500207|t | 0
57637|1500180|t | 1 57637|1500208|t | 1
57637|1500181|t | 0 57637|1500209|t | 0
57638|1500176|t | 0 57638|1500204|t | 0
(4 rows) (4 rows)
id|value id|value
@ -1076,7 +1256,7 @@ step s1-isolate-tenant-no-same-coloc-blocking:
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500194 1500222
(1 row) (1 row)
step s3-release-advisory-lock: step s3-release-advisory-lock:
@ -1090,7 +1270,7 @@ t
step s2-isolate-tenant: <... completed> step s2-isolate-tenant: <... completed>
isolate_tenant_to_new_shard isolate_tenant_to_new_shard
--------------------------------------------------------------------- ---------------------------------------------------------------------
1500191 1500219
(1 row) (1 row)
step s2-print-cluster: step s2-print-cluster:
@ -1106,10 +1286,10 @@ step s2-print-cluster:
nodeport|shardid|success|result nodeport|shardid|success|result
--------------------------------------------------------------------- ---------------------------------------------------------------------
57637|1500190|t | 0 57637|1500218|t | 0
57637|1500191|t | 1 57637|1500219|t | 1
57637|1500192|t | 0 57637|1500220|t | 0
57638|1500187|t | 0 57638|1500215|t | 0
(4 rows) (4 rows)
id|value id|value

View File

@ -108,6 +108,22 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SET application_name to 'citus_internal gpid=not a correct gpid'; SET application_name to 'citus_internal gpid=not a correct gpid';
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 's'); SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 's');
ERROR: This is an internal Citus function can only be used in a distributed transaction ERROR: This is an internal Citus function can only be used in a distributed transaction
ROLLBACK;
-- application_name with suffix is ok (e.g. pgbouncer might add this)
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
assign_distributed_transaction_id
---------------------------------------------------------------------
(1 row)
SET application_name to 'citus_internal gpid=10000000001 - from 10.12.14.16:10370';
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 's');
citus_internal_add_partition_metadata
---------------------------------------------------------------------
(1 row)
ROLLBACK; ROLLBACK;
-- application_name with empty gpid -- application_name with empty gpid
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;

View File

@ -1150,6 +1150,38 @@ CREATE TABLE test (x int, y int references ref(a));
SELECT create_distributed_table('test','x'); SELECT create_distributed_table('test','x');
ERROR: canceling the transaction since it was involved in a distributed deadlock ERROR: canceling the transaction since it was involved in a distributed deadlock
END; END;
-- verify the split fails if we still need to replicate reference tables
SELECT citus_remove_node('localhost', :worker_2_port);
citus_remove_node
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table('test','x');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT citus_add_node('localhost', :worker_2_port);
citus_add_node
---------------------------------------------------------------------
1370022
(1 row)
SELECT
citus_split_shard_by_split_points(shardid,
ARRAY[(shardminvalue::int + (shardmaxvalue::int - shardminvalue::int)/2)::text],
ARRAY[nodeid, nodeid],
'force_logical')
FROM
pg_dist_shard, pg_dist_node
WHERE
logicalrelid = 'replicate_reference_table.test'::regclass AND nodename = 'localhost' AND nodeport = :worker_2_port
ORDER BY shardid LIMIT 1;
ERROR: reference tables have not been replicated to node localhost:xxxxx yet
DETAIL: Reference tables are lazily replicated after adding a node, but must exist before shards can be created on that node.
HINT: Run SELECT replicate_reference_tables(); to ensure reference tables exist on all nodes.
-- test adding an invalid node while we have reference tables to replicate -- test adding an invalid node while we have reference tables to replicate
-- set client message level to ERROR and verbosity to terse to supporess -- set client message level to ERROR and verbosity to terse to supporess
-- OS-dependent host name resolution warnings -- OS-dependent host name resolution warnings

View File

@ -3,6 +3,13 @@
-- --
-- Tests select distinct, and select distinct on features. -- Tests select distinct, and select distinct on features.
-- --
SHOW server_version \gset
SELECT substring(:'server_version', '\d+')::int >= 15 AS server_version_ge_15;
server_version_ge_15
---------------------------------------------------------------------
t
(1 row)
ANALYZE lineitem_hash_part; ANALYZE lineitem_hash_part;
-- function calls are supported -- function calls are supported
SELECT DISTINCT l_orderkey, now() FROM lineitem_hash_part LIMIT 0; SELECT DISTINCT l_orderkey, now() FROM lineitem_hash_part LIMIT 0;
@ -306,14 +313,12 @@ EXPLAIN (COSTS FALSE)
-- check the plan if the hash aggreate is disabled. We expect to see sort+unique -- check the plan if the hash aggreate is disabled. We expect to see sort+unique
-- instead of aggregate plan node to handle distinct. -- instead of aggregate plan node to handle distinct.
SET enable_hashagg TO off; SET enable_hashagg TO off;
SELECT public.plan_without_result_lines($Q$
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT DISTINCT count(*) SELECT DISTINCT count(*)
FROM lineitem_hash_part FROM lineitem_hash_part
GROUP BY l_suppkey, l_linenumber GROUP BY l_suppkey, l_linenumber
ORDER BY 1; ORDER BY 1;
$Q$); QUERY PLAN
plan_without_result_lines
--------------------------------------------------------------------- ---------------------------------------------------------------------
Unique Unique
-> Sort -> Sort
@ -382,15 +387,13 @@ EXPLAIN (COSTS FALSE)
-- check the plan if the hash aggreate is disabled. Similar to the explain of -- check the plan if the hash aggreate is disabled. Similar to the explain of
-- the query above. -- the query above.
SET enable_hashagg TO off; SET enable_hashagg TO off;
SELECT public.plan_without_result_lines($Q$
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT DISTINCT l_suppkey, count(*) SELECT DISTINCT l_suppkey, count(*)
FROM lineitem_hash_part FROM lineitem_hash_part
GROUP BY l_suppkey, l_linenumber GROUP BY l_suppkey, l_linenumber
ORDER BY 1 ORDER BY 1
LIMIT 10; LIMIT 10;
$Q$); QUERY PLAN
plan_without_result_lines
--------------------------------------------------------------------- ---------------------------------------------------------------------
Limit Limit
-> Unique -> Unique
@ -443,9 +446,10 @@ EXPLAIN (COSTS FALSE)
QUERY PLAN QUERY PLAN
--------------------------------------------------------------------- ---------------------------------------------------------------------
Limit Limit
-> Unique
-> Sort -> Sort
Sort Key: remote_scan.l_suppkey, ((pg_catalog.sum(remote_scan.avg) / pg_catalog.sum(remote_scan.avg_1))) Sort Key: remote_scan.l_suppkey, ((pg_catalog.sum(remote_scan.avg) / pg_catalog.sum(remote_scan.avg_1)))
-> HashAggregate
Group Key: remote_scan.l_suppkey, (pg_catalog.sum(remote_scan.avg) / pg_catalog.sum(remote_scan.avg_1))
-> HashAggregate -> HashAggregate
Group Key: remote_scan.l_suppkey, remote_scan.worker_column_4 Group Key: remote_scan.l_suppkey, remote_scan.worker_column_4
-> Custom Scan (Citus Adaptive) -> Custom Scan (Citus Adaptive)
@ -456,20 +460,18 @@ EXPLAIN (COSTS FALSE)
-> HashAggregate -> HashAggregate
Group Key: l_suppkey, l_linenumber Group Key: l_suppkey, l_linenumber
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part -> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
(14 rows) (15 rows)
-- check the plan if the hash aggreate is disabled. This explain errors out due -- check the plan if the hash aggreate is disabled. This explain errors out due
-- to a bug right now, expectation must be corrected after fixing it. -- to a bug right now, expectation must be corrected after fixing it.
SET enable_hashagg TO off; SET enable_hashagg TO off;
SELECT public.plan_without_result_lines($Q$
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT DISTINCT l_suppkey, avg(l_partkey) SELECT DISTINCT l_suppkey, avg(l_partkey)
FROM lineitem_hash_part FROM lineitem_hash_part
GROUP BY l_suppkey, l_linenumber GROUP BY l_suppkey, l_linenumber
ORDER BY 1,2 ORDER BY 1,2
LIMIT 10; LIMIT 10;
$Q$); QUERY PLAN
plan_without_result_lines
--------------------------------------------------------------------- ---------------------------------------------------------------------
Limit Limit
-> Unique -> Unique
@ -539,15 +541,13 @@ EXPLAIN (COSTS FALSE)
-- check the plan if the hash aggreate is disabled. We expect to see sort+unique to -- check the plan if the hash aggreate is disabled. We expect to see sort+unique to
-- handle distinct on. -- handle distinct on.
SET enable_hashagg TO off; SET enable_hashagg TO off;
SELECT public.plan_without_result_lines($Q$
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT DISTINCT ON (l_suppkey) avg(l_partkey) SELECT DISTINCT ON (l_suppkey) avg(l_partkey)
FROM lineitem_hash_part FROM lineitem_hash_part
GROUP BY l_suppkey, l_linenumber GROUP BY l_suppkey, l_linenumber
ORDER BY l_suppkey,1 ORDER BY l_suppkey,1
LIMIT 10; LIMIT 10;
$Q$); QUERY PLAN
plan_without_result_lines
--------------------------------------------------------------------- ---------------------------------------------------------------------
Limit Limit
-> Unique -> Unique
@ -598,9 +598,10 @@ EXPLAIN (COSTS FALSE)
QUERY PLAN QUERY PLAN
--------------------------------------------------------------------- ---------------------------------------------------------------------
Limit Limit
-> Unique
-> Sort -> Sort
Sort Key: ((sum(remote_scan.avg) / (pg_catalog.sum(remote_scan.avg_1))::double precision)) Sort Key: ((sum(remote_scan.avg) / (pg_catalog.sum(remote_scan.avg_1))::double precision))
-> HashAggregate
Group Key: (sum(remote_scan.avg) / (pg_catalog.sum(remote_scan.avg_1))::double precision)
-> HashAggregate -> HashAggregate
Group Key: remote_scan.worker_column_3, remote_scan.worker_column_4 Group Key: remote_scan.worker_column_3, remote_scan.worker_column_4
-> Custom Scan (Citus Adaptive) -> Custom Scan (Citus Adaptive)
@ -611,20 +612,18 @@ EXPLAIN (COSTS FALSE)
-> HashAggregate -> HashAggregate
Group Key: l_suppkey, l_linenumber Group Key: l_suppkey, l_linenumber
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part -> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
(14 rows) (15 rows)
-- check the plan if the hash aggreate is disabled. This explain errors out due -- check the plan if the hash aggreate is disabled. This explain errors out due
-- to a bug right now, expectation must be corrected after fixing it. -- to a bug right now, expectation must be corrected after fixing it.
SET enable_hashagg TO off; SET enable_hashagg TO off;
SELECT public.plan_without_result_lines($Q$
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT DISTINCT avg(ceil(l_partkey / 2)) SELECT DISTINCT avg(ceil(l_partkey / 2))
FROM lineitem_hash_part FROM lineitem_hash_part
GROUP BY l_suppkey, l_linenumber GROUP BY l_suppkey, l_linenumber
ORDER BY 1 ORDER BY 1
LIMIT 10; LIMIT 10;
$Q$); QUERY PLAN
plan_without_result_lines
--------------------------------------------------------------------- ---------------------------------------------------------------------
Limit Limit
-> Unique -> Unique
@ -675,9 +674,10 @@ EXPLAIN (COSTS FALSE)
QUERY PLAN QUERY PLAN
--------------------------------------------------------------------- ---------------------------------------------------------------------
Limit Limit
-> Unique
-> Sort -> Sort
Sort Key: (((pg_catalog.sum(remote_scan.dis))::bigint + COALESCE((pg_catalog.sum(remote_scan.dis_1))::bigint, '0'::bigint))) Sort Key: (((pg_catalog.sum(remote_scan.dis))::bigint + COALESCE((pg_catalog.sum(remote_scan.dis_1))::bigint, '0'::bigint)))
-> HashAggregate
Group Key: ((pg_catalog.sum(remote_scan.dis))::bigint + COALESCE((pg_catalog.sum(remote_scan.dis_1))::bigint, '0'::bigint))
-> HashAggregate -> HashAggregate
Group Key: remote_scan.worker_column_3, remote_scan.worker_column_4 Group Key: remote_scan.worker_column_3, remote_scan.worker_column_4
-> Custom Scan (Citus Adaptive) -> Custom Scan (Citus Adaptive)
@ -688,20 +688,18 @@ EXPLAIN (COSTS FALSE)
-> HashAggregate -> HashAggregate
Group Key: l_suppkey, l_linenumber Group Key: l_suppkey, l_linenumber
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part -> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
(14 rows) (15 rows)
-- check the plan if the hash aggreate is disabled. This explain errors out due -- check the plan if the hash aggreate is disabled. This explain errors out due
-- to a bug right now, expectation must be corrected after fixing it. -- to a bug right now, expectation must be corrected after fixing it.
SET enable_hashagg TO off; SET enable_hashagg TO off;
SELECT public.plan_without_result_lines($Q$
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT DISTINCT sum(l_suppkey) + count(l_partkey) AS dis SELECT DISTINCT sum(l_suppkey) + count(l_partkey) AS dis
FROM lineitem_hash_part FROM lineitem_hash_part
GROUP BY l_suppkey, l_linenumber GROUP BY l_suppkey, l_linenumber
ORDER BY 1 ORDER BY 1
LIMIT 10; LIMIT 10;
$Q$); QUERY PLAN
plan_without_result_lines
--------------------------------------------------------------------- ---------------------------------------------------------------------
Limit Limit
-> Unique -> Unique
@ -910,14 +908,12 @@ EXPLAIN (COSTS FALSE)
-- check the plan if the hash aggreate is disabled -- check the plan if the hash aggreate is disabled
SET enable_hashagg TO off; SET enable_hashagg TO off;
SELECT public.plan_without_result_lines($Q$
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT DISTINCT ceil(count(case when l_partkey > 100000 THEN 1 ELSE 0 END) / 2) AS count SELECT DISTINCT ceil(count(case when l_partkey > 100000 THEN 1 ELSE 0 END) / 2) AS count
FROM lineitem_hash_part FROM lineitem_hash_part
GROUP BY l_suppkey GROUP BY l_suppkey
ORDER BY 1; ORDER BY 1;
$Q$); QUERY PLAN
plan_without_result_lines
--------------------------------------------------------------------- ---------------------------------------------------------------------
Unique Unique
-> Sort -> Sort

File diff suppressed because it is too large Load Diff

View File

@ -320,6 +320,9 @@ INSERT INTO test_parameterized_sql VALUES(1, 1);
SELECT * FROM test_parameterized_sql_function(1); SELECT * FROM test_parameterized_sql_function(1);
ERROR: cannot perform distributed planning on this query because parameterized queries for SQL functions referencing distributed tables are not supported ERROR: cannot perform distributed planning on this query because parameterized queries for SQL functions referencing distributed tables are not supported
HINT: Consider using PL/pgSQL functions instead. HINT: Consider using PL/pgSQL functions instead.
SELECT (SELECT 1 FROM test_parameterized_sql limit 1) FROM test_parameterized_sql_function(1);
ERROR: cannot perform distributed planning on this query because parameterized queries for SQL functions referencing distributed tables are not supported
HINT: Consider using PL/pgSQL functions instead.
SELECT test_parameterized_sql_function_in_subquery_where(1); SELECT test_parameterized_sql_function_in_subquery_where(1);
ERROR: could not create distributed plan ERROR: could not create distributed plan
DETAIL: Possibly this is caused by the use of parameters in SQL functions, which is not supported in Citus. DETAIL: Possibly this is caused by the use of parameters in SQL functions, which is not supported in Citus.

View File

@ -17,15 +17,10 @@ BEGIN
END; END;
$$LANGUAGE plpgsql; $$LANGUAGE plpgsql;
-- Create a function to ignore worker plans in explain output -- Create a function to ignore worker plans in explain output
-- Also remove extra "-> Result" lines for PG15 support
CREATE OR REPLACE FUNCTION coordinator_plan(explain_command text, out query_plan text) CREATE OR REPLACE FUNCTION coordinator_plan(explain_command text, out query_plan text)
RETURNS SETOF TEXT AS $$ RETURNS SETOF TEXT AS $$
BEGIN BEGIN
FOR query_plan IN execute explain_command LOOP FOR query_plan IN execute explain_command LOOP
IF (query_plan LIKE '%-> Result%' OR query_plan = 'Result')
THEN
CONTINUE;
END IF;
RETURN next; RETURN next;
IF query_plan LIKE '%Task Count:%' IF query_plan LIKE '%Task Count:%'
THEN THEN
@ -36,16 +31,12 @@ BEGIN
END; $$ language plpgsql; END; $$ language plpgsql;
-- Create a function to ignore worker plans in explain output -- Create a function to ignore worker plans in explain output
-- It also shows task count for plan and subplans -- It also shows task count for plan and subplans
-- Also remove extra "-> Result" lines for PG15 support
CREATE OR REPLACE FUNCTION coordinator_plan_with_subplans(explain_command text, out query_plan text) CREATE OR REPLACE FUNCTION coordinator_plan_with_subplans(explain_command text, out query_plan text)
RETURNS SETOF TEXT AS $$ RETURNS SETOF TEXT AS $$
DECLARE DECLARE
task_count_line_reached boolean := false; task_count_line_reached boolean := false;
BEGIN BEGIN
FOR query_plan IN execute explain_command LOOP FOR query_plan IN execute explain_command LOOP
IF (query_plan LIKE '%-> Result%' OR query_plan = 'Result') THEN
CONTINUE;
END IF;
IF NOT task_count_line_reached THEN IF NOT task_count_line_reached THEN
RETURN next; RETURN next;
END IF; END IF;
@ -59,19 +50,6 @@ BEGIN
END LOOP; END LOOP;
RETURN; RETURN;
END; $$ language plpgsql; END; $$ language plpgsql;
-- Create a function to ignore "-> Result" lines for PG15 support
-- In PG15 there are some extra "-> Result" lines
CREATE OR REPLACE FUNCTION plan_without_result_lines(explain_command text, out query_plan text)
RETURNS SETOF TEXT AS $$
BEGIN
FOR query_plan IN execute explain_command LOOP
IF (query_plan LIKE '%-> Result%' OR query_plan = 'Result') THEN
CONTINUE;
END IF;
RETURN next;
END LOOP;
RETURN;
END; $$ language plpgsql;
-- Create a function to normalize Memory Usage, Buckets, Batches -- Create a function to normalize Memory Usage, Buckets, Batches
CREATE OR REPLACE FUNCTION plan_normalize_memory(explain_command text, out query_plan text) CREATE OR REPLACE FUNCTION plan_normalize_memory(explain_command text, out query_plan text)
RETURNS SETOF TEXT AS $$ RETURNS SETOF TEXT AS $$
@ -81,18 +59,6 @@ BEGIN
RETURN NEXT; RETURN NEXT;
END LOOP; END LOOP;
END; $$ language plpgsql; END; $$ language plpgsql;
-- Create a function to remove arrows from the explain plan
CREATE OR REPLACE FUNCTION plan_without_arrows(explain_command text, out query_plan text)
RETURNS SETOF TEXT AS $$
BEGIN
FOR query_plan IN execute explain_command LOOP
IF (query_plan LIKE '%-> Result%' OR query_plan = 'Result') THEN
CONTINUE;
END IF;
query_plan := regexp_replace(query_plan, '( )*-> (.*)', '\2', 'g');
RETURN NEXT;
END LOOP;
END; $$ language plpgsql;
-- helper function that returns true if output of given explain has "is not null" (case in-sensitive) -- helper function that returns true if output of given explain has "is not null" (case in-sensitive)
CREATE OR REPLACE FUNCTION explain_has_is_not_null(explain_command text) CREATE OR REPLACE FUNCTION explain_has_is_not_null(explain_command text)
RETURNS BOOLEAN AS $$ RETURNS BOOLEAN AS $$

View File

@ -168,6 +168,10 @@ SELECT * FROM sale_triggers ORDER BY 1, 2;
truncate_trigger_xxxxxxx | sale_newyork | O truncate_trigger_xxxxxxx | sale_newyork | O
(6 rows) (6 rows)
-- test that we can't rename a distributed clone trigger
ALTER TRIGGER "new_record_sale_trigger" ON "pg15"."sale_newyork" RENAME TO "another_trigger_name";
ERROR: cannot rename trigger "new_record_sale_trigger" on table "sale_newyork"
HINT: Rename trigger on partitioned table "sale" instead.
-- --
-- In PG15, For GENERATED columns, all dependencies of the generation -- In PG15, For GENERATED columns, all dependencies of the generation
-- expression are recorded as NORMAL dependencies of the column itself. -- expression are recorded as NORMAL dependencies of the column itself.
@ -388,9 +392,37 @@ ON (true)
WHEN MATCHED THEN WHEN MATCHED THEN
UPDATE SET x = (SELECT count(*) FROM tbl2); UPDATE SET x = (SELECT count(*) FROM tbl2);
ERROR: MERGE command is not supported on Citus tables yet ERROR: MERGE command is not supported on Citus tables yet
-- test numeric types with negative scale
CREATE TABLE numeric_negative_scale(numeric_column numeric(3,-1), orig_value int);
INSERT into numeric_negative_scale SELECT x,x FROM generate_series(111, 115) x;
-- verify that we can not distribute by a column that has numeric type with negative scale
SELECT create_distributed_table('numeric_negative_scale','numeric_column');
ERROR: cannot distribute relation: numeric_negative_scale
DETAIL: Distribution column must not use numeric type with negative scale
-- However, we can distribute by other columns
SELECT create_distributed_table('numeric_negative_scale','orig_value');
NOTICE: Copying data from local table...
NOTICE: copying the data has completed
DETAIL: The local data in the table is no longer visible, but is still on disk.
HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$pg15.numeric_negative_scale$$)
create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT * FROM numeric_negative_scale ORDER BY 1,2;
numeric_column | orig_value
---------------------------------------------------------------------
110 | 111
110 | 112
110 | 113
110 | 114
120 | 115
(5 rows)
-- Clean up -- Clean up
DROP SCHEMA pg15 CASCADE; DROP SCHEMA pg15 CASCADE;
NOTICE: drop cascades to 9 other objects NOTICE: drop cascades to 10 other objects
DETAIL: drop cascades to collation german_phonebook_test DETAIL: drop cascades to collation german_phonebook_test
drop cascades to collation default_provider drop cascades to collation default_provider
drop cascades to table sale drop cascades to table sale
@ -400,3 +432,4 @@ drop cascades to view sale_triggers
drop cascades to table generated_stored_ref drop cascades to table generated_stored_ref
drop cascades to table tbl1 drop cascades to table tbl1
drop cascades to table tbl2 drop cascades to table tbl2
drop cascades to table numeric_negative_scale

View File

@ -0,0 +1,243 @@
--
-- UPGRADE_DISTRIBUTED_TRIGGERS_AFTER
--
-- In PG15, Renaming triggers on partitioned tables
-- recurses to renaming the triggers on the partitions as well.
-- Relevant PG commit:
-- 80ba4bb383538a2ee846fece6a7b8da9518b6866
--
-- this test is relevant only for pg14-15 upgrade
--
SHOW server_version \gset
SELECT substring(:'server_version', '\d+')::int = 15 AS server_version_eq_15
\gset
\if :server_version_eq_15
\else
\q
\endif
SET search_path TO upgrade_distributed_triggers, public;
SET citus.shard_count TO 4;
SET citus.enable_unsafe_triggers TO true;
SELECT run_command_on_workers('ALTER SYSTEM SET citus.enable_unsafe_triggers TO true;');
run_command_on_workers
---------------------------------------------------------------------
(localhost,10201,t,"ALTER SYSTEM")
(localhost,10202,t,"ALTER SYSTEM")
(2 rows)
SELECT run_command_on_workers('SELECT pg_reload_conf();');
run_command_on_workers
---------------------------------------------------------------------
(localhost,10201,t,t)
(localhost,10202,t,t)
(2 rows)
-- after PG15 upgrade, all child triggers have the same name with the parent triggers
-- check that the workers are also updated
SELECT * FROM sale_triggers ORDER BY 1, 2;
tgname | tgrelid | tgenabled
---------------------------------------------------------------------
another_renamed_trigger | sale | O
another_renamed_trigger | sale_newyork | O
another_renamed_trigger | sale_california | O
not_renamed_trigger | sale | O
not_renamed_trigger | sale_newyork | O
not_renamed_trigger | sale_california | O
renamed_record_sale_trigger | sale | O
renamed_record_sale_trigger | sale_newyork | O
renamed_record_sale_trigger | sale_california | O
truncate_trigger_xxxxxxx | sale | O
truncate_trigger_xxxxxxx | sale_newyork | O
truncate_trigger_xxxxxxx | sale_california | O
(12 rows)
SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'record_sale_trigger%' ORDER BY 1,2;
tgrelid | tgname
---------------------------------------------------------------------
(0 rows)
SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'renamed_record_sale_trigger%' ORDER BY 1,2;
tgrelid | tgname
---------------------------------------------------------------------
sale | renamed_record_sale_trigger
sale_california | renamed_record_sale_trigger
sale_newyork | renamed_record_sale_trigger
(3 rows)
SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'record_sale_trigger%';$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,10201,t,0)
(localhost,10202,t,0)
(2 rows)
SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'renamed_record_sale_trigger%';$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,10201,t,9)
(localhost,10202,t,9)
(2 rows)
-- create another partition to verify that all is safe and sound
CREATE TABLE sale_alabama PARTITION OF sale FOR VALUES IN ('AL');
SELECT * FROM sale_triggers ORDER BY 1, 2;
tgname | tgrelid | tgenabled
---------------------------------------------------------------------
another_renamed_trigger | sale | O
another_renamed_trigger | sale_newyork | O
another_renamed_trigger | sale_california | O
another_renamed_trigger | sale_alabama | O
not_renamed_trigger | sale | O
not_renamed_trigger | sale_newyork | O
not_renamed_trigger | sale_california | O
not_renamed_trigger | sale_alabama | O
renamed_record_sale_trigger | sale | O
renamed_record_sale_trigger | sale_newyork | O
renamed_record_sale_trigger | sale_california | O
renamed_record_sale_trigger | sale_alabama | O
truncate_trigger_xxxxxxx | sale | O
truncate_trigger_xxxxxxx | sale_newyork | O
truncate_trigger_xxxxxxx | sale_california | O
truncate_trigger_xxxxxxx | sale_alabama | O
(16 rows)
SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'record_sale_trigger%' ORDER BY 1,2;
tgrelid | tgname
---------------------------------------------------------------------
(0 rows)
SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'renamed_record_sale_trigger%' ORDER BY 1,2;
tgrelid | tgname
---------------------------------------------------------------------
sale | renamed_record_sale_trigger
sale_alabama | renamed_record_sale_trigger
sale_california | renamed_record_sale_trigger
sale_newyork | renamed_record_sale_trigger
(4 rows)
SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'record_sale_trigger%';$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,10201,t,0)
(localhost,10202,t,0)
(2 rows)
SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'renamed_record_sale_trigger%';$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,10201,t,12)
(localhost,10202,t,12)
(2 rows)
-- drop a trigger to verify that all is safe and sound
DROP TRIGGER another_renamed_trigger ON sale;
SELECT * FROM sale_triggers ORDER BY 1, 2;
tgname | tgrelid | tgenabled
---------------------------------------------------------------------
not_renamed_trigger | sale | O
not_renamed_trigger | sale_newyork | O
not_renamed_trigger | sale_california | O
not_renamed_trigger | sale_alabama | O
renamed_record_sale_trigger | sale | O
renamed_record_sale_trigger | sale_newyork | O
renamed_record_sale_trigger | sale_california | O
renamed_record_sale_trigger | sale_alabama | O
truncate_trigger_xxxxxxx | sale | O
truncate_trigger_xxxxxxx | sale_newyork | O
truncate_trigger_xxxxxxx | sale_california | O
truncate_trigger_xxxxxxx | sale_alabama | O
(12 rows)
-- rename a trigger - note that it also renames the triggers on the partitions
ALTER TRIGGER "renamed_record_sale_trigger" ON "sale" RENAME TO "final_renamed_record_sale_trigger";
SELECT * FROM sale_triggers ORDER BY 1, 2;
tgname | tgrelid | tgenabled
---------------------------------------------------------------------
final_renamed_record_sale_trigger | sale | O
final_renamed_record_sale_trigger | sale_newyork | O
final_renamed_record_sale_trigger | sale_california | O
final_renamed_record_sale_trigger | sale_alabama | O
not_renamed_trigger | sale | O
not_renamed_trigger | sale_newyork | O
not_renamed_trigger | sale_california | O
not_renamed_trigger | sale_alabama | O
truncate_trigger_xxxxxxx | sale | O
truncate_trigger_xxxxxxx | sale_newyork | O
truncate_trigger_xxxxxxx | sale_california | O
truncate_trigger_xxxxxxx | sale_alabama | O
(12 rows)
SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'renamed_record_sale_trigger%' ORDER BY 1,2;
tgrelid | tgname
---------------------------------------------------------------------
(0 rows)
SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'final_renamed_record_sale_trigger%' ORDER BY 1,2;
tgrelid | tgname
---------------------------------------------------------------------
sale | final_renamed_record_sale_trigger
sale_alabama | final_renamed_record_sale_trigger
sale_california | final_renamed_record_sale_trigger
sale_newyork | final_renamed_record_sale_trigger
(4 rows)
SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'renamed_record_sale_trigger%';$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,10201,t,0)
(localhost,10202,t,0)
(2 rows)
SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'final_renamed_record_sale_trigger%';$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,10201,t,12)
(localhost,10202,t,12)
(2 rows)
DROP TRIGGER final_renamed_record_sale_trigger ON sale;
-- create another trigger and rename it
CREATE TRIGGER yet_another_trigger
AFTER INSERT OR UPDATE OR DELETE ON sale
FOR EACH ROW EXECUTE FUNCTION upgrade_distributed_triggers.record_sale();
ALTER TRIGGER "yet_another_trigger" ON "sale" RENAME TO "renamed_yet_another_trigger";
SELECT * FROM sale_triggers ORDER BY 1, 2;
tgname | tgrelid | tgenabled
---------------------------------------------------------------------
not_renamed_trigger | sale | O
not_renamed_trigger | sale_newyork | O
not_renamed_trigger | sale_california | O
not_renamed_trigger | sale_alabama | O
renamed_yet_another_trigger | sale | O
renamed_yet_another_trigger | sale_newyork | O
renamed_yet_another_trigger | sale_california | O
renamed_yet_another_trigger | sale_alabama | O
truncate_trigger_xxxxxxx | sale | O
truncate_trigger_xxxxxxx | sale_newyork | O
truncate_trigger_xxxxxxx | sale_california | O
truncate_trigger_xxxxxxx | sale_alabama | O
(12 rows)
-- after upgrade to PG15, test that we can't rename a distributed clone trigger
ALTER TRIGGER "renamed_yet_another_trigger" ON "sale_alabama" RENAME TO "another_trigger_name";
ERROR: cannot rename trigger "renamed_yet_another_trigger" on table "sale_alabama"
HINT: Rename trigger on partitioned table "sale" instead.
SELECT count(*) FROM pg_trigger WHERE tgname like 'another_trigger_name%';
count
---------------------------------------------------------------------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'another_trigger_name%';$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,10201,t,0)
(localhost,10202,t,0)
(2 rows)
DROP SCHEMA upgrade_distributed_triggers CASCADE;
NOTICE: drop cascades to 4 other objects
DETAIL: drop cascades to function record_sale()
drop cascades to table sale
drop cascades to table record_sale
drop cascades to view sale_triggers

View File

@ -0,0 +1,16 @@
--
-- UPGRADE_DISTRIBUTED_TRIGGERS_AFTER
--
-- In PG15, Renaming triggers on partitioned tables
-- recurses to renaming the triggers on the partitions as well.
-- Relevant PG commit:
-- 80ba4bb383538a2ee846fece6a7b8da9518b6866
--
-- this test is relevant only for pg14-15 upgrade
--
SHOW server_version \gset
SELECT substring(:'server_version', '\d+')::int = 15 AS server_version_eq_15
\gset
\if :server_version_eq_15
\else
\q

View File

@ -0,0 +1,246 @@
--
-- UPGRADE_DISTRIBUTED_TRIGGERS_BEFORE
--
-- PRE PG15, Renaming the parent triggers on partitioned tables doesn't
-- recurse to renaming the child triggers on the partitions as well.
--
-- this test is relevant only for pg14-15 upgrade
--
SHOW server_version \gset
SELECT substring(:'server_version', '\d+')::int = 14 AS server_version_eq_14
\gset
\if :server_version_eq_14
\else
\q
\endif
CREATE SCHEMA upgrade_distributed_triggers;
SET search_path TO upgrade_distributed_triggers, public;
SET citus.shard_count TO 4;
SET citus.enable_unsafe_triggers TO true;
SELECT run_command_on_workers('ALTER SYSTEM SET citus.enable_unsafe_triggers TO true;');
run_command_on_workers
---------------------------------------------------------------------
(localhost,10201,t,"ALTER SYSTEM")
(localhost,10202,t,"ALTER SYSTEM")
(2 rows)
SELECT run_command_on_workers('SELECT pg_reload_conf();');
run_command_on_workers
---------------------------------------------------------------------
(localhost,10201,t,t)
(localhost,10202,t,t)
(2 rows)
CREATE TABLE sale(
sale_date date not null,
state_code text,
product_sku text,
units integer)
PARTITION BY list (state_code);
ALTER TABLE sale ADD CONSTRAINT sale_pk PRIMARY KEY (state_code, sale_date);
CREATE TABLE sale_newyork PARTITION OF sale FOR VALUES IN ('NY');
CREATE TABLE record_sale(
operation_type text not null,
product_sku text,
state_code text,
units integer,
PRIMARY KEY(state_code, product_sku, operation_type, units));
SELECT create_distributed_table('sale', 'state_code');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE sale_california PARTITION OF sale FOR VALUES IN ('CA');
SELECT create_distributed_table('record_sale', 'state_code', colocate_with := 'sale');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE OR REPLACE FUNCTION record_sale()
RETURNS trigger
AS $$
BEGIN
INSERT INTO upgrade_distributed_triggers.record_sale(operation_type, product_sku, state_code, units)
VALUES (TG_OP, NEW.product_sku, NEW.state_code, NEW.units);
RETURN NULL;
END;
$$ LANGUAGE plpgsql;
-- will rename this trigger
CREATE TRIGGER record_sale_trigger
AFTER INSERT OR UPDATE OR DELETE ON sale
FOR EACH ROW EXECUTE FUNCTION upgrade_distributed_triggers.record_sale();
-- will rename this trigger
CREATE TRIGGER another_trigger
AFTER INSERT OR UPDATE OR DELETE ON sale
FOR EACH ROW EXECUTE FUNCTION upgrade_distributed_triggers.record_sale();
-- won't rename this trigger
CREATE TRIGGER not_renamed_trigger
AFTER INSERT OR UPDATE OR DELETE ON sale
FOR EACH ROW EXECUTE FUNCTION upgrade_distributed_triggers.record_sale();
-- Trigger function should appear on workers
SELECT proname from pg_proc WHERE oid='upgrade_distributed_triggers.record_sale'::regproc;
proname
---------------------------------------------------------------------
record_sale
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM pg_proc WHERE oid='upgrade_distributed_triggers.record_sale'::regproc$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,10201,t,1)
(localhost,10202,t,1)
(2 rows)
-- Trigger should appear on workers
SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'record_sale_trigger%' ORDER BY 1,2;
tgrelid | tgname
---------------------------------------------------------------------
sale | record_sale_trigger
sale_california | record_sale_trigger
sale_newyork | record_sale_trigger
(3 rows)
SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'record_sale_trigger%';$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,10201,t,9)
(localhost,10202,t,9)
(2 rows)
CREATE VIEW sale_triggers AS
SELECT tgname, tgrelid::regclass, tgenabled
FROM pg_trigger
WHERE tgrelid::regclass::text like 'sale%'
ORDER BY 1, 2;
SELECT * FROM sale_triggers ORDER BY 1, 2;
tgname | tgrelid | tgenabled
---------------------------------------------------------------------
another_trigger | sale | O
another_trigger | sale_newyork | O
another_trigger | sale_california | O
not_renamed_trigger | sale | O
not_renamed_trigger | sale_newyork | O
not_renamed_trigger | sale_california | O
record_sale_trigger | sale | O
record_sale_trigger | sale_newyork | O
record_sale_trigger | sale_california | O
truncate_trigger_xxxxxxx | sale | O
truncate_trigger_xxxxxxx | sale_newyork | O
truncate_trigger_xxxxxxx | sale_california | O
(12 rows)
-- rename the triggers - note that it doesn't rename the
-- triggers on the partitions
ALTER TRIGGER record_sale_trigger ON sale RENAME TO renamed_record_sale_trigger;
SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'record_sale_trigger%' ORDER BY 1,2;
tgrelid | tgname
---------------------------------------------------------------------
sale_california | record_sale_trigger
sale_newyork | record_sale_trigger
(2 rows)
SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'renamed_record_sale_trigger%' ORDER BY 1,2;
tgrelid | tgname
---------------------------------------------------------------------
sale | renamed_record_sale_trigger
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'record_sale_trigger%';$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,10201,t,6)
(localhost,10202,t,6)
(2 rows)
SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'renamed_record_sale_trigger%';$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,10201,t,3)
(localhost,10202,t,3)
(2 rows)
ALTER TRIGGER another_trigger ON sale RENAME TO another_renamed_trigger;
SELECT * FROM sale_triggers ORDER BY 1, 2;
tgname | tgrelid | tgenabled
---------------------------------------------------------------------
another_renamed_trigger | sale | O
another_trigger | sale_newyork | O
another_trigger | sale_california | O
not_renamed_trigger | sale | O
not_renamed_trigger | sale_newyork | O
not_renamed_trigger | sale_california | O
record_sale_trigger | sale_newyork | O
record_sale_trigger | sale_california | O
renamed_record_sale_trigger | sale | O
truncate_trigger_xxxxxxx | sale | O
truncate_trigger_xxxxxxx | sale_newyork | O
truncate_trigger_xxxxxxx | sale_california | O
(12 rows)
-- although the child triggers haven't been renamed to
-- another_renamed_trigger, they are dropped when the parent is dropped
DROP TRIGGER another_renamed_trigger ON sale;
SELECT * FROM sale_triggers ORDER BY 1, 2;
tgname | tgrelid | tgenabled
---------------------------------------------------------------------
not_renamed_trigger | sale | O
not_renamed_trigger | sale_newyork | O
not_renamed_trigger | sale_california | O
record_sale_trigger | sale_newyork | O
record_sale_trigger | sale_california | O
renamed_record_sale_trigger | sale | O
truncate_trigger_xxxxxxx | sale | O
truncate_trigger_xxxxxxx | sale_newyork | O
truncate_trigger_xxxxxxx | sale_california | O
(9 rows)
SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'another_renamed_trigger%' ORDER BY 1,2;
tgrelid | tgname
---------------------------------------------------------------------
(0 rows)
SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'another_trigger%' ORDER BY 1,2;
tgrelid | tgname
---------------------------------------------------------------------
(0 rows)
SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'another_renamed_trigger%';$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,10201,t,0)
(localhost,10202,t,0)
(2 rows)
SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'another_trigger%';$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,10201,t,0)
(localhost,10202,t,0)
(2 rows)
CREATE TRIGGER another_trigger
AFTER INSERT OR UPDATE OR DELETE ON sale
FOR EACH ROW EXECUTE FUNCTION upgrade_distributed_triggers.record_sale();
ALTER TRIGGER another_trigger ON sale RENAME TO another_renamed_trigger;
SELECT * FROM sale_triggers ORDER BY 1, 2;
tgname | tgrelid | tgenabled
---------------------------------------------------------------------
another_renamed_trigger | sale | O
another_trigger | sale_newyork | O
another_trigger | sale_california | O
not_renamed_trigger | sale | O
not_renamed_trigger | sale_newyork | O
not_renamed_trigger | sale_california | O
record_sale_trigger | sale_newyork | O
record_sale_trigger | sale_california | O
renamed_record_sale_trigger | sale | O
truncate_trigger_xxxxxxx | sale | O
truncate_trigger_xxxxxxx | sale_newyork | O
truncate_trigger_xxxxxxx | sale_california | O
(12 rows)
-- check that we can't rename child triggers on partitions of distributed tables
ALTER TRIGGER another_trigger ON sale_newyork RENAME TO another_renamed_trigger;
ERROR: cannot rename child triggers on distributed partitions

View File

@ -0,0 +1,14 @@
--
-- UPGRADE_DISTRIBUTED_TRIGGERS_BEFORE
--
-- PRE PG15, Renaming the parent triggers on partitioned tables doesn't
-- recurse to renaming the child triggers on the partitions as well.
--
-- this test is relevant only for pg14-15 upgrade
--
SHOW server_version \gset
SELECT substring(:'server_version', '\d+')::int = 14 AS server_version_eq_14
\gset
\if :server_version_eq_14
\else
\q

View File

@ -1491,12 +1491,10 @@ LIMIT 5;
(17 rows) (17 rows)
-- Grouping can be pushed down with aggregates even when window function can't -- Grouping can be pushed down with aggregates even when window function can't
SELECT public.plan_without_result_lines($Q$
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT user_id, count(value_1), stddev(value_1), count(user_id) OVER (PARTITION BY random()) SELECT user_id, count(value_1), stddev(value_1), count(user_id) OVER (PARTITION BY random())
FROM users_table GROUP BY user_id HAVING avg(value_1) > 2 LIMIT 1; FROM users_table GROUP BY user_id HAVING avg(value_1) > 2 LIMIT 1;
$Q$); QUERY PLAN
plan_without_result_lines
--------------------------------------------------------------------- ---------------------------------------------------------------------
Limit Limit
-> WindowAgg -> WindowAgg

View File

@ -1495,12 +1495,10 @@ LIMIT 5;
(18 rows) (18 rows)
-- Grouping can be pushed down with aggregates even when window function can't -- Grouping can be pushed down with aggregates even when window function can't
SELECT public.plan_without_result_lines($Q$
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT user_id, count(value_1), stddev(value_1), count(user_id) OVER (PARTITION BY random()) SELECT user_id, count(value_1), stddev(value_1), count(user_id) OVER (PARTITION BY random())
FROM users_table GROUP BY user_id HAVING avg(value_1) > 2 LIMIT 1; FROM users_table GROUP BY user_id HAVING avg(value_1) > 2 LIMIT 1;
$Q$); QUERY PLAN
plan_without_result_lines
--------------------------------------------------------------------- ---------------------------------------------------------------------
Limit Limit
-> WindowAgg -> WindowAgg

View File

@ -86,7 +86,7 @@ test: multi_agg_type_conversion multi_count_type_conversion recursive_relation_p
test: multi_partition_pruning single_hash_repartition_join unsupported_lateral_subqueries test: multi_partition_pruning single_hash_repartition_join unsupported_lateral_subqueries
test: multi_join_pruning multi_hash_pruning intermediate_result_pruning test: multi_join_pruning multi_hash_pruning intermediate_result_pruning
test: multi_null_minmax_value_pruning cursors test: multi_null_minmax_value_pruning cursors
test: modification_correctness test: modification_correctness adv_lock_permission
test: multi_query_directory_cleanup test: multi_query_directory_cleanup
test: multi_task_assignment_policy multi_cross_shard test: multi_task_assignment_policy multi_cross_shard
test: multi_utility_statements test: multi_utility_statements

View File

@ -9,7 +9,7 @@ setup
AS 'citus', $$test_assign_global_pid$$; AS 'citus', $$test_assign_global_pid$$;
SET citus.shard_replication_factor TO 1; SET citus.shard_replication_factor TO 1;
SET citus.shard_count TO 4; SET citus.shard_count TO 4;
select setval('pg_dist_shardid_seq', GREATEST(1300000, nextval('pg_dist_shardid_seq'))); ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1300001;
CREATE TABLE test_table(column1 int, column2 int); CREATE TABLE test_table(column1 int, column2 int);
SELECT create_distributed_table('test_table', 'column1'); SELECT create_distributed_table('test_table', 'column1');

View File

@ -44,6 +44,13 @@ step "s1-update"
UPDATE isolation_table SET value = 5 WHERE id = 5; UPDATE isolation_table SET value = 5 WHERE id = 5;
} }
step "s1-update-complex"
{
UPDATE isolation_table SET value = 5 WHERE id IN (
SELECT max(id) FROM isolation_table
);
}
step "s1-delete" step "s1-delete"
{ {
DELETE FROM isolation_table WHERE id = 5; DELETE FROM isolation_table WHERE id = 5;
@ -119,6 +126,7 @@ step "s2-print-index-count"
// we expect DML/DDL queries to fail because the shard they are waiting for is destroyed // we expect DML/DDL queries to fail because the shard they are waiting for is destroyed
permutation "s1-load-cache" "s1-insert" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-update" "s2-commit" "s1-commit" "s2-print-cluster" permutation "s1-load-cache" "s1-insert" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-update" "s2-commit" "s1-commit" "s2-print-cluster"
permutation "s1-load-cache" "s1-insert" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-delete" "s2-commit" "s1-commit" "s2-print-cluster" permutation "s1-load-cache" "s1-insert" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-delete" "s2-commit" "s1-commit" "s2-print-cluster"
permutation "s1-load-cache" "s1-insert" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-update-complex" "s2-commit" "s1-commit" "s2-print-cluster"
permutation "s1-load-cache" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-insert" "s2-commit" "s1-commit" "s2-print-cluster" permutation "s1-load-cache" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-insert" "s2-commit" "s1-commit" "s2-print-cluster"
permutation "s1-load-cache" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-copy" "s2-commit" "s1-commit" "s2-print-cluster" permutation "s1-load-cache" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-copy" "s2-commit" "s1-commit" "s2-print-cluster"
permutation "s1-load-cache" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-ddl" "s2-commit" "s1-commit" "s2-print-cluster" "s2-print-index-count" permutation "s1-load-cache" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-ddl" "s2-commit" "s1-commit" "s2-print-cluster" "s2-print-index-count"
@ -127,6 +135,7 @@ permutation "s1-load-cache" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant
// the same tests without loading the cache at first // the same tests without loading the cache at first
permutation "s1-insert" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-update" "s2-commit" "s1-commit" "s2-print-cluster" permutation "s1-insert" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-update" "s2-commit" "s1-commit" "s2-print-cluster"
permutation "s1-insert" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-delete" "s2-commit" "s1-commit" "s2-print-cluster" permutation "s1-insert" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-delete" "s2-commit" "s1-commit" "s2-print-cluster"
permutation "s1-insert" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-update-complex" "s2-commit" "s1-commit" "s2-print-cluster"
permutation "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-insert" "s2-commit" "s1-commit" "s2-print-cluster" permutation "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-insert" "s2-commit" "s1-commit" "s2-print-cluster"
permutation "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-copy" "s2-commit" "s1-commit" "s2-print-cluster" permutation "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-copy" "s2-commit" "s1-commit" "s2-print-cluster"
permutation "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-ddl" "s2-commit" "s1-commit" "s2-print-cluster" "s2-print-index-count" permutation "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-ddl" "s2-commit" "s1-commit" "s2-print-cluster" "s2-print-index-count"

View File

@ -51,6 +51,13 @@ step "s1-update"
UPDATE isolation_table SET value = 5 WHERE id = 5; UPDATE isolation_table SET value = 5 WHERE id = 5;
} }
step "s1-update-complex"
{
UPDATE isolation_table SET value = 5 WHERE id IN (
SELECT max(id) FROM isolation_table
);
}
step "s1-delete" step "s1-delete"
{ {
DELETE FROM isolation_table WHERE id = 5; DELETE FROM isolation_table WHERE id = 5;
@ -148,12 +155,14 @@ step "s3-release-advisory-lock"
// we expect DML queries of s2 to succeed without being blocked. // we expect DML queries of s2 to succeed without being blocked.
permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-update" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster" permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-update" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"
permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-delete" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster" permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-delete" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"
permutation "s1-load-cache" "s1-insert" "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-update-complex" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"
permutation "s1-load-cache" "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-insert" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster" permutation "s1-load-cache" "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-insert" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"
permutation "s1-load-cache" "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-copy" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster" permutation "s1-load-cache" "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-copy" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"
// the same tests without loading the cache at first // the same tests without loading the cache at first
permutation "s1-insert" "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-update" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster" permutation "s1-insert" "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-update" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"
permutation "s1-insert" "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-delete" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster" permutation "s1-insert" "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-delete" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"
permutation "s1-insert" "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-update-complex" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"
permutation "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-insert" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster" permutation "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-insert" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"
permutation "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-copy" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster" permutation "s3-acquire-advisory-lock" "s1-begin" "s1-select" "s2-begin" "s2-isolate-tenant" "s1-copy" "s1-commit" "s3-release-advisory-lock" "s2-commit" "s2-print-cluster"

View File

@ -40,4 +40,22 @@ SELECT sum(result::bigint) FROM run_command_on_workers($$
$$); $$);
END; END;
CREATE OR REPLACE FUNCTION select_for_update()
RETURNS void
AS $$
DECLARE
my int;
BEGIN
SELECT y INTO my FROM test WHERE x = 1 FOR UPDATE;
END;
$$ LANGUAGE plpgsql;
-- so that we can prove that we open a transaction block by logging below:
-- "NOTICE: issuing BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;..."
SET citus.log_remote_commands TO on;
SELECT select_for_update();
SET citus.log_remote_commands TO off;
DROP SCHEMA adaptive_executor CASCADE; DROP SCHEMA adaptive_executor CASCADE;

View File

@ -0,0 +1,42 @@
CREATE SCHEMA adv_lock_permission;
SET search_path to adv_lock_permission;
-- do not cache any connections, we change some settings and don't want old ones cached
SET citus.max_cached_conns_per_worker TO 0;
CREATE ROLE user_1 WITH LOGIN;
CREATE TABLE reference_table_1 (A int);
SELECT create_reference_table('reference_table_1');
CREATE TABLE reference_table_2 (A int);
SELECT create_reference_table('reference_table_2');
GRANT USAGE ON SCHEMA adv_lock_permission TO user_1;
GRANT SELECT ON reference_table_1 TO user_1;
GRANT INSERT, UPDATE ON reference_table_2 TO user_1;
SET ROLE user_1;
-- do not cache any connections, we change some settings and don't want old ones cached
SET citus.max_cached_conns_per_worker TO 0;
SET search_path to adv_lock_permission;
INSERT INTO reference_table_2 SELECT * FROM reference_table_1;
SET ROLE postgres;
-- do not cache any connections, we change some settings and don't want old ones cached
SET citus.max_cached_conns_per_worker TO 0;
-- change the role so that it can skip permission checks
ALTER ROLE user_1 SET citus.skip_advisory_lock_permission_checks TO on;
SET ROLE user_1;
SET citus.max_cached_conns_per_worker TO 0;
INSERT INTO reference_table_2 SELECT * FROM reference_table_1;
SET ROLE postgres;
SET client_min_messages TO ERROR;
DROP SCHEMA adv_lock_permission CASCADE;
DROP ROLE user_1;

View File

@ -465,9 +465,19 @@ ALTER TABLE loc_tb ADD CONSTRAINT fkey FOREIGN KEY (a) references ref_tb(a);
-- drop the view&matview with circular dependency -- drop the view&matview with circular dependency
DROP VIEW v103 CASCADE; DROP VIEW v103 CASCADE;
SET client_min_messages TO DEBUG1;
-- now it should successfully add to metadata and create the views on workers -- now it should successfully add to metadata and create the views on workers
ALTER TABLE loc_tb ADD CONSTRAINT fkey FOREIGN KEY (a) references ref_tb(a); ALTER TABLE loc_tb ADD CONSTRAINT fkey FOREIGN KEY (a) references ref_tb(a);
-- verify the views are created on workers
select run_command_on_workers($$SELECT count(*)=0 from citus_local_tables_mx.v100$$);
select run_command_on_workers($$SELECT count(*)=0 from citus_local_tables_mx.v101$$);
select run_command_on_workers($$SELECT count(*)=0 from citus_local_tables_mx.v102$$);
CREATE TABLE loc_tb_2 (a int);
CREATE VIEW v104 AS SELECT * from loc_tb_2;
SET client_min_messages TO DEBUG1;
-- verify the CREATE command for the view is generated correctly
ALTER TABLE loc_tb_2 ADD CONSTRAINT fkey_2 FOREIGN KEY (a) references ref_tb(a);
SET client_min_messages TO WARNING; SET client_min_messages TO WARNING;
-- works fine -- works fine
@ -477,8 +487,8 @@ select run_command_on_workers($$SELECT count(*) from citus_local_tables_mx.v100,
ALTER TABLE loc_tb DROP CONSTRAINT fkey; ALTER TABLE loc_tb DROP CONSTRAINT fkey;
-- fails because fkey is dropped and table is converted to local table -- fails because fkey is dropped and table is converted to local table
select run_command_on_workers($$SELECT count(*) from citus_local_tables_mx.v100$$); select run_command_on_workers($$SELECT count(*) from citus_local_tables_mx.v100$$);
select run_command_on_workers($$SELECT count(*) from citus_local_tables_mx.v101$$); select run_command_on_workers($$SELECT count(*) from citus_local_tables_mx.v101$$);
select run_command_on_workers($$SELECT count(*) from citus_local_tables_mx.v102$$); select run_command_on_workers($$SELECT count(*) from citus_local_tables_mx.v102$$);
INSERT INTO loc_tb VALUES (1), (2); INSERT INTO loc_tb VALUES (1), (2);
-- test a matview with columnar -- test a matview with columnar

View File

@ -130,15 +130,11 @@ INSERT INTO another_columnar_table SELECT generate_series(0,5);
EXPLAIN (analyze on, costs off, timing off, summary off) EXPLAIN (analyze on, costs off, timing off, summary off)
SELECT a, y FROM multi_column_chunk_filtering, another_columnar_table WHERE x > 1; SELECT a, y FROM multi_column_chunk_filtering, another_columnar_table WHERE x > 1;
SELECT plan_without_arrows($Q$
EXPLAIN (costs off, timing off, summary off) EXPLAIN (costs off, timing off, summary off)
SELECT y, * FROM another_columnar_table; SELECT y, * FROM another_columnar_table;
$Q$);
SELECT plan_without_arrows($Q$
EXPLAIN (costs off, timing off, summary off) EXPLAIN (costs off, timing off, summary off)
SELECT *, x FROM another_columnar_table; SELECT *, x FROM another_columnar_table;
$Q$);
EXPLAIN (costs off, timing off, summary off) EXPLAIN (costs off, timing off, summary off)
SELECT y, another_columnar_table FROM another_columnar_table; SELECT y, another_columnar_table FROM another_columnar_table;

View File

@ -429,13 +429,11 @@ EXPLAIN (COSTS OFF, SUMMARY OFF)
SELECT * FROM weird_col_explain; SELECT * FROM weird_col_explain;
\set VERBOSITY terse \set VERBOSITY terse
SELECT public.plan_without_result_lines($Q$
EXPLAIN (COSTS OFF, SUMMARY OFF) EXPLAIN (COSTS OFF, SUMMARY OFF)
SELECT *, "bbbbbbbbbbbbbbbbbbbbbbbbb\!bbbb'bbbbbbbbbbbbbbbbbbbbb''bbbbbbbb" SELECT *, "bbbbbbbbbbbbbbbbbbbbbbbbb\!bbbb'bbbbbbbbbbbbbbbbbbbbb''bbbbbbbb"
FROM weird_col_explain FROM weird_col_explain
WHERE "bbbbbbbbbbbbbbbbbbbbbbbbb\!bbbb'bbbbbbbbbbbbbbbbbbbbb''bbbbbbbb" * 2 > WHERE "bbbbbbbbbbbbbbbbbbbbbbbbb\!bbbb'bbbbbbbbbbbbbbbbbbbbb''bbbbbbbb" * 2 >
"aaaaaaaaaaaa$aaaaaa$$aaaaaaaaaaaaaaaaaaaaaaaaaaaaa'aaaaaaaa'$a'!"; "aaaaaaaaaaaa$aaaaaa$$aaaaaaaaaaaaaaaaaaaaaaaaaaaaa'aaaaaaaa'$a'!";
$Q$);
\set VERBOSITY default \set VERBOSITY default
-- should not project any columns -- should not project any columns

View File

@ -635,9 +635,7 @@ DO UPDATE SET
create table table_with_sequences (x int, y int, z bigserial); create table table_with_sequences (x int, y int, z bigserial);
insert into table_with_sequences values (1,1); insert into table_with_sequences values (1,1);
select create_distributed_table('table_with_sequences','x'); select create_distributed_table('table_with_sequences','x');
SELECT public.plan_without_result_lines($Q$
explain (costs off) insert into table_with_sequences select y, x from table_with_sequences; explain (costs off) insert into table_with_sequences select y, x from table_with_sequences;
$Q$);
-- verify that we don't report repartitioned insert/select for tables -- verify that we don't report repartitioned insert/select for tables
-- with user-defined sequences. -- with user-defined sequences.
@ -645,9 +643,7 @@ CREATE SEQUENCE user_defined_sequence;
create table table_with_user_sequences (x int, y int, z bigint default nextval('user_defined_sequence')); create table table_with_user_sequences (x int, y int, z bigint default nextval('user_defined_sequence'));
insert into table_with_user_sequences values (1,1); insert into table_with_user_sequences values (1,1);
select create_distributed_table('table_with_user_sequences','x'); select create_distributed_table('table_with_user_sequences','x');
SELECT public.plan_without_result_lines($Q$
explain (costs off) insert into table_with_user_sequences select y, x from table_with_user_sequences; explain (costs off) insert into table_with_user_sequences select y, x from table_with_user_sequences;
$Q$);
-- clean-up -- clean-up
SET client_min_messages TO WARNING; SET client_min_messages TO WARNING;

View File

@ -75,6 +75,13 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 's'); SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 's');
ROLLBACK; ROLLBACK;
-- application_name with suffix is ok (e.g. pgbouncer might add this)
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
SET application_name to 'citus_internal gpid=10000000001 - from 10.12.14.16:10370';
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 's');
ROLLBACK;
-- application_name with empty gpid -- application_name with empty gpid
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED; BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02'); SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');

View File

@ -700,6 +700,21 @@ CREATE TABLE test (x int, y int references ref(a));
SELECT create_distributed_table('test','x'); SELECT create_distributed_table('test','x');
END; END;
-- verify the split fails if we still need to replicate reference tables
SELECT citus_remove_node('localhost', :worker_2_port);
SELECT create_distributed_table('test','x');
SELECT citus_add_node('localhost', :worker_2_port);
SELECT
citus_split_shard_by_split_points(shardid,
ARRAY[(shardminvalue::int + (shardmaxvalue::int - shardminvalue::int)/2)::text],
ARRAY[nodeid, nodeid],
'force_logical')
FROM
pg_dist_shard, pg_dist_node
WHERE
logicalrelid = 'replicate_reference_table.test'::regclass AND nodename = 'localhost' AND nodeport = :worker_2_port
ORDER BY shardid LIMIT 1;
-- test adding an invalid node while we have reference tables to replicate -- test adding an invalid node while we have reference tables to replicate
-- set client message level to ERROR and verbosity to terse to supporess -- set client message level to ERROR and verbosity to terse to supporess
-- OS-dependent host name resolution warnings -- OS-dependent host name resolution warnings

View File

@ -3,6 +3,8 @@
-- --
-- Tests select distinct, and select distinct on features. -- Tests select distinct, and select distinct on features.
-- --
SHOW server_version \gset
SELECT substring(:'server_version', '\d+')::int >= 15 AS server_version_ge_15;
ANALYZE lineitem_hash_part; ANALYZE lineitem_hash_part;
@ -113,13 +115,11 @@ EXPLAIN (COSTS FALSE)
-- check the plan if the hash aggreate is disabled. We expect to see sort+unique -- check the plan if the hash aggreate is disabled. We expect to see sort+unique
-- instead of aggregate plan node to handle distinct. -- instead of aggregate plan node to handle distinct.
SET enable_hashagg TO off; SET enable_hashagg TO off;
SELECT public.plan_without_result_lines($Q$
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT DISTINCT count(*) SELECT DISTINCT count(*)
FROM lineitem_hash_part FROM lineitem_hash_part
GROUP BY l_suppkey, l_linenumber GROUP BY l_suppkey, l_linenumber
ORDER BY 1; ORDER BY 1;
$Q$);
SET enable_hashagg TO on; SET enable_hashagg TO on;
@ -142,14 +142,12 @@ EXPLAIN (COSTS FALSE)
-- check the plan if the hash aggreate is disabled. Similar to the explain of -- check the plan if the hash aggreate is disabled. Similar to the explain of
-- the query above. -- the query above.
SET enable_hashagg TO off; SET enable_hashagg TO off;
SELECT public.plan_without_result_lines($Q$
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT DISTINCT l_suppkey, count(*) SELECT DISTINCT l_suppkey, count(*)
FROM lineitem_hash_part FROM lineitem_hash_part
GROUP BY l_suppkey, l_linenumber GROUP BY l_suppkey, l_linenumber
ORDER BY 1 ORDER BY 1
LIMIT 10; LIMIT 10;
$Q$);
SET enable_hashagg TO on; SET enable_hashagg TO on;
@ -173,14 +171,12 @@ EXPLAIN (COSTS FALSE)
-- check the plan if the hash aggreate is disabled. This explain errors out due -- check the plan if the hash aggreate is disabled. This explain errors out due
-- to a bug right now, expectation must be corrected after fixing it. -- to a bug right now, expectation must be corrected after fixing it.
SET enable_hashagg TO off; SET enable_hashagg TO off;
SELECT public.plan_without_result_lines($Q$
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT DISTINCT l_suppkey, avg(l_partkey) SELECT DISTINCT l_suppkey, avg(l_partkey)
FROM lineitem_hash_part FROM lineitem_hash_part
GROUP BY l_suppkey, l_linenumber GROUP BY l_suppkey, l_linenumber
ORDER BY 1,2 ORDER BY 1,2
LIMIT 10; LIMIT 10;
$Q$);
SET enable_hashagg TO on; SET enable_hashagg TO on;
@ -203,14 +199,12 @@ EXPLAIN (COSTS FALSE)
-- check the plan if the hash aggreate is disabled. We expect to see sort+unique to -- check the plan if the hash aggreate is disabled. We expect to see sort+unique to
-- handle distinct on. -- handle distinct on.
SET enable_hashagg TO off; SET enable_hashagg TO off;
SELECT public.plan_without_result_lines($Q$
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT DISTINCT ON (l_suppkey) avg(l_partkey) SELECT DISTINCT ON (l_suppkey) avg(l_partkey)
FROM lineitem_hash_part FROM lineitem_hash_part
GROUP BY l_suppkey, l_linenumber GROUP BY l_suppkey, l_linenumber
ORDER BY l_suppkey,1 ORDER BY l_suppkey,1
LIMIT 10; LIMIT 10;
$Q$);
SET enable_hashagg TO on; SET enable_hashagg TO on;
@ -232,14 +226,12 @@ EXPLAIN (COSTS FALSE)
-- check the plan if the hash aggreate is disabled. This explain errors out due -- check the plan if the hash aggreate is disabled. This explain errors out due
-- to a bug right now, expectation must be corrected after fixing it. -- to a bug right now, expectation must be corrected after fixing it.
SET enable_hashagg TO off; SET enable_hashagg TO off;
SELECT public.plan_without_result_lines($Q$
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT DISTINCT avg(ceil(l_partkey / 2)) SELECT DISTINCT avg(ceil(l_partkey / 2))
FROM lineitem_hash_part FROM lineitem_hash_part
GROUP BY l_suppkey, l_linenumber GROUP BY l_suppkey, l_linenumber
ORDER BY 1 ORDER BY 1
LIMIT 10; LIMIT 10;
$Q$);
SET enable_hashagg TO on; SET enable_hashagg TO on;
@ -261,14 +253,12 @@ EXPLAIN (COSTS FALSE)
-- check the plan if the hash aggreate is disabled. This explain errors out due -- check the plan if the hash aggreate is disabled. This explain errors out due
-- to a bug right now, expectation must be corrected after fixing it. -- to a bug right now, expectation must be corrected after fixing it.
SET enable_hashagg TO off; SET enable_hashagg TO off;
SELECT public.plan_without_result_lines($Q$
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT DISTINCT sum(l_suppkey) + count(l_partkey) AS dis SELECT DISTINCT sum(l_suppkey) + count(l_partkey) AS dis
FROM lineitem_hash_part FROM lineitem_hash_part
GROUP BY l_suppkey, l_linenumber GROUP BY l_suppkey, l_linenumber
ORDER BY 1 ORDER BY 1
LIMIT 10; LIMIT 10;
$Q$);
SET enable_hashagg TO on; SET enable_hashagg TO on;
@ -345,13 +335,11 @@ EXPLAIN (COSTS FALSE)
-- check the plan if the hash aggreate is disabled -- check the plan if the hash aggreate is disabled
SET enable_hashagg TO off; SET enable_hashagg TO off;
SELECT public.plan_without_result_lines($Q$
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT DISTINCT ceil(count(case when l_partkey > 100000 THEN 1 ELSE 0 END) / 2) AS count SELECT DISTINCT ceil(count(case when l_partkey > 100000 THEN 1 ELSE 0 END) / 2) AS count
FROM lineitem_hash_part FROM lineitem_hash_part
GROUP BY l_suppkey GROUP BY l_suppkey
ORDER BY 1; ORDER BY 1;
$Q$);
SET enable_hashagg TO on; SET enable_hashagg TO on;

View File

@ -144,6 +144,9 @@ INSERT INTO test_parameterized_sql VALUES(1, 1);
-- all of them should fail -- all of them should fail
SELECT * FROM test_parameterized_sql_function(1); SELECT * FROM test_parameterized_sql_function(1);
SELECT (SELECT 1 FROM test_parameterized_sql limit 1) FROM test_parameterized_sql_function(1);
SELECT test_parameterized_sql_function_in_subquery_where(1); SELECT test_parameterized_sql_function_in_subquery_where(1);
-- postgres behaves slightly differently for the following -- postgres behaves slightly differently for the following

View File

@ -20,15 +20,10 @@ END;
$$LANGUAGE plpgsql; $$LANGUAGE plpgsql;
-- Create a function to ignore worker plans in explain output -- Create a function to ignore worker plans in explain output
-- Also remove extra "-> Result" lines for PG15 support
CREATE OR REPLACE FUNCTION coordinator_plan(explain_command text, out query_plan text) CREATE OR REPLACE FUNCTION coordinator_plan(explain_command text, out query_plan text)
RETURNS SETOF TEXT AS $$ RETURNS SETOF TEXT AS $$
BEGIN BEGIN
FOR query_plan IN execute explain_command LOOP FOR query_plan IN execute explain_command LOOP
IF (query_plan LIKE '%-> Result%' OR query_plan = 'Result')
THEN
CONTINUE;
END IF;
RETURN next; RETURN next;
IF query_plan LIKE '%Task Count:%' IF query_plan LIKE '%Task Count:%'
THEN THEN
@ -40,16 +35,12 @@ END; $$ language plpgsql;
-- Create a function to ignore worker plans in explain output -- Create a function to ignore worker plans in explain output
-- It also shows task count for plan and subplans -- It also shows task count for plan and subplans
-- Also remove extra "-> Result" lines for PG15 support
CREATE OR REPLACE FUNCTION coordinator_plan_with_subplans(explain_command text, out query_plan text) CREATE OR REPLACE FUNCTION coordinator_plan_with_subplans(explain_command text, out query_plan text)
RETURNS SETOF TEXT AS $$ RETURNS SETOF TEXT AS $$
DECLARE DECLARE
task_count_line_reached boolean := false; task_count_line_reached boolean := false;
BEGIN BEGIN
FOR query_plan IN execute explain_command LOOP FOR query_plan IN execute explain_command LOOP
IF (query_plan LIKE '%-> Result%' OR query_plan = 'Result') THEN
CONTINUE;
END IF;
IF NOT task_count_line_reached THEN IF NOT task_count_line_reached THEN
RETURN next; RETURN next;
END IF; END IF;
@ -64,20 +55,6 @@ BEGIN
RETURN; RETURN;
END; $$ language plpgsql; END; $$ language plpgsql;
-- Create a function to ignore "-> Result" lines for PG15 support
-- In PG15 there are some extra "-> Result" lines
CREATE OR REPLACE FUNCTION plan_without_result_lines(explain_command text, out query_plan text)
RETURNS SETOF TEXT AS $$
BEGIN
FOR query_plan IN execute explain_command LOOP
IF (query_plan LIKE '%-> Result%' OR query_plan = 'Result') THEN
CONTINUE;
END IF;
RETURN next;
END LOOP;
RETURN;
END; $$ language plpgsql;
-- Create a function to normalize Memory Usage, Buckets, Batches -- Create a function to normalize Memory Usage, Buckets, Batches
CREATE OR REPLACE FUNCTION plan_normalize_memory(explain_command text, out query_plan text) CREATE OR REPLACE FUNCTION plan_normalize_memory(explain_command text, out query_plan text)
RETURNS SETOF TEXT AS $$ RETURNS SETOF TEXT AS $$
@ -88,19 +65,6 @@ BEGIN
END LOOP; END LOOP;
END; $$ language plpgsql; END; $$ language plpgsql;
-- Create a function to remove arrows from the explain plan
CREATE OR REPLACE FUNCTION plan_without_arrows(explain_command text, out query_plan text)
RETURNS SETOF TEXT AS $$
BEGIN
FOR query_plan IN execute explain_command LOOP
IF (query_plan LIKE '%-> Result%' OR query_plan = 'Result') THEN
CONTINUE;
END IF;
query_plan := regexp_replace(query_plan, '( )*-> (.*)', '\2', 'g');
RETURN NEXT;
END LOOP;
END; $$ language plpgsql;
-- helper function that returns true if output of given explain has "is not null" (case in-sensitive) -- helper function that returns true if output of given explain has "is not null" (case in-sensitive)
CREATE OR REPLACE FUNCTION explain_has_is_not_null(explain_command text) CREATE OR REPLACE FUNCTION explain_has_is_not_null(explain_command text)
RETURNS BOOLEAN AS $$ RETURNS BOOLEAN AS $$

View File

@ -114,6 +114,9 @@ SELECT * FROM sale_triggers ORDER BY 1, 2;
ALTER TRIGGER "record_sale_trigger" ON "pg15"."sale" RENAME TO "new_record_sale_trigger"; ALTER TRIGGER "record_sale_trigger" ON "pg15"."sale" RENAME TO "new_record_sale_trigger";
SELECT * FROM sale_triggers ORDER BY 1, 2; SELECT * FROM sale_triggers ORDER BY 1, 2;
-- test that we can't rename a distributed clone trigger
ALTER TRIGGER "new_record_sale_trigger" ON "pg15"."sale_newyork" RENAME TO "another_trigger_name";
-- --
-- In PG15, For GENERATED columns, all dependencies of the generation -- In PG15, For GENERATED columns, all dependencies of the generation
-- expression are recorded as NORMAL dependencies of the column itself. -- expression are recorded as NORMAL dependencies of the column itself.
@ -242,5 +245,15 @@ ON (true)
WHEN MATCHED THEN WHEN MATCHED THEN
UPDATE SET x = (SELECT count(*) FROM tbl2); UPDATE SET x = (SELECT count(*) FROM tbl2);
-- test numeric types with negative scale
CREATE TABLE numeric_negative_scale(numeric_column numeric(3,-1), orig_value int);
INSERT into numeric_negative_scale SELECT x,x FROM generate_series(111, 115) x;
-- verify that we can not distribute by a column that has numeric type with negative scale
SELECT create_distributed_table('numeric_negative_scale','numeric_column');
-- However, we can distribute by other columns
SELECT create_distributed_table('numeric_negative_scale','orig_value');
SELECT * FROM numeric_negative_scale ORDER BY 1,2;
-- Clean up -- Clean up
DROP SCHEMA pg15 CASCADE; DROP SCHEMA pg15 CASCADE;

View File

@ -0,0 +1,77 @@
--
-- UPGRADE_DISTRIBUTED_TRIGGERS_AFTER
--
-- In PG15, Renaming triggers on partitioned tables
-- recurses to renaming the triggers on the partitions as well.
-- Relevant PG commit:
-- 80ba4bb383538a2ee846fece6a7b8da9518b6866
--
-- this test is relevant only for pg14-15 upgrade
--
SHOW server_version \gset
SELECT substring(:'server_version', '\d+')::int = 15 AS server_version_eq_15
\gset
\if :server_version_eq_15
\else
\q
\endif
SET search_path TO upgrade_distributed_triggers, public;
SET citus.shard_count TO 4;
SET citus.enable_unsafe_triggers TO true;
SELECT run_command_on_workers('ALTER SYSTEM SET citus.enable_unsafe_triggers TO true;');
SELECT run_command_on_workers('SELECT pg_reload_conf();');
-- after PG15 upgrade, all child triggers have the same name with the parent triggers
-- check that the workers are also updated
SELECT * FROM sale_triggers ORDER BY 1, 2;
SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'record_sale_trigger%' ORDER BY 1,2;
SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'renamed_record_sale_trigger%' ORDER BY 1,2;
SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'record_sale_trigger%';$$);
SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'renamed_record_sale_trigger%';$$);
-- create another partition to verify that all is safe and sound
CREATE TABLE sale_alabama PARTITION OF sale FOR VALUES IN ('AL');
SELECT * FROM sale_triggers ORDER BY 1, 2;
SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'record_sale_trigger%' ORDER BY 1,2;
SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'renamed_record_sale_trigger%' ORDER BY 1,2;
SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'record_sale_trigger%';$$);
SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'renamed_record_sale_trigger%';$$);
-- drop a trigger to verify that all is safe and sound
DROP TRIGGER another_renamed_trigger ON sale;
SELECT * FROM sale_triggers ORDER BY 1, 2;
-- rename a trigger - note that it also renames the triggers on the partitions
ALTER TRIGGER "renamed_record_sale_trigger" ON "sale" RENAME TO "final_renamed_record_sale_trigger";
SELECT * FROM sale_triggers ORDER BY 1, 2;
SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'renamed_record_sale_trigger%' ORDER BY 1,2;
SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'final_renamed_record_sale_trigger%' ORDER BY 1,2;
SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'renamed_record_sale_trigger%';$$);
SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'final_renamed_record_sale_trigger%';$$);
DROP TRIGGER final_renamed_record_sale_trigger ON sale;
-- create another trigger and rename it
CREATE TRIGGER yet_another_trigger
AFTER INSERT OR UPDATE OR DELETE ON sale
FOR EACH ROW EXECUTE FUNCTION upgrade_distributed_triggers.record_sale();
ALTER TRIGGER "yet_another_trigger" ON "sale" RENAME TO "renamed_yet_another_trigger";
SELECT * FROM sale_triggers ORDER BY 1, 2;
-- after upgrade to PG15, test that we can't rename a distributed clone trigger
ALTER TRIGGER "renamed_yet_another_trigger" ON "sale_alabama" RENAME TO "another_trigger_name";
SELECT count(*) FROM pg_trigger WHERE tgname like 'another_trigger_name%';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'another_trigger_name%';$$);
DROP SCHEMA upgrade_distributed_triggers CASCADE;

View File

@ -0,0 +1,119 @@
--
-- UPGRADE_DISTRIBUTED_TRIGGERS_BEFORE
--
-- PRE PG15, Renaming the parent triggers on partitioned tables doesn't
-- recurse to renaming the child triggers on the partitions as well.
--
-- this test is relevant only for pg14-15 upgrade
--
SHOW server_version \gset
SELECT substring(:'server_version', '\d+')::int = 14 AS server_version_eq_14
\gset
\if :server_version_eq_14
\else
\q
\endif
CREATE SCHEMA upgrade_distributed_triggers;
SET search_path TO upgrade_distributed_triggers, public;
SET citus.shard_count TO 4;
SET citus.enable_unsafe_triggers TO true;
SELECT run_command_on_workers('ALTER SYSTEM SET citus.enable_unsafe_triggers TO true;');
SELECT run_command_on_workers('SELECT pg_reload_conf();');
CREATE TABLE sale(
sale_date date not null,
state_code text,
product_sku text,
units integer)
PARTITION BY list (state_code);
ALTER TABLE sale ADD CONSTRAINT sale_pk PRIMARY KEY (state_code, sale_date);
CREATE TABLE sale_newyork PARTITION OF sale FOR VALUES IN ('NY');
CREATE TABLE record_sale(
operation_type text not null,
product_sku text,
state_code text,
units integer,
PRIMARY KEY(state_code, product_sku, operation_type, units));
SELECT create_distributed_table('sale', 'state_code');
CREATE TABLE sale_california PARTITION OF sale FOR VALUES IN ('CA');
SELECT create_distributed_table('record_sale', 'state_code', colocate_with := 'sale');
CREATE OR REPLACE FUNCTION record_sale()
RETURNS trigger
AS $$
BEGIN
INSERT INTO upgrade_distributed_triggers.record_sale(operation_type, product_sku, state_code, units)
VALUES (TG_OP, NEW.product_sku, NEW.state_code, NEW.units);
RETURN NULL;
END;
$$ LANGUAGE plpgsql;
-- will rename this trigger
CREATE TRIGGER record_sale_trigger
AFTER INSERT OR UPDATE OR DELETE ON sale
FOR EACH ROW EXECUTE FUNCTION upgrade_distributed_triggers.record_sale();
-- will rename this trigger
CREATE TRIGGER another_trigger
AFTER INSERT OR UPDATE OR DELETE ON sale
FOR EACH ROW EXECUTE FUNCTION upgrade_distributed_triggers.record_sale();
-- won't rename this trigger
CREATE TRIGGER not_renamed_trigger
AFTER INSERT OR UPDATE OR DELETE ON sale
FOR EACH ROW EXECUTE FUNCTION upgrade_distributed_triggers.record_sale();
-- Trigger function should appear on workers
SELECT proname from pg_proc WHERE oid='upgrade_distributed_triggers.record_sale'::regproc;
SELECT run_command_on_workers($$SELECT count(*) FROM pg_proc WHERE oid='upgrade_distributed_triggers.record_sale'::regproc$$);
-- Trigger should appear on workers
SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'record_sale_trigger%' ORDER BY 1,2;
SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'record_sale_trigger%';$$);
CREATE VIEW sale_triggers AS
SELECT tgname, tgrelid::regclass, tgenabled
FROM pg_trigger
WHERE tgrelid::regclass::text like 'sale%'
ORDER BY 1, 2;
SELECT * FROM sale_triggers ORDER BY 1, 2;
-- rename the triggers - note that it doesn't rename the
-- triggers on the partitions
ALTER TRIGGER record_sale_trigger ON sale RENAME TO renamed_record_sale_trigger;
SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'record_sale_trigger%' ORDER BY 1,2;
SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'renamed_record_sale_trigger%' ORDER BY 1,2;
SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'record_sale_trigger%';$$);
SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'renamed_record_sale_trigger%';$$);
ALTER TRIGGER another_trigger ON sale RENAME TO another_renamed_trigger;
SELECT * FROM sale_triggers ORDER BY 1, 2;
-- although the child triggers haven't been renamed to
-- another_renamed_trigger, they are dropped when the parent is dropped
DROP TRIGGER another_renamed_trigger ON sale;
SELECT * FROM sale_triggers ORDER BY 1, 2;
SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'another_renamed_trigger%' ORDER BY 1,2;
SELECT tgrelid::regclass::text, tgname FROM pg_trigger WHERE tgname like 'another_trigger%' ORDER BY 1,2;
SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'another_renamed_trigger%';$$);
SELECT run_command_on_workers($$SELECT count(*) FROM pg_trigger WHERE tgname like 'another_trigger%';$$);
CREATE TRIGGER another_trigger
AFTER INSERT OR UPDATE OR DELETE ON sale
FOR EACH ROW EXECUTE FUNCTION upgrade_distributed_triggers.record_sale();
ALTER TRIGGER another_trigger ON sale RENAME TO another_renamed_trigger;
SELECT * FROM sale_triggers ORDER BY 1, 2;
-- check that we can't rename child triggers on partitions of distributed tables
ALTER TRIGGER another_trigger ON sale_newyork RENAME TO another_renamed_trigger;

View File

@ -576,11 +576,9 @@ ORDER BY user_id, avg(value_1) DESC
LIMIT 5; LIMIT 5;
-- Grouping can be pushed down with aggregates even when window function can't -- Grouping can be pushed down with aggregates even when window function can't
SELECT public.plan_without_result_lines($Q$
EXPLAIN (COSTS FALSE) EXPLAIN (COSTS FALSE)
SELECT user_id, count(value_1), stddev(value_1), count(user_id) OVER (PARTITION BY random()) SELECT user_id, count(value_1), stddev(value_1), count(user_id) OVER (PARTITION BY random())
FROM users_table GROUP BY user_id HAVING avg(value_1) > 2 LIMIT 1; FROM users_table GROUP BY user_id HAVING avg(value_1) > 2 LIMIT 1;
$Q$);
-- Window function with inlined CTE -- Window function with inlined CTE
WITH cte as ( WITH cte as (