Merge branch 'master' into dev/yanwjin/separate_columnar2

pull/5754/head
ywj 2022-03-02 15:27:16 -08:00 committed by GitHub
commit dc073629c7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
115 changed files with 5111 additions and 929 deletions

View File

@ -0,0 +1,107 @@
/*-------------------------------------------------------------------------
*
* aggregate.c
* Commands for distributing AGGREGATE statements.
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "distributed/commands.h"
#include "distributed/commands/utility_hook.h"
#include "distributed/deparser.h"
#include "distributed/listutils.h"
#include "distributed/metadata/dependency.h"
#include "distributed/metadata_sync.h"
#include "distributed/metadata/distobject.h"
#include "distributed/multi_executor.h"
#include "nodes/parsenodes.h"
#include "utils/lsyscache.h"
/*
* PostprocessDefineAggregateStmt actually creates the plan we need to execute for
* aggregate propagation.
* This is the downside of using the locally created aggregate to get the sql statement.
*
* If the aggregate depends on any non-distributed relation, Citus can not distribute it.
* In order to not to prevent users from creating local aggregates on the coordinator,
* a WARNING message will be sent to the user about the case instead of erroring out.
*
* Besides creating the plan we also make sure all (new) dependencies of the aggregate
* are created on all nodes.
*/
List *
PostprocessDefineAggregateStmt(Node *node, const char *queryString)
{
QualifyTreeNode((Node *) node);
DefineStmt *stmt = castNode(DefineStmt, node);
if (!ShouldPropagate())
{
return NIL;
}
if (!ShouldPropagateCreateInCoordinatedTransction())
{
return NIL;
}
ObjectAddress address = GetObjectAddressFromParseTree((Node *) stmt, false);
EnsureCoordinator();
EnsureSequentialMode(OBJECT_AGGREGATE);
ObjectAddress *undistributableDependency = GetUndistributableDependency(
&address);
if (undistributableDependency != NULL)
{
if (SupportedDependencyByCitus(undistributableDependency))
{
/*
* Citus can't distribute some relations as dependency, although those
* types as supported by Citus. So we can use get_rel_name directly
*/
RangeVar *aggRangeVar = makeRangeVarFromNameList(stmt->defnames);
char *aggName = aggRangeVar->relname;
char *dependentRelationName =
get_rel_name(undistributableDependency->objectId);
ereport(WARNING, (errmsg("Citus can't distribute aggregate \"%s\" having "
"dependency on non-distributed relation \"%s\"",
aggName, dependentRelationName),
errdetail("Aggregate will be created only locally"),
errhint("To distribute aggregate, distribute dependent "
"relations first. Then, re-create the aggregate")));
}
else
{
char *objectType = NULL;
#if PG_VERSION_NUM >= PG_VERSION_14
objectType = getObjectTypeDescription(undistributableDependency, false);
#else
objectType = getObjectTypeDescription(undistributableDependency);
#endif
ereport(WARNING, (errmsg("Citus can't distribute functions having "
"dependency on unsupported object of type \"%s\"",
objectType),
errdetail("Aggregate will be created only locally")));
}
return NIL;
}
EnsureDependenciesExistOnAllNodes(&address);
List *commands = CreateFunctionDDLCommandsIdempotent(&address);
commands = lcons(DISABLE_DDL_PROPAGATION, commands);
commands = lappend(commands, ENABLE_DDL_PROPAGATION);
return NodeDDLTaskList(NON_COORDINATOR_NODES, commands);
}

View File

@ -87,10 +87,11 @@ CitusSignalBackend(uint64 globalPID, uint64 timeout, int sig)
} }
#endif #endif
int nodeId = ExtractNodeIdFromGlobalPID(globalPID); bool missingOk = false;
int nodeId = ExtractNodeIdFromGlobalPID(globalPID, missingOk);
int processId = ExtractProcessIdFromGlobalPID(globalPID); int processId = ExtractProcessIdFromGlobalPID(globalPID);
WorkerNode *workerNode = FindNodeWithNodeId(nodeId); WorkerNode *workerNode = FindNodeWithNodeId(nodeId, missingOk);
StringInfo cancelQuery = makeStringInfo(); StringInfo cancelQuery = makeStringInfo();

View File

@ -530,11 +530,14 @@ PreprocessDefineCollationStmt(Node *node, const char *queryString,
{ {
Assert(castNode(DefineStmt, node)->kind == OBJECT_COLLATION); Assert(castNode(DefineStmt, node)->kind == OBJECT_COLLATION);
if (ShouldPropagateDefineCollationStmt()) if (!ShouldPropagateDefineCollationStmt())
{ {
EnsureCoordinator(); return NIL;
} }
EnsureCoordinator();
EnsureSequentialMode(OBJECT_COLLATION);
return NIL; return NIL;
} }
@ -575,8 +578,7 @@ PostprocessDefineCollationStmt(Node *node, const char *queryString)
* ShouldPropagateDefineCollationStmt checks if collation define * ShouldPropagateDefineCollationStmt checks if collation define
* statement should be propagated. Don't propagate if: * statement should be propagated. Don't propagate if:
* - metadata syncing if off * - metadata syncing if off
* - statement is part of a multi stmt transaction and the multi shard connection * - create statement should be propagated according the the ddl propagation policy
* type is not sequential
*/ */
static bool static bool
ShouldPropagateDefineCollationStmt() ShouldPropagateDefineCollationStmt()
@ -586,8 +588,7 @@ ShouldPropagateDefineCollationStmt()
return false; return false;
} }
if (IsMultiStatementTransaction() && if (!ShouldPropagateCreateInCoordinatedTransction())
MultiShardConnectionType != SEQUENTIAL_CONNECTION)
{ {
return false; return false;
} }

View File

@ -20,6 +20,8 @@
#include "distributed/metadata/dependency.h" #include "distributed/metadata/dependency.h"
#include "distributed/metadata/distobject.h" #include "distributed/metadata/distobject.h"
#include "distributed/metadata_sync.h" #include "distributed/metadata_sync.h"
#include "distributed/multi_executor.h"
#include "distributed/relation_access_tracking.h"
#include "distributed/remote_commands.h" #include "distributed/remote_commands.h"
#include "distributed/worker_manager.h" #include "distributed/worker_manager.h"
#include "distributed/worker_transaction.h" #include "distributed/worker_transaction.h"
@ -247,7 +249,8 @@ GetDependencyCreateDDLCommands(const ObjectAddress *dependency)
* The commands will be added to both shards and metadata tables via the table * The commands will be added to both shards and metadata tables via the table
* creation commands. * creation commands.
*/ */
if (relKind == RELKIND_INDEX) if (relKind == RELKIND_INDEX ||
relKind == RELKIND_PARTITIONED_INDEX)
{ {
return NIL; return NIL;
} }
@ -458,6 +461,88 @@ ShouldPropagate(void)
} }
/*
* ShouldPropagateCreateInCoordinatedTransction returns based the current state of the
* session and policies if Citus needs to propagate the creation of new objects.
*
* Creation of objects on other nodes could be postponed till the object is actually used
* in a sharded object (eg. distributed table or index on a distributed table). In certain
* use cases the opportunity for parallelism in a transaction block is preferred. When
* configured like that the creation of an object might be postponed and backfilled till
* the object is actually used.
*/
bool
ShouldPropagateCreateInCoordinatedTransction()
{
if (!IsMultiStatementTransaction())
{
/*
* If we are in a single statement transaction we will always propagate the
* creation of objects. There are no downsides in regard to performance or
* transactional limitations. These only arise with transaction blocks consisting
* of multiple statements.
*/
return true;
}
if (MultiShardConnectionType == SEQUENTIAL_CONNECTION)
{
/*
* If we are in a transaction that is already switched to sequential, either by
* the user, or automatically by an other command, we will always propagate the
* creation of new objects to the workers.
*
* This guarantees no strange anomalies when the transaction aborts or on
* visibility of the newly created object.
*/
return true;
}
switch (CreateObjectPropagationMode)
{
case CREATE_OBJECT_PROPAGATION_DEFERRED:
{
/*
* We prefer parallelism at this point. Since we did not already return while
* checking for sequential mode we are still in parallel mode. We don't want
* to switch that now, thus not propagating the creation.
*/
return false;
}
case CREATE_OBJECT_PROPAGATION_AUTOMATIC:
{
/*
* When we run in optimistic mode we want to switch to sequential mode, only
* if this would _not_ give an error to the user. Meaning, we either are
* already in sequential mode (checked earlier), or there has been no parallel
* execution in the current transaction block.
*
* If switching to sequential would throw an error we would stay in parallel
* mode while creating new objects. We will rely on Citus' mechanism to ensure
* the existence if the object would be used in the same transaction.
*/
if (ParallelQueryExecutedInTransaction())
{
return false;
}
return true;
}
case CREATE_OBJECT_PROPAGATION_IMMEDIATE:
{
return true;
}
default:
{
elog(ERROR, "unsupported ddl propagation mode");
}
}
}
/* /*
* ShouldPropagateObject determines if we should be propagating DDLs based * ShouldPropagateObject determines if we should be propagating DDLs based
* on their object address. * on their object address.

View File

@ -43,11 +43,11 @@ static DistributeObjectOps Aggregate_AlterOwner = {
}; };
static DistributeObjectOps Aggregate_Define = { static DistributeObjectOps Aggregate_Define = {
.deparse = NULL, .deparse = NULL,
.qualify = NULL, .qualify = QualifyDefineAggregateStmt,
.preprocess = NULL, .preprocess = NULL,
.postprocess = NULL, .postprocess = PostprocessDefineAggregateStmt,
.address = DefineAggregateStmtObjectAddress, .address = DefineAggregateStmtObjectAddress,
.markDistributed = false, .markDistributed = true,
}; };
static DistributeObjectOps Aggregate_Drop = { static DistributeObjectOps Aggregate_Drop = {
.deparse = DeparseDropFunctionStmt, .deparse = DeparseDropFunctionStmt,

View File

@ -136,11 +136,8 @@ PostprocessCreateExtensionStmt(Node *node, const char *queryString)
return NIL; return NIL;
} }
/* /* check creation against multi-statement transaction policy */
* If the extension command is a part of a multi-statement transaction, if (!ShouldPropagateCreateInCoordinatedTransction())
* do not propagate it
*/
if (IsMultiStatementTransaction())
{ {
return NIL; return NIL;
} }
@ -148,16 +145,6 @@ PostprocessCreateExtensionStmt(Node *node, const char *queryString)
/* extension management can only be done via coordinator node */ /* extension management can only be done via coordinator node */
EnsureCoordinator(); EnsureCoordinator();
/*
* Make sure that no new nodes are added after this point until the end of the
* transaction by taking a RowShareLock on pg_dist_node, which conflicts with the
* ExclusiveLock taken by citus_add_node.
* This guarantees that all active nodes will have the extension, because they will
* either get it now, or get it in citus_add_node after this transaction finishes and
* the pg_dist_object record becomes visible.
*/
LockRelationOid(DistNodeRelationId(), RowShareLock);
/* /*
* Make sure that the current transaction is already in sequential mode, * Make sure that the current transaction is already in sequential mode,
* or can still safely be put in sequential mode * or can still safely be put in sequential mode
@ -259,16 +246,6 @@ PreprocessDropExtensionStmt(Node *node, const char *queryString,
/* extension management can only be done via coordinator node */ /* extension management can only be done via coordinator node */
EnsureCoordinator(); EnsureCoordinator();
/*
* Make sure that no new nodes are added after this point until the end of the
* transaction by taking a RowShareLock on pg_dist_node, which conflicts with the
* ExclusiveLock taken by citus_add_node.
* This guarantees that all active nodes will drop the extension, because they will
* either get it now, or get it in citus_add_node after this transaction finishes and
* the pg_dist_object record becomes visible.
*/
LockRelationOid(DistNodeRelationId(), RowShareLock);
/* /*
* Make sure that the current transaction is already in sequential mode, * Make sure that the current transaction is already in sequential mode,
* or can still safely be put in sequential mode * or can still safely be put in sequential mode
@ -395,15 +372,6 @@ PreprocessAlterExtensionSchemaStmt(Node *node, const char *queryString,
/* extension management can only be done via coordinator node */ /* extension management can only be done via coordinator node */
EnsureCoordinator(); EnsureCoordinator();
/*
* Make sure that no new nodes are added after this point until the end of the
* transaction by taking a RowShareLock on pg_dist_node, which conflicts with the
* ExclusiveLock taken by citus_add_node.
* This guarantees that all active nodes will update the extension schema after
* this transaction finishes and the pg_dist_object record becomes visible.
*/
LockRelationOid(DistNodeRelationId(), RowShareLock);
/* /*
* Make sure that the current transaction is already in sequential mode, * Make sure that the current transaction is already in sequential mode,
* or can still safely be put in sequential mode * or can still safely be put in sequential mode
@ -463,16 +431,6 @@ PreprocessAlterExtensionUpdateStmt(Node *node, const char *queryString,
/* extension management can only be done via coordinator node */ /* extension management can only be done via coordinator node */
EnsureCoordinator(); EnsureCoordinator();
/*
* Make sure that no new nodes are added after this point until the end of the
* transaction by taking a RowShareLock on pg_dist_node, which conflicts with the
* ExclusiveLock taken by citus_add_node.
* This guarantees that all active nodes will update the extension version, because
* they will either get it now, or get it in citus_add_node after this transaction
* finishes and the pg_dist_object record becomes visible.
*/
LockRelationOid(DistNodeRelationId(), RowShareLock);
/* /*
* Make sure that the current transaction is already in sequential mode, * Make sure that the current transaction is already in sequential mode,
* or can still safely be put in sequential mode * or can still safely be put in sequential mode
@ -555,6 +513,16 @@ MarkExistingObjectDependenciesDistributedIfSupported()
ObjectAddress tableAddress = { 0 }; ObjectAddress tableAddress = { 0 };
ObjectAddressSet(tableAddress, RelationRelationId, citusTableId); ObjectAddressSet(tableAddress, RelationRelationId, citusTableId);
if (ShouldSyncTableMetadata(citusTableId))
{
/* we need to pass pointer allocated in the heap */
ObjectAddress *addressPointer = palloc0(sizeof(ObjectAddress));
*addressPointer = tableAddress;
/* as of Citus 11, tables that should be synced are also considered object */
resultingObjectAddresses = lappend(resultingObjectAddresses, addressPointer);
}
List *distributableDependencyObjectAddresses = List *distributableDependencyObjectAddresses =
GetDistributableDependenciesForObject(&tableAddress); GetDistributableDependenciesForObject(&tableAddress);
@ -578,11 +546,22 @@ MarkExistingObjectDependenciesDistributedIfSupported()
/* remove duplicates from object addresses list for efficiency */ /* remove duplicates from object addresses list for efficiency */
List *uniqueObjectAddresses = GetUniqueDependenciesList(resultingObjectAddresses); List *uniqueObjectAddresses = GetUniqueDependenciesList(resultingObjectAddresses);
/*
* We should sync the new dependencies during ALTER EXTENSION because
* we cannot know whether the nodes has already been upgraded or not. If
* the nodes are not upgraded at this point, we cannot sync the object. Also,
* when the workers upgraded, they'd get the same objects anyway.
*/
bool prevMetadataSyncValue = EnableMetadataSync;
SetLocalEnableMetadataSync(false);
ObjectAddress *objectAddress = NULL; ObjectAddress *objectAddress = NULL;
foreach_ptr(objectAddress, uniqueObjectAddresses) foreach_ptr(objectAddress, uniqueObjectAddresses)
{ {
MarkObjectDistributed(objectAddress); MarkObjectDistributed(objectAddress);
} }
SetLocalEnableMetadataSync(prevMetadataSyncValue);
} }

View File

@ -17,6 +17,7 @@
#include "distributed/listutils.h" #include "distributed/listutils.h"
#include "distributed/metadata/distobject.h" #include "distributed/metadata/distobject.h"
#include "distributed/metadata_sync.h" #include "distributed/metadata_sync.h"
#include "distributed/multi_executor.h"
#include "distributed/worker_transaction.h" #include "distributed/worker_transaction.h"
#include "foreign/foreign.h" #include "foreign/foreign.h"
#include "nodes/makefuncs.h" #include "nodes/makefuncs.h"
@ -41,7 +42,14 @@ PreprocessCreateForeignServerStmt(Node *node, const char *queryString,
return NIL; return NIL;
} }
/* check creation against multi-statement transaction policy */
if (!ShouldPropagateCreateInCoordinatedTransction())
{
return NIL;
}
EnsureCoordinator(); EnsureCoordinator();
EnsureSequentialMode(OBJECT_FOREIGN_SERVER);
char *sql = DeparseTreeNode(node); char *sql = DeparseTreeNode(node);
@ -209,7 +217,18 @@ PreprocessDropForeignServerStmt(Node *node, const char *queryString,
List * List *
PostprocessCreateForeignServerStmt(Node *node, const char *queryString) PostprocessCreateForeignServerStmt(Node *node, const char *queryString)
{ {
bool missingOk = false; if (!ShouldPropagate())
{
return NIL;
}
/* check creation against multi-statement transaction policy */
if (!ShouldPropagateCreateInCoordinatedTransction())
{
return NIL;
}
const bool missingOk = false;
ObjectAddress address = GetObjectAddressFromParseTree(node, missingOk); ObjectAddress address = GetObjectAddressFromParseTree(node, missingOk);
EnsureDependenciesExistOnAllNodes(&address); EnsureDependenciesExistOnAllNodes(&address);
@ -224,8 +243,14 @@ PostprocessCreateForeignServerStmt(Node *node, const char *queryString)
List * List *
PostprocessAlterForeignServerOwnerStmt(Node *node, const char *queryString) PostprocessAlterForeignServerOwnerStmt(Node *node, const char *queryString)
{ {
bool missingOk = false; const bool missingOk = false;
ObjectAddress address = GetObjectAddressFromParseTree(node, missingOk); ObjectAddress address = GetObjectAddressFromParseTree(node, missingOk);
if (!ShouldPropagateObject(&address))
{
return NIL;
}
EnsureDependenciesExistOnAllNodes(&address); EnsureDependenciesExistOnAllNodes(&address);
return NIL; return NIL;

View File

@ -82,7 +82,6 @@ static void EnsureFunctionCanBeColocatedWithTable(Oid functionOid, Oid
static bool ShouldPropagateCreateFunction(CreateFunctionStmt *stmt); static bool ShouldPropagateCreateFunction(CreateFunctionStmt *stmt);
static bool ShouldPropagateAlterFunction(const ObjectAddress *address); static bool ShouldPropagateAlterFunction(const ObjectAddress *address);
static bool ShouldAddFunctionSignature(FunctionParameterMode mode); static bool ShouldAddFunctionSignature(FunctionParameterMode mode);
static ObjectAddress * GetUndistributableDependency(ObjectAddress *functionAddress);
static ObjectAddress FunctionToObjectAddress(ObjectType objectType, static ObjectAddress FunctionToObjectAddress(ObjectType objectType,
ObjectWithArgs *objectWithArgs, ObjectWithArgs *objectWithArgs,
bool missing_ok); bool missing_ok);
@ -1352,7 +1351,7 @@ PostprocessCreateFunctionStmt(Node *node, const char *queryString)
* GetUndistributableDependency checks whether object has any non-distributable * GetUndistributableDependency checks whether object has any non-distributable
* dependency. If any one found, it will be returned. * dependency. If any one found, it will be returned.
*/ */
static ObjectAddress * ObjectAddress *
GetUndistributableDependency(ObjectAddress *objectAddress) GetUndistributableDependency(ObjectAddress *objectAddress)
{ {
List *dependencies = GetAllDependenciesForObject(objectAddress); List *dependencies = GetAllDependenciesForObject(objectAddress);
@ -1443,10 +1442,18 @@ DefineAggregateStmtObjectAddress(Node *node, bool missing_ok)
ObjectWithArgs *objectWithArgs = makeNode(ObjectWithArgs); ObjectWithArgs *objectWithArgs = makeNode(ObjectWithArgs);
objectWithArgs->objname = stmt->defnames; objectWithArgs->objname = stmt->defnames;
if (stmt->args != NIL)
{
FunctionParameter *funcParam = NULL; FunctionParameter *funcParam = NULL;
foreach_ptr(funcParam, linitial(stmt->args)) foreach_ptr(funcParam, linitial(stmt->args))
{ {
objectWithArgs->objargs = lappend(objectWithArgs->objargs, funcParam->argType); objectWithArgs->objargs = lappend(objectWithArgs->objargs,
funcParam->argType);
}
}
else
{
objectWithArgs->objargs = list_make1(makeTypeName("anyelement"));
} }
return FunctionToObjectAddress(OBJECT_AGGREGATE, objectWithArgs, missing_ok); return FunctionToObjectAddress(OBJECT_AGGREGATE, objectWithArgs, missing_ok);
@ -2019,10 +2026,10 @@ ShouldAddFunctionSignature(FunctionParameterMode mode)
/* /*
* FunctionToObjectAddress returns the ObjectAddress of a Function or Procedure based on * FunctionToObjectAddress returns the ObjectAddress of a Function, Procedure or
* its type and ObjectWithArgs describing the Function/Procedure. If missing_ok is set to * Aggregate based on its type and ObjectWithArgs describing the
* false an error will be raised by postgres explaining the Function/Procedure could not * Function/Procedure/Aggregate. If missing_ok is set to false an error will be
* be found. * raised by postgres explaining the Function/Procedure could not be found.
*/ */
static ObjectAddress static ObjectAddress
FunctionToObjectAddress(ObjectType objectType, ObjectWithArgs *objectWithArgs, FunctionToObjectAddress(ObjectType objectType, ObjectWithArgs *objectWithArgs,

View File

@ -140,13 +140,6 @@ PostprocessAlterRoleStmt(Node *node, const char *queryString)
AlterRoleStmt *stmt = castNode(AlterRoleStmt, node); AlterRoleStmt *stmt = castNode(AlterRoleStmt, node);
/*
* Make sure that no new nodes are added after this point until the end of the
* transaction by taking a RowShareLock on pg_dist_node, which conflicts with the
* ExclusiveLock taken by citus_add_node.
*/
LockRelationOid(DistNodeRelationId(), RowShareLock);
DefElem *option = NULL; DefElem *option = NULL;
foreach_ptr(option, stmt->options) foreach_ptr(option, stmt->options)
{ {

View File

@ -367,8 +367,8 @@ ShouldPropagateCreateSchemaStmt()
return false; return false;
} }
if (IsMultiStatementTransaction() && /* check creation against multi-statement transaction policy */
MultiShardConnectionType != SEQUENTIAL_CONNECTION) if (!ShouldPropagateCreateInCoordinatedTransction())
{ {
return false; return false;
} }

View File

@ -73,17 +73,11 @@ PostprocessCreateTextSearchConfigurationStmt(Node *node, const char *queryString
return NIL; return NIL;
} }
/* /* check creation against multi-statement transaction policy */
* If the create command is a part of a multi-statement transaction that is not in if (!ShouldPropagateCreateInCoordinatedTransction())
* sequential mode, don't propagate. Instead we will rely on back filling.
*/
if (IsMultiStatementTransaction())
{
if (MultiShardConnectionType != SEQUENTIAL_CONNECTION)
{ {
return NIL; return NIL;
} }
}
EnsureCoordinator(); EnsureCoordinator();
EnsureSequentialMode(OBJECT_TSCONFIGURATION); EnsureSequentialMode(OBJECT_TSCONFIGURATION);

View File

@ -129,16 +129,6 @@ PreprocessCompositeTypeStmt(Node *node, const char *queryString,
*/ */
EnsureCoordinator(); EnsureCoordinator();
/*
* Make sure that no new nodes are added after this point until the end of the
* transaction by taking a RowShareLock on pg_dist_node, which conflicts with the
* ExclusiveLock taken by citus_add_node.
* This guarantees that all active nodes will have the object, because they will
* either get it now, or get it in citus_add_node after this transaction finishes and
* the pg_dist_object record becomes visible.
*/
LockRelationOid(DistNodeRelationId(), RowShareLock);
/* fully qualify before lookup and later deparsing */ /* fully qualify before lookup and later deparsing */
QualifyTreeNode(node); QualifyTreeNode(node);
@ -1161,7 +1151,7 @@ ShouldPropagateTypeCreate()
* this type will be used as a column in a table that will be created and distributed * this type will be used as a column in a table that will be created and distributed
* in this same transaction. * in this same transaction.
*/ */
if (IsMultiStatementTransaction()) if (!ShouldPropagateCreateInCoordinatedTransction())
{ {
return false; return false;
} }

View File

@ -78,6 +78,7 @@
#include "utils/syscache.h" #include "utils/syscache.h"
bool EnableDDLPropagation = true; /* ddl propagation is enabled */ bool EnableDDLPropagation = true; /* ddl propagation is enabled */
int CreateObjectPropagationMode = CREATE_OBJECT_PROPAGATION_DEFERRED;
PropSetCmdBehavior PropagateSetCommands = PROPSETCMD_NONE; /* SET prop off */ PropSetCmdBehavior PropagateSetCommands = PROPSETCMD_NONE; /* SET prop off */
static bool shouldInvalidateForeignKeyGraph = false; static bool shouldInvalidateForeignKeyGraph = false;
static int activeAlterTables = 0; static int activeAlterTables = 0;
@ -258,7 +259,7 @@ multi_ProcessUtility(PlannedStmt *pstmt,
else if (IsA(parsetree, DoStmt)) else if (IsA(parsetree, DoStmt))
{ {
/* /*
* All statements in a DO block are executed in a single transaciton, * All statements in a DO block are executed in a single transaction,
* so we need to keep track of whether we are inside a DO block. * so we need to keep track of whether we are inside a DO block.
*/ */
DoBlockLevel += 1; DoBlockLevel += 1;
@ -1563,7 +1564,8 @@ DDLTaskList(Oid relationId, const char *commandString)
List * List *
NodeDDLTaskList(TargetWorkerSet targets, List *commands) NodeDDLTaskList(TargetWorkerSet targets, List *commands)
{ {
List *workerNodes = TargetWorkerSetNodeList(targets, NoLock); /* don't allow concurrent node list changes that require an exclusive lock */
List *workerNodes = TargetWorkerSetNodeList(targets, RowShareLock);
if (list_length(workerNodes) <= 0) if (list_length(workerNodes) <= 0)
{ {

View File

@ -0,0 +1,34 @@
/*-------------------------------------------------------------------------
*
* qualify_aggregate_stmts.c
* Functions specialized in fully qualifying all aggregate statements.
* These functions are dispatched from qualify.c
*
* Fully qualifying aggregate statements consists of adding the schema name
* to the subject of the types as well as any other branch of the parsetree.
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "catalog/namespace.h"
#include "distributed/deparser.h"
#include "nodes/makefuncs.h"
#include "utils/lsyscache.h"
void
QualifyDefineAggregateStmt(Node *node)
{
DefineStmt *stmt = castNode(DefineStmt, node);
if (list_length(stmt->defnames) == 1)
{
char *objname = NULL;
Oid creationSchema = QualifiedNameGetCreationNamespace(stmt->defnames, &objname);
stmt->defnames = list_make2(makeString(get_namespace_name(creationSchema)),
linitial(stmt->defnames));
}
}

View File

@ -819,6 +819,9 @@ AdaptiveExecutor(CitusScanState *scanState)
bool hasDependentJobs = HasDependentJobs(job); bool hasDependentJobs = HasDependentJobs(job);
if (hasDependentJobs) if (hasDependentJobs)
{ {
/* jobs use intermediate results, which require a distributed transaction */
UseCoordinatedTransaction();
jobIdList = ExecuteDependentTasks(taskList, job); jobIdList = ExecuteDependentTasks(taskList, job);
} }
@ -828,9 +831,10 @@ AdaptiveExecutor(CitusScanState *scanState)
targetPoolSize = 1; targetPoolSize = 1;
} }
bool excludeFromXact = false;
TransactionProperties xactProperties = DecideTransactionPropertiesForTaskList( TransactionProperties xactProperties = DecideTransactionPropertiesForTaskList(
distributedPlan->modLevel, taskList, distributedPlan->modLevel, taskList, excludeFromXact);
hasDependentJobs);
bool localExecutionSupported = true; bool localExecutionSupported = true;
DistributedExecution *execution = CreateDistributedExecution( DistributedExecution *execution = CreateDistributedExecution(
@ -873,11 +877,6 @@ AdaptiveExecutor(CitusScanState *scanState)
FinishDistributedExecution(execution); FinishDistributedExecution(execution);
if (hasDependentJobs)
{
DoRepartitionCleanup(jobIdList);
}
if (SortReturning && distributedPlan->expectResults && commandType != CMD_SELECT) if (SortReturning && distributedPlan->expectResults && commandType != CMD_SELECT)
{ {
SortTupleStore(scanState); SortTupleStore(scanState);
@ -963,6 +962,26 @@ ExecuteUtilityTaskListExtended(List *utilityTaskList, int poolSize,
} }
/*
* ExecuteTaskList is a proxy to ExecuteTaskListExtended
* with defaults for some of the arguments.
*/
uint64
ExecuteTaskList(RowModifyLevel modLevel, List *taskList)
{
bool localExecutionSupported = true;
ExecutionParams *executionParams = CreateBasicExecutionParams(
modLevel, taskList, MaxAdaptiveExecutorPoolSize, localExecutionSupported
);
bool excludeFromXact = false;
executionParams->xactProperties = DecideTransactionPropertiesForTaskList(
modLevel, taskList, excludeFromXact);
return ExecuteTaskListExtended(executionParams);
}
/* /*
* ExecuteTaskListOutsideTransaction is a proxy to ExecuteTaskListExtended * ExecuteTaskListOutsideTransaction is a proxy to ExecuteTaskListExtended
* with defaults for some of the arguments. * with defaults for some of the arguments.
@ -2639,12 +2658,6 @@ RunDistributedExecution(DistributedExecution *execution)
*/ */
UnclaimAllSessionConnections(execution->sessionList); UnclaimAllSessionConnections(execution->sessionList);
/* do repartition cleanup if this is a repartition query*/
if (list_length(execution->jobIdList) > 0)
{
DoRepartitionCleanup(execution->jobIdList);
}
if (execution->waitEventSet != NULL) if (execution->waitEventSet != NULL)
{ {
FreeWaitEventSet(execution->waitEventSet); FreeWaitEventSet(execution->waitEventSet);

View File

@ -150,8 +150,6 @@ RegisterCitusCustomScanMethods(void)
static void static void
CitusBeginScan(CustomScanState *node, EState *estate, int eflags) CitusBeginScan(CustomScanState *node, EState *estate, int eflags)
{ {
MarkCitusInitiatedCoordinatorBackend();
CitusScanState *scanState = (CitusScanState *) node; CitusScanState *scanState = (CitusScanState *) node;
/* /*

View File

@ -39,6 +39,7 @@ static HTAB * CreateTaskHashTable(void);
static bool IsAllDependencyCompleted(Task *task, HTAB *completedTasks); static bool IsAllDependencyCompleted(Task *task, HTAB *completedTasks);
static void AddCompletedTasks(List *curCompletedTasks, HTAB *completedTasks); static void AddCompletedTasks(List *curCompletedTasks, HTAB *completedTasks);
static List * FindExecutableTasks(List *allTasks, HTAB *completedTasks); static List * FindExecutableTasks(List *allTasks, HTAB *completedTasks);
static List * RemoveMergeTasks(List *taskList);
static int TaskHashCompare(const void *key1, const void *key2, Size keysize); static int TaskHashCompare(const void *key1, const void *key2, Size keysize);
static uint32 TaskHash(const void *key, Size keysize); static uint32 TaskHash(const void *key, Size keysize);
static bool IsTaskAlreadyCompleted(Task *task, HTAB *completedTasks); static bool IsTaskAlreadyCompleted(Task *task, HTAB *completedTasks);
@ -64,8 +65,13 @@ ExecuteTasksInDependencyOrder(List *allTasks, List *excludedTasks, List *jobIds)
{ {
break; break;
} }
ExecuteTaskListOutsideTransaction(ROW_MODIFY_NONE, curTasks,
MaxAdaptiveExecutorPoolSize, jobIds); /* merge tasks do not need to be executed */
List *executableTasks = RemoveMergeTasks(curTasks);
if (list_length(executableTasks) > 0)
{
ExecuteTaskList(ROW_MODIFY_NONE, executableTasks);
}
AddCompletedTasks(curTasks, completedTasks); AddCompletedTasks(curTasks, completedTasks);
curTasks = NIL; curTasks = NIL;
@ -97,6 +103,29 @@ FindExecutableTasks(List *allTasks, HTAB *completedTasks)
} }
/*
* RemoveMergeTasks returns a copy of taskList that excludes all the
* merge tasks. We do this because merge tasks are currently only a
* logical concept that does not need to be executed.
*/
static List *
RemoveMergeTasks(List *taskList)
{
List *prunedTaskList = NIL;
Task *task = NULL;
foreach_ptr(task, taskList)
{
if (task->taskType != MERGE_TASK)
{
prunedTaskList = lappend(prunedTaskList, task);
}
}
return prunedTaskList;
}
/* /*
* AddCompletedTasks adds the givens tasks to completedTasks HTAB. * AddCompletedTasks adds the givens tasks to completedTasks HTAB.
*/ */

View File

@ -59,28 +59,6 @@ typedef struct PartitioningTupleDest
} PartitioningTupleDest; } PartitioningTupleDest;
/*
* NodePair contains the source and destination node in a NodeToNodeFragmentsTransfer.
* It is a separate struct to use it as a key in a hash table.
*/
typedef struct NodePair
{
uint32 sourceNodeId;
uint32 targetNodeId;
} NodePair;
/*
* NodeToNodeFragmentsTransfer contains all fragments that need to be fetched from
* the source node to the destination node in the NodePair.
*/
typedef struct NodeToNodeFragmentsTransfer
{
NodePair nodes;
List *fragmentList;
} NodeToNodeFragmentsTransfer;
/* forward declarations of local functions */ /* forward declarations of local functions */
static List * WrapTasksForPartitioning(const char *resultIdPrefix, static List * WrapTasksForPartitioning(const char *resultIdPrefix,
List *selectTaskList, List *selectTaskList,
@ -98,9 +76,6 @@ static TupleDesc PartitioningTupleDestTupleDescForQuery(TupleDestination *self,
queryNumber); queryNumber);
static ArrayType * CreateArrayFromDatums(Datum *datumArray, bool *nullsArray, int static ArrayType * CreateArrayFromDatums(Datum *datumArray, bool *nullsArray, int
datumCount, Oid typeId); datumCount, Oid typeId);
static void ShardMinMaxValueArrays(ShardInterval **shardIntervalArray, int shardCount,
Oid intervalTypeId, ArrayType **minValueArray,
ArrayType **maxValueArray);
static char * SourceShardPrefix(const char *resultPrefix, uint64 shardId); static char * SourceShardPrefix(const char *resultPrefix, uint64 shardId);
static DistributedResultFragment * TupleToDistributedResultFragment(HeapTuple heapTuple, static DistributedResultFragment * TupleToDistributedResultFragment(HeapTuple heapTuple,
TupleDesc tupleDesc, TupleDesc tupleDesc,
@ -115,8 +90,6 @@ static List ** ColocateFragmentsWithRelation(List *fragmentList,
static List * ColocationTransfers(List *fragmentList, static List * ColocationTransfers(List *fragmentList,
CitusTableCacheEntry *targetRelation); CitusTableCacheEntry *targetRelation);
static List * FragmentTransferTaskList(List *fragmentListTransfers); static List * FragmentTransferTaskList(List *fragmentListTransfers);
static char * QueryStringForFragmentsTransfer(
NodeToNodeFragmentsTransfer *fragmentsTransfer);
static void ExecuteFetchTaskList(List *fetchTaskList); static void ExecuteFetchTaskList(List *fetchTaskList);
@ -360,7 +333,7 @@ SourceShardPrefix(const char *resultPrefix, uint64 shardId)
* ShardMinMaxValueArrays returns min values and max values of given shard * ShardMinMaxValueArrays returns min values and max values of given shard
* intervals. Returned arrays are text arrays. * intervals. Returned arrays are text arrays.
*/ */
static void void
ShardMinMaxValueArrays(ShardInterval **shardIntervalArray, int shardCount, ShardMinMaxValueArrays(ShardInterval **shardIntervalArray, int shardCount,
Oid intervalTypeOutFunc, ArrayType **minValueArray, Oid intervalTypeOutFunc, ArrayType **minValueArray,
ArrayType **maxValueArray) ArrayType **maxValueArray)
@ -632,7 +605,7 @@ FragmentTransferTaskList(List *fragmentListTransfers)
* result fragments from source node to target node. See the structure of * result fragments from source node to target node. See the structure of
* NodeToNodeFragmentsTransfer for details of how these are decided. * NodeToNodeFragmentsTransfer for details of how these are decided.
*/ */
static char * char *
QueryStringForFragmentsTransfer(NodeToNodeFragmentsTransfer *fragmentsTransfer) QueryStringForFragmentsTransfer(NodeToNodeFragmentsTransfer *fragmentsTransfer)
{ {
StringInfo queryString = makeStringInfo(); StringInfo queryString = makeStringInfo();
@ -667,7 +640,7 @@ QueryStringForFragmentsTransfer(NodeToNodeFragmentsTransfer *fragmentsTransfer)
quote_literal_cstr(sourceNode->workerName), quote_literal_cstr(sourceNode->workerName),
sourceNode->workerPort); sourceNode->workerPort);
ereport(DEBUG3, (errmsg("fetch task on %s:%d: %s", sourceNode->workerName, ereport(DEBUG4, (errmsg("fetch task on %s:%d: %s", sourceNode->workerName,
sourceNode->workerPort, queryString->data))); sourceNode->workerPort, queryString->data)));
return queryString->data; return queryString->data;

View File

@ -17,6 +17,7 @@
#include "pgstat.h" #include "pgstat.h"
#include "catalog/pg_enum.h" #include "catalog/pg_enum.h"
#include "catalog/pg_type.h"
#include "commands/copy.h" #include "commands/copy.h"
#include "distributed/commands/multi_copy.h" #include "distributed/commands/multi_copy.h"
#include "distributed/connection_management.h" #include "distributed/connection_management.h"
@ -916,6 +917,8 @@ fetch_intermediate_results(PG_FUNCTION_ARGS)
StringInfo beginAndSetXactId = BeginAndSetDistributedTransactionIdCommand(); StringInfo beginAndSetXactId = BeginAndSetDistributedTransactionIdCommand();
ExecuteCriticalRemoteCommand(connection, beginAndSetXactId->data); ExecuteCriticalRemoteCommand(connection, beginAndSetXactId->data);
CreateIntermediateResultsDirectory();
for (resultIndex = 0; resultIndex < resultCount; resultIndex++) for (resultIndex = 0; resultIndex < resultCount; resultIndex++)
{ {
char *resultId = TextDatumGetCString(resultIdArray[resultIndex]); char *resultId = TextDatumGetCString(resultIdArray[resultIndex]);
@ -938,6 +941,19 @@ fetch_intermediate_results(PG_FUNCTION_ARGS)
static uint64 static uint64
FetchRemoteIntermediateResult(MultiConnection *connection, char *resultId) FetchRemoteIntermediateResult(MultiConnection *connection, char *resultId)
{ {
char *localPath = QueryResultFileName(resultId);
struct stat fileStat;
int statOK = stat(localPath, &fileStat);
if (statOK == 0)
{
/*
* File exists, most likely because we are trying to fetch a
* a file from a node to itself. Skip doing work.
*/
return fileStat.st_size;
}
uint64 totalBytesWritten = 0; uint64 totalBytesWritten = 0;
StringInfo copyCommand = makeStringInfo(); StringInfo copyCommand = makeStringInfo();
@ -948,8 +964,6 @@ FetchRemoteIntermediateResult(MultiConnection *connection, char *resultId)
int socket = PQsocket(pgConn); int socket = PQsocket(pgConn);
bool raiseErrors = true; bool raiseErrors = true;
CreateIntermediateResultsDirectory();
appendStringInfo(copyCommand, "COPY \"%s\" TO STDOUT WITH (format result)", appendStringInfo(copyCommand, "COPY \"%s\" TO STDOUT WITH (format result)",
resultId); resultId);
@ -966,7 +980,6 @@ FetchRemoteIntermediateResult(MultiConnection *connection, char *resultId)
PQclear(result); PQclear(result);
char *localPath = QueryResultFileName(resultId);
File fileDesc = FileOpenForTransmit(localPath, fileFlags, fileMode); File fileDesc = FileOpenForTransmit(localPath, fileFlags, fileMode);
FileCompat fileCompat = FileCompatFromFileStart(fileDesc); FileCompat fileCompat = FileCompatFromFileStart(fileDesc);

View File

@ -915,6 +915,25 @@ TaskAccessesLocalNode(Task *task)
} }
/*
* EnsureCompatibleLocalExecutionState makes sure that the tasks won't have
* any visibility problems because of local execution.
*/
void
EnsureCompatibleLocalExecutionState(List *taskList)
{
/*
* We have LOCAL_EXECUTION_REQUIRED check here to avoid unnecessarily
* iterating the task list in AnyTaskAccessesLocalNode.
*/
if (GetCurrentLocalExecutionStatus() == LOCAL_EXECUTION_REQUIRED &&
AnyTaskAccessesLocalNode(taskList))
{
ErrorIfTransactionAccessedPlacementsLocally();
}
}
/* /*
* ErrorIfTransactionAccessedPlacementsLocally errors out if a local query * ErrorIfTransactionAccessedPlacementsLocally errors out if a local query
* on any shard has already been executed in the same transaction. * on any shard has already been executed in the same transaction.

View File

@ -748,6 +748,11 @@ GetObjectTypeString(ObjectType objType)
{ {
switch (objType) switch (objType)
{ {
case OBJECT_AGGREGATE:
{
return "aggregate";
}
case OBJECT_COLLATION: case OBJECT_COLLATION:
{ {
return "collation"; return "collation";
@ -763,6 +768,11 @@ GetObjectTypeString(ObjectType objType)
return "extension"; return "extension";
} }
case OBJECT_FOREIGN_SERVER:
{
return "foreign server";
}
case OBJECT_FUNCTION: case OBJECT_FUNCTION:
{ {
return "function"; return "function";

View File

@ -86,6 +86,9 @@ typedef struct PartitionedResultDestReceiver
/* keeping track of which partitionDestReceivers have been started */ /* keeping track of which partitionDestReceivers have been started */
Bitmapset *startedDestReceivers; Bitmapset *startedDestReceivers;
/* whether NULL partition column values are allowed */
bool allowNullPartitionColumnValues;
} PartitionedResultDestReceiver; } PartitionedResultDestReceiver;
static Portal StartPortalForQueryExecution(const char *queryString); static Portal StartPortalForQueryExecution(const char *queryString);
@ -99,7 +102,8 @@ static DestReceiver * CreatePartitionedResultDestReceiver(int partitionColumnInd
shardSearchInfo, shardSearchInfo,
DestReceiver ** DestReceiver **
partitionedDestReceivers, partitionedDestReceivers,
bool lazyStartup); bool lazyStartup,
bool allowNullPartitionValues);
static void PartitionedResultDestReceiverStartup(DestReceiver *dest, int operation, static void PartitionedResultDestReceiverStartup(DestReceiver *dest, int operation,
TupleDesc inputTupleDescriptor); TupleDesc inputTupleDescriptor);
static bool PartitionedResultDestReceiverReceive(TupleTableSlot *slot, static bool PartitionedResultDestReceiverReceive(TupleTableSlot *slot,
@ -148,6 +152,8 @@ worker_partition_query_result(PG_FUNCTION_ARGS)
int32 maxValuesCount = ArrayObjectCount(maxValuesArray); int32 maxValuesCount = ArrayObjectCount(maxValuesArray);
bool binaryCopy = PG_GETARG_BOOL(6); bool binaryCopy = PG_GETARG_BOOL(6);
bool allowNullPartitionColumnValues = PG_GETARG_BOOL(7);
bool generateEmptyResults = PG_GETARG_BOOL(8);
if (!IsMultiStatementTransaction()) if (!IsMultiStatementTransaction())
{ {
@ -226,13 +232,21 @@ worker_partition_query_result(PG_FUNCTION_ARGS)
dests[partitionIndex] = partitionDest; dests[partitionIndex] = partitionDest;
} }
const bool lazyStartup = true; /*
* If we are asked to generated empty results, use non-lazy startup.
*
* The rStartup of the FileDestReceiver will be called for all partitions
* and generate empty files, which may still have binary header/footer.
*/
const bool lazyStartup = !generateEmptyResults;
DestReceiver *dest = CreatePartitionedResultDestReceiver( DestReceiver *dest = CreatePartitionedResultDestReceiver(
partitionColumnIndex, partitionColumnIndex,
partitionCount, partitionCount,
shardSearchInfo, shardSearchInfo,
dests, dests,
lazyStartup); lazyStartup,
allowNullPartitionColumnValues);
/* execute the query */ /* execute the query */
PortalRun(portal, FETCH_ALL, false, true, dest, dest, NULL); PortalRun(portal, FETCH_ALL, false, true, dest, dest, NULL);
@ -390,7 +404,8 @@ CreatePartitionedResultDestReceiver(int partitionColumnIndex,
int partitionCount, int partitionCount,
CitusTableCacheEntry *shardSearchInfo, CitusTableCacheEntry *shardSearchInfo,
DestReceiver **partitionedDestReceivers, DestReceiver **partitionedDestReceivers,
bool lazyStartup) bool lazyStartup,
bool allowNullPartitionColumnValues)
{ {
PartitionedResultDestReceiver *resultDest = PartitionedResultDestReceiver *resultDest =
palloc0(sizeof(PartitionedResultDestReceiver)); palloc0(sizeof(PartitionedResultDestReceiver));
@ -409,6 +424,7 @@ CreatePartitionedResultDestReceiver(int partitionColumnIndex,
resultDest->partitionDestReceivers = partitionedDestReceivers; resultDest->partitionDestReceivers = partitionedDestReceivers;
resultDest->startedDestReceivers = NULL; resultDest->startedDestReceivers = NULL;
resultDest->lazyStartup = lazyStartup; resultDest->lazyStartup = lazyStartup;
resultDest->allowNullPartitionColumnValues = allowNullPartitionColumnValues;
return (DestReceiver *) resultDest; return (DestReceiver *) resultDest;
} }
@ -458,12 +474,27 @@ PartitionedResultDestReceiverReceive(TupleTableSlot *slot, DestReceiver *dest)
Datum *columnValues = slot->tts_values; Datum *columnValues = slot->tts_values;
bool *columnNulls = slot->tts_isnull; bool *columnNulls = slot->tts_isnull;
int partitionIndex;
if (columnNulls[self->partitionColumnIndex]) if (columnNulls[self->partitionColumnIndex])
{
if (self->allowNullPartitionColumnValues)
{
/*
* NULL values go into the first partition for both hash- and range-
* partitioning, since that is the only way to guarantee that there is
* always a partition for NULL and that it is always the same partition.
*/
partitionIndex = 0;
}
else
{ {
ereport(ERROR, (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED), ereport(ERROR, (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
errmsg("the partition column value cannot be NULL"))); errmsg("the partition column value cannot be NULL")));
} }
}
else
{
Datum partitionColumnValue = columnValues[self->partitionColumnIndex]; Datum partitionColumnValue = columnValues[self->partitionColumnIndex];
ShardInterval *shardInterval = FindShardInterval(partitionColumnValue, ShardInterval *shardInterval = FindShardInterval(partitionColumnValue,
self->shardSearchInfo); self->shardSearchInfo);
@ -474,7 +505,9 @@ PartitionedResultDestReceiverReceive(TupleTableSlot *slot, DestReceiver *dest)
"value"))); "value")));
} }
int partitionIndex = shardInterval->shardIndex; partitionIndex = shardInterval->shardIndex;
}
DestReceiver *partitionDest = self->partitionDestReceivers[partitionIndex]; DestReceiver *partitionDest = self->partitionDestReceivers[partitionIndex];
/* check if this partitionDestReceiver has been started before, start if not */ /* check if this partitionDestReceiver has been started before, start if not */

View File

@ -44,12 +44,8 @@
#include "distributed/worker_transaction.h" #include "distributed/worker_transaction.h"
static List * CreateTemporarySchemasForMergeTasks(Job *topLevelJob);
static List * ExtractJobsInJobTree(Job *job); static List * ExtractJobsInJobTree(Job *job);
static void TraverseJobTree(Job *curJob, List **jobs); static void TraverseJobTree(Job *curJob, List **jobs);
static char * GenerateCreateSchemasCommand(List *jobIds, char *schemaOwner);
static char * GenerateJobCommands(List *jobIds, char *templateCommand);
static char * GenerateDeleteJobsCommand(List *jobIds);
/* /*
@ -60,13 +56,8 @@ static char * GenerateDeleteJobsCommand(List *jobIds);
List * List *
ExecuteDependentTasks(List *topLevelTasks, Job *topLevelJob) ExecuteDependentTasks(List *topLevelTasks, Job *topLevelJob)
{ {
EnsureNoModificationsHaveBeenDone();
List *allTasks = CreateTaskListForJobTree(topLevelTasks); List *allTasks = CreateTaskListForJobTree(topLevelTasks);
List *jobIds = ExtractJobsInJobTree(topLevelJob);
EnsureCompatibleLocalExecutionState(allTasks);
List *jobIds = CreateTemporarySchemasForMergeTasks(topLevelJob);
ExecuteTasksInDependencyOrder(allTasks, topLevelTasks, jobIds); ExecuteTasksInDependencyOrder(allTasks, topLevelTasks, jobIds);
@ -74,40 +65,6 @@ ExecuteDependentTasks(List *topLevelTasks, Job *topLevelJob)
} }
/*
* EnsureCompatibleLocalExecutionState makes sure that the tasks won't have
* any visibility problems because of local execution.
*/
void
EnsureCompatibleLocalExecutionState(List *taskList)
{
/*
* We have LOCAL_EXECUTION_REQUIRED check here to avoid unnecessarily
* iterating the task list in AnyTaskAccessesLocalNode.
*/
if (GetCurrentLocalExecutionStatus() == LOCAL_EXECUTION_REQUIRED &&
AnyTaskAccessesLocalNode(taskList))
{
ErrorIfTransactionAccessedPlacementsLocally();
}
}
/*
* CreateTemporarySchemasForMergeTasks creates the necessary schemas that will be used
* later in each worker. Single transaction is used to create the schemas.
*/
static List *
CreateTemporarySchemasForMergeTasks(Job *topLeveLJob)
{
List *jobIds = ExtractJobsInJobTree(topLeveLJob);
char *createSchemasCommand = GenerateCreateSchemasCommand(jobIds, CurrentUserName());
SendCommandToWorkersInParallel(ALL_SHARD_NODES, createSchemasCommand,
CitusExtensionOwnerName());
return jobIds;
}
/* /*
* ExtractJobsInJobTree returns all job ids in the job tree * ExtractJobsInJobTree returns all job ids in the job tree
* where the given job is root. * where the given job is root.
@ -139,67 +96,3 @@ TraverseJobTree(Job *curJob, List **jobIds)
TraverseJobTree(childJob, jobIds); TraverseJobTree(childJob, jobIds);
} }
} }
/*
* GenerateCreateSchemasCommand returns concatanated create schema commands.
*/
static char *
GenerateCreateSchemasCommand(List *jobIds, char *ownerName)
{
StringInfo createSchemaCommand = makeStringInfo();
uint64 *jobIdPointer = NULL;
foreach_ptr(jobIdPointer, jobIds)
{
uint64 jobId = *jobIdPointer;
appendStringInfo(createSchemaCommand, WORKER_CREATE_SCHEMA_QUERY,
jobId, quote_literal_cstr(ownerName));
}
return createSchemaCommand->data;
}
/*
* GenerateJobCommands returns concatenated commands with the given template
* command for each job id from the given job ids. The returned command is
* exactly list_length(jobIds) subcommands.
* E.g create_schema(jobId1); create_schema(jobId2); ...
* This way we can send the command in just one latency to a worker.
*/
static char *
GenerateJobCommands(List *jobIds, char *templateCommand)
{
StringInfo createSchemaCommand = makeStringInfo();
uint64 *jobIdPointer = NULL;
foreach_ptr(jobIdPointer, jobIds)
{
uint64 jobId = *jobIdPointer;
appendStringInfo(createSchemaCommand, templateCommand, jobId);
}
return createSchemaCommand->data;
}
/*
* DoRepartitionCleanup removes the temporary job directories and schemas that are
* used for repartition queries for the given job ids.
*/
void
DoRepartitionCleanup(List *jobIds)
{
SendCommandToWorkersOptionalInParallel(ALL_SHARD_NODES, GenerateDeleteJobsCommand(
jobIds),
CitusExtensionOwnerName());
}
/*
* GenerateDeleteJobsCommand returns concatanated remove job dir commands.
*/
static char *
GenerateDeleteJobsCommand(List *jobIds)
{
return GenerateJobCommands(jobIds, WORKER_REPARTITION_CLEANUP_QUERY);
}

View File

@ -122,6 +122,7 @@ typedef struct ViewDependencyNode
static List * GetRelationSequenceDependencyList(Oid relationId); static List * GetRelationSequenceDependencyList(Oid relationId);
static List * GetRelationFunctionDependencyList(Oid relationId);
static List * GetRelationTriggerFunctionDependencyList(Oid relationId); static List * GetRelationTriggerFunctionDependencyList(Oid relationId);
static List * GetRelationStatsSchemaDependencyList(Oid relationId); static List * GetRelationStatsSchemaDependencyList(Oid relationId);
static List * GetRelationIndicesDependencyList(Oid relationId); static List * GetRelationIndicesDependencyList(Oid relationId);
@ -722,7 +723,8 @@ SupportedDependencyByCitus(const ObjectAddress *address)
relKind == RELKIND_PARTITIONED_TABLE || relKind == RELKIND_PARTITIONED_TABLE ||
relKind == RELKIND_FOREIGN_TABLE || relKind == RELKIND_FOREIGN_TABLE ||
relKind == RELKIND_SEQUENCE || relKind == RELKIND_SEQUENCE ||
relKind == RELKIND_INDEX) relKind == RELKIND_INDEX ||
relKind == RELKIND_PARTITIONED_INDEX)
{ {
return true; return true;
} }
@ -1090,9 +1092,15 @@ ExpandCitusSupportedTypes(ObjectAddressCollector *collector, ObjectAddress targe
* with them. * with them.
*/ */
List *sequenceDependencyList = GetRelationSequenceDependencyList(relationId); List *sequenceDependencyList = GetRelationSequenceDependencyList(relationId);
result = list_concat(result, sequenceDependencyList); result = list_concat(result, sequenceDependencyList);
/*
* Get the dependent functions for tables as columns has default values
* and contraints, then expand dependency list with them.
*/
List *functionDependencyList = GetRelationFunctionDependencyList(relationId);
result = list_concat(result, functionDependencyList);
/* /*
* Tables could have indexes. Indexes themself could have dependencies that * Tables could have indexes. Indexes themself could have dependencies that
* need to be propagated. eg. TEXT SEARCH CONFIGRUATIONS. Here we add the * need to be propagated. eg. TEXT SEARCH CONFIGRUATIONS. Here we add the
@ -1133,6 +1141,21 @@ GetRelationSequenceDependencyList(Oid relationId)
} }
/*
* GetRelationFunctionDependencyList returns the function dependency definition
* list for the given relation.
*/
static List *
GetRelationFunctionDependencyList(Oid relationId)
{
List *dependentFunctionOids = GetDependentFunctionsWithRelation(relationId);
List *functionDependencyDefList =
CreateObjectAddressDependencyDefList(ProcedureRelationId, dependentFunctionOids);
return functionDependencyDefList;
}
/* /*
* GetRelationStatsSchemaDependencyList returns a list of DependencyDefinition * GetRelationStatsSchemaDependencyList returns a list of DependencyDefinition
* objects for the schemas that statistics' of the relation with relationId depends. * objects for the schemas that statistics' of the relation with relationId depends.

View File

@ -28,9 +28,11 @@
#include "catalog/indexing.h" #include "catalog/indexing.h"
#include "catalog/pg_am.h" #include "catalog/pg_am.h"
#include "catalog/pg_attrdef.h" #include "catalog/pg_attrdef.h"
#include "catalog/pg_constraint.h"
#include "catalog/pg_depend.h" #include "catalog/pg_depend.h"
#include "catalog/pg_foreign_server.h" #include "catalog/pg_foreign_server.h"
#include "catalog/pg_namespace.h" #include "catalog/pg_namespace.h"
#include "catalog/pg_proc.h"
#include "catalog/pg_type.h" #include "catalog/pg_type.h"
#include "commands/async.h" #include "commands/async.h"
#include "distributed/argutils.h" #include "distributed/argutils.h"
@ -85,6 +87,7 @@ char *EnableManualMetadataChangesForUser = "";
static void EnsureObjectMetadataIsSane(int distributionArgumentIndex, static void EnsureObjectMetadataIsSane(int distributionArgumentIndex,
int colocationId); int colocationId);
static List * GetFunctionDependenciesForObjects(ObjectAddress *objectAddress);
static char * SchemaOwnerName(Oid objectId); static char * SchemaOwnerName(Oid objectId);
static bool HasMetadataWorkers(void); static bool HasMetadataWorkers(void);
static void CreateShellTableOnWorkers(Oid relationId); static void CreateShellTableOnWorkers(Oid relationId);
@ -104,7 +107,6 @@ static List * GetObjectsForGrantStmt(ObjectType objectType, Oid objectId);
static AccessPriv * GetAccessPrivObjectForGrantStmt(char *permission); static AccessPriv * GetAccessPrivObjectForGrantStmt(char *permission);
static List * GenerateGrantOnSchemaQueriesFromAclItem(Oid schemaOid, static List * GenerateGrantOnSchemaQueriesFromAclItem(Oid schemaOid,
AclItem *aclItem); AclItem *aclItem);
static void SetLocalEnableMetadataSync(bool state);
static void SetLocalReplicateReferenceTablesOnActivate(bool state); static void SetLocalReplicateReferenceTablesOnActivate(bool state);
static char * GenerateSetRoleQuery(Oid roleOid); static char * GenerateSetRoleQuery(Oid roleOid);
static void MetadataSyncSigTermHandler(SIGNAL_ARGS); static void MetadataSyncSigTermHandler(SIGNAL_ARGS);
@ -1544,6 +1546,119 @@ GetSequencesFromAttrDef(Oid attrdefOid)
} }
/*
* GetDependentFunctionsWithRelation returns the dependent functions for the
* given relation id.
*/
List *
GetDependentFunctionsWithRelation(Oid relationId)
{
List *referencingObjects = NIL;
List *functionOids = NIL;
ScanKeyData key[2];
HeapTuple tup;
Relation depRel = table_open(DependRelationId, AccessShareLock);
ScanKeyInit(&key[0],
Anum_pg_depend_refclassid,
BTEqualStrategyNumber, F_OIDEQ,
ObjectIdGetDatum(RelationRelationId));
ScanKeyInit(&key[1],
Anum_pg_depend_refobjid,
BTEqualStrategyNumber, F_OIDEQ,
ObjectIdGetDatum(relationId));
SysScanDesc scan = systable_beginscan(depRel, DependReferenceIndexId, true,
NULL, 2, key);
while (HeapTupleIsValid(tup = systable_getnext(scan)))
{
Form_pg_depend deprec = (Form_pg_depend) GETSTRUCT(tup);
/*
* objsubid is nonzero only for table columns and zero for anything else.
* Since we are trying to find a dependency from the column of a table to
* function we've added deprec->refobjsubid != 0 check.
*
* We are following DEPENDENCY_AUTO for dependencies via column and
* DEPENDENCY_NORMAL anything else. Since only procedure dependencies
* for those dependencies will be obtained in GetFunctionDependenciesForObjects
* following both dependency types are not harmful.
*/
if ((deprec->refobjsubid != 0 && deprec->deptype == DEPENDENCY_AUTO) ||
deprec->deptype == DEPENDENCY_NORMAL)
{
ObjectAddress *refAddress = palloc(sizeof(ObjectAddress));
ObjectAddressSubSet(*refAddress, deprec->classid,
deprec->objid,
deprec->objsubid);
referencingObjects = lappend(referencingObjects, refAddress);
}
}
systable_endscan(scan);
table_close(depRel, AccessShareLock);
ObjectAddress *referencingObject = NULL;
foreach_ptr(referencingObject, referencingObjects)
{
functionOids = list_concat(functionOids,
GetFunctionDependenciesForObjects(referencingObject));
}
return functionOids;
}
/*
* GetFunctionDependenciesForObjects returns a list of function OIDs that have
* dependency with the given object
*/
static List *
GetFunctionDependenciesForObjects(ObjectAddress *objectAddress)
{
List *functionOids = NIL;
ScanKeyData key[3];
HeapTuple tup;
Relation depRel = table_open(DependRelationId, AccessShareLock);
ScanKeyInit(&key[0],
Anum_pg_depend_classid,
BTEqualStrategyNumber, F_OIDEQ,
ObjectIdGetDatum(objectAddress->classId));
ScanKeyInit(&key[1],
Anum_pg_depend_objid,
BTEqualStrategyNumber, F_OIDEQ,
ObjectIdGetDatum(objectAddress->objectId));
ScanKeyInit(&key[2],
Anum_pg_depend_objsubid,
BTEqualStrategyNumber, F_OIDEQ,
ObjectIdGetDatum(objectAddress->objectSubId));
SysScanDesc scan = systable_beginscan(depRel, DependDependerIndexId, true,
NULL, 3, key);
while (HeapTupleIsValid(tup = systable_getnext(scan)))
{
Form_pg_depend deprec = (Form_pg_depend) GETSTRUCT(tup);
if (deprec->refclassid == ProcedureRelationId)
{
functionOids = lappend_oid(functionOids, deprec->refobjid);
}
}
systable_endscan(scan);
table_close(depRel, AccessShareLock);
return functionOids;
}
/* /*
* SequenceDependencyCommandList generates commands to record the dependency * SequenceDependencyCommandList generates commands to record the dependency
* of sequences on tables on the worker. This dependency does not exist by * of sequences on tables on the worker. This dependency does not exist by
@ -1832,7 +1947,7 @@ GetAccessPrivObjectForGrantStmt(char *permission)
/* /*
* SetLocalEnableMetadataSync sets the enable_metadata_sync locally * SetLocalEnableMetadataSync sets the enable_metadata_sync locally
*/ */
static void void
SetLocalEnableMetadataSync(bool state) SetLocalEnableMetadataSync(bool state)
{ {
set_config_option("citus.enable_metadata_sync", state == true ? "on" : "off", set_config_option("citus.enable_metadata_sync", state == true ? "on" : "off",

View File

@ -1291,6 +1291,52 @@ NodeGroupHasShardPlacements(int32 groupId, bool onlyConsiderActivePlacements)
} }
/*
* IsActiveShardPlacement checks if the shard placement is labelled as
* active, and that it is placed in an active worker.
* Expects shard worker to not be NULL.
*/
bool
IsActiveShardPlacement(ShardPlacement *shardPlacement)
{
WorkerNode *workerNode =
FindWorkerNode(shardPlacement->nodeName, shardPlacement->nodePort);
if (!workerNode)
{
ereport(ERROR, (errmsg("There is a shard placement on node %s:%d but "
"could not find the node.", shardPlacement->nodeName,
shardPlacement->nodePort)));
}
return shardPlacement->shardState == SHARD_STATE_ACTIVE &&
workerNode->isActive;
}
/*
* FilterShardPlacementList filters a list of shard placements based on a filter.
* Keep only the shard for which the filter function returns true.
*/
List *
FilterShardPlacementList(List *shardPlacementList, bool (*filter)(ShardPlacement *))
{
List *filteredShardPlacementList = NIL;
ShardPlacement *shardPlacement = NULL;
foreach_ptr(shardPlacement, shardPlacementList)
{
if (filter(shardPlacement))
{
filteredShardPlacementList = lappend(filteredShardPlacementList,
shardPlacement);
}
}
return filteredShardPlacementList;
}
/* /*
* ActiveShardPlacementListOnGroup returns a list of active shard placements * ActiveShardPlacementListOnGroup returns a list of active shard placements
* that are sitting on group with groupId for given shardId. * that are sitting on group with groupId for given shardId.
@ -1323,53 +1369,39 @@ ActiveShardPlacementListOnGroup(uint64 shardId, int32 groupId)
List * List *
ActiveShardPlacementList(uint64 shardId) ActiveShardPlacementList(uint64 shardId)
{ {
List *activePlacementList = NIL;
List *shardPlacementList = List *shardPlacementList =
ShardPlacementListIncludingOrphanedPlacements(shardId); ShardPlacementListIncludingOrphanedPlacements(shardId);
ShardPlacement *shardPlacement = NULL; List *activePlacementList = FilterShardPlacementList(shardPlacementList,
foreach_ptr(shardPlacement, shardPlacementList) IsActiveShardPlacement);
{
WorkerNode *workerNode =
FindWorkerNode(shardPlacement->nodeName, shardPlacement->nodePort);
/*
* We have already resolved the placement to node, so would have
* errored out earlier.
*/
Assert(workerNode != NULL);
if (shardPlacement->shardState == SHARD_STATE_ACTIVE &&
workerNode->isActive)
{
activePlacementList = lappend(activePlacementList, shardPlacement);
}
}
return SortList(activePlacementList, CompareShardPlacementsByWorker); return SortList(activePlacementList, CompareShardPlacementsByWorker);
} }
/*
* IsShardPlacementNotOrphaned checks returns true if a shard placement is not orphaned
* Orphaned shards are shards marked to be deleted at a later point (shardstate = 4).
*/
static inline bool
IsShardPlacementNotOrphaned(ShardPlacement *shardPlacement)
{
return shardPlacement->shardState != SHARD_STATE_TO_DELETE;
}
/* /*
* ShardPlacementListWithoutOrphanedPlacements returns shard placements exluding * ShardPlacementListWithoutOrphanedPlacements returns shard placements exluding
* the ones that are orphaned, because they are marked to be deleted at a later * the ones that are orphaned.
* point (shardstate = 4).
*/ */
List * List *
ShardPlacementListWithoutOrphanedPlacements(uint64 shardId) ShardPlacementListWithoutOrphanedPlacements(uint64 shardId)
{ {
List *activePlacementList = NIL;
List *shardPlacementList = List *shardPlacementList =
ShardPlacementListIncludingOrphanedPlacements(shardId); ShardPlacementListIncludingOrphanedPlacements(shardId);
ShardPlacement *shardPlacement = NULL; List *activePlacementList = FilterShardPlacementList(shardPlacementList,
foreach_ptr(shardPlacement, shardPlacementList) IsShardPlacementNotOrphaned);
{
if (shardPlacement->shardState != SHARD_STATE_TO_DELETE)
{
activePlacementList = lappend(activePlacementList, shardPlacement);
}
}
return SortList(activePlacementList, CompareShardPlacementsByWorker); return SortList(activePlacementList, CompareShardPlacementsByWorker);
} }

View File

@ -188,9 +188,6 @@ citus_set_coordinator_host(PG_FUNCTION_ARGS)
Name nodeClusterName = PG_GETARG_NAME(3); Name nodeClusterName = PG_GETARG_NAME(3);
nodeMetadata.nodeCluster = NameStr(*nodeClusterName); nodeMetadata.nodeCluster = NameStr(*nodeClusterName);
/* prevent concurrent modification */
LockRelationOid(DistNodeRelationId(), RowShareLock);
bool isCoordinatorInMetadata = false; bool isCoordinatorInMetadata = false;
WorkerNode *coordinatorNode = PrimaryNodeForGroup(COORDINATOR_GROUP_ID, WorkerNode *coordinatorNode = PrimaryNodeForGroup(COORDINATOR_GROUP_ID,
&isCoordinatorInMetadata); &isCoordinatorInMetadata);
@ -1544,7 +1541,7 @@ FindWorkerNodeAnyCluster(const char *nodeName, int32 nodePort)
* If the node cannot be found this functions errors. * If the node cannot be found this functions errors.
*/ */
WorkerNode * WorkerNode *
FindNodeWithNodeId(int nodeId) FindNodeWithNodeId(int nodeId, bool missingOk)
{ {
List *workerList = ActiveReadableNodeList(); List *workerList = ActiveReadableNodeList();
WorkerNode *workerNode = NULL; WorkerNode *workerNode = NULL;
@ -1558,7 +1555,10 @@ FindNodeWithNodeId(int nodeId)
} }
/* there isn't any node with nodeId in pg_dist_node */ /* there isn't any node with nodeId in pg_dist_node */
if (!missingOk)
{
elog(ERROR, "worker node with node id %d could not be found", nodeId); elog(ERROR, "worker node with node id %d could not be found", nodeId);
}
return NULL; return NULL;
} }
@ -1780,12 +1780,6 @@ AddNodeMetadata(char *nodeName, int32 nodePort,
*nodeAlreadyExists = false; *nodeAlreadyExists = false;
/*
* Prevent / wait for concurrent modification before checking whether
* the worker already exists in pg_dist_node.
*/
LockRelationOid(DistNodeRelationId(), RowShareLock);
WorkerNode *workerNode = FindWorkerNodeAnyCluster(nodeName, nodePort); WorkerNode *workerNode = FindWorkerNodeAnyCluster(nodeName, nodePort);
if (workerNode != NULL) if (workerNode != NULL)
{ {

View File

@ -34,6 +34,7 @@
#include "distributed/lock_graph.h" #include "distributed/lock_graph.h"
#include "distributed/coordinator_protocol.h" #include "distributed/coordinator_protocol.h"
#include "distributed/metadata_cache.h" #include "distributed/metadata_cache.h"
#include "distributed/metadata_utility.h"
#include "distributed/multi_client_executor.h" #include "distributed/multi_client_executor.h"
#include "distributed/multi_progress.h" #include "distributed/multi_progress.h"
#include "distributed/multi_server_executor.h" #include "distributed/multi_server_executor.h"
@ -190,7 +191,7 @@ static void UpdateShardPlacement(PlacementUpdateEvent *placementUpdateEvent,
List *responsiveNodeList, Oid shardReplicationModeOid); List *responsiveNodeList, Oid shardReplicationModeOid);
/* static declarations for main logic's utility functions */ /* static declarations for main logic's utility functions */
static HTAB * ActivePlacementsHash(List *shardPlacementList); static HTAB * ShardPlacementsListToHash(List *shardPlacementList);
static bool PlacementsHashFind(HTAB *placementsHash, uint64 shardId, static bool PlacementsHashFind(HTAB *placementsHash, uint64 shardId,
WorkerNode *workerNode); WorkerNode *workerNode);
static void PlacementsHashEnter(HTAB *placementsHash, uint64 shardId, static void PlacementsHashEnter(HTAB *placementsHash, uint64 shardId,
@ -396,6 +397,7 @@ FullShardPlacementList(Oid relationId, ArrayType *excludedShardArray)
placement->shardId = groupPlacement->shardId; placement->shardId = groupPlacement->shardId;
placement->shardLength = groupPlacement->shardLength; placement->shardLength = groupPlacement->shardLength;
placement->shardState = groupPlacement->shardState; placement->shardState = groupPlacement->shardState;
placement->nodeId = worker->nodeId;
placement->nodeName = pstrdup(worker->workerName); placement->nodeName = pstrdup(worker->workerName);
placement->nodePort = worker->workerPort; placement->nodePort = worker->workerPort;
placement->placementId = groupPlacement->placementId; placement->placementId = groupPlacement->placementId;
@ -446,14 +448,17 @@ GetRebalanceSteps(RebalanceOptions *options)
/* sort the lists to make the function more deterministic */ /* sort the lists to make the function more deterministic */
List *activeWorkerList = SortedActiveWorkers(); List *activeWorkerList = SortedActiveWorkers();
List *shardPlacementListList = NIL; List *activeShardPlacementListList = NIL;
Oid relationId = InvalidOid; Oid relationId = InvalidOid;
foreach_oid(relationId, options->relationIdList) foreach_oid(relationId, options->relationIdList)
{ {
List *shardPlacementList = FullShardPlacementList(relationId, List *shardPlacementList = FullShardPlacementList(relationId,
options->excludedShardArray); options->excludedShardArray);
shardPlacementListList = lappend(shardPlacementListList, shardPlacementList); List *activeShardPlacementListForRelation =
FilterShardPlacementList(shardPlacementList, IsActiveShardPlacement);
activeShardPlacementListList =
lappend(activeShardPlacementListList, activeShardPlacementListForRelation);
} }
if (options->threshold < options->rebalanceStrategy->minimumThreshold) if (options->threshold < options->rebalanceStrategy->minimumThreshold)
@ -471,7 +476,7 @@ GetRebalanceSteps(RebalanceOptions *options)
} }
return RebalancePlacementUpdates(activeWorkerList, return RebalancePlacementUpdates(activeWorkerList,
shardPlacementListList, activeShardPlacementListList,
options->threshold, options->threshold,
options->maxShardMoves, options->maxShardMoves,
options->drainOnly, options->drainOnly,
@ -795,7 +800,6 @@ rebalance_table_shards(PG_FUNCTION_ARGS)
{ {
Oid relationId = PG_GETARG_OID(0); Oid relationId = PG_GETARG_OID(0);
ErrorIfMoveUnsupportedTableType(relationId); ErrorIfMoveUnsupportedTableType(relationId);
relationIdList = list_make1_oid(relationId); relationIdList = list_make1_oid(relationId);
} }
else else
@ -951,9 +955,11 @@ replicate_table_shards(PG_FUNCTION_ARGS)
List *activeWorkerList = SortedActiveWorkers(); List *activeWorkerList = SortedActiveWorkers();
List *shardPlacementList = FullShardPlacementList(relationId, excludedShardArray); List *shardPlacementList = FullShardPlacementList(relationId, excludedShardArray);
List *activeShardPlacementList = FilterShardPlacementList(shardPlacementList,
IsActiveShardPlacement);
List *placementUpdateList = ReplicationPlacementUpdates(activeWorkerList, List *placementUpdateList = ReplicationPlacementUpdates(activeWorkerList,
shardPlacementList, activeShardPlacementList,
shardReplicationFactor); shardReplicationFactor);
placementUpdateList = list_truncate(placementUpdateList, maxShardCopies); placementUpdateList = list_truncate(placementUpdateList, maxShardCopies);
@ -1737,13 +1743,13 @@ ExecuteRebalancerCommandInSeparateTransaction(char *command)
* which is placed in the source node but not in the target node as the shard to * which is placed in the source node but not in the target node as the shard to
* move. * move.
* *
* The shardPlacementListList argument contains a list of lists of shard * The activeShardPlacementListList argument contains a list of lists of active shard
* placements. Each of these lists are balanced independently. This is used to * placements. Each of these lists are balanced independently. This is used to
* make sure different colocation groups are balanced separately, so each list * make sure different colocation groups are balanced separately, so each list
* contains the placements of a colocation group. * contains the placements of a colocation group.
*/ */
List * List *
RebalancePlacementUpdates(List *workerNodeList, List *shardPlacementListList, RebalancePlacementUpdates(List *workerNodeList, List *activeShardPlacementListList,
double threshold, double threshold,
int32 maxShardMoves, int32 maxShardMoves,
bool drainOnly, bool drainOnly,
@ -1755,7 +1761,7 @@ RebalancePlacementUpdates(List *workerNodeList, List *shardPlacementListList,
List *shardPlacementList = NIL; List *shardPlacementList = NIL;
List *placementUpdateList = NIL; List *placementUpdateList = NIL;
foreach_ptr(shardPlacementList, shardPlacementListList) foreach_ptr(shardPlacementList, activeShardPlacementListList)
{ {
state = InitRebalanceState(workerNodeList, shardPlacementList, state = InitRebalanceState(workerNodeList, shardPlacementList,
functions); functions);
@ -1861,7 +1867,7 @@ InitRebalanceState(List *workerNodeList, List *shardPlacementList,
RebalanceState *state = palloc0(sizeof(RebalanceState)); RebalanceState *state = palloc0(sizeof(RebalanceState));
state->functions = functions; state->functions = functions;
state->placementsHash = ActivePlacementsHash(shardPlacementList); state->placementsHash = ShardPlacementsListToHash(shardPlacementList);
/* create empty fill state for all of the worker nodes */ /* create empty fill state for all of the worker nodes */
foreach_ptr(workerNode, workerNodeList) foreach_ptr(workerNode, workerNodeList)
@ -2413,29 +2419,25 @@ FindAndMoveShardCost(float4 utilizationLowerBound,
/* /*
* ReplicationPlacementUpdates returns a list of placement updates which * ReplicationPlacementUpdates returns a list of placement updates which
* replicates shard placements that need re-replication. To do this, the * replicates shard placements that need re-replication. To do this, the
* function loops over the shard placements, and for each shard placement * function loops over the active shard placements, and for each shard placement
* which needs to be re-replicated, it chooses an active worker node with * which needs to be re-replicated, it chooses an active worker node with
* smallest number of shards as the target node. * smallest number of shards as the target node.
*/ */
List * List *
ReplicationPlacementUpdates(List *workerNodeList, List *shardPlacementList, ReplicationPlacementUpdates(List *workerNodeList, List *activeShardPlacementList,
int shardReplicationFactor) int shardReplicationFactor)
{ {
List *placementUpdateList = NIL; List *placementUpdateList = NIL;
ListCell *shardPlacementCell = NULL; ListCell *shardPlacementCell = NULL;
uint32 workerNodeIndex = 0; uint32 workerNodeIndex = 0;
HTAB *placementsHash = ActivePlacementsHash(shardPlacementList); HTAB *placementsHash = ShardPlacementsListToHash(activeShardPlacementList);
uint32 workerNodeCount = list_length(workerNodeList); uint32 workerNodeCount = list_length(workerNodeList);
/* get number of shards per node */ /* get number of shards per node */
uint32 *shardCountArray = palloc0(workerNodeCount * sizeof(uint32)); uint32 *shardCountArray = palloc0(workerNodeCount * sizeof(uint32));
foreach(shardPlacementCell, shardPlacementList) foreach(shardPlacementCell, activeShardPlacementList)
{ {
ShardPlacement *placement = lfirst(shardPlacementCell); ShardPlacement *placement = lfirst(shardPlacementCell);
if (placement->shardState != SHARD_STATE_ACTIVE)
{
continue;
}
for (workerNodeIndex = 0; workerNodeIndex < workerNodeCount; workerNodeIndex++) for (workerNodeIndex = 0; workerNodeIndex < workerNodeCount; workerNodeIndex++)
{ {
@ -2449,7 +2451,7 @@ ReplicationPlacementUpdates(List *workerNodeList, List *shardPlacementList,
} }
} }
foreach(shardPlacementCell, shardPlacementList) foreach(shardPlacementCell, activeShardPlacementList)
{ {
WorkerNode *sourceNode = NULL; WorkerNode *sourceNode = NULL;
WorkerNode *targetNode = NULL; WorkerNode *targetNode = NULL;
@ -2586,11 +2588,11 @@ ShardActivePlacementCount(HTAB *activePlacementsHash, uint64 shardId,
/* /*
* ActivePlacementsHash creates and returns a hash set for the placements in * ShardPlacementsListToHash creates and returns a hash set from a shard
* the given list of shard placements which are in active state. * placement list.
*/ */
static HTAB * static HTAB *
ActivePlacementsHash(List *shardPlacementList) ShardPlacementsListToHash(List *shardPlacementList)
{ {
ListCell *shardPlacementCell = NULL; ListCell *shardPlacementCell = NULL;
HASHCTL info; HASHCTL info;
@ -2609,12 +2611,9 @@ ActivePlacementsHash(List *shardPlacementList)
foreach(shardPlacementCell, shardPlacementList) foreach(shardPlacementCell, shardPlacementList)
{ {
ShardPlacement *shardPlacement = (ShardPlacement *) lfirst(shardPlacementCell); ShardPlacement *shardPlacement = (ShardPlacement *) lfirst(shardPlacementCell);
if (shardPlacement->shardState == SHARD_STATE_ACTIVE)
{
void *hashKey = (void *) shardPlacement; void *hashKey = (void *) shardPlacement;
hash_search(shardPlacementsHash, hashKey, HASH_ENTER, NULL); hash_search(shardPlacementsHash, hashKey, HASH_ENTER, NULL);
} }
}
return shardPlacementsHash; return shardPlacementsHash;
} }

View File

@ -40,6 +40,7 @@
#include "distributed/colocation_utils.h" #include "distributed/colocation_utils.h"
#include "distributed/deparse_shard_query.h" #include "distributed/deparse_shard_query.h"
#include "distributed/coordinator_protocol.h" #include "distributed/coordinator_protocol.h"
#include "distributed/intermediate_results.h"
#include "distributed/metadata_cache.h" #include "distributed/metadata_cache.h"
#include "distributed/multi_router_planner.h" #include "distributed/multi_router_planner.h"
#include "distributed/multi_join_order.h" #include "distributed/multi_join_order.h"
@ -52,15 +53,16 @@
#include "distributed/pg_dist_shard.h" #include "distributed/pg_dist_shard.h"
#include "distributed/query_pushdown_planning.h" #include "distributed/query_pushdown_planning.h"
#include "distributed/query_utils.h" #include "distributed/query_utils.h"
#include "distributed/recursive_planning.h"
#include "distributed/shardinterval_utils.h" #include "distributed/shardinterval_utils.h"
#include "distributed/shard_pruning.h" #include "distributed/shard_pruning.h"
#include "distributed/string_utils.h" #include "distributed/string_utils.h"
#include "distributed/worker_manager.h" #include "distributed/worker_manager.h"
#include "distributed/worker_protocol.h" #include "distributed/worker_protocol.h"
#include "distributed/version_compat.h" #include "distributed/version_compat.h"
#include "nodes/makefuncs.h" #include "nodes/makefuncs.h"
#include "nodes/nodeFuncs.h" #include "nodes/nodeFuncs.h"
#include "nodes/print.h"
#include "optimizer/clauses.h" #include "optimizer/clauses.h"
#include "nodes/pathnodes.h" #include "nodes/pathnodes.h"
#include "optimizer/optimizer.h" #include "optimizer/optimizer.h"
@ -157,8 +159,6 @@ static MapMergeJob * BuildMapMergeJob(Query *jobQuery, List *dependentJobList,
Oid baseRelationId, Oid baseRelationId,
BoundaryNodeJobType boundaryNodeJobType); BoundaryNodeJobType boundaryNodeJobType);
static uint32 HashPartitionCount(void); static uint32 HashPartitionCount(void);
static ArrayType * SplitPointObject(ShardInterval **shardIntervalArray,
uint32 shardIntervalCount);
/* Local functions forward declarations for task list creation and helper functions */ /* Local functions forward declarations for task list creation and helper functions */
static Job * BuildJobTreeTaskList(Job *jobTree, static Job * BuildJobTreeTaskList(Job *jobTree,
@ -195,11 +195,11 @@ static bool JoinPrunable(RangeTableFragment *leftFragment,
static ShardInterval * FragmentInterval(RangeTableFragment *fragment); static ShardInterval * FragmentInterval(RangeTableFragment *fragment);
static StringInfo FragmentIntervalString(ShardInterval *fragmentInterval); static StringInfo FragmentIntervalString(ShardInterval *fragmentInterval);
static List * DataFetchTaskList(uint64 jobId, uint32 taskIdIndex, List *fragmentList); static List * DataFetchTaskList(uint64 jobId, uint32 taskIdIndex, List *fragmentList);
static StringInfo DatumArrayString(Datum *datumArray, uint32 datumCount, Oid datumTypeId);
static List * BuildRelationShardList(List *rangeTableList, List *fragmentList); static List * BuildRelationShardList(List *rangeTableList, List *fragmentList);
static void UpdateRangeTableAlias(List *rangeTableList, List *fragmentList); static void UpdateRangeTableAlias(List *rangeTableList, List *fragmentList);
static Alias * FragmentAlias(RangeTblEntry *rangeTableEntry, static Alias * FragmentAlias(RangeTblEntry *rangeTableEntry,
RangeTableFragment *fragment); RangeTableFragment *fragment);
static List * FetchTaskResultNameList(List *mapOutputFetchTaskList);
static uint64 AnchorShardId(List *fragmentList, uint32 anchorRangeTableId); static uint64 AnchorShardId(List *fragmentList, uint32 anchorRangeTableId);
static List * PruneSqlTaskDependencies(List *sqlTaskList); static List * PruneSqlTaskDependencies(List *sqlTaskList);
static List * AssignTaskList(List *sqlTaskList); static List * AssignTaskList(List *sqlTaskList);
@ -218,11 +218,13 @@ static void AssignDataFetchDependencies(List *taskList);
static uint32 TaskListHighestTaskId(List *taskList); static uint32 TaskListHighestTaskId(List *taskList);
static List * MapTaskList(MapMergeJob *mapMergeJob, List *filterTaskList); static List * MapTaskList(MapMergeJob *mapMergeJob, List *filterTaskList);
static StringInfo CreateMapQueryString(MapMergeJob *mapMergeJob, Task *filterTask, static StringInfo CreateMapQueryString(MapMergeJob *mapMergeJob, Task *filterTask,
uint32 partitionColumnIndex); uint32 partitionColumnIndex, bool useBinaryFormat);
static char * PartitionResultNamePrefix(uint64 jobId, int32 taskId);
static char * PartitionResultName(uint64 jobId, uint32 taskId, uint32 partitionId);
static ShardInterval ** RangeIntervalArrayWithNullBucket(ShardInterval **intervalArray,
int intervalCount);
static List * MergeTaskList(MapMergeJob *mapMergeJob, List *mapTaskList, static List * MergeTaskList(MapMergeJob *mapMergeJob, List *mapTaskList,
uint32 taskIdIndex); uint32 taskIdIndex);
static StringInfo ColumnNameArrayString(uint32 columnCount, uint64 generatingJobId);
static StringInfo ColumnTypeArrayString(List *targetEntryList);
static List * FetchEqualityAttrNumsForRTEOpExpr(OpExpr *opExpr); static List * FetchEqualityAttrNumsForRTEOpExpr(OpExpr *opExpr);
static List * FetchEqualityAttrNumsForRTEBoolExpr(BoolExpr *boolExpr); static List * FetchEqualityAttrNumsForRTEBoolExpr(BoolExpr *boolExpr);
@ -853,10 +855,14 @@ TargetEntryList(List *expressionList)
foreach(expressionCell, expressionList) foreach(expressionCell, expressionList)
{ {
Expr *expression = (Expr *) lfirst(expressionCell); Expr *expression = (Expr *) lfirst(expressionCell);
int columnNumber = list_length(targetEntryList) + 1;
StringInfo columnName = makeStringInfo();
appendStringInfo(columnName, "column%d", columnNumber);
TargetEntry *targetEntry = makeTargetEntry(expression, columnNumber,
columnName->data, false);
TargetEntry *targetEntry = makeTargetEntry(expression,
list_length(targetEntryList) + 1,
NULL, false);
targetEntryList = lappend(targetEntryList, targetEntry); targetEntryList = lappend(targetEntryList, targetEntry);
} }
@ -2043,45 +2049,6 @@ HashPartitionCount(void)
} }
/*
* SplitPointObject walks over shard intervals in the given array, extracts each
* shard interval's minimum value, sorts and inserts these minimum values into a
* new array. This sorted array is then used by the MapMerge job.
*/
static ArrayType *
SplitPointObject(ShardInterval **shardIntervalArray, uint32 shardIntervalCount)
{
Oid typeId = InvalidOid;
bool typeByValue = false;
char typeAlignment = 0;
int16 typeLength = 0;
/* allocate an array for shard min values */
uint32 minDatumCount = shardIntervalCount;
Datum *minDatumArray = palloc0(minDatumCount * sizeof(Datum));
for (uint32 intervalIndex = 0; intervalIndex < shardIntervalCount; intervalIndex++)
{
ShardInterval *shardInterval = shardIntervalArray[intervalIndex];
minDatumArray[intervalIndex] = shardInterval->minValue;
Assert(shardInterval->minValueExists);
/* resolve the datum type on the first pass */
if (intervalIndex == 0)
{
typeId = shardInterval->valueTypeId;
}
}
/* construct the split point object from the sorted array */
get_typlenbyvalalign(typeId, &typeLength, &typeByValue, &typeAlignment);
ArrayType *splitPointObject = construct_array(minDatumArray, minDatumCount, typeId,
typeLength, typeByValue, typeAlignment);
return splitPointObject;
}
/* ------------------------------------------------------------ /* ------------------------------------------------------------
* Functions that relate to building and assigning tasks follow * Functions that relate to building and assigning tasks follow
* ------------------------------------------------------------ * ------------------------------------------------------------
@ -4097,34 +4064,6 @@ DataFetchTaskList(uint64 jobId, uint32 taskIdIndex, List *fragmentList)
} }
/* Helper function to return a datum array's external string representation. */
static StringInfo
DatumArrayString(Datum *datumArray, uint32 datumCount, Oid datumTypeId)
{
int16 typeLength = 0;
bool typeByValue = false;
char typeAlignment = 0;
/* construct the array object from the given array */
get_typlenbyvalalign(datumTypeId, &typeLength, &typeByValue, &typeAlignment);
ArrayType *arrayObject = construct_array(datumArray, datumCount, datumTypeId,
typeLength, typeByValue, typeAlignment);
Datum arrayObjectDatum = PointerGetDatum(arrayObject);
/* convert the array object to its string representation */
FmgrInfo *arrayOutFunction = (FmgrInfo *) palloc0(sizeof(FmgrInfo));
fmgr_info(F_ARRAY_OUT, arrayOutFunction);
Datum arrayStringDatum = FunctionCall1(arrayOutFunction, arrayObjectDatum);
char *arrayString = DatumGetCString(arrayStringDatum);
StringInfo arrayStringInfo = makeStringInfo();
appendStringInfo(arrayStringInfo, "%s", arrayString);
return arrayStringInfo;
}
/* /*
* CreateBasicTask creates a task, initializes fields that are common to each task, * CreateBasicTask creates a task, initializes fields that are common to each task,
* and returns the created task. * and returns the created task.
@ -4234,19 +4173,26 @@ FragmentAlias(RangeTblEntry *rangeTableEntry, RangeTableFragment *fragment)
else if (fragmentType == CITUS_RTE_REMOTE_QUERY) else if (fragmentType == CITUS_RTE_REMOTE_QUERY)
{ {
Task *mergeTask = (Task *) fragment->fragmentReference; Task *mergeTask = (Task *) fragment->fragmentReference;
uint64 jobId = mergeTask->jobId; List *mapOutputFetchTaskList = mergeTask->dependentTaskList;
uint32 taskId = mergeTask->taskId; List *resultNameList = FetchTaskResultNameList(mapOutputFetchTaskList);
List *mapJobTargetList = mergeTask->mapJobTargetList;
StringInfo jobSchemaName = JobSchemaName(jobId); /* determine whether all types have binary input/output functions */
StringInfo taskTableName = TaskTableName(taskId); bool useBinaryFormat = CanUseBinaryCopyFormatForTargetList(mapJobTargetList);
StringInfo aliasNameString = makeStringInfo(); /* generate the query on the intermediate result */
appendStringInfo(aliasNameString, "%s.%s", Query *fragmentSetQuery = BuildReadIntermediateResultsArrayQuery(mapJobTargetList,
jobSchemaName->data, taskTableName->data); NIL,
resultNameList,
useBinaryFormat);
aliasName = aliasNameString->data; /* we only really care about the function RTE */
fragmentName = taskTableName->data; RangeTblEntry *readIntermediateResultsRTE = linitial(fragmentSetQuery->rtable);
schemaName = jobSchemaName->data;
/* crudely override the fragment RTE */
*rangeTableEntry = *readIntermediateResultsRTE;
return rangeTableEntry->alias;
} }
/* /*
@ -4267,6 +4213,30 @@ FragmentAlias(RangeTblEntry *rangeTableEntry, RangeTableFragment *fragment)
} }
/*
* FetchTaskResultNameList builds a list of result names that reflect
* the output of map-fetch tasks.
*/
static List *
FetchTaskResultNameList(List *mapOutputFetchTaskList)
{
List *resultNameList = NIL;
Task *mapOutputFetchTask = NULL;
foreach_ptr(mapOutputFetchTask, mapOutputFetchTaskList)
{
Task *mapTask = linitial(mapOutputFetchTask->dependentTaskList);
int partitionId = mapOutputFetchTask->partitionId;
char *resultName =
PartitionResultName(mapTask->jobId, mapTask->taskId, partitionId);
resultNameList = lappend(resultNameList, resultName);
}
return resultNameList;
}
/* /*
* AnchorShardId walks over each fragment in the given fragment list, finds the * AnchorShardId walks over each fragment in the given fragment list, finds the
* fragment that corresponds to the given anchor range tableId, and returns this * fragment that corresponds to the given anchor range tableId, and returns this
@ -4377,17 +4347,28 @@ MapTaskList(MapMergeJob *mapMergeJob, List *filterTaskList)
filterQuery->targetList); filterQuery->targetList);
} }
/* determine whether all types have binary input/output functions */
bool useBinaryFormat = CanUseBinaryCopyFormatForTargetList(filterQuery->targetList);
foreach(filterTaskCell, filterTaskList) foreach(filterTaskCell, filterTaskList)
{ {
Task *filterTask = (Task *) lfirst(filterTaskCell); Task *filterTask = (Task *) lfirst(filterTaskCell);
StringInfo mapQueryString = CreateMapQueryString(mapMergeJob, filterTask, StringInfo mapQueryString = CreateMapQueryString(mapMergeJob, filterTask,
partitionColumnResNo); partitionColumnResNo,
useBinaryFormat);
/* convert filter query task into map task */ /* convert filter query task into map task */
Task *mapTask = filterTask; Task *mapTask = filterTask;
SetTaskQueryString(mapTask, mapQueryString->data); SetTaskQueryString(mapTask, mapQueryString->data);
mapTask->taskType = MAP_TASK; mapTask->taskType = MAP_TASK;
/*
* We do not support fail-over in case of map tasks, since we would also
* have to fail over the corresponding merge tasks. We therefore truncate
* the list down to the first element.
*/
mapTask->taskPlacementList = list_truncate(mapTask->taskPlacementList, 1);
mapTaskList = lappend(mapTaskList, mapTask); mapTaskList = lappend(mapTaskList, mapTask);
} }
@ -4428,21 +4409,19 @@ PartitionColumnIndex(Var *targetVar, List *targetList)
*/ */
static StringInfo static StringInfo
CreateMapQueryString(MapMergeJob *mapMergeJob, Task *filterTask, CreateMapQueryString(MapMergeJob *mapMergeJob, Task *filterTask,
uint32 partitionColumnIndex) uint32 partitionColumnIndex, bool useBinaryFormat)
{ {
uint64 jobId = filterTask->jobId; uint64 jobId = filterTask->jobId;
uint32 taskId = filterTask->taskId; uint32 taskId = filterTask->taskId;
char *resultNamePrefix = PartitionResultNamePrefix(jobId, taskId);
/* wrap repartition query string around filter query string */ /* wrap repartition query string around filter query string */
StringInfo mapQueryString = makeStringInfo(); StringInfo mapQueryString = makeStringInfo();
char *filterQueryString = TaskQueryString(filterTask); char *filterQueryString = TaskQueryString(filterTask);
char *filterQueryEscapedText = quote_literal_cstr(filterQueryString);
PartitionType partitionType = mapMergeJob->partitionType; PartitionType partitionType = mapMergeJob->partitionType;
Var *partitionColumn = mapMergeJob->partitionColumn; Var *partitionColumn = mapMergeJob->partitionColumn;
Oid partitionColumnType = partitionColumn->vartype; Oid partitionColumnType = partitionColumn->vartype;
char *partitionColumnTypeFullName = format_type_be_qualified(partitionColumnType);
int32 partitionColumnTypeMod = partitionColumn->vartypmod;
ShardInterval **intervalArray = mapMergeJob->sortedShardIntervalArray; ShardInterval **intervalArray = mapMergeJob->sortedShardIntervalArray;
uint32 intervalCount = mapMergeJob->partitionCount; uint32 intervalCount = mapMergeJob->partitionCount;
@ -4450,38 +4429,101 @@ CreateMapQueryString(MapMergeJob *mapMergeJob, Task *filterTask,
if (partitionType == DUAL_HASH_PARTITION_TYPE) if (partitionType == DUAL_HASH_PARTITION_TYPE)
{ {
partitionColumnType = INT4OID; partitionColumnType = INT4OID;
partitionColumnTypeMod = get_typmodin(INT4OID);
intervalArray = GenerateSyntheticShardIntervalArray(intervalCount); intervalArray = GenerateSyntheticShardIntervalArray(intervalCount);
} }
else if (partitionType == SINGLE_HASH_PARTITION_TYPE) else if (partitionType == SINGLE_HASH_PARTITION_TYPE)
{ {
partitionColumnType = INT4OID; partitionColumnType = INT4OID;
partitionColumnTypeMod = get_typmodin(INT4OID);
} }
else if (partitionType == RANGE_PARTITION_TYPE)
ArrayType *splitPointObject = SplitPointObject(intervalArray, intervalCount);
StringInfo splitPointString = ArrayObjectToString(splitPointObject,
partitionColumnType,
partitionColumnTypeMod);
char *partitionCommand = NULL;
if (partitionType == RANGE_PARTITION_TYPE)
{ {
partitionCommand = RANGE_PARTITION_COMMAND; /* add a partition for NULL values at index 0 */
} intervalArray = RangeIntervalArrayWithNullBucket(intervalArray, intervalCount);
else intervalCount++;
{
partitionCommand = HASH_PARTITION_COMMAND;
} }
char *partitionColumnIndextText = ConvertIntToString(partitionColumnIndex); Oid intervalTypeOutFunc = InvalidOid;
appendStringInfo(mapQueryString, partitionCommand, jobId, taskId, bool intervalTypeVarlena = false;
filterQueryEscapedText, partitionColumnIndextText, ArrayType *minValueArray = NULL;
partitionColumnTypeFullName, splitPointString->data); ArrayType *maxValueArray = NULL;
getTypeOutputInfo(partitionColumnType, &intervalTypeOutFunc, &intervalTypeVarlena);
ShardMinMaxValueArrays(intervalArray, intervalCount, intervalTypeOutFunc,
&minValueArray, &maxValueArray);
StringInfo minValuesString = ArrayObjectToString(minValueArray, TEXTOID,
InvalidOid);
StringInfo maxValuesString = ArrayObjectToString(maxValueArray, TEXTOID,
InvalidOid);
char *partitionMethodString = partitionType == RANGE_PARTITION_TYPE ?
"range" : "hash";
/*
* Non-partition columns can easily contain NULL values, so we allow NULL
* values in the column by which we re-partition. They will end up in the
* first partition.
*/
bool allowNullPartitionColumnValue = true;
/*
* We currently generate empty results for each partition and fetch all of them.
*/
bool generateEmptyResults = true;
appendStringInfo(mapQueryString,
"SELECT partition_index"
", %s || '_' || partition_index::text "
", rows_written "
"FROM pg_catalog.worker_partition_query_result"
"(%s,%s,%d,%s,%s,%s,%s,%s,%s) WHERE rows_written > 0",
quote_literal_cstr(resultNamePrefix),
quote_literal_cstr(resultNamePrefix),
quote_literal_cstr(filterQueryString),
partitionColumnIndex - 1,
quote_literal_cstr(partitionMethodString),
minValuesString->data,
maxValuesString->data,
useBinaryFormat ? "true" : "false",
allowNullPartitionColumnValue ? "true" : "false",
generateEmptyResults ? "true" : "false");
return mapQueryString; return mapQueryString;
} }
/*
* PartitionResultNamePrefix returns the prefix we use for worker_partition_query_result
* results. Each result will have a _<partition index> suffix.
*/
static char *
PartitionResultNamePrefix(uint64 jobId, int32 taskId)
{
StringInfo resultNamePrefix = makeStringInfo();
appendStringInfo(resultNamePrefix, "repartition_" UINT64_FORMAT "_%u", jobId, taskId);
return resultNamePrefix->data;
}
/*
* PartitionResultName returns the name of a worker_partition_query_result result for
* a specific partition.
*/
static char *
PartitionResultName(uint64 jobId, uint32 taskId, uint32 partitionId)
{
StringInfo resultName = makeStringInfo();
char *resultNamePrefix = PartitionResultNamePrefix(jobId, taskId);
appendStringInfo(resultName, "%s_%d", resultNamePrefix, partitionId);
return resultName->data;
}
/* /*
* GenerateSyntheticShardIntervalArray returns a shard interval pointer array * GenerateSyntheticShardIntervalArray returns a shard interval pointer array
* which has a uniform hash distribution for the given input partitionCount. * which has a uniform hash distribution for the given input partitionCount.
@ -4504,6 +4546,12 @@ GenerateSyntheticShardIntervalArray(int partitionCount)
int32 shardMinHashToken = PG_INT32_MIN + (shardIndex * hashTokenIncrement); int32 shardMinHashToken = PG_INT32_MIN + (shardIndex * hashTokenIncrement);
int32 shardMaxHashToken = shardMinHashToken + (hashTokenIncrement - 1); int32 shardMaxHashToken = shardMinHashToken + (hashTokenIncrement - 1);
/* extend the last range to cover the full range of integers */
if (shardIndex == (partitionCount - 1))
{
shardMaxHashToken = PG_INT32_MAX;
}
shardInterval->relationId = InvalidOid; shardInterval->relationId = InvalidOid;
shardInterval->minValueExists = true; shardInterval->minValueExists = true;
shardInterval->minValue = Int32GetDatum(shardMinHashToken); shardInterval->minValue = Int32GetDatum(shardMinHashToken);
@ -4521,6 +4569,34 @@ GenerateSyntheticShardIntervalArray(int partitionCount)
} }
/*
* RangeIntervalArrayWithNullBucket prepends an additional bucket for NULL values
* to intervalArray and returns the result.
*
* When we support NULL values in (range-partitioned) shards, we will need to revise
* this logic, since there may already be an interval for NULL values.
*/
static ShardInterval **
RangeIntervalArrayWithNullBucket(ShardInterval **intervalArray, int intervalCount)
{
int fullIntervalCount = intervalCount + 1;
ShardInterval **fullIntervalArray =
palloc0(fullIntervalCount * sizeof(ShardInterval *));
fullIntervalArray[0] = CitusMakeNode(ShardInterval);
fullIntervalArray[0]->minValueExists = true;
fullIntervalArray[0]->maxValueExists = true;
fullIntervalArray[0]->valueTypeId = intervalArray[0]->valueTypeId;
for (int intervalIndex = 1; intervalIndex < fullIntervalCount; intervalIndex++)
{
fullIntervalArray[intervalIndex] = intervalArray[intervalIndex - 1];
}
return fullIntervalArray;
}
/* /*
* Determine RowModifyLevel required for given query * Determine RowModifyLevel required for given query
*/ */
@ -4598,7 +4674,7 @@ ArrayObjectToString(ArrayType *arrayObject, Oid columnType, int32 columnTypeMod)
char *arrayOutputEscapedText = quote_literal_cstr(arrayOutputText); char *arrayOutputEscapedText = quote_literal_cstr(arrayOutputText);
/* add an explicit cast to array's string representation */ /* add an explicit cast to array's string representation */
char *arrayOutTypeName = format_type_with_typemod(arrayOutType, columnTypeMod); char *arrayOutTypeName = format_type_be(arrayOutType);
StringInfo arrayString = makeStringInfo(); StringInfo arrayString = makeStringInfo();
appendStringInfo(arrayString, "%s::%s", appendStringInfo(arrayString, "%s::%s",
@ -4660,17 +4736,9 @@ MergeTaskList(MapMergeJob *mapMergeJob, List *mapTaskList, uint32 taskIdIndex)
Query *reduceQuery = mapMergeJob->reduceQuery; Query *reduceQuery = mapMergeJob->reduceQuery;
if (reduceQuery == NULL) if (reduceQuery == NULL)
{ {
uint32 columnCount = (uint32) list_length(targetEntryList); /* create logical merge task (not executed, but useful for bookkeeping) */
StringInfo columnNames = ColumnNameArrayString(columnCount, jobId);
StringInfo columnTypes = ColumnTypeArrayString(targetEntryList);
StringInfo mergeQueryString = makeStringInfo();
appendStringInfo(mergeQueryString, MERGE_FILES_INTO_TABLE_COMMAND,
jobId, taskIdIndex, columnNames->data, columnTypes->data);
/* create merge task */
mergeTask = CreateBasicTask(jobId, mergeTaskId, MERGE_TASK, mergeTask = CreateBasicTask(jobId, mergeTaskId, MERGE_TASK,
mergeQueryString->data); "<merge>");
} }
mergeTask->partitionId = partitionId; mergeTask->partitionId = partitionId;
taskIdIndex++; taskIdIndex++;
@ -4682,26 +4750,35 @@ MergeTaskList(MapMergeJob *mapMergeJob, List *mapTaskList, uint32 taskIdIndex)
/* find the node name/port for map task's execution */ /* find the node name/port for map task's execution */
List *mapTaskPlacementList = mapTask->taskPlacementList; List *mapTaskPlacementList = mapTask->taskPlacementList;
ShardPlacement *mapTaskPlacement = linitial(mapTaskPlacementList); ShardPlacement *mapTaskPlacement = linitial(mapTaskPlacementList);
char *mapTaskNodeName = mapTaskPlacement->nodeName;
uint32 mapTaskNodePort = mapTaskPlacement->nodePort; char *partitionResultName =
PartitionResultName(jobId, mapTask->taskId, partitionId);
/* we currently only fetch a single fragment at a time */
DistributedResultFragment singleFragmentTransfer;
singleFragmentTransfer.resultId = partitionResultName;
singleFragmentTransfer.nodeId = mapTaskPlacement->nodeId;
singleFragmentTransfer.rowCount = 0;
singleFragmentTransfer.targetShardId = INVALID_SHARD_ID;
singleFragmentTransfer.targetShardIndex = partitionId;
NodeToNodeFragmentsTransfer fragmentsTransfer;
fragmentsTransfer.nodes.sourceNodeId = mapTaskPlacement->nodeId;
/* /*
* We will use the first node even if replication factor is greater than 1 * Target node is not yet decided, and not necessary for
* When replication factor is greater than 1 and there * QueryStringForFragmentsTransfer.
* is a connection problem to the node that has done the map task, we will get
* an error in fetch task execution.
*/ */
StringInfo mapFetchQueryString = makeStringInfo(); fragmentsTransfer.nodes.targetNodeId = -1;
appendStringInfo(mapFetchQueryString, MAP_OUTPUT_FETCH_COMMAND,
mapTask->jobId, mapTask->taskId, partitionId, fragmentsTransfer.fragmentList = list_make1(&singleFragmentTransfer);
mergeTaskId, /* fetch results to merge task */
mapTaskNodeName, mapTaskNodePort); char *fetchQueryString = QueryStringForFragmentsTransfer(&fragmentsTransfer);
Task *mapOutputFetchTask = CreateBasicTask(jobId, taskIdIndex, Task *mapOutputFetchTask = CreateBasicTask(jobId, taskIdIndex,
MAP_OUTPUT_FETCH_TASK, MAP_OUTPUT_FETCH_TASK,
mapFetchQueryString->data); fetchQueryString);
mapOutputFetchTask->partitionId = partitionId; mapOutputFetchTask->partitionId = partitionId;
mapOutputFetchTask->upstreamTaskId = mergeTaskId; mapOutputFetchTask->upstreamTaskId = mergeTaskId;
mapOutputFetchTask->dependentTaskList = list_make1(mapTask); mapOutputFetchTask->dependentTaskList = list_make1(mapTask);
@ -4712,6 +4789,7 @@ MergeTaskList(MapMergeJob *mapMergeJob, List *mapTaskList, uint32 taskIdIndex)
/* merge task depends on completion of fetch tasks */ /* merge task depends on completion of fetch tasks */
mergeTask->dependentTaskList = mapOutputFetchTaskList; mergeTask->dependentTaskList = mapOutputFetchTaskList;
mergeTask->mapJobTargetList = targetEntryList;
/* if single repartitioned, each merge task represents an interval */ /* if single repartitioned, each merge task represents an interval */
if (mapMergeJob->partitionType == RANGE_PARTITION_TYPE) if (mapMergeJob->partitionType == RANGE_PARTITION_TYPE)
@ -4738,71 +4816,6 @@ MergeTaskList(MapMergeJob *mapMergeJob, List *mapTaskList, uint32 taskIdIndex)
} }
/*
* ColumnNameArrayString creates a list of column names for a merged table, and
* outputs this list of column names in their (array) string representation.
*/
static StringInfo
ColumnNameArrayString(uint32 columnCount, uint64 generatingJobId)
{
Datum *columnNameArray = palloc0(columnCount * sizeof(Datum));
uint32 columnNameIndex = 0;
/* build list of intermediate column names, generated by given jobId */
List *columnNameList = DerivedColumnNameList(columnCount, generatingJobId);
ListCell *columnNameCell = NULL;
foreach(columnNameCell, columnNameList)
{
Value *columnNameValue = (Value *) lfirst(columnNameCell);
char *columnNameString = strVal(columnNameValue);
Datum columnName = CStringGetDatum(columnNameString);
columnNameArray[columnNameIndex] = columnName;
columnNameIndex++;
}
StringInfo columnNameArrayString = DatumArrayString(columnNameArray, columnCount,
CSTRINGOID);
return columnNameArrayString;
}
/*
* ColumnTypeArrayString resolves a list of column types for a merged table, and
* outputs this list of column types in their (array) string representation.
*/
static StringInfo
ColumnTypeArrayString(List *targetEntryList)
{
ListCell *targetEntryCell = NULL;
uint32 columnCount = (uint32) list_length(targetEntryList);
Datum *columnTypeArray = palloc0(columnCount * sizeof(Datum));
uint32 columnTypeIndex = 0;
foreach(targetEntryCell, targetEntryList)
{
TargetEntry *targetEntry = (TargetEntry *) lfirst(targetEntryCell);
Node *columnExpression = (Node *) targetEntry->expr;
Oid columnTypeId = exprType(columnExpression);
int32 columnTypeMod = exprTypmod(columnExpression);
char *columnTypeName = format_type_with_typemod(columnTypeId, columnTypeMod);
Datum columnType = CStringGetDatum(columnTypeName);
columnTypeArray[columnTypeIndex] = columnType;
columnTypeIndex++;
}
StringInfo columnTypeArrayString = DatumArrayString(columnTypeArray, columnCount,
CSTRINGOID);
return columnTypeArrayString;
}
/* /*
* AssignTaskList assigns locations to given tasks based on dependencies between * AssignTaskList assigns locations to given tasks based on dependencies between
* tasks and configured task assignment policies. The function also handles the * tasks and configured task assignment policies. The function also handles the
@ -5392,6 +5405,7 @@ ActiveShardPlacementLists(List *taskList)
/* sort shard placements by their creation time */ /* sort shard placements by their creation time */
activeShardPlacementList = SortList(activeShardPlacementList, activeShardPlacementList = SortList(activeShardPlacementList,
CompareShardPlacements); CompareShardPlacements);
shardPlacementLists = lappend(shardPlacementLists, activeShardPlacementList); shardPlacementLists = lappend(shardPlacementLists, activeShardPlacementList);
} }

View File

@ -218,6 +218,13 @@ static const struct config_enum_entry explain_analyze_sort_method_options[] = {
{ NULL, 0, false } { NULL, 0, false }
}; };
static const struct config_enum_entry create_object_propagation_options[] = {
{"deferred", CREATE_OBJECT_PROPAGATION_DEFERRED, false},
{"automatic", CREATE_OBJECT_PROPAGATION_AUTOMATIC, false},
{"immediate", CREATE_OBJECT_PROPAGATION_IMMEDIATE, false},
{NULL, 0, false}
};
/* *INDENT-ON* */ /* *INDENT-ON* */
@ -609,7 +616,7 @@ RegisterCitusConfigVariables(void)
false, false,
#endif #endif
PGC_SIGHUP, PGC_SIGHUP,
GUC_STANDARD, GUC_NO_SHOW_ALL,
NULL, NULL, NULL); NULL, NULL, NULL);
DefineCustomBoolVariable( DefineCustomBoolVariable(
@ -675,6 +682,24 @@ RegisterCitusConfigVariables(void)
GUC_STANDARD, GUC_STANDARD,
NULL, NULL, NULL); NULL, NULL, NULL);
DefineCustomEnumVariable(
"citus.create_object_propagation",
gettext_noop("Controls the behavior of CREATE statements in transactions for "
"supported objects"),
gettext_noop("When creating new objects in transactions this setting is used to "
"determine the behavior for propagating. When objects are created "
"in a multi-statement transaction block Citus needs to switch to "
"sequential mode (if not already) to make sure the objects are "
"visible to later statements on shards. The switch to sequential is "
"not always desired. By changing this behavior the user can trade "
"off performance for full transactional consistency on the creation "
"of new objects."),
&CreateObjectPropagationMode,
CREATE_OBJECT_PROPAGATION_DEFERRED, create_object_propagation_options,
PGC_USERSET,
GUC_NO_SHOW_ALL,
NULL, NULL, NULL);
DefineCustomBoolVariable( DefineCustomBoolVariable(
"citus.defer_drop_after_shard_move", "citus.defer_drop_after_shard_move",
gettext_noop("When enabled a shard move will mark the original shards " gettext_noop("When enabled a shard move will mark the original shards "

View File

@ -42,8 +42,9 @@ GRANT SELECT ON pg_catalog.citus_dist_stat_activity TO PUBLIC;
#include "udfs/pg_cancel_backend/11.0-1.sql" #include "udfs/pg_cancel_backend/11.0-1.sql"
#include "udfs/pg_terminate_backend/11.0-1.sql" #include "udfs/pg_terminate_backend/11.0-1.sql"
#include "udfs/worker_partition_query_result/11.0-1.sql"
DROP FUNCTION IF EXISTS pg_catalog.master_apply_delete_command(text); DROP FUNCTION pg_catalog.master_apply_delete_command(text);
DROP FUNCTION pg_catalog.master_get_table_metadata(text); DROP FUNCTION pg_catalog.master_get_table_metadata(text);
DROP FUNCTION pg_catalog.master_append_table_to_shard(bigint, text, text, integer); DROP FUNCTION pg_catalog.master_append_table_to_shard(bigint, text, text, integer);
@ -79,3 +80,6 @@ BEGIN
UPDATE pg_dist_node_metadata SET metadata=jsonb_set(metadata, '{partitioned_citus_table_exists_pre_11}', to_jsonb(partitioned_table_exists), true); UPDATE pg_dist_node_metadata SET metadata=jsonb_set(metadata, '{partitioned_citus_table_exists_pre_11}', to_jsonb(partitioned_table_exists), true);
END; END;
$$; $$;
#include "udfs/citus_finalize_upgrade_to_citus11/11.0-1.sql"

View File

@ -1,6 +1,10 @@
-- citus--11.0-1--10.2-4 -- citus--11.0-1--10.2-4
DROP FUNCTION pg_catalog.create_distributed_function(regprocedure, text, text, bool); DROP FUNCTION pg_catalog.create_distributed_function(regprocedure, text, text, bool);
DROP FUNCTION pg_catalog.worker_partition_query_result(text, text, int, citus.distribution_type, text[], text[], boolean, boolean, boolean);
#include "../udfs/worker_partition_query_result/9.2-1.sql"
CREATE FUNCTION pg_catalog.master_apply_delete_command(text) CREATE FUNCTION pg_catalog.master_apply_delete_command(text)
RETURNS integer RETURNS integer
LANGUAGE C STRICT LANGUAGE C STRICT
@ -21,6 +25,7 @@ CREATE FUNCTION pg_catalog.master_get_table_metadata(
AS 'MODULE_PATHNAME', $$master_get_table_metadata$$; AS 'MODULE_PATHNAME', $$master_get_table_metadata$$;
COMMENT ON FUNCTION master_get_table_metadata(relation_name text) COMMENT ON FUNCTION master_get_table_metadata(relation_name text)
IS 'fetch metadata values for the table'; IS 'fetch metadata values for the table';
ALTER TABLE pg_catalog.pg_dist_partition DROP COLUMN autoconverted; ALTER TABLE pg_catalog.pg_dist_partition DROP COLUMN autoconverted;
CREATE FUNCTION master_append_table_to_shard(bigint, text, text, integer) CREATE FUNCTION master_append_table_to_shard(bigint, text, text, integer)
@ -85,7 +90,6 @@ DROP FUNCTION pg_catalog.citus_shard_indexes_on_worker();
#include "../udfs/create_distributed_function/9.0-1.sql" #include "../udfs/create_distributed_function/9.0-1.sql"
ALTER TABLE citus.pg_dist_object DROP COLUMN force_delegation; ALTER TABLE citus.pg_dist_object DROP COLUMN force_delegation;
SET search_path = 'pg_catalog'; SET search_path = 'pg_catalog';
@ -342,4 +346,6 @@ JOIN
ALTER VIEW citus.citus_lock_waits SET SCHEMA pg_catalog; ALTER VIEW citus.citus_lock_waits SET SCHEMA pg_catalog;
GRANT SELECT ON pg_catalog.citus_lock_waits TO PUBLIC; GRANT SELECT ON pg_catalog.citus_lock_waits TO PUBLIC;
DROP FUNCTION pg_catalog.citus_finalize_upgrade_to_citus11(bool);
RESET search_path; RESET search_path;

View File

@ -0,0 +1,224 @@
-- citus_finalize_upgrade_to_citus11() is a helper UDF ensures
-- the upgrade to Citus 11 is finished successfully. Upgrade to
-- Citus 11 requires all active primary worker nodes to get the
-- metadata. And, this function's job is to sync the metadata to
-- the nodes that does not already have
-- once the function finishes without any errors and returns true
-- the cluster is ready for running distributed queries from
-- the worker nodes. When debug is enabled, the function provides
-- more information to the user.
CREATE OR REPLACE FUNCTION pg_catalog.citus_finalize_upgrade_to_citus11(enforce_version_check bool default true)
RETURNS bool
LANGUAGE plpgsql
AS $$
BEGIN
---------------------------------------------
-- This script consists of N stages
-- Each step is documented, and if log level
-- is reduced to DEBUG1, each step is logged
-- as well
---------------------------------------------
------------------------------------------------------------------------------------------
-- STAGE 0: Ensure no concurrent node metadata changing operation happens while this
-- script is running via acquiring a strong lock on the pg_dist_node
------------------------------------------------------------------------------------------
BEGIN
LOCK TABLE pg_dist_node IN EXCLUSIVE MODE NOWAIT;
EXCEPTION WHEN OTHERS THEN
RAISE 'Another node metadata changing operation is in progress, try again.';
END;
------------------------------------------------------------------------------------------
-- STAGE 1: We want all the commands to run in the same transaction block. Without
-- sequential mode, metadata syncing cannot be done in a transaction block along with
-- other commands
------------------------------------------------------------------------------------------
SET LOCAL citus.multi_shard_modify_mode TO 'sequential';
------------------------------------------------------------------------------------------
-- STAGE 2: Ensure we have the prerequisites
-- (a) only superuser can run this script
-- (b) cannot be executed when enable_ddl_propagation is False
-- (c) can only be executed from the coordinator
------------------------------------------------------------------------------------------
DECLARE
is_superuser_running boolean := False;
enable_ddl_prop boolean:= False;
local_group_id int := 0;
BEGIN
SELECT rolsuper INTO is_superuser_running FROM pg_roles WHERE rolname = current_user;
IF is_superuser_running IS NOT True THEN
RAISE EXCEPTION 'This operation can only be initiated by superuser';
END IF;
SELECT current_setting('citus.enable_ddl_propagation') INTO enable_ddl_prop;
IF enable_ddl_prop IS NOT True THEN
RAISE EXCEPTION 'This operation cannot be completed when citus.enable_ddl_propagation is False.';
END IF;
SELECT groupid INTO local_group_id FROM pg_dist_local_group;
IF local_group_id != 0 THEN
RAISE EXCEPTION 'Operation is not allowed on this node. Connect to the coordinator and run it again.';
ELSE
RAISE DEBUG 'We are on the coordinator, continue to sync metadata';
END IF;
END;
------------------------------------------------------------------------------------------
-- STAGE 3: Ensure all primary nodes are active
------------------------------------------------------------------------------------------
DECLARE
primary_disabled_worker_node_count int := 0;
BEGIN
SELECT count(*) INTO primary_disabled_worker_node_count FROM pg_dist_node
WHERE groupid != 0 AND noderole = 'primary' AND NOT isactive;
IF primary_disabled_worker_node_count != 0 THEN
RAISE EXCEPTION 'There are inactive primary worker nodes, you need to activate the nodes first.'
'Use SELECT citus_activate_node() to activate the disabled nodes';
ELSE
RAISE DEBUG 'There are no disabled worker nodes, continue to sync metadata';
END IF;
END;
------------------------------------------------------------------------------------------
-- STAGE 4: Ensure there is no connectivity issues in the cluster
------------------------------------------------------------------------------------------
DECLARE
all_nodes_can_connect_to_each_other boolean := False;
BEGIN
SELECT bool_and(coalesce(result, false)) INTO all_nodes_can_connect_to_each_other FROM citus_check_cluster_node_health();
IF all_nodes_can_connect_to_each_other != True THEN
RAISE EXCEPTION 'There are unhealth primary nodes, you need to ensure all '
'nodes are up and runnnig. Also, make sure that all nodes can connect '
'to each other. Use SELECT * FROM citus_check_cluster_node_health(); '
'to check the cluster health';
ELSE
RAISE DEBUG 'Cluster is healthy, all nodes can connect to each other';
END IF;
END;
------------------------------------------------------------------------------------------
-- STAGE 5: Ensure all nodes are on the same version
------------------------------------------------------------------------------------------
DECLARE
coordinator_version text := '';
worker_node_version text := '';
worker_node_version_count int := 0;
BEGIN
SELECT extversion INTO coordinator_version from pg_extension WHERE extname = 'citus';
-- first, check if all nodes have the same versions
SELECT
count(*) INTO worker_node_version_count
FROM
run_command_on_workers('SELECT extversion from pg_extension WHERE extname = ''citus'';')
GROUP BY result;
IF enforce_version_check AND worker_node_version_count != 1 THEN
RAISE EXCEPTION 'All nodes should have the same Citus version installed. Currently '
'the some of the workers has version different versions';
ELSE
RAISE DEBUG 'All worker nodes have the same Citus version';
END IF;
-- second, check if all nodes have the same versions
SELECT
result INTO worker_node_version
FROM
run_command_on_workers('SELECT extversion from pg_extension WHERE extname = ''citus'';')
GROUP BY result;
IF enforce_version_check AND coordinator_version != worker_node_version THEN
RAISE EXCEPTION 'All nodes should have the same Citus version installed. Currently '
'the coordinator has version % and the worker(s) has %',
coordinator_version, worker_node_version;
ELSE
RAISE DEBUG 'All nodes have the same Citus version';
END IF;
END;
------------------------------------------------------------------------------------------
-- STAGE 6: Ensure all the partitioned tables have the proper naming structure
-- As described on https://github.com/citusdata/citus/issues/4962
-- existing indexes on partitioned distributed tables can collide
-- with the index names exists on the shards
-- luckily, we know how to fix it.
-- And, note that we should do this even if the cluster is a basic plan
-- (e.g., single node Citus) such that when cluster scaled out, everything
-- works as intended
-- And, this should be done only ONCE for a cluster as it can be a pretty
-- time consuming operation. Thus, even if the function is called multiple time,
-- we keep track of it and do not re-execute this part if not needed.
------------------------------------------------------------------------------------------
DECLARE
partitioned_table_exists_pre_11 boolean:=False;
BEGIN
-- we recorded if partitioned tables exists during upgrade to Citus 11
SELECT metadata->>'partitioned_citus_table_exists_pre_11' INTO partitioned_table_exists_pre_11
FROM pg_dist_node_metadata;
IF partitioned_table_exists_pre_11 IS NOT NULL AND partitioned_table_exists_pre_11 THEN
-- this might take long depending on the number of partitions and shards...
RAISE NOTICE 'Preparing all the existing partitioned table indexes';
PERFORM pg_catalog.fix_all_partition_shard_index_names();
-- great, we are done with fixing the existing wrong index names
-- so, lets remove this
UPDATE pg_dist_node_metadata
SET metadata=jsonb_delete(metadata, 'partitioned_citus_table_exists_pre_11');
ELSE
RAISE DEBUG 'There are no partitioned tables that should be fixed';
END IF;
END;
------------------------------------------------------------------------------------------
-- STAGE 7: Return early if there are no primary worker nodes
-- We don't strictly need this step, but it gives a nicer notice message
------------------------------------------------------------------------------------------
DECLARE
primary_worker_node_count bigint :=0;
BEGIN
SELECT count(*) INTO primary_worker_node_count FROM pg_dist_node WHERE groupid != 0 AND noderole = 'primary';
IF primary_worker_node_count = 0 THEN
RAISE NOTICE 'There are no primary worker nodes, no need to sync metadata to any node';
RETURN true;
ELSE
RAISE DEBUG 'There are % primary worker nodes, continue to sync metadata', primary_worker_node_count;
END IF;
END;
------------------------------------------------------------------------------------------
-- STAGE 8: Do the actual metadata & object syncing to the worker nodes
-- For the "already synced" metadata nodes, we do not strictly need to
-- sync the objects & metadata, but there is no harm to do it anyway
-- it'll only cost some execution time but makes sure that we have a
-- a consistent metadata & objects across all the nodes
------------------------------------------------------------------------------------------
DECLARE
BEGIN
-- this might take long depending on the number of tables & objects ...
RAISE NOTICE 'Preparing to sync the metadata to all nodes';
PERFORM start_metadata_sync_to_node(nodename,nodeport)
FROM
pg_dist_node WHERE groupid != 0 AND noderole = 'primary';
END;
RETURN true;
END;
$$;
COMMENT ON FUNCTION pg_catalog.citus_finalize_upgrade_to_citus11(bool)
IS 'finalizes upgrade to Citus';
REVOKE ALL ON FUNCTION pg_catalog.citus_finalize_upgrade_to_citus11(bool) FROM PUBLIC;

View File

@ -0,0 +1,224 @@
-- citus_finalize_upgrade_to_citus11() is a helper UDF ensures
-- the upgrade to Citus 11 is finished successfully. Upgrade to
-- Citus 11 requires all active primary worker nodes to get the
-- metadata. And, this function's job is to sync the metadata to
-- the nodes that does not already have
-- once the function finishes without any errors and returns true
-- the cluster is ready for running distributed queries from
-- the worker nodes. When debug is enabled, the function provides
-- more information to the user.
CREATE OR REPLACE FUNCTION pg_catalog.citus_finalize_upgrade_to_citus11(enforce_version_check bool default true)
RETURNS bool
LANGUAGE plpgsql
AS $$
BEGIN
---------------------------------------------
-- This script consists of N stages
-- Each step is documented, and if log level
-- is reduced to DEBUG1, each step is logged
-- as well
---------------------------------------------
------------------------------------------------------------------------------------------
-- STAGE 0: Ensure no concurrent node metadata changing operation happens while this
-- script is running via acquiring a strong lock on the pg_dist_node
------------------------------------------------------------------------------------------
BEGIN
LOCK TABLE pg_dist_node IN EXCLUSIVE MODE NOWAIT;
EXCEPTION WHEN OTHERS THEN
RAISE 'Another node metadata changing operation is in progress, try again.';
END;
------------------------------------------------------------------------------------------
-- STAGE 1: We want all the commands to run in the same transaction block. Without
-- sequential mode, metadata syncing cannot be done in a transaction block along with
-- other commands
------------------------------------------------------------------------------------------
SET LOCAL citus.multi_shard_modify_mode TO 'sequential';
------------------------------------------------------------------------------------------
-- STAGE 2: Ensure we have the prerequisites
-- (a) only superuser can run this script
-- (b) cannot be executed when enable_ddl_propagation is False
-- (c) can only be executed from the coordinator
------------------------------------------------------------------------------------------
DECLARE
is_superuser_running boolean := False;
enable_ddl_prop boolean:= False;
local_group_id int := 0;
BEGIN
SELECT rolsuper INTO is_superuser_running FROM pg_roles WHERE rolname = current_user;
IF is_superuser_running IS NOT True THEN
RAISE EXCEPTION 'This operation can only be initiated by superuser';
END IF;
SELECT current_setting('citus.enable_ddl_propagation') INTO enable_ddl_prop;
IF enable_ddl_prop IS NOT True THEN
RAISE EXCEPTION 'This operation cannot be completed when citus.enable_ddl_propagation is False.';
END IF;
SELECT groupid INTO local_group_id FROM pg_dist_local_group;
IF local_group_id != 0 THEN
RAISE EXCEPTION 'Operation is not allowed on this node. Connect to the coordinator and run it again.';
ELSE
RAISE DEBUG 'We are on the coordinator, continue to sync metadata';
END IF;
END;
------------------------------------------------------------------------------------------
-- STAGE 3: Ensure all primary nodes are active
------------------------------------------------------------------------------------------
DECLARE
primary_disabled_worker_node_count int := 0;
BEGIN
SELECT count(*) INTO primary_disabled_worker_node_count FROM pg_dist_node
WHERE groupid != 0 AND noderole = 'primary' AND NOT isactive;
IF primary_disabled_worker_node_count != 0 THEN
RAISE EXCEPTION 'There are inactive primary worker nodes, you need to activate the nodes first.'
'Use SELECT citus_activate_node() to activate the disabled nodes';
ELSE
RAISE DEBUG 'There are no disabled worker nodes, continue to sync metadata';
END IF;
END;
------------------------------------------------------------------------------------------
-- STAGE 4: Ensure there is no connectivity issues in the cluster
------------------------------------------------------------------------------------------
DECLARE
all_nodes_can_connect_to_each_other boolean := False;
BEGIN
SELECT bool_and(coalesce(result, false)) INTO all_nodes_can_connect_to_each_other FROM citus_check_cluster_node_health();
IF all_nodes_can_connect_to_each_other != True THEN
RAISE EXCEPTION 'There are unhealth primary nodes, you need to ensure all '
'nodes are up and runnnig. Also, make sure that all nodes can connect '
'to each other. Use SELECT * FROM citus_check_cluster_node_health(); '
'to check the cluster health';
ELSE
RAISE DEBUG 'Cluster is healthy, all nodes can connect to each other';
END IF;
END;
------------------------------------------------------------------------------------------
-- STAGE 5: Ensure all nodes are on the same version
------------------------------------------------------------------------------------------
DECLARE
coordinator_version text := '';
worker_node_version text := '';
worker_node_version_count int := 0;
BEGIN
SELECT extversion INTO coordinator_version from pg_extension WHERE extname = 'citus';
-- first, check if all nodes have the same versions
SELECT
count(*) INTO worker_node_version_count
FROM
run_command_on_workers('SELECT extversion from pg_extension WHERE extname = ''citus'';')
GROUP BY result;
IF enforce_version_check AND worker_node_version_count != 1 THEN
RAISE EXCEPTION 'All nodes should have the same Citus version installed. Currently '
'the some of the workers has version different versions';
ELSE
RAISE DEBUG 'All worker nodes have the same Citus version';
END IF;
-- second, check if all nodes have the same versions
SELECT
result INTO worker_node_version
FROM
run_command_on_workers('SELECT extversion from pg_extension WHERE extname = ''citus'';')
GROUP BY result;
IF enforce_version_check AND coordinator_version != worker_node_version THEN
RAISE EXCEPTION 'All nodes should have the same Citus version installed. Currently '
'the coordinator has version % and the worker(s) has %',
coordinator_version, worker_node_version;
ELSE
RAISE DEBUG 'All nodes have the same Citus version';
END IF;
END;
------------------------------------------------------------------------------------------
-- STAGE 6: Ensure all the partitioned tables have the proper naming structure
-- As described on https://github.com/citusdata/citus/issues/4962
-- existing indexes on partitioned distributed tables can collide
-- with the index names exists on the shards
-- luckily, we know how to fix it.
-- And, note that we should do this even if the cluster is a basic plan
-- (e.g., single node Citus) such that when cluster scaled out, everything
-- works as intended
-- And, this should be done only ONCE for a cluster as it can be a pretty
-- time consuming operation. Thus, even if the function is called multiple time,
-- we keep track of it and do not re-execute this part if not needed.
------------------------------------------------------------------------------------------
DECLARE
partitioned_table_exists_pre_11 boolean:=False;
BEGIN
-- we recorded if partitioned tables exists during upgrade to Citus 11
SELECT metadata->>'partitioned_citus_table_exists_pre_11' INTO partitioned_table_exists_pre_11
FROM pg_dist_node_metadata;
IF partitioned_table_exists_pre_11 IS NOT NULL AND partitioned_table_exists_pre_11 THEN
-- this might take long depending on the number of partitions and shards...
RAISE NOTICE 'Preparing all the existing partitioned table indexes';
PERFORM pg_catalog.fix_all_partition_shard_index_names();
-- great, we are done with fixing the existing wrong index names
-- so, lets remove this
UPDATE pg_dist_node_metadata
SET metadata=jsonb_delete(metadata, 'partitioned_citus_table_exists_pre_11');
ELSE
RAISE DEBUG 'There are no partitioned tables that should be fixed';
END IF;
END;
------------------------------------------------------------------------------------------
-- STAGE 7: Return early if there are no primary worker nodes
-- We don't strictly need this step, but it gives a nicer notice message
------------------------------------------------------------------------------------------
DECLARE
primary_worker_node_count bigint :=0;
BEGIN
SELECT count(*) INTO primary_worker_node_count FROM pg_dist_node WHERE groupid != 0 AND noderole = 'primary';
IF primary_worker_node_count = 0 THEN
RAISE NOTICE 'There are no primary worker nodes, no need to sync metadata to any node';
RETURN true;
ELSE
RAISE DEBUG 'There are % primary worker nodes, continue to sync metadata', primary_worker_node_count;
END IF;
END;
------------------------------------------------------------------------------------------
-- STAGE 8: Do the actual metadata & object syncing to the worker nodes
-- For the "already synced" metadata nodes, we do not strictly need to
-- sync the objects & metadata, but there is no harm to do it anyway
-- it'll only cost some execution time but makes sure that we have a
-- a consistent metadata & objects across all the nodes
------------------------------------------------------------------------------------------
DECLARE
BEGIN
-- this might take long depending on the number of tables & objects ...
RAISE NOTICE 'Preparing to sync the metadata to all nodes';
PERFORM start_metadata_sync_to_node(nodename,nodeport)
FROM
pg_dist_node WHERE groupid != 0 AND noderole = 'primary';
END;
RETURN true;
END;
$$;
COMMENT ON FUNCTION pg_catalog.citus_finalize_upgrade_to_citus11(bool)
IS 'finalizes upgrade to Citus';
REVOKE ALL ON FUNCTION pg_catalog.citus_finalize_upgrade_to_citus11(bool) FROM PUBLIC;

View File

@ -12,15 +12,22 @@ BEGIN
IF substring(current_Setting('server_version'), '\d+')::int >= 14 THEN IF substring(current_Setting('server_version'), '\d+')::int >= 14 THEN
EXECUTE $cmd$ EXECUTE $cmd$
-- disable propagation to prevent EnsureCoordinator errors
-- the aggregate created here does not depend on Citus extension (yet)
-- since we add the dependency with the next command
SET citus.enable_ddl_propagation TO OFF;
CREATE AGGREGATE array_cat_agg(anycompatiblearray) (SFUNC = array_cat, STYPE = anycompatiblearray); CREATE AGGREGATE array_cat_agg(anycompatiblearray) (SFUNC = array_cat, STYPE = anycompatiblearray);
COMMENT ON AGGREGATE array_cat_agg(anycompatiblearray) COMMENT ON AGGREGATE array_cat_agg(anycompatiblearray)
IS 'concatenate input arrays into a single array'; IS 'concatenate input arrays into a single array';
RESET citus.enable_ddl_propagation;
$cmd$; $cmd$;
ELSE ELSE
EXECUTE $cmd$ EXECUTE $cmd$
SET citus.enable_ddl_propagation TO OFF;
CREATE AGGREGATE array_cat_agg(anyarray) (SFUNC = array_cat, STYPE = anyarray); CREATE AGGREGATE array_cat_agg(anyarray) (SFUNC = array_cat, STYPE = anyarray);
COMMENT ON AGGREGATE array_cat_agg(anyarray) COMMENT ON AGGREGATE array_cat_agg(anyarray)
IS 'concatenate input arrays into a single array'; IS 'concatenate input arrays into a single array';
RESET citus.enable_ddl_propagation;
$cmd$; $cmd$;
END IF; END IF;

View File

@ -12,15 +12,22 @@ BEGIN
IF substring(current_Setting('server_version'), '\d+')::int >= 14 THEN IF substring(current_Setting('server_version'), '\d+')::int >= 14 THEN
EXECUTE $cmd$ EXECUTE $cmd$
-- disable propagation to prevent EnsureCoordinator errors
-- the aggregate created here does not depend on Citus extension (yet)
-- since we add the dependency with the next command
SET citus.enable_ddl_propagation TO OFF;
CREATE AGGREGATE array_cat_agg(anycompatiblearray) (SFUNC = array_cat, STYPE = anycompatiblearray); CREATE AGGREGATE array_cat_agg(anycompatiblearray) (SFUNC = array_cat, STYPE = anycompatiblearray);
COMMENT ON AGGREGATE array_cat_agg(anycompatiblearray) COMMENT ON AGGREGATE array_cat_agg(anycompatiblearray)
IS 'concatenate input arrays into a single array'; IS 'concatenate input arrays into a single array';
RESET citus.enable_ddl_propagation;
$cmd$; $cmd$;
ELSE ELSE
EXECUTE $cmd$ EXECUTE $cmd$
SET citus.enable_ddl_propagation TO OFF;
CREATE AGGREGATE array_cat_agg(anyarray) (SFUNC = array_cat, STYPE = anyarray); CREATE AGGREGATE array_cat_agg(anyarray) (SFUNC = array_cat, STYPE = anyarray);
COMMENT ON AGGREGATE array_cat_agg(anyarray) COMMENT ON AGGREGATE array_cat_agg(anyarray)
IS 'concatenate input arrays into a single array'; IS 'concatenate input arrays into a single array';
RESET citus.enable_ddl_propagation;
$cmd$; $cmd$;
END IF; END IF;

View File

@ -12,4 +12,4 @@ CREATE OR REPLACE FUNCTION pg_catalog.worker_create_or_replace_object(statements
AS 'MODULE_PATHNAME', $$worker_create_or_replace_object_array$$; AS 'MODULE_PATHNAME', $$worker_create_or_replace_object_array$$;
COMMENT ON FUNCTION pg_catalog.worker_create_or_replace_object(statements text[]) COMMENT ON FUNCTION pg_catalog.worker_create_or_replace_object(statements text[])
IS 'takes a lost of sql statements, before executing these it will check if the object already exists in that exact state otherwise replaces that named object with the new object'; IS 'takes an array of sql statements, before executing these it will check if the object already exists in that exact state otherwise replaces that named object with the new object';

View File

@ -12,4 +12,4 @@ CREATE OR REPLACE FUNCTION pg_catalog.worker_create_or_replace_object(statements
AS 'MODULE_PATHNAME', $$worker_create_or_replace_object_array$$; AS 'MODULE_PATHNAME', $$worker_create_or_replace_object_array$$;
COMMENT ON FUNCTION pg_catalog.worker_create_or_replace_object(statements text[]) COMMENT ON FUNCTION pg_catalog.worker_create_or_replace_object(statements text[])
IS 'takes a lost of sql statements, before executing these it will check if the object already exists in that exact state otherwise replaces that named object with the new object'; IS 'takes an array of sql statements, before executing these it will check if the object already exists in that exact state otherwise replaces that named object with the new object';

View File

@ -0,0 +1,20 @@
DROP FUNCTION pg_catalog.worker_partition_query_result(text, text, int, citus.distribution_type, text[], text[], boolean);
CREATE OR REPLACE FUNCTION pg_catalog.worker_partition_query_result(
result_prefix text,
query text,
partition_column_index int,
partition_method citus.distribution_type,
partition_min_values text[],
partition_max_values text[],
binary_copy boolean,
allow_null_partition_column boolean DEFAULT false,
generate_empty_results boolean DEFAULT false,
OUT partition_index int,
OUT rows_written bigint,
OUT bytes_written bigint)
RETURNS SETOF record
LANGUAGE C STRICT VOLATILE
AS 'MODULE_PATHNAME', $$worker_partition_query_result$$;
COMMENT ON FUNCTION pg_catalog.worker_partition_query_result(text, text, int, citus.distribution_type, text[], text[], boolean, boolean, boolean)
IS 'execute a query and partitions its results in set of local result files';

View File

@ -1,3 +1,5 @@
DROP FUNCTION pg_catalog.worker_partition_query_result(text, text, int, citus.distribution_type, text[], text[], boolean);
CREATE OR REPLACE FUNCTION pg_catalog.worker_partition_query_result( CREATE OR REPLACE FUNCTION pg_catalog.worker_partition_query_result(
result_prefix text, result_prefix text,
query text, query text,
@ -5,12 +7,14 @@ CREATE OR REPLACE FUNCTION pg_catalog.worker_partition_query_result(
partition_method citus.distribution_type, partition_method citus.distribution_type,
partition_min_values text[], partition_min_values text[],
partition_max_values text[], partition_max_values text[],
binaryCopy boolean, binary_copy boolean,
allow_null_partition_column boolean DEFAULT false,
generate_empty_results boolean DEFAULT false,
OUT partition_index int, OUT partition_index int,
OUT rows_written bigint, OUT rows_written bigint,
OUT bytes_written bigint) OUT bytes_written bigint)
RETURNS SETOF record RETURNS SETOF record
LANGUAGE C STRICT VOLATILE LANGUAGE C STRICT VOLATILE
AS 'MODULE_PATHNAME', $$worker_partition_query_result$$; AS 'MODULE_PATHNAME', $$worker_partition_query_result$$;
COMMENT ON FUNCTION pg_catalog.worker_partition_query_result(text, text, int, citus.distribution_type, text[], text[], boolean) COMMENT ON FUNCTION pg_catalog.worker_partition_query_result(text, text, int, citus.distribution_type, text[], text[], boolean, boolean, boolean)
IS 'execute a query and partitions its results in set of local result files'; IS 'execute a query and partitions its results in set of local result files';

View File

@ -20,9 +20,11 @@
#include "distributed/citus_ruleutils.h" #include "distributed/citus_ruleutils.h"
#include "distributed/connection_management.h" #include "distributed/connection_management.h"
#include "distributed/listutils.h" #include "distributed/listutils.h"
#include "distributed/metadata_utility.h"
#include "distributed/multi_physical_planner.h" #include "distributed/multi_physical_planner.h"
#include "distributed/shard_cleaner.h" #include "distributed/shard_cleaner.h"
#include "distributed/shard_rebalancer.h" #include "distributed/shard_rebalancer.h"
#include "distributed/relay_utility.h"
#include "funcapi.h" #include "funcapi.h"
#include "miscadmin.h" #include "miscadmin.h"
#include "utils/builtins.h" #include "utils/builtins.h"
@ -85,6 +87,18 @@ run_try_drop_marked_shards(PG_FUNCTION_ARGS)
} }
/*
* IsActiveTestShardPlacement checks if the dummy shard placement created in tests
* are labelled as active. Note that this function does not check if the worker is also
* active, because the dummy test workers are not registered as actual workers.
*/
static inline bool
IsActiveTestShardPlacement(ShardPlacement *shardPlacement)
{
return shardPlacement->shardState == SHARD_STATE_ACTIVE;
}
/* /*
* shard_placement_rebalance_array returns a list of operations which can make a * shard_placement_rebalance_array returns a list of operations which can make a
* cluster consisting of given shard placements and worker nodes balanced with * cluster consisting of given shard placements and worker nodes balanced with
@ -138,7 +152,9 @@ shard_placement_rebalance_array(PG_FUNCTION_ARGS)
if (shardPlacementTestInfo->nextColocationGroup) if (shardPlacementTestInfo->nextColocationGroup)
{ {
shardPlacementList = SortList(shardPlacementList, CompareShardPlacements); shardPlacementList = SortList(shardPlacementList, CompareShardPlacements);
shardPlacementListList = lappend(shardPlacementListList, shardPlacementList); shardPlacementListList = lappend(shardPlacementListList,
FilterShardPlacementList(shardPlacementList,
IsActiveTestShardPlacement));
shardPlacementList = NIL; shardPlacementList = NIL;
} }
shardPlacementList = lappend(shardPlacementList, shardPlacementList = lappend(shardPlacementList,
@ -290,12 +306,15 @@ shard_placement_replication_array(PG_FUNCTION_ARGS)
shardPlacementTestInfo->placement); shardPlacementTestInfo->placement);
} }
List *activeShardPlacementList = FilterShardPlacementList(shardPlacementList,
IsActiveTestShardPlacement);
/* sort the lists to make the function more deterministic */ /* sort the lists to make the function more deterministic */
workerNodeList = SortList(workerNodeList, CompareWorkerNodes); workerNodeList = SortList(workerNodeList, CompareWorkerNodes);
shardPlacementList = SortList(shardPlacementList, CompareShardPlacements); activeShardPlacementList = SortList(activeShardPlacementList, CompareShardPlacements);
List *placementUpdateList = ReplicationPlacementUpdates(workerNodeList, List *placementUpdateList = ReplicationPlacementUpdates(workerNodeList,
shardPlacementList, activeShardPlacementList,
shardReplicationFactor); shardReplicationFactor);
ArrayType *placementUpdateJsonArray = PlacementUpdateListToJsonArray( ArrayType *placementUpdateJsonArray = PlacementUpdateListToJsonArray(
placementUpdateList); placementUpdateList);
@ -426,6 +445,9 @@ JsonArrayToWorkerTestInfoList(ArrayType *workerNodeJsonArrayObject)
workerTestInfo->capacity = JsonFieldValueUInt64Default(workerNodeJson, workerTestInfo->capacity = JsonFieldValueUInt64Default(workerNodeJson,
"capacity", 1); "capacity", 1);
workerNode->isActive = JsonFieldValueBoolDefault(workerNodeJson,
"isActive", true);
workerTestInfoList = lappend(workerTestInfoList, workerTestInfo); workerTestInfoList = lappend(workerTestInfoList, workerTestInfo);
char *disallowedShardsString = JsonFieldValueString( char *disallowedShardsString = JsonFieldValueString(
workerNodeJson, "disallowed_shards"); workerNodeJson, "disallowed_shards");

View File

@ -83,6 +83,7 @@ typedef struct BackendManagementShmemData
static void StoreAllActiveTransactions(Tuplestorestate *tupleStore, TupleDesc static void StoreAllActiveTransactions(Tuplestorestate *tupleStore, TupleDesc
tupleDescriptor); tupleDescriptor);
static bool UserHasPermissionToViewStatsOf(Oid currentUserId, Oid backendOwnedId);
static uint64 GenerateGlobalPID(void); static uint64 GenerateGlobalPID(void);
static shmem_startup_hook_type prev_shmem_startup_hook = NULL; static shmem_startup_hook_type prev_shmem_startup_hook = NULL;
@ -114,8 +115,6 @@ assign_distributed_transaction_id(PG_FUNCTION_ARGS)
{ {
CheckCitusVersion(ERROR); CheckCitusVersion(ERROR);
Oid userId = GetUserId();
/* prepare data before acquiring spinlock to protect against errors */ /* prepare data before acquiring spinlock to protect against errors */
int32 initiatorNodeIdentifier = PG_GETARG_INT32(0); int32 initiatorNodeIdentifier = PG_GETARG_INT32(0);
uint64 transactionNumber = PG_GETARG_INT64(1); uint64 transactionNumber = PG_GETARG_INT64(1);
@ -144,17 +143,11 @@ assign_distributed_transaction_id(PG_FUNCTION_ARGS)
"transaction id"))); "transaction id")));
} }
MyBackendData->databaseId = MyDatabaseId;
MyBackendData->userId = userId;
MyBackendData->transactionId.initiatorNodeIdentifier = initiatorNodeIdentifier; MyBackendData->transactionId.initiatorNodeIdentifier = initiatorNodeIdentifier;
MyBackendData->transactionId.transactionNumber = transactionNumber; MyBackendData->transactionId.transactionNumber = transactionNumber;
MyBackendData->transactionId.timestamp = timestamp; MyBackendData->transactionId.timestamp = timestamp;
MyBackendData->transactionId.transactionOriginator = false; MyBackendData->transactionId.transactionOriginator = false;
MyBackendData->citusBackend.initiatorNodeIdentifier =
MyBackendData->transactionId.initiatorNodeIdentifier;
SpinLockRelease(&MyBackendData->mutex); SpinLockRelease(&MyBackendData->mutex);
PG_RETURN_VOID(); PG_RETURN_VOID();
@ -360,55 +353,48 @@ StoreAllActiveTransactions(Tuplestorestate *tupleStore, TupleDesc tupleDescripto
{ {
Datum values[ACTIVE_TRANSACTION_COLUMN_COUNT]; Datum values[ACTIVE_TRANSACTION_COLUMN_COUNT];
bool isNulls[ACTIVE_TRANSACTION_COLUMN_COUNT]; bool isNulls[ACTIVE_TRANSACTION_COLUMN_COUNT];
bool showAllTransactions = superuser(); bool showAllBackends = superuser();
const Oid userId = GetUserId(); const Oid userId = GetUserId();
/* if (!showAllBackends && is_member_of_role(userId, ROLE_PG_MONITOR))
* We don't want to initialize memory while spinlock is held so we
* prefer to do it here. This initialization is done only for the first
* row.
*/
memset(values, 0, sizeof(values));
memset(isNulls, false, sizeof(isNulls));
if (is_member_of_role(userId, ROLE_PG_MONITOR))
{ {
showAllTransactions = true; showAllBackends = true;
} }
/* we're reading all distributed transactions, prevent new backends */ /* we're reading all distributed transactions, prevent new backends */
LockBackendSharedMemory(LW_SHARED); LockBackendSharedMemory(LW_SHARED);
for (int backendIndex = 0; backendIndex < MaxBackends; ++backendIndex) for (int backendIndex = 0; backendIndex < TotalProcCount(); ++backendIndex)
{ {
bool showCurrentBackendDetails = showAllBackends;
BackendData *currentBackend = BackendData *currentBackend =
&backendManagementShmemData->backends[backendIndex]; &backendManagementShmemData->backends[backendIndex];
PGPROC *currentProc = &ProcGlobal->allProcs[backendIndex];
/* to work on data after releasing g spinlock to protect against errors */ /* to work on data after releasing g spinlock to protect against errors */
int initiatorNodeIdentifier = -1;
uint64 transactionNumber = 0; uint64 transactionNumber = 0;
SpinLockAcquire(&currentBackend->mutex); SpinLockAcquire(&currentBackend->mutex);
if (currentBackend->globalPID == INVALID_CITUS_INTERNAL_BACKEND_GPID) if (currentProc->pid == 0)
{ {
/* unused PGPROC slot */
SpinLockRelease(&currentBackend->mutex); SpinLockRelease(&currentBackend->mutex);
continue; continue;
} }
/* /*
* Unless the user has a role that allows seeing all transactions (superuser, * Unless the user has a role that allows seeing all transactions (superuser,
* pg_monitor), skip over transactions belonging to other users. * pg_monitor), we only follow pg_stat_statements owner checks.
*/ */
if (!showAllTransactions && currentBackend->userId != userId) if (!showCurrentBackendDetails &&
UserHasPermissionToViewStatsOf(userId, currentProc->roleId))
{ {
SpinLockRelease(&currentBackend->mutex); showCurrentBackendDetails = true;
continue;
} }
Oid databaseId = currentBackend->databaseId; Oid databaseId = currentBackend->databaseId;
int backendPid = ProcGlobal->allProcs[backendIndex].pid; int backendPid = ProcGlobal->allProcs[backendIndex].pid;
initiatorNodeIdentifier = currentBackend->citusBackend.initiatorNodeIdentifier;
/* /*
* We prefer to use worker_query instead of distributedCommandOriginator in * We prefer to use worker_query instead of distributedCommandOriginator in
@ -423,13 +409,42 @@ StoreAllActiveTransactions(Tuplestorestate *tupleStore, TupleDesc tupleDescripto
SpinLockRelease(&currentBackend->mutex); SpinLockRelease(&currentBackend->mutex);
memset(values, 0, sizeof(values));
memset(isNulls, false, sizeof(isNulls));
/*
* We imitate pg_stat_activity such that if a user doesn't have enough
* privileges, we only show the minimal information including the pid,
* global pid and distributedCommandOriginator.
*
* pid is already can be found in pg_stat_activity for any process, and
* the rest doesn't reveal anything critial for under priviledge users
* but still could be useful for monitoring purposes of Citus.
*/
if (showCurrentBackendDetails)
{
bool missingOk = true;
int initiatorNodeId =
ExtractNodeIdFromGlobalPID(currentBackend->globalPID, missingOk);
values[0] = ObjectIdGetDatum(databaseId); values[0] = ObjectIdGetDatum(databaseId);
values[1] = Int32GetDatum(backendPid); values[1] = Int32GetDatum(backendPid);
values[2] = Int32GetDatum(initiatorNodeIdentifier); values[2] = Int32GetDatum(initiatorNodeId);
values[3] = !distributedCommandOriginator; values[3] = !distributedCommandOriginator;
values[4] = UInt64GetDatum(transactionNumber); values[4] = UInt64GetDatum(transactionNumber);
values[5] = TimestampTzGetDatum(transactionIdTimestamp); values[5] = TimestampTzGetDatum(transactionIdTimestamp);
values[6] = UInt64GetDatum(currentBackend->globalPID); values[6] = UInt64GetDatum(currentBackend->globalPID);
}
else
{
isNulls[0] = true;
values[1] = Int32GetDatum(backendPid);
isNulls[2] = true;
values[3] = !distributedCommandOriginator;
isNulls[4] = true;
isNulls[5] = true;
values[6] = UInt64GetDatum(currentBackend->globalPID);
}
tuplestore_putvalues(tupleStore, tupleDescriptor, values, isNulls); tuplestore_putvalues(tupleStore, tupleDescriptor, values, isNulls);
@ -446,6 +461,35 @@ StoreAllActiveTransactions(Tuplestorestate *tupleStore, TupleDesc tupleDescripto
} }
/*
* UserHasPermissionToViewStatsOf returns true if currentUserId can
* see backends of backendOwnedId.
*
* We follow the same approach with pg_stat_activity.
*/
static
bool
UserHasPermissionToViewStatsOf(Oid currentUserId, Oid backendOwnedId)
{
if (has_privs_of_role(currentUserId, backendOwnedId))
{
return true;
}
if (is_member_of_role(currentUserId,
#if PG_VERSION_NUM >= PG_VERSION_14
ROLE_PG_READ_ALL_STATS))
#else
DEFAULT_ROLE_READ_ALL_STATS))
#endif
{
return true;
}
return false;
}
/* /*
* InitializeBackendManagement requests the necessary shared memory * InitializeBackendManagement requests the necessary shared memory
* from Postgres and sets up the shared memory startup hook. * from Postgres and sets up the shared memory startup hook.
@ -520,7 +564,6 @@ BackendManagementShmemInit(void)
{ {
BackendData *backendData = BackendData *backendData =
&backendManagementShmemData->backends[backendIndex]; &backendManagementShmemData->backends[backendIndex];
backendData->citusBackend.initiatorNodeIdentifier = -1;
SpinLockInit(&backendData->mutex); SpinLockInit(&backendData->mutex);
} }
} }
@ -652,16 +695,12 @@ UnSetDistributedTransactionId(void)
{ {
SpinLockAcquire(&MyBackendData->mutex); SpinLockAcquire(&MyBackendData->mutex);
MyBackendData->databaseId = 0;
MyBackendData->userId = 0;
MyBackendData->cancelledDueToDeadlock = false; MyBackendData->cancelledDueToDeadlock = false;
MyBackendData->transactionId.initiatorNodeIdentifier = 0; MyBackendData->transactionId.initiatorNodeIdentifier = 0;
MyBackendData->transactionId.transactionOriginator = false; MyBackendData->transactionId.transactionOriginator = false;
MyBackendData->transactionId.transactionNumber = 0; MyBackendData->transactionId.transactionNumber = 0;
MyBackendData->transactionId.timestamp = 0; MyBackendData->transactionId.timestamp = 0;
MyBackendData->citusBackend.initiatorNodeIdentifier = -1;
SpinLockRelease(&MyBackendData->mutex); SpinLockRelease(&MyBackendData->mutex);
} }
} }
@ -679,6 +718,8 @@ UnSetGlobalPID(void)
SpinLockAcquire(&MyBackendData->mutex); SpinLockAcquire(&MyBackendData->mutex);
MyBackendData->globalPID = 0; MyBackendData->globalPID = 0;
MyBackendData->databaseId = 0;
MyBackendData->userId = 0;
SpinLockRelease(&MyBackendData->mutex); SpinLockRelease(&MyBackendData->mutex);
} }
@ -760,41 +801,14 @@ AssignDistributedTransactionId(void)
uint64 nextTransactionNumber = pg_atomic_fetch_add_u64(transactionNumberSequence, 1); uint64 nextTransactionNumber = pg_atomic_fetch_add_u64(transactionNumberSequence, 1);
int32 localGroupId = GetLocalGroupId(); int32 localGroupId = GetLocalGroupId();
TimestampTz currentTimestamp = GetCurrentTimestamp(); TimestampTz currentTimestamp = GetCurrentTimestamp();
Oid userId = GetUserId();
SpinLockAcquire(&MyBackendData->mutex); SpinLockAcquire(&MyBackendData->mutex);
MyBackendData->databaseId = MyDatabaseId;
MyBackendData->userId = userId;
MyBackendData->transactionId.initiatorNodeIdentifier = localGroupId; MyBackendData->transactionId.initiatorNodeIdentifier = localGroupId;
MyBackendData->transactionId.transactionOriginator = true; MyBackendData->transactionId.transactionOriginator = true;
MyBackendData->transactionId.transactionNumber = nextTransactionNumber; MyBackendData->transactionId.transactionNumber = nextTransactionNumber;
MyBackendData->transactionId.timestamp = currentTimestamp; MyBackendData->transactionId.timestamp = currentTimestamp;
MyBackendData->citusBackend.initiatorNodeIdentifier = localGroupId;
SpinLockRelease(&MyBackendData->mutex);
}
/*
* MarkCitusInitiatedCoordinatorBackend sets that coordinator backend is
* initiated by Citus.
*/
void
MarkCitusInitiatedCoordinatorBackend(void)
{
/*
* GetLocalGroupId may throw exception which can cause leaving spin lock
* unreleased. Calling GetLocalGroupId function before the lock to avoid this.
*/
int32 localGroupId = GetLocalGroupId();
SpinLockAcquire(&MyBackendData->mutex);
MyBackendData->citusBackend.initiatorNodeIdentifier = localGroupId;
SpinLockRelease(&MyBackendData->mutex); SpinLockRelease(&MyBackendData->mutex);
} }
@ -821,9 +835,15 @@ AssignGlobalPID(void)
globalPID = ExtractGlobalPID(application_name); globalPID = ExtractGlobalPID(application_name);
} }
Oid userId = GetUserId();
SpinLockAcquire(&MyBackendData->mutex); SpinLockAcquire(&MyBackendData->mutex);
MyBackendData->globalPID = globalPID; MyBackendData->globalPID = globalPID;
MyBackendData->distributedCommandOriginator = distributedCommandOriginator; MyBackendData->distributedCommandOriginator = distributedCommandOriginator;
MyBackendData->databaseId = MyDatabaseId;
MyBackendData->userId = userId;
SpinLockRelease(&MyBackendData->mutex); SpinLockRelease(&MyBackendData->mutex);
} }
@ -926,11 +946,12 @@ ExtractGlobalPID(char *applicationName)
* gives us the node id. * gives us the node id.
*/ */
int int
ExtractNodeIdFromGlobalPID(uint64 globalPID) ExtractNodeIdFromGlobalPID(uint64 globalPID, bool missingOk)
{ {
int nodeId = (int) (globalPID / GLOBAL_PID_NODE_ID_MULTIPLIER); int nodeId = (int) (globalPID / GLOBAL_PID_NODE_ID_MULTIPLIER);
if (nodeId == GLOBAL_PID_NODE_ID_FOR_NODES_NOT_IN_METADATA) if (!missingOk &&
nodeId == GLOBAL_PID_NODE_ID_FOR_NODES_NOT_IN_METADATA)
{ {
ereport(ERROR, (errmsg("originator node of the query with the global pid " ereport(ERROR, (errmsg("originator node of the query with the global pid "
"%lu is not in Citus' metadata", globalPID), "%lu is not in Citus' metadata", globalPID),

View File

@ -538,33 +538,35 @@ ReplaceInitiatorNodeIdentifier(int initiator_node_identifier,
* transaction. However, we cannot know which node has initiated * transaction. However, we cannot know which node has initiated
* the worker query. * the worker query.
*/ */
if (initiator_node_identifier > 0) if (initiator_node_identifier == GLOBAL_PID_NODE_ID_FOR_NODES_NOT_IN_METADATA &&
{ IsCoordinator())
bool nodeExists = false;
initiatorWorkerNode = PrimaryNodeForGroup(initiator_node_identifier, &nodeExists);
/* a query should run on an existing node */
Assert(nodeExists);
if (initiatorWorkerNode == NULL)
{
ereport(ERROR, (errmsg("no primary node found for group %d",
initiator_node_identifier)));
}
citusDistStat->master_query_host_name =
cstring_to_text(initiatorWorkerNode->workerName);
citusDistStat->master_query_host_port = initiatorWorkerNode->workerPort;
}
else if (initiator_node_identifier == 0 && IsCoordinator())
{ {
citusDistStat->master_query_host_name = cstring_to_text(coordinator_host_name); citusDistStat->master_query_host_name = cstring_to_text(coordinator_host_name);
citusDistStat->master_query_host_port = PostPortNumber; citusDistStat->master_query_host_port = PostPortNumber;
} }
else if (initiator_node_identifier == 0) else if (initiator_node_identifier == GLOBAL_PID_NODE_ID_FOR_NODES_NOT_IN_METADATA)
{ {
citusDistStat->master_query_host_name = cstring_to_text(coordinator_host_name); citusDistStat->master_query_host_name = cstring_to_text(coordinator_host_name);
citusDistStat->master_query_host_port = 0; citusDistStat->master_query_host_port = 0;
} }
else if (initiator_node_identifier > 0)
{
/* a query should run on an existing node, but lets be defensive */
bool missingOk = true;
initiatorWorkerNode = FindNodeWithNodeId(initiator_node_identifier, missingOk);
if (initiatorWorkerNode)
{
citusDistStat->master_query_host_name =
cstring_to_text(initiatorWorkerNode->workerName);
citusDistStat->master_query_host_port = initiatorWorkerNode->workerPort;
}
else
{
citusDistStat->master_query_host_name = NULL;
citusDistStat->master_query_host_port = 0;
}
}
else else
{ {
citusDistStat->master_query_host_name = NULL; citusDistStat->master_query_host_name = NULL;

View File

@ -15,6 +15,7 @@ extern bool EnableCostBasedConnectionEstablishment;
extern bool PreventIncompleteConnectionEstablishment; extern bool PreventIncompleteConnectionEstablishment;
extern bool ShouldRunTasksSequentially(List *taskList); extern bool ShouldRunTasksSequentially(List *taskList);
extern uint64 ExecuteTaskList(RowModifyLevel modLevel, List *taskList);
extern uint64 ExecuteUtilityTaskList(List *utilityTaskList, bool localExecutionSupported); extern uint64 ExecuteUtilityTaskList(List *utilityTaskList, bool localExecutionSupported);
extern uint64 ExecuteUtilityTaskListExtended(List *utilityTaskList, int poolSize, extern uint64 ExecuteUtilityTaskListExtended(List *utilityTaskList, int poolSize,
bool localExecutionSupported); bool localExecutionSupported);

View File

@ -22,16 +22,6 @@
#include "storage/s_lock.h" #include "storage/s_lock.h"
/*
* CitusInitiatedBackend keeps some information about the backends that are
* initiated by Citus.
*/
typedef struct CitusInitiatedBackend
{
int initiatorNodeIdentifier;
} CitusInitiatedBackend;
/* /*
* Each backend's active distributed transaction information is tracked via * Each backend's active distributed transaction information is tracked via
* BackendData in shared memory. * BackendData in shared memory.
@ -51,7 +41,6 @@ typedef struct BackendData
bool cancelledDueToDeadlock; bool cancelledDueToDeadlock;
uint64 globalPID; uint64 globalPID;
bool distributedCommandOriginator; bool distributedCommandOriginator;
CitusInitiatedBackend citusBackend;
DistributedTransactionId transactionId; DistributedTransactionId transactionId;
} BackendData; } BackendData;
@ -64,13 +53,12 @@ extern void UnlockBackendSharedMemory(void);
extern void UnSetDistributedTransactionId(void); extern void UnSetDistributedTransactionId(void);
extern void UnSetGlobalPID(void); extern void UnSetGlobalPID(void);
extern void AssignDistributedTransactionId(void); extern void AssignDistributedTransactionId(void);
extern void MarkCitusInitiatedCoordinatorBackend(void);
extern void AssignGlobalPID(void); extern void AssignGlobalPID(void);
extern uint64 GetGlobalPID(void); extern uint64 GetGlobalPID(void);
extern void OverrideBackendDataDistributedCommandOriginator(bool extern void OverrideBackendDataDistributedCommandOriginator(bool
distributedCommandOriginator); distributedCommandOriginator);
extern uint64 ExtractGlobalPID(char *applicationName); extern uint64 ExtractGlobalPID(char *applicationName);
extern int ExtractNodeIdFromGlobalPID(uint64 globalPID); 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);
extern void CancelTransactionDueToDeadlock(PGPROC *proc); extern void CancelTransactionDueToDeadlock(PGPROC *proc);

View File

@ -122,6 +122,9 @@ typedef enum SearchForeignKeyColumnFlags
} SearchForeignKeyColumnFlags; } SearchForeignKeyColumnFlags;
/* aggregate.c - forward declarations */
extern List * PostprocessDefineAggregateStmt(Node *node, const char *queryString);
/* cluster.c - forward declarations */ /* cluster.c - forward declarations */
extern List * PreprocessClusterStmt(Node *node, const char *clusterCommand, extern List * PreprocessClusterStmt(Node *node, const char *clusterCommand,
ProcessUtilityContext processUtilityContext); ProcessUtilityContext processUtilityContext);
@ -264,6 +267,7 @@ extern List * PreprocessCreateFunctionStmt(Node *stmt, const char *queryString,
ProcessUtilityContext processUtilityContext); ProcessUtilityContext processUtilityContext);
extern List * PostprocessCreateFunctionStmt(Node *stmt, extern List * PostprocessCreateFunctionStmt(Node *stmt,
const char *queryString); const char *queryString);
extern ObjectAddress * GetUndistributableDependency(ObjectAddress *functionAddress);
extern ObjectAddress CreateFunctionStmtObjectAddress(Node *stmt, extern ObjectAddress CreateFunctionStmtObjectAddress(Node *stmt,
bool missing_ok); bool missing_ok);
extern ObjectAddress DefineAggregateStmtObjectAddress(Node *stmt, extern ObjectAddress DefineAggregateStmtObjectAddress(Node *stmt,

View File

@ -22,6 +22,13 @@
#include "distributed/version_compat.h" #include "distributed/version_compat.h"
#include "distributed/worker_transaction.h" #include "distributed/worker_transaction.h"
typedef enum
{
CREATE_OBJECT_PROPAGATION_DEFERRED = 0,
CREATE_OBJECT_PROPAGATION_AUTOMATIC = 1,
CREATE_OBJECT_PROPAGATION_IMMEDIATE = 2
} CreateObjectPropagationOptions;
typedef enum typedef enum
{ {
PROPSETCMD_INVALID = -1, PROPSETCMD_INVALID = -1,
@ -32,6 +39,7 @@ typedef enum
} PropSetCmdBehavior; } PropSetCmdBehavior;
extern PropSetCmdBehavior PropagateSetCommands; extern PropSetCmdBehavior PropagateSetCommands;
extern bool EnableDDLPropagation; extern bool EnableDDLPropagation;
extern int CreateObjectPropagationMode;
extern bool EnableCreateTypePropagation; extern bool EnableCreateTypePropagation;
extern bool EnableAlterRolePropagation; extern bool EnableAlterRolePropagation;
extern bool EnableAlterRoleSetPropagation; extern bool EnableAlterRoleSetPropagation;

View File

@ -33,6 +33,9 @@ extern void QualifyTreeNode(Node *stmt);
extern char * DeparseTreeNode(Node *stmt); extern char * DeparseTreeNode(Node *stmt);
extern List * DeparseTreeNodes(List *stmts); extern List * DeparseTreeNodes(List *stmts);
/* forward declarations for qualify_aggregate_stmts.c */
extern void QualifyDefineAggregateStmt(Node *node);
/* forward declarations for deparse_attribute_stmts.c */ /* forward declarations for deparse_attribute_stmts.c */
extern char * DeparseRenameAttributeStmt(Node *); extern char * DeparseRenameAttributeStmt(Node *);

View File

@ -48,6 +48,28 @@ typedef struct DistributedResultFragment
} DistributedResultFragment; } DistributedResultFragment;
/*
* NodePair contains the source and destination node in a NodeToNodeFragmentsTransfer.
* It is a separate struct to use it as a key in a hash table.
*/
typedef struct NodePair
{
uint32 sourceNodeId;
uint32 targetNodeId;
} NodePair;
/*
* NodeToNodeFragmentsTransfer contains all fragments that need to be fetched from
* the source node to the destination node in the NodePair.
*/
typedef struct NodeToNodeFragmentsTransfer
{
NodePair nodes;
List *fragmentList;
} NodeToNodeFragmentsTransfer;
/* intermediate_results.c */ /* intermediate_results.c */
extern DestReceiver * CreateRemoteFileDestReceiver(const char *resultId, extern DestReceiver * CreateRemoteFileDestReceiver(const char *resultId,
EState *executorState, EState *executorState,
@ -72,5 +94,10 @@ extern List * PartitionTasklistResults(const char *resultIdPrefix, List *selectT
int partitionColumnIndex, int partitionColumnIndex,
CitusTableCacheEntry *distributionScheme, CitusTableCacheEntry *distributionScheme,
bool binaryFormat); bool binaryFormat);
extern char * QueryStringForFragmentsTransfer(
NodeToNodeFragmentsTransfer *fragmentsTransfer);
extern void ShardMinMaxValueArrays(ShardInterval **shardIntervalArray, int shardCount,
Oid intervalTypeId, ArrayType **minValueArray,
ArrayType **maxValueArray);
#endif /* INTERMEDIATE_RESULTS_H */ #endif /* INTERMEDIATE_RESULTS_H */

View File

@ -43,6 +43,7 @@ extern void ExecuteUtilityCommand(const char *utilityCommand);
extern bool ShouldExecuteTasksLocally(List *taskList); extern bool ShouldExecuteTasksLocally(List *taskList);
extern bool AnyTaskAccessesLocalNode(List *taskList); extern bool AnyTaskAccessesLocalNode(List *taskList);
extern bool TaskAccessesLocalNode(Task *task); extern bool TaskAccessesLocalNode(Task *task);
extern void EnsureCompatibleLocalExecutionState(List *taskList);
extern void ErrorIfTransactionAccessedPlacementsLocally(void); extern void ErrorIfTransactionAccessedPlacementsLocally(void);
extern void DisableLocalExecution(void); extern void DisableLocalExecution(void);
extern void SetLocalExecutionStatus(LocalExecutionStatus newStatus); extern void SetLocalExecutionStatus(LocalExecutionStatus newStatus);

View File

@ -73,7 +73,9 @@ extern List * GetSequencesFromAttrDef(Oid attrdefOid);
extern void GetDependentSequencesWithRelation(Oid relationId, List **attnumList, extern void GetDependentSequencesWithRelation(Oid relationId, List **attnumList,
List **dependentSequenceList, AttrNumber List **dependentSequenceList, AttrNumber
attnum); attnum);
extern List * GetDependentFunctionsWithRelation(Oid relationId);
extern Oid GetAttributeTypeOid(Oid relationId, AttrNumber attnum); extern Oid GetAttributeTypeOid(Oid relationId, AttrNumber attnum);
extern void SetLocalEnableMetadataSync(bool state);
#define DELETE_ALL_NODES "DELETE FROM pg_dist_node" #define DELETE_ALL_NODES "DELETE FROM pg_dist_node"
#define DELETE_ALL_PLACEMENTS "DELETE FROM pg_dist_placement" #define DELETE_ALL_PLACEMENTS "DELETE FROM pg_dist_placement"

View File

@ -209,6 +209,9 @@ extern ShardInterval * CopyShardInterval(ShardInterval *srcInterval);
extern uint64 ShardLength(uint64 shardId); extern uint64 ShardLength(uint64 shardId);
extern bool NodeGroupHasShardPlacements(int32 groupId, extern bool NodeGroupHasShardPlacements(int32 groupId,
bool onlyConsiderActivePlacements); bool onlyConsiderActivePlacements);
extern bool IsActiveShardPlacement(ShardPlacement *ShardPlacement);
extern List * FilterShardPlacementList(List *shardPlacementList, bool (*filter)(
ShardPlacement *));
extern List * ActiveShardPlacementListOnGroup(uint64 shardId, int32 groupId); extern List * ActiveShardPlacementListOnGroup(uint64 shardId, int32 groupId);
extern List * ActiveShardPlacementList(uint64 shardId); extern List * ActiveShardPlacementList(uint64 shardId);
extern List * ShardPlacementListWithoutOrphanedPlacements(uint64 shardId); extern List * ShardPlacementListWithoutOrphanedPlacements(uint64 shardId);
@ -248,6 +251,7 @@ extern TableConversionReturn * UndistributeTable(TableConversionParameters *para
extern void EnsureDependenciesExistOnAllNodes(const ObjectAddress *target); extern void EnsureDependenciesExistOnAllNodes(const ObjectAddress *target);
extern List * GetDistributableDependenciesForObject(const ObjectAddress *target); extern List * GetDistributableDependenciesForObject(const ObjectAddress *target);
extern bool ShouldPropagate(void); extern bool ShouldPropagate(void);
extern bool ShouldPropagateCreateInCoordinatedTransction(void);
extern bool ShouldPropagateObject(const ObjectAddress *address); extern bool ShouldPropagateObject(const ObjectAddress *address);
extern List * ReplicateAllObjectsToNodeCommandList(const char *nodeName, int nodePort); extern List * ReplicateAllObjectsToNodeCommandList(const char *nodeName, int nodePort);

View File

@ -36,15 +36,6 @@
/* Definitions local to the physical planner */ /* Definitions local to the physical planner */
#define NON_PRUNABLE_JOIN -1 #define NON_PRUNABLE_JOIN -1
#define RESERVED_HASHED_COLUMN_ID MaxAttrNumber #define RESERVED_HASHED_COLUMN_ID MaxAttrNumber
#define MERGE_COLUMN_FORMAT "merge_column_%u"
#define MAP_OUTPUT_FETCH_COMMAND "SELECT worker_fetch_partition_file \
(" UINT64_FORMAT ", %u, %u, %u, '%s', %u)"
#define RANGE_PARTITION_COMMAND "SELECT worker_range_partition_table \
(" UINT64_FORMAT ", %d, %s, '%s', '%s'::regtype, %s)"
#define HASH_PARTITION_COMMAND "SELECT worker_hash_partition_table \
(" UINT64_FORMAT ", %d, %s, '%s', '%s'::regtype, %s)"
#define MERGE_FILES_INTO_TABLE_COMMAND "SELECT worker_merge_files_into_table \
(" UINT64_FORMAT ", %d, '%s', '%s')"
extern int RepartitionJoinBucketCountPerNode; extern int RepartitionJoinBucketCountPerNode;
@ -262,6 +253,10 @@ typedef struct Task
uint32 upstreamTaskId; /* only applies to data fetch tasks */ uint32 upstreamTaskId; /* only applies to data fetch tasks */
ShardInterval *shardInterval; /* only applies to merge tasks */ ShardInterval *shardInterval; /* only applies to merge tasks */
bool assignmentConstrained; /* only applies to merge tasks */ bool assignmentConstrained; /* only applies to merge tasks */
/* for merge tasks, this is set to the target list of the map task */
List *mapJobTargetList;
char replicationModel; /* only applies to modify tasks */ char replicationModel; /* only applies to modify tasks */
/* /*

View File

@ -13,8 +13,6 @@
#include "nodes/pg_list.h" #include "nodes/pg_list.h"
extern List * ExecuteDependentTasks(List *taskList, Job *topLevelJob); extern List * ExecuteDependentTasks(List *taskList, Job *topLevelJob);
extern void EnsureCompatibleLocalExecutionState(List *taskList);
extern void DoRepartitionCleanup(List *jobIds);
#endif /* REPARTITION_JOIN_EXECUTION_H */ #endif /* REPARTITION_JOIN_EXECUTION_H */

View File

@ -182,7 +182,8 @@ extern Datum init_rebalance_monitor(PG_FUNCTION_ARGS);
extern Datum finalize_rebalance_monitor(PG_FUNCTION_ARGS); extern Datum finalize_rebalance_monitor(PG_FUNCTION_ARGS);
extern Datum get_rebalance_progress(PG_FUNCTION_ARGS); extern Datum get_rebalance_progress(PG_FUNCTION_ARGS);
extern List * RebalancePlacementUpdates(List *workerNodeList, List *shardPlacementList, extern List * RebalancePlacementUpdates(List *workerNodeList,
List *shardPlacementListList,
double threshold, double threshold,
int32 maxShardMoves, int32 maxShardMoves,
bool drainOnly, bool drainOnly,

View File

@ -86,7 +86,7 @@ extern List * ActiveReadableNodeList(void);
extern WorkerNode * FindWorkerNode(const char *nodeName, int32 nodePort); extern WorkerNode * FindWorkerNode(const char *nodeName, int32 nodePort);
extern WorkerNode * FindWorkerNodeOrError(const char *nodeName, int32 nodePort); extern WorkerNode * FindWorkerNodeOrError(const char *nodeName, int32 nodePort);
extern WorkerNode * FindWorkerNodeAnyCluster(const char *nodeName, int32 nodePort); extern WorkerNode * FindWorkerNodeAnyCluster(const char *nodeName, int32 nodePort);
extern WorkerNode * FindNodeWithNodeId(int nodeId); extern WorkerNode * FindNodeWithNodeId(int nodeId, bool missingOk);
extern List * ReadDistNode(bool includeNodesFromOtherClusters); extern List * ReadDistNode(bool includeNodesFromOtherClusters);
extern void EnsureCoordinator(void); extern void EnsureCoordinator(void);
extern void InsertCoordinatorIfClusterEmpty(void); extern void InsertCoordinatorIfClusterEmpty(void);

View File

@ -4,3 +4,4 @@ test: upgrade_basic_after
test: upgrade_partition_constraints_after test: upgrade_partition_constraints_after
test: upgrade_pg_dist_object_test_after test: upgrade_pg_dist_object_test_after
test: upgrade_columnar_metapage_after test: upgrade_columnar_metapage_after
test: upgrade_post_11_after

View File

@ -4,3 +4,4 @@ test: upgrade_basic_before
test: upgrade_partition_constraints_before test: upgrade_partition_constraints_before
test: upgrade_pg_dist_object_test_before test: upgrade_pg_dist_object_test_before
test: upgrade_columnar_metapage_before test: upgrade_columnar_metapage_before
test: upgrade_post_11_before

View File

@ -19,12 +19,19 @@ test=$(basename "$file1" .out | sed -E "s/_[0-9]+$//")
args=${@:1:$#-2} args=${@:1:$#-2}
BASEDIR=$(dirname "$0") BASEDIR=$(dirname "$0")
# whereis searches for standard unix places before $PATH. So select the first DIFF=/usr/bin/diff
# entry as the original diff tool. if [ ! -f "$DIFF" ]
DIFF=$(whereis diff | sed "s/diff://g" | awk '{print $1}')
if [ -z "$DIFF" ]
then then
DIFF=/usr/bin/diff # whereis searches for standard unix places before $PATH. So select the
# first entry as the original diff tool.
# With the default WSL2 configuration whereis is very slow though ~400ms,
# so we only use it if /usr/bin/diff does not exist.
DIFF=$(whereis diff | sed "s/diff://g" | awk '{print $1}')
if [ -z "$DIFF" ]
then
echo "ERROR: could not find diff command" 1>&2
exit 1
fi
fi fi
if test -z "${VANILLATEST:-}" if test -z "${VANILLATEST:-}"

View File

@ -227,8 +227,7 @@ class CitusSmallSharedPoolSizeConfig(CitusMXBaseClusterConfig):
def __init__(self, arguments): def __init__(self, arguments):
super().__init__(arguments) super().__init__(arguments)
self.new_settings = { self.new_settings = {
# can be uncommented after https://github.com/citusdata/citus/issues/5342 "citus.local_shared_pool_size": 5,
# "citus.local_shared_pool_size": 5,
"citus.max_shared_pool_size": 5, "citus.max_shared_pool_size": 5,
} }

View File

@ -52,9 +52,12 @@ SELECT count(*) FROM (SELECT k.a FROM ab k, ab l WHERE k.a = l.b) first, (SELECT
ROLLBACK; ROLLBACK;
BEGIN; BEGIN;
INSERT INTO ab values(1, 2); INSERT INTO ab values(1, 2);
-- DDL happened before repartition query in a transaction block, so this should error.
SELECT count(*) FROM (SELECT k.a FROM ab k, ab l WHERE k.a = l.b) first, (SELECT * FROM ab) second WHERE first.a = second.b; SELECT count(*) FROM (SELECT k.a FROM ab k, ab l WHERE k.a = l.b) first, (SELECT * FROM ab) second WHERE first.a = second.b;
ERROR: cannot open new connections after the first modification command within a transaction count
---------------------------------------------------------------------
14
(1 row)
ROLLBACK; ROLLBACK;
SET citus.enable_single_hash_repartition_joins TO ON; SET citus.enable_single_hash_repartition_joins TO ON;
CREATE TABLE single_hash_repartition_first (id int, sum int, avg float); CREATE TABLE single_hash_repartition_first (id int, sum int, avg float);

View File

@ -282,6 +282,21 @@ create aggregate binstragg(text, text)(
combinefunc=binstragg_combinefunc, combinefunc=binstragg_combinefunc,
stype=text stype=text
); );
-- verify that the aggregate is added into pg_dist_object, on each worker
SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.binstragg'::regproc;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,1)
(localhost,57638,t,1)
(2 rows)
SELECT run_command_on_workers($$select count(*) from pg_aggregate where aggfnoid::text like '%binstragg%';$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,1)
(localhost,57638,t,1)
(2 rows)
select create_distributed_function('binstragg(text,text)'); select create_distributed_function('binstragg(text,text)');
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -549,7 +564,6 @@ SELECT id%5, first(val ORDER BY key), last(val ORDER BY key)
FROM aggdata GROUP BY id%5 ORDER BY id%5; FROM aggdata GROUP BY id%5 ORDER BY id%5;
ERROR: unsupported aggregate function first ERROR: unsupported aggregate function first
-- test aggregate with stype which is not a by-value datum -- test aggregate with stype which is not a by-value datum
-- also test our handling of the aggregate not existing on workers
create function sumstring_sfunc(state text, x text) create function sumstring_sfunc(state text, x text)
returns text immutable language plpgsql as $$ returns text immutable language plpgsql as $$
begin return (state::float8 + x::float8)::text; begin return (state::float8 + x::float8)::text;
@ -561,9 +575,20 @@ create aggregate sumstring(text) (
combinefunc = sumstring_sfunc, combinefunc = sumstring_sfunc,
initcond = '0' initcond = '0'
); );
select sumstring(valf::text) from aggdata where valf is not null; -- verify that the aggregate is propagated
ERROR: function "aggregate_support.sumstring(text)" does not exist select aggfnoid from pg_aggregate where aggfnoid::text like '%sumstring%';
CONTEXT: while executing command on localhost:xxxxx aggfnoid
---------------------------------------------------------------------
sumstring
(1 row)
SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%sumstring%';$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,aggregate_support.sumstring)
(localhost,57638,t,aggregate_support.sumstring)
(2 rows)
select create_distributed_function('sumstring(text)'); select create_distributed_function('sumstring(text)');
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -1042,5 +1067,122 @@ LEFT JOIN ref_table ON TRUE;
109 109
(1 row) (1 row)
-- try createing aggregate having non-distributable dependency type
create table dummy_tbl (a int);
create function dummy_fnc(a dummy_tbl, d double precision) RETURNS dummy_tbl
AS $$SELECT 1;$$ LANGUAGE sql;
WARNING: Citus can't distribute function "dummy_fnc" having dependency on non-distributed relation "dummy_tbl"
DETAIL: Function will be created only locally
HINT: To distribute function, distribute dependent relations first. Then, re-create the function
-- should give warning and create aggregate local only
create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc);
WARNING: Citus can't distribute aggregate "dependent_agg" having dependency on non-distributed relation "dummy_tbl"
DETAIL: Aggregate will be created only locally
HINT: To distribute aggregate, distribute dependent relations first. Then, re-create the aggregate
-- clear and try again with distributed table
DROP TABLE dummy_tbl CASCADE;
NOTICE: drop cascades to 2 other objects
DETAIL: drop cascades to function dummy_fnc(dummy_tbl,double precision)
drop cascades to function dependent_agg(double precision)
create table dummy_tbl (a int);
SELECT create_distributed_table('dummy_tbl','a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
create function dummy_fnc(a dummy_tbl, d double precision) RETURNS dummy_tbl
AS $$SELECT 1;$$ LANGUAGE sql;
-- test in tx block
-- shouldn't distribute, as citus.create_object_propagation is set to deferred
BEGIN;
create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc);
COMMIT;
-- verify not distributed
SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"")
(localhost,57638,t,"")
(2 rows)
drop aggregate dependent_agg ( double precision);
-- now try with create_object_propagation = immediate
SET citus.create_object_propagation TO immediate;
-- should distribute, as citus.create_object_propagation is set to immediate
-- will switch to sequential mode
BEGIN;
create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc);
COMMIT;
-- verify distributed
SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,aggregate_support.dependent_agg)
(localhost,57638,t,aggregate_support.dependent_agg)
(2 rows)
drop aggregate dependent_agg ( double precision);
-- now try with create_object_propagation = automatic
SET citus.create_object_propagation TO automatic;
-- should distribute, as citus.create_object_propagation is set to automatic
-- will switch to sequential mode
BEGIN;
create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc);
COMMIT;
-- verify distributed
SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,aggregate_support.dependent_agg)
(localhost,57638,t,aggregate_support.dependent_agg)
(2 rows)
-- verify that the aggregate is added into pg_dist_object, on each worker
SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,1)
(localhost,57638,t,1)
(2 rows)
RESET citus.create_object_propagation;
-- drop and test outside of tx block
drop aggregate dependent_agg (float8);
-- verify that the aggregate is removed from pg_dist_object, on each worker
SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,f,"ERROR: function ""aggregate_support.dependent_agg"" does not exist")
(localhost,57638,f,"ERROR: function ""aggregate_support.dependent_agg"" does not exist")
(2 rows)
create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc);
--verify
SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,aggregate_support.dependent_agg)
(localhost,57638,t,aggregate_support.dependent_agg)
(2 rows)
DROP TABLE dummy_tbl CASCADE;
NOTICE: drop cascades to 2 other objects
DETAIL: drop cascades to function dummy_fnc(dummy_tbl,double precision)
drop cascades to function dependent_agg(double precision)
SET citus.create_object_propagation TO automatic;
begin;
create type typ1 as (a int);
create or replace function fnagg(a typ1, d double precision) RETURNS typ1 AS $$SELECT 1;$$LANGUAGE sql;
create aggregate dependent_agg (float8) (stype=typ1, sfunc=fnagg);
commit;
RESET citus.create_object_propagation;
SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,aggregate_support.dependent_agg)
(localhost,57638,t,aggregate_support.dependent_agg)
(2 rows)
set client_min_messages to error; set client_min_messages to error;
drop schema aggregate_support cascade; drop schema aggregate_support cascade;

View File

@ -249,8 +249,30 @@ SELECT count(*) FROM test t1, test t2 WHERE t1.x = t2.y;
(1 row) (1 row)
BEGIN; BEGIN;
SET citus.enable_repartition_joins TO ON; SET citus.enable_unique_job_ids TO off;
SELECT count(*) FROM test t1, test t2 WHERE t1.x = t2.y; SELECT count(*) FROM test t1, test t2 WHERE t1.x = t2.y;
NOTICE: executing the command locally: SELECT partition_index, 'repartition_25_1' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_25_1','SELECT x AS column1 FROM coordinator_shouldhaveshards.test_1503000 t1 WHERE true',0,'hash','{-2147483648,-1431655766,-715827884,-2,715827880,1431655762}'::text[],'{-1431655767,-715827885,-3,715827879,1431655761,2147483647}'::text[],true,true,true) WHERE rows_written > 0
NOTICE: executing the command locally: SELECT partition_index, 'repartition_25_4' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_25_4','SELECT x AS column1 FROM coordinator_shouldhaveshards.test_1503003 t1 WHERE true',0,'hash','{-2147483648,-1431655766,-715827884,-2,715827880,1431655762}'::text[],'{-1431655767,-715827885,-3,715827879,1431655761,2147483647}'::text[],true,true,true) WHERE rows_written > 0
NOTICE: executing the command locally: SELECT partition_index, 'repartition_26_1' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_26_1','SELECT y AS column1 FROM coordinator_shouldhaveshards.test_1503000 t2 WHERE true',0,'hash','{-2147483648,-1431655766,-715827884,-2,715827880,1431655762}'::text[],'{-1431655767,-715827885,-3,715827879,1431655761,2147483647}'::text[],true,true,true) WHERE rows_written > 0
NOTICE: executing the command locally: SELECT partition_index, 'repartition_26_4' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_26_4','SELECT y AS column1 FROM coordinator_shouldhaveshards.test_1503003 t2 WHERE true',0,'hash','{-2147483648,-1431655766,-715827884,-2,715827880,1431655762}'::text[],'{-1431655767,-715827885,-3,715827879,1431655761,2147483647}'::text[],true,true,true) WHERE rows_written > 0
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_25_1_0']::text[],'localhost',57636) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_25_2_0']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_25_3_0']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_25_4_0']::text[],'localhost',57636) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_26_1_0']::text[],'localhost',57636) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_26_2_0']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_26_3_0']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_26_4_0']::text[],'localhost',57636) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_25_1_3']::text[],'localhost',57636) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_25_2_3']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_25_3_3']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_25_4_3']::text[],'localhost',57636) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_26_1_3']::text[],'localhost',57636) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_26_2_3']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_26_3_3']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_26_4_3']::text[],'localhost',57636) bytes
NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_25_1_0,repartition_25_2_0,repartition_25_3_0,repartition_25_4_0}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 integer) JOIN read_intermediate_results('{repartition_26_1_0,repartition_26_2_0,repartition_26_3_0,repartition_26_4_0}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 integer) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true
NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_25_1_3,repartition_25_2_3,repartition_25_3_3,repartition_25_4_3}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 integer) JOIN read_intermediate_results('{repartition_26_1_3,repartition_26_2_3,repartition_26_3_3,repartition_26_4_3}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 integer) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
100 100
@ -268,9 +290,33 @@ NOTICE: executing the command locally: SELECT y FROM coordinator_shouldhaveshar
(1 row) (1 row)
SELECT count(*) FROM test t1, test t2 WHERE t1.x = t2.y; SELECT count(*) FROM test t1, test t2 WHERE t1.x = t2.y;
ERROR: cannot execute command because a local execution has accessed a placement in the transaction NOTICE: executing the command locally: SELECT partition_index, 'repartition_29_1' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_29_1','SELECT x AS column1 FROM coordinator_shouldhaveshards.test_1503000 t1 WHERE true',0,'hash','{-2147483648,-1431655766,-715827884,-2,715827880,1431655762}'::text[],'{-1431655767,-715827885,-3,715827879,1431655761,2147483647}'::text[],true,true,true) WHERE rows_written > 0
DETAIL: Some parallel commands cannot be executed if a previous command has already been executed locally NOTICE: executing the command locally: SELECT partition_index, 'repartition_29_4' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_29_4','SELECT x AS column1 FROM coordinator_shouldhaveshards.test_1503003 t1 WHERE true',0,'hash','{-2147483648,-1431655766,-715827884,-2,715827880,1431655762}'::text[],'{-1431655767,-715827885,-3,715827879,1431655761,2147483647}'::text[],true,true,true) WHERE rows_written > 0
HINT: Try re-running the transaction with "SET LOCAL citus.enable_local_execution TO OFF;" NOTICE: executing the command locally: SELECT partition_index, 'repartition_30_1' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_30_1','SELECT y AS column1 FROM coordinator_shouldhaveshards.test_1503000 t2 WHERE true',0,'hash','{-2147483648,-1431655766,-715827884,-2,715827880,1431655762}'::text[],'{-1431655767,-715827885,-3,715827879,1431655761,2147483647}'::text[],true,true,true) WHERE rows_written > 0
NOTICE: executing the command locally: SELECT partition_index, 'repartition_30_4' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_30_4','SELECT y AS column1 FROM coordinator_shouldhaveshards.test_1503003 t2 WHERE true',0,'hash','{-2147483648,-1431655766,-715827884,-2,715827880,1431655762}'::text[],'{-1431655767,-715827885,-3,715827879,1431655761,2147483647}'::text[],true,true,true) WHERE rows_written > 0
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_29_1_2']::text[],'localhost',57636) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_29_2_2']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_29_3_2']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_29_4_2']::text[],'localhost',57636) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_30_1_2']::text[],'localhost',57636) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_30_2_2']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_30_3_2']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_30_4_2']::text[],'localhost',57636) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_29_1_5']::text[],'localhost',57636) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_29_2_5']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_29_3_5']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_29_4_5']::text[],'localhost',57636) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_30_1_5']::text[],'localhost',57636) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_30_2_5']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_30_3_5']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_30_4_5']::text[],'localhost',57636) bytes
NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_29_1_2,repartition_29_2_2,repartition_29_3_2,repartition_29_4_2}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 integer) JOIN read_intermediate_results('{repartition_30_1_2,repartition_30_2_2,repartition_30_3_2,repartition_30_4_2}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 integer) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true
NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_29_1_5,repartition_29_2_5,repartition_29_3_5,repartition_29_4_5}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 integer) JOIN read_intermediate_results('{repartition_30_1_5,repartition_30_2_5,repartition_30_3_5,repartition_30_4_5}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 integer) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true
count
---------------------------------------------------------------------
100
(1 row)
ROLLBACK; ROLLBACK;
CREATE TABLE ref (a int, b int); CREATE TABLE ref (a int, b int);
SELECT create_reference_table('ref'); SELECT create_reference_table('ref');

View File

@ -386,7 +386,7 @@ ORDER BY shard_key_value, object_id, change_id;
-- Triggers (tables) which are not colocated -- Triggers (tables) which are not colocated
-- --
CREATE TABLE emptest ( CREATE TABLE emptest (
empname text NOT NULL, empname text NOT NULL PRIMARY KEY,
salary integer salary integer
); );
CREATE TABLE emptest_audit( CREATE TABLE emptest_audit(
@ -394,7 +394,8 @@ CREATE TABLE emptest_audit(
stamp timestamp NOT NULL, stamp timestamp NOT NULL,
userid text NOT NULL, userid text NOT NULL,
empname text NOT NULL, empname text NOT NULL,
salary integer salary integer,
PRIMARY KEY (empname, userid, stamp, operation, salary)
); );
SELECT create_distributed_table('emptest','empname',colocate_with :='none'); SELECT create_distributed_table('emptest','empname',colocate_with :='none');
create_distributed_table create_distributed_table
@ -477,6 +478,7 @@ CREATE TABLE record_op (
operation_type text not null, operation_type text not null,
stamp timestamp NOT NULL stamp timestamp NOT NULL
); );
ALTER TABLE record_op REPLICA IDENTITY FULL;
SELECT create_distributed_table('record_op', 'empname', colocate_with := 'emptest'); SELECT create_distributed_table('record_op', 'empname', colocate_with := 'emptest');
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -654,7 +656,7 @@ PARTITION BY list (state_code);
ALTER TABLE sale ADD CONSTRAINT sale_pk PRIMARY KEY (state_code, sale_date); 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 sale_newyork PARTITION OF sale FOR VALUES IN ('NY');
CREATE TABLE sale_california PARTITION OF sale FOR VALUES IN ('CA'); CREATE TABLE sale_california PARTITION OF sale FOR VALUES IN ('CA');
CREATE TABLE record_sale(operation_type text not null, product_sku text, state_code text); 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'); SELECT create_distributed_table('sale', 'state_code');
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -671,8 +673,8 @@ CREATE OR REPLACE FUNCTION record_sale()
RETURNS trigger RETURNS trigger
AS $$ AS $$
BEGIN BEGIN
INSERT INTO distributed_triggers.record_sale(operation_type, product_sku, state_code) INSERT INTO distributed_triggers.record_sale(operation_type, product_sku, state_code, units)
VALUES (TG_OP, NEW.product_sku, NEW.state_code); VALUES (TG_OP, NEW.product_sku, NEW.state_code, NEW.units);
RETURN NULL; RETURN NULL;
END; END;
$$ LANGUAGE plpgsql; $$ LANGUAGE plpgsql;
@ -697,7 +699,7 @@ TABLE sale ORDER BY state_code, sale_date;
02-03-2019 | NY | AZ-000A1 | 47 02-03-2019 | NY | AZ-000A1 | 47
(6 rows) (6 rows)
TABLE record_sale ORDER BY 1,2,3; SELECT operation_type, product_sku, state_code FROM record_sale ORDER BY 1,2,3;
operation_type | product_sku | state_code operation_type | product_sku | state_code
--------------------------------------------------------------------- ---------------------------------------------------------------------
INSERT | AZ-000A1 | CA INSERT | AZ-000A1 | CA

View File

@ -453,6 +453,107 @@ SELECT * FROM field_indirection_test_2 ORDER BY 1,2,3;
8 | (10," text10",20) | (40,50) 8 | (10," text10",20) | (40,50)
(2 rows) (2 rows)
-- test different ddl propagation modes
SET citus.create_object_propagation TO deferred;
BEGIN;
CREATE TYPE deferred_type AS (a int);
SHOW citus.multi_shard_modify_mode;
citus.multi_shard_modify_mode
---------------------------------------------------------------------
parallel
(1 row)
CREATE TABLE deferred_table(a int,b deferred_type);
SELECT create_distributed_table('deferred_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SHOW citus.multi_shard_modify_mode;
citus.multi_shard_modify_mode
---------------------------------------------------------------------
parallel
(1 row)
COMMIT;
SET citus.create_object_propagation TO automatic;
BEGIN;
CREATE TYPE automatic_type AS (a int);
SHOW citus.multi_shard_modify_mode;
citus.multi_shard_modify_mode
---------------------------------------------------------------------
sequential
(1 row)
CREATE TABLE automatic_table(a int,b automatic_type);
SELECT create_distributed_table('automatic_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SHOW citus.multi_shard_modify_mode;
citus.multi_shard_modify_mode
---------------------------------------------------------------------
sequential
(1 row)
COMMIT;
SET citus.create_object_propagation TO automatic;
BEGIN;
-- force parallel execution by preceding with a analytical query
SET LOCAL citus.force_max_query_parallelization TO on;
SELECT count(*) FROM automatic_table;
count
---------------------------------------------------------------------
0
(1 row)
CREATE TYPE automatic2_type AS (a int);
SHOW citus.multi_shard_modify_mode;
citus.multi_shard_modify_mode
---------------------------------------------------------------------
parallel
(1 row)
CREATE TABLE automatic2_table(a int,b automatic2_type);
SELECT create_distributed_table('automatic2_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SHOW citus.multi_shard_modify_mode;
citus.multi_shard_modify_mode
---------------------------------------------------------------------
parallel
(1 row)
COMMIT;
SET citus.create_object_propagation TO immediate;
BEGIN;
CREATE TYPE immediate_type AS (a int);
SHOW citus.multi_shard_modify_mode;
citus.multi_shard_modify_mode
---------------------------------------------------------------------
sequential
(1 row)
CREATE TABLE immediate_table(a int,b immediate_type);
SELECT create_distributed_table('immediate_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SHOW citus.multi_shard_modify_mode;
citus.multi_shard_modify_mode
---------------------------------------------------------------------
sequential
(1 row)
COMMIT;
-- clear objects -- clear objects
SET client_min_messages TO error; -- suppress cascading objects dropping SET client_min_messages TO error; -- suppress cascading objects dropping
DROP SCHEMA type_tests CASCADE; DROP SCHEMA type_tests CASCADE;

View File

@ -157,25 +157,50 @@ SELECT * FROM ref, local WHERE a = c ORDER BY a;
SET citus.enable_repartition_joins TO ON; SET citus.enable_repartition_joins TO ON;
SELECT * FROM test t1, test t2 WHERE t1.x = t2.y ORDER BY t1.x; SELECT * FROM test t1, test t2 WHERE t1.x = t2.y ORDER BY t1.x;
ERROR: writing to worker nodes is not currently allowed x | y | x | y
DETAIL: the database is read-only ---------------------------------------------------------------------
2 | 7 | 1 | 2
4 | 5 | 3 | 4
5 | 6 | 4 | 5
(3 rows)
SET citus.enable_single_hash_repartition_joins TO ON; SET citus.enable_single_hash_repartition_joins TO ON;
SELECT * FROM test t1, test t2 WHERE t1.x = t2.y ORDER BY t1.x; SELECT * FROM test t1, test t2 WHERE t1.x = t2.y ORDER BY t1.x;
ERROR: writing to worker nodes is not currently allowed x | y | x | y
DETAIL: the database is read-only ---------------------------------------------------------------------
2 | 7 | 1 | 2
4 | 5 | 3 | 4
5 | 6 | 4 | 5
(3 rows)
SET citus.task_assignment_policy TO 'round-robin'; SET citus.task_assignment_policy TO 'round-robin';
SET citus.enable_single_hash_repartition_joins TO ON; SET citus.enable_single_hash_repartition_joins TO ON;
SELECT * FROM test t1, test t2 WHERE t1.x = t2.y ORDER BY t1.x; SELECT * FROM test t1, test t2 WHERE t1.x = t2.y ORDER BY t1.x;
ERROR: writing to worker nodes is not currently allowed x | y | x | y
DETAIL: the database is read-only ---------------------------------------------------------------------
2 | 7 | 1 | 2
4 | 5 | 3 | 4
5 | 6 | 4 | 5
(3 rows)
SET citus.task_assignment_policy TO 'greedy'; SET citus.task_assignment_policy TO 'greedy';
SELECT * FROM test t1, test t2 WHERE t1.x = t2.y ORDER BY t1.x; SELECT * FROM test t1, test t2 WHERE t1.x = t2.y ORDER BY t1.x;
ERROR: writing to worker nodes is not currently allowed x | y | x | y
DETAIL: the database is read-only ---------------------------------------------------------------------
2 | 7 | 1 | 2
4 | 5 | 3 | 4
5 | 6 | 4 | 5
(3 rows)
SET citus.task_assignment_policy TO 'first-replica'; SET citus.task_assignment_policy TO 'first-replica';
SELECT * FROM test t1, test t2 WHERE t1.x = t2.y ORDER BY t1.x; SELECT * FROM test t1, test t2 WHERE t1.x = t2.y ORDER BY t1.x;
ERROR: writing to worker nodes is not currently allowed x | y | x | y
DETAIL: the database is read-only ---------------------------------------------------------------------
2 | 7 | 1 | 2
4 | 5 | 3 | 4
5 | 6 | 4 | 5
(3 rows)
RESET citus.enable_repartition_joins; RESET citus.enable_repartition_joins;
RESET citus.enable_single_hash_repartition_joins; RESET citus.enable_single_hash_repartition_joins;
-- Confirm that dummy placements work -- Confirm that dummy placements work
@ -319,12 +344,22 @@ SELECT * FROM ref, local WHERE a = c ORDER BY a;
SET citus.enable_repartition_joins TO ON; SET citus.enable_repartition_joins TO ON;
SELECT * FROM test t1, test t2 WHERE t1.x = t2.y ORDER BY t1.x; SELECT * FROM test t1, test t2 WHERE t1.x = t2.y ORDER BY t1.x;
ERROR: writing to worker nodes is not currently allowed x | y | x | y
DETAIL: the database is read-only ---------------------------------------------------------------------
2 | 7 | 1 | 2
4 | 5 | 3 | 4
5 | 6 | 4 | 5
(3 rows)
SET citus.enable_single_hash_repartition_joins TO ON; SET citus.enable_single_hash_repartition_joins TO ON;
SELECT * FROM test t1, test t2 WHERE t1.x = t2.y ORDER BY t1.x; SELECT * FROM test t1, test t2 WHERE t1.x = t2.y ORDER BY t1.x;
ERROR: writing to worker nodes is not currently allowed x | y | x | y
DETAIL: the database is read-only ---------------------------------------------------------------------
2 | 7 | 1 | 2
4 | 5 | 3 | 4
5 | 6 | 4 | 5
(3 rows)
RESET citus.enable_repartition_joins; RESET citus.enable_repartition_joins;
RESET citus.enable_single_hash_repartition_joins; RESET citus.enable_single_hash_repartition_joins;
-- Confirm that dummy placements work -- Confirm that dummy placements work

View File

@ -316,6 +316,710 @@ BEGIN
return 1; return 1;
END; END;
$$; $$;
-- Show that functions are propagated (or not) as a dependency
-- Function as a default column
BEGIN;
CREATE OR REPLACE FUNCTION func_in_transaction_def()
RETURNS int
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
-- Function shouldn't be propagated within transaction
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(0 rows)
CREATE TABLE table_to_prop_func(id int, col_1 int default func_in_transaction_def());
SELECT create_distributed_table('table_to_prop_func','id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- Function should be marked as distributed after distributing the table that depends on it
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(function,"{function_propagation_schema,func_in_transaction_def}",{})
(1 row)
COMMIT;
-- Function should be marked as distributed on the worker after committing changes
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid;$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_def}",{})
localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_def}",{})
(2 rows)
-- Multiple functions as a default column
BEGIN;
CREATE OR REPLACE FUNCTION func_in_transaction_1()
RETURNS int
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
CREATE OR REPLACE FUNCTION func_in_transaction_2()
RETURNS int
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
-- Functions shouldn't be propagated within transaction
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(0 rows)
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(0 rows)
CREATE TABLE table_to_prop_func_2(id int, col_1 int default func_in_transaction_1() + func_in_transaction_2());
SELECT create_distributed_table('table_to_prop_func_2','id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- Functions should be marked as distribued after distributing the table that depends on it
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(function,"{function_propagation_schema,func_in_transaction_1}",{})
(1 row)
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(function,"{function_propagation_schema,func_in_transaction_2}",{})
(1 row)
COMMIT;
-- Functions should be marked as distributed on the worker after committing changes
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid;$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_1}",{})
localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_1}",{})
(2 rows)
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid;$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_2}",{})
localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_2}",{})
(2 rows)
-- If function has dependency on non-distributed table it should error out
BEGIN;
CREATE TABLE non_dist_table(id int);
CREATE OR REPLACE FUNCTION func_in_transaction_3(param_1 non_dist_table)
RETURNS int
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
CREATE TABLE table_to_prop_func_3(id int, col_1 int default func_in_transaction_3(NULL::non_dist_table));
-- It should error out as there is a non-distributed table dependency
SELECT create_distributed_table('table_to_prop_func_3','id');
ERROR: type function_propagation_schema.non_dist_table does not exist
CONTEXT: while executing command on localhost:xxxxx
COMMIT;
-- Adding a column with default value should propagate the function
BEGIN;
CREATE TABLE table_to_prop_func_4(id int);
SELECT create_distributed_table('table_to_prop_func_4', 'id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE OR REPLACE FUNCTION func_in_transaction_4()
RETURNS int
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
-- Function shouldn't be propagated within transaction
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(0 rows)
ALTER TABLE table_to_prop_func_4 ADD COLUMN col_1 int default function_propagation_schema.func_in_transaction_4();
-- Function should be marked as distributed after adding the column
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(function,"{function_propagation_schema,func_in_transaction_4}",{})
(1 row)
COMMIT;
-- Functions should be marked as distributed on the worker after committing changes
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid;$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_4}",{})
localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_4}",{})
(2 rows)
-- Adding multiple columns with default values should propagate the function
BEGIN;
CREATE OR REPLACE FUNCTION func_in_transaction_5()
RETURNS int
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
CREATE OR REPLACE FUNCTION func_in_transaction_6()
RETURNS int
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
-- Functions shouldn't be propagated within transaction
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(0 rows)
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(0 rows)
CREATE TABLE table_to_prop_func_5(id int, col_1 int default func_in_transaction_5(), col_2 int default func_in_transaction_6());
SELECT create_distributed_table('table_to_prop_func_5', 'id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- Functions should be marked as distributed after adding the column
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(function,"{function_propagation_schema,func_in_transaction_5}",{})
(1 row)
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(function,"{function_propagation_schema,func_in_transaction_6}",{})
(1 row)
COMMIT;
-- Functions should be marked as distributed on the worker after committing changes
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid;$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_5}",{})
localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_5}",{})
(2 rows)
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid;$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_6}",{})
localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_6}",{})
(2 rows)
-- Adding a constraint with function check should propagate the function
BEGIN;
CREATE OR REPLACE FUNCTION func_in_transaction_7(param_1 int)
RETURNS boolean
LANGUAGE plpgsql AS
$$
BEGIN
return param_1 > 5;
END;
$$;
-- Functions shouldn't be propagated within transaction
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(0 rows)
CREATE TABLE table_to_prop_func_6(id int, col_1 int check (function_propagation_schema.func_in_transaction_7(col_1)));
SELECT create_distributed_table('table_to_prop_func_6', 'id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- Function should be marked as distributed after adding the column
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(function,"{function_propagation_schema,func_in_transaction_7}",{integer})
(1 row)
COMMIT;
-- Function should be marked as distributed on the worker after committing changes
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid;$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_7}",{integer})
localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_7}",{integer})
(2 rows)
-- Adding a constraint with multiple functions check should propagate the function
BEGIN;
CREATE OR REPLACE FUNCTION func_in_transaction_8(param_1 int)
RETURNS boolean
LANGUAGE plpgsql AS
$$
BEGIN
return param_1 > 5;
END;
$$;
CREATE OR REPLACE FUNCTION func_in_transaction_9(param_1 int)
RETURNS boolean
LANGUAGE plpgsql AS
$$
BEGIN
return param_1 > 5;
END;
$$;
-- Functions shouldn't be propagated within transaction
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(0 rows)
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(0 rows)
CREATE TABLE table_to_prop_func_7(id int, col_1 int check (function_propagation_schema.func_in_transaction_8(col_1) and function_propagation_schema.func_in_transaction_9(col_1)));
SELECT create_distributed_table('table_to_prop_func_7', 'id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- Function should be marked as distributed after adding the column
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(function,"{function_propagation_schema,func_in_transaction_8}",{integer})
(1 row)
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(function,"{function_propagation_schema,func_in_transaction_9}",{integer})
(1 row)
COMMIT;
-- Functions should be marked as distributed on the worker after committing changes
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid;$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_8}",{integer})
localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_8}",{integer})
(2 rows)
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid;$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_9}",{integer})
localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_9}",{integer})
(2 rows)
-- Adding a column with constraint should propagate the function
BEGIN;
CREATE TABLE table_to_prop_func_8(id int, col_1 int);
SELECT create_distributed_table('table_to_prop_func_8', 'id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE OR REPLACE FUNCTION func_in_transaction_10(param_1 int)
RETURNS boolean
LANGUAGE plpgsql AS
$$
BEGIN
return param_1 > 5;
END;
$$;
-- Functions shouldn't be propagated within transaction
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(0 rows)
ALTER TABLE table_to_prop_func_8 ADD CONSTRAINT col1_check CHECK (function_propagation_schema.func_in_transaction_10(col_1));
-- Function should be marked as distributed after adding the constraint
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(function,"{function_propagation_schema,func_in_transaction_10}",{integer})
(1 row)
COMMIT;
-- Function should be marked as distributed on the worker after committing changes
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid;$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_10}",{integer})
localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_10}",{integer})
(2 rows)
-- If constraint depends on a non-distributed table it should error out
BEGIN;
CREATE TABLE local_table_for_const(id int);
CREATE OR REPLACE FUNCTION func_in_transaction_11(param_1 int, param_2 local_table_for_const)
RETURNS boolean
LANGUAGE plpgsql AS
$$
BEGIN
return param_1 > 5;
END;
$$;
CREATE TABLE table_to_prop_func_9(id int, col_1 int check (func_in_transaction_11(col_1, NULL::local_table_for_const)));
-- It should error out since there is non-distributed table dependency exists
SELECT create_distributed_table('table_to_prop_func_9', 'id');
ERROR: type function_propagation_schema.local_table_for_const does not exist
CONTEXT: while executing command on localhost:xxxxx
COMMIT;
-- Show that function as a part of generated always is supporte
BEGIN;
CREATE OR REPLACE FUNCTION non_sense_func_for_generated_always()
RETURNS int
LANGUAGE plpgsql IMMUTABLE AS
$$
BEGIN
return 1;
END;
$$;
-- Functions shouldn't be propagated within transaction
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_generated_always'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(0 rows)
CREATE TABLE people (
id int,
height_cm numeric,
height_in numeric GENERATED ALWAYS AS (height_cm / non_sense_func_for_generated_always()) STORED);
SELECT create_distributed_table('people', 'id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- Show that function is distributed after distributing the table
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_generated_always'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(function,"{function_propagation_schema,non_sense_func_for_generated_always}",{})
(1 row)
COMMIT;
-- Show that functions depending table via rule are also distributed
BEGIN;
CREATE OR REPLACE FUNCTION func_for_rule()
RETURNS int
LANGUAGE plpgsql STABLE AS
$$
BEGIN
return 4;
END;
$$;
-- Functions shouldn't be propagated within transaction
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(0 rows)
CREATE TABLE table_1_for_rule(id int, col_1 int);
CREATE TABLE table_2_for_rule(id int, col_1 int);
CREATE RULE rule_1 AS ON UPDATE TO table_1_for_rule DO ALSO UPDATE table_2_for_rule SET col_1 = col_1 * func_for_rule();
SELECT create_distributed_table('table_1_for_rule','id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- Functions should be distributed after distributing the table
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(function,"{function_propagation_schema,func_for_rule}",{})
(1 row)
COMMIT;
-- Function should be marked as distributed on the worker after committing changes
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid;$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (function,"{function_propagation_schema,func_for_rule}",{})
localhost | 57638 | t | (function,"{function_propagation_schema,func_for_rule}",{})
(2 rows)
-- Show that functions as partitioning functions are supported
BEGIN;
CREATE OR REPLACE FUNCTION non_sense_func_for_partitioning(int)
RETURNS int
LANGUAGE plpgsql IMMUTABLE AS
$$
BEGIN
return 1;
END;
$$;
-- Functions shouldn't be propagated within transaction
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(0 rows)
CREATE TABLE partitioned_table_to_test_func_prop(id INT, a INT) PARTITION BY RANGE (non_sense_func_for_partitioning(id));
SELECT create_distributed_table('partitioned_table_to_test_func_prop', 'id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- Show that function is distributed after distributing the table
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(function,"{function_propagation_schema,non_sense_func_for_partitioning}",{integer})
(1 row)
COMMIT;
-- Function should be marked as distributed on the worker after committing changes
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid;$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (function,"{function_propagation_schema,non_sense_func_for_partitioning}",{integer})
localhost | 57638 | t | (function,"{function_propagation_schema,non_sense_func_for_partitioning}",{integer})
(2 rows)
-- Test function dependency on citus local table
BEGIN;
CREATE OR REPLACE FUNCTION func_in_transaction_for_local_table()
RETURNS int
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
-- Function shouldn't be propagated within transaction
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_for_local_table'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(0 rows)
CREATE TABLE citus_local_table_to_test_func(l1 int DEFAULT func_in_transaction_for_local_table());
SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0);
NOTICE: localhost:xxxxx is the coordinator and already contains metadata, skipping syncing the metadata
?column?
---------------------------------------------------------------------
1
(1 row)
SELECT citus_add_local_table_to_metadata('citus_local_table_to_test_func');
citus_add_local_table_to_metadata
---------------------------------------------------------------------
(1 row)
-- Function should be marked as distributed after distributing the table that depends on it
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_for_local_table'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(function,"{function_propagation_schema,func_in_transaction_for_local_table}",{})
(1 row)
ROLLBACK;
-- Show that having a function dependency on exlude also works
BEGIN;
CREATE OR REPLACE FUNCTION exclude_bool_func()
RETURNS boolean
LANGUAGE plpgsql IMMUTABLE AS
$$
BEGIN
return true;
END;
$$;
-- Functions shouldn't be propagated within transaction
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(0 rows)
CREATE TABLE exclusion_func_prop_table (id int, EXCLUDE USING btree (id WITH =) WHERE (exclude_bool_func()));
SELECT create_distributed_table('exclusion_func_prop_table', 'id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- Function should be marked as distributed after distributing the table that depends on it
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(function,"{function_propagation_schema,exclude_bool_func}",{})
(1 row)
COMMIT;
-- Function should be marked as distributed on the worker after committing changes
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid;$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (function,"{function_propagation_schema,exclude_bool_func}",{})
localhost | 57638 | t | (function,"{function_propagation_schema,exclude_bool_func}",{})
(2 rows)
-- Show that having a function dependency for index also works
BEGIN;
CREATE OR REPLACE FUNCTION func_for_index_predicate(col_1 int)
RETURNS boolean
LANGUAGE plpgsql IMMUTABLE AS
$$
BEGIN
return col_1 > 5;
END;
$$;
-- Functions shouldn't be propagated within transaction
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(0 rows)
CREATE TABLE table_to_check_func_index_dep (id int, col_2 int);
CREATE INDEX on table_to_check_func_index_dep(col_2) WHERE (func_for_index_predicate(col_2));
SELECT create_distributed_table('table_to_check_func_index_dep', 'id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- Function should be marked as distributed after distributing the table that depends on it
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(function,"{function_propagation_schema,func_for_index_predicate}",{integer})
(1 row)
COMMIT;
-- Function should be marked as distributed on the worker after committing changes
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid;$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (function,"{function_propagation_schema,func_for_index_predicate}",{integer})
localhost | 57638 | t | (function,"{function_propagation_schema,func_for_index_predicate}",{integer})
(2 rows)
-- Test function to function dependency
BEGIN;
CREATE OR REPLACE FUNCTION func_for_func_dep_1()
RETURNS int
LANGUAGE plpgsql IMMUTABLE AS
$$
BEGIN
return 5;
END;
$$;
CREATE TABLE func_dep_table(a int, b int default func_for_func_dep_1());
CREATE OR REPLACE FUNCTION func_for_func_dep_2(col_1 func_dep_table)
RETURNS int
LANGUAGE plpgsql IMMUTABLE AS
$$
BEGIN
return 5;
END;
$$;
SELECT create_distributed_table('func_dep_table', 'a');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- Function should be marked as distributed after distributing the table that depends on it
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_func_dep_1'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(function,"{function_propagation_schema,func_for_func_dep_1}",{})
(1 row)
COMMIT;
-- Function should be marked as distributed on the worker after committing changes
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_func_dep_1'::regproc::oid;$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (function,"{function_propagation_schema,func_for_func_dep_1}",{})
localhost | 57638 | t | (function,"{function_propagation_schema,func_for_func_dep_1}",{})
(2 rows)
-- Test function with SQL language and sequence dependency
BEGIN;
CREATE OR REPLACE FUNCTION func_in_transaction_def_with_seq(val bigint)
RETURNS bigint
LANGUAGE SQL AS
$$
SELECT 2 * val;
$$;
CREATE OR REPLACE FUNCTION func_in_transaction_def_with_func(val bigint)
RETURNS bigint
LANGUAGE SQL AS
$$
SELECT func_in_transaction_def_with_seq(val);
$$;
-- Function shouldn't be propagated within transaction
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(0 rows)
CREATE SEQUENCE myseq;
CREATE TABLE table_to_prop_seq_func(id int, col_1 bigint default func_in_transaction_def_with_func(func_in_transaction_def_with_seq(nextval('myseq'))));
SELECT create_distributed_table('table_to_prop_seq_func','id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- Function should be marked as distributed after distributing the table that depends on it
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid;
pg_identify_object_as_address
---------------------------------------------------------------------
(function,"{function_propagation_schema,func_in_transaction_def_with_seq}",{bigint})
(1 row)
COMMIT;
-- Function should be marked as distributed on the worker after committing changes
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid;$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_def_with_seq}",{bigint})
localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_def_with_seq}",{bigint})
(2 rows)
RESET search_path; RESET search_path;
SET client_min_messages TO WARNING; SET client_min_messages TO WARNING;
DROP SCHEMA function_propagation_schema CASCADE; DROP SCHEMA function_propagation_schema CASCADE;

View File

@ -251,9 +251,6 @@ SELECT * FROM squares ORDER BY x;
5 | 25 5 | 25
(5 rows) (5 rows)
-- empty shard interval array should raise error
SELECT worker_hash_partition_table(42,1,'SELECT a FROM generate_series(1,100) AS a', 'a', 23, ARRAY[0]);
ERROR: invalid distribution column value
-- cannot use DDL commands -- cannot use DDL commands
select broadcast_intermediate_result('a', 'create table foo(int serial)'); select broadcast_intermediate_result('a', 'create table foo(int serial)');
ERROR: cannot execute utility commands ERROR: cannot execute utility commands
@ -507,10 +504,9 @@ WARNING: Query could not find the intermediate result file "squares_2", it was
(0 rows) (0 rows)
ROLLBACK TO SAVEPOINT s1; ROLLBACK TO SAVEPOINT s1;
-- fetch from worker 2 should fail -- fetch from invalid worker port should fail
SELECT * FROM fetch_intermediate_results(ARRAY['squares_1', 'squares_2']::text[], 'localhost', :worker_2_port); SELECT * FROM fetch_intermediate_results(ARRAY['squares_1', 'squares_2']::text[], 'localhost', 57635);
ERROR: could not open file "base/pgsql_job_cache/xx_x_xxx/squares_1.data": No such file or directory ERROR: cannot connect to localhost:xxxxx to fetch intermediate results
CONTEXT: while executing command on localhost:xxxxx
ROLLBACK TO SAVEPOINT s1; ROLLBACK TO SAVEPOINT s1;
-- still, results aren't available on coordinator yet -- still, results aren't available on coordinator yet
SELECT * FROM read_intermediate_results(ARRAY['squares_1', 'squares_2']::text[], 'binary') AS res (x int, x2 int); SELECT * FROM read_intermediate_results(ARRAY['squares_1', 'squares_2']::text[], 'binary') AS res (x int, x2 int);

View File

@ -42,7 +42,7 @@ query |query_hostname |query_hostport|d
(1 row) (1 row)
step s3-view-worker: step s3-view-worker:
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC; SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND backend_type = 'client backend' ORDER BY query DESC;
query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -112,7 +112,7 @@ query |query_hostname |query_hostport|di
(1 row) (1 row)
step s3-view-worker: step s3-view-worker:
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC; SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND backend_type = 'client backend' ORDER BY query DESC;
query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -176,7 +176,7 @@ query |query_hostname |query_hostport|distribute
(1 row) (1 row)
step s3-view-worker: step s3-view-worker:
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC; SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND backend_type = 'client backend' ORDER BY query DESC;
query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -243,7 +243,7 @@ query |query_hostname |query_
(1 row) (1 row)
step s3-view-worker: step s3-view-worker:
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC; SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND backend_type = 'client backend' ORDER BY query DESC;
query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -89,11 +89,11 @@ step s1-insert:
step s1-verify-current-xact-is-on-worker: step s1-verify-current-xact-is-on-worker:
SELECT SELECT
remote.nodeport, remote.nodeport,
remote.result = row(xact.initiator_node_identifier, xact.transaction_number)::text AS xact_exists remote.result = row(xact.transaction_number)::text AS xact_exists
FROM FROM
get_current_transaction_id() as xact, get_current_transaction_id() as xact,
run_command_on_workers($$ run_command_on_workers($$
SELECT row(initiator_node_identifier, transaction_number) SELECT row(transaction_number)
FROM get_all_active_transactions() FROM get_all_active_transactions()
WHERE transaction_number != 0; WHERE transaction_number != 0;
$$) as remote $$) as remote

View File

@ -1,12 +1,13 @@
Parsed test spec with 2 sessions Parsed test spec with 2 sessions
starting permutation: s1-initialize s1-begin s2-begin s1-drop s2-drop s1-commit s2-commit s1-select-count starting permutation: s1-initialize s2-initialize s1-begin s2-begin s1-drop s2-drop s1-commit s2-commit s1-select-count
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV;
step s2-initialize: SET search_path TO 'drop_tests';
step s1-begin: BEGIN; step s1-begin: BEGIN;
step s2-begin: BEGIN; step s2-begin: BEGIN;
step s1-drop: DROP TABLE drop_hash; step s1-drop: DROP TABLE drop_hash;
@ -23,13 +24,60 @@ restore_isolation_tester_func
(1 row) (1 row)
starting permutation: s1-initialize s1-begin s2-begin s1-drop s2-ddl-create-index s1-commit s2-commit s1-select-count s1-show-indexes starting permutation: s1-initialize s2-initialize s1-begin s2-begin s1-drop-schema s2-drop-schema s1-commit s2-commit s1-select-count
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV;
step s2-initialize: SET search_path TO 'drop_tests';
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-drop-schema: DROP SCHEMA drop_tests CASCADE;
step s2-drop-schema: DROP SCHEMA drop_tests CASCADE; <waiting ...>
step s1-commit: COMMIT;
step s2-drop-schema: <... completed>
ERROR: schema "drop_tests" does not exist
step s2-commit: COMMIT;
step s1-select-count: SELECT COUNT(*) FROM drop_hash;
ERROR: relation "drop_hash" does not exist
restore_isolation_tester_func
---------------------------------------------------------------------
(1 row)
starting permutation: s1-initialize s2-initialize s1-begin s2-begin s1-drop-schema s2-drop-schema-2 s1-commit s2-commit s1-select-count
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV;
step s2-initialize: SET search_path TO 'drop_tests';
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-drop-schema: DROP SCHEMA drop_tests CASCADE;
step s2-drop-schema-2: DROP SCHEMA drop_tests_2 CASCADE;
step s1-commit: COMMIT;
step s2-commit: COMMIT;
step s1-select-count: SELECT COUNT(*) FROM drop_hash;
ERROR: relation "drop_hash" does not exist
restore_isolation_tester_func
---------------------------------------------------------------------
(1 row)
starting permutation: s1-initialize s2-initialize s1-begin s2-begin s1-drop s2-ddl-create-index s1-commit s2-commit s1-select-count s1-show-indexes
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV;
step s2-initialize: SET search_path TO 'drop_tests';
step s1-begin: BEGIN; step s1-begin: BEGIN;
step s2-begin: BEGIN; step s2-begin: BEGIN;
step s1-drop: DROP TABLE drop_hash; step s1-drop: DROP TABLE drop_hash;
@ -53,13 +101,14 @@ restore_isolation_tester_func
(1 row) (1 row)
starting permutation: s1-initialize s1-ddl-create-index s1-begin s2-begin s1-drop s2-ddl-drop-index s1-commit s2-commit s1-select-count s1-show-indexes starting permutation: s1-initialize s2-initialize s1-ddl-create-index s1-begin s2-begin s1-drop s2-ddl-drop-index s1-commit s2-commit s1-select-count s1-show-indexes
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV;
step s2-initialize: SET search_path TO 'drop_tests';
step s1-ddl-create-index: CREATE INDEX drop_hash_index ON drop_hash(id); step s1-ddl-create-index: CREATE INDEX drop_hash_index ON drop_hash(id);
step s1-begin: BEGIN; step s1-begin: BEGIN;
step s2-begin: BEGIN; step s2-begin: BEGIN;
@ -84,13 +133,14 @@ restore_isolation_tester_func
(1 row) (1 row)
starting permutation: s1-initialize s1-begin s1-drop s2-ddl-create-index-concurrently s1-commit s1-select-count s1-show-indexes starting permutation: s1-initialize s2-initialize s1-begin s1-drop s2-ddl-create-index-concurrently s1-commit s1-select-count s1-show-indexes
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV;
step s2-initialize: SET search_path TO 'drop_tests';
step s1-begin: BEGIN; step s1-begin: BEGIN;
step s1-drop: DROP TABLE drop_hash; step s1-drop: DROP TABLE drop_hash;
step s2-ddl-create-index-concurrently: CREATE INDEX CONCURRENTLY drop_hash_index ON drop_hash(id); <waiting ...> step s2-ddl-create-index-concurrently: CREATE INDEX CONCURRENTLY drop_hash_index ON drop_hash(id); <waiting ...>
@ -112,13 +162,14 @@ restore_isolation_tester_func
(1 row) (1 row)
starting permutation: s1-initialize s1-begin s2-begin s1-drop s2-ddl-add-column s1-commit s2-commit s1-select-count s1-show-columns starting permutation: s1-initialize s2-initialize s1-begin s2-begin s1-drop s2-ddl-add-column s1-commit s2-commit s1-select-count s1-show-columns
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV;
step s2-initialize: SET search_path TO 'drop_tests';
step s1-begin: BEGIN; step s1-begin: BEGIN;
step s2-begin: BEGIN; step s2-begin: BEGIN;
step s1-drop: DROP TABLE drop_hash; step s1-drop: DROP TABLE drop_hash;
@ -142,13 +193,14 @@ restore_isolation_tester_func
(1 row) (1 row)
starting permutation: s1-initialize s1-ddl-add-column s1-begin s2-begin s1-drop s2-ddl-drop-column s1-commit s2-commit s1-select-count s1-show-columns starting permutation: s1-initialize s2-initialize s1-ddl-add-column s1-begin s2-begin s1-drop s2-ddl-drop-column s1-commit s2-commit s1-select-count s1-show-columns
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV;
step s2-initialize: SET search_path TO 'drop_tests';
step s1-ddl-add-column: ALTER TABLE drop_hash ADD new_column int DEFAULT 0; step s1-ddl-add-column: ALTER TABLE drop_hash ADD new_column int DEFAULT 0;
step s1-begin: BEGIN; step s1-begin: BEGIN;
step s2-begin: BEGIN; step s2-begin: BEGIN;
@ -173,13 +225,14 @@ restore_isolation_tester_func
(1 row) (1 row)
starting permutation: s1-initialize s1-begin s2-begin s1-drop s2-ddl-rename-column s1-commit s2-commit s1-select-count s1-show-columns starting permutation: s1-initialize s2-initialize s1-begin s2-begin s1-drop s2-ddl-rename-column s1-commit s2-commit s1-select-count s1-show-columns
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV;
step s2-initialize: SET search_path TO 'drop_tests';
step s1-begin: BEGIN; step s1-begin: BEGIN;
step s2-begin: BEGIN; step s2-begin: BEGIN;
step s1-drop: DROP TABLE drop_hash; step s1-drop: DROP TABLE drop_hash;
@ -203,13 +256,14 @@ restore_isolation_tester_func
(1 row) (1 row)
starting permutation: s1-initialize s1-begin s2-begin s1-drop s2-table-size s1-commit s2-commit s1-select-count starting permutation: s1-initialize s2-initialize s1-begin s2-begin s1-drop s2-table-size s1-commit s2-commit s1-select-count
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV;
step s2-initialize: SET search_path TO 'drop_tests';
step s1-begin: BEGIN; step s1-begin: BEGIN;
step s2-begin: BEGIN; step s2-begin: BEGIN;
step s1-drop: DROP TABLE drop_hash; step s1-drop: DROP TABLE drop_hash;
@ -226,7 +280,7 @@ restore_isolation_tester_func
(1 row) (1 row)
starting permutation: s1-drop s1-create-non-distributed-table s1-initialize s1-begin s2-begin s1-drop s2-distribute-table s1-commit s2-commit s1-select-count starting permutation: s1-drop s1-create-non-distributed-table s1-initialize s2-initialize s1-begin s2-begin s1-drop s2-distribute-table s1-commit s2-commit s1-select-count
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -234,7 +288,8 @@ create_distributed_table
step s1-drop: DROP TABLE drop_hash; step s1-drop: DROP TABLE drop_hash;
step s1-create-non-distributed-table: CREATE TABLE drop_hash(id integer, data text); COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; step s1-create-non-distributed-table: CREATE TABLE drop_hash(id integer, data text); COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV;
step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV;
step s2-initialize: SET search_path TO 'drop_tests';
step s1-begin: BEGIN; step s1-begin: BEGIN;
step s2-begin: BEGIN; step s2-begin: BEGIN;
step s1-drop: DROP TABLE drop_hash; step s1-drop: DROP TABLE drop_hash;
@ -251,13 +306,14 @@ restore_isolation_tester_func
(1 row) (1 row)
starting permutation: s1-initialize s1-begin s2-begin s1-ddl-create-index s2-drop s1-commit s2-commit s1-select-count s1-show-indexes starting permutation: s1-initialize s2-initialize s1-begin s2-begin s1-ddl-create-index s2-drop s1-commit s2-commit s1-select-count s1-show-indexes
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV;
step s2-initialize: SET search_path TO 'drop_tests';
step s1-begin: BEGIN; step s1-begin: BEGIN;
step s2-begin: BEGIN; step s2-begin: BEGIN;
step s1-ddl-create-index: CREATE INDEX drop_hash_index ON drop_hash(id); step s1-ddl-create-index: CREATE INDEX drop_hash_index ON drop_hash(id);
@ -280,13 +336,14 @@ restore_isolation_tester_func
(1 row) (1 row)
starting permutation: s1-initialize s1-ddl-create-index s1-begin s2-begin s1-ddl-drop-index s2-drop s1-commit s2-commit s1-select-count s1-show-indexes starting permutation: s1-initialize s2-initialize s1-ddl-create-index s1-begin s2-begin s1-ddl-drop-index s2-drop s1-commit s2-commit s1-select-count s1-show-indexes
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV;
step s2-initialize: SET search_path TO 'drop_tests';
step s1-ddl-create-index: CREATE INDEX drop_hash_index ON drop_hash(id); step s1-ddl-create-index: CREATE INDEX drop_hash_index ON drop_hash(id);
step s1-begin: BEGIN; step s1-begin: BEGIN;
step s2-begin: BEGIN; step s2-begin: BEGIN;
@ -310,13 +367,14 @@ restore_isolation_tester_func
(1 row) (1 row)
starting permutation: s1-initialize s1-begin s2-begin s1-ddl-add-column s2-drop s1-commit s2-commit s1-select-count s1-show-columns starting permutation: s1-initialize s2-initialize s1-begin s2-begin s1-ddl-add-column s2-drop s1-commit s2-commit s1-select-count s1-show-columns
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV;
step s2-initialize: SET search_path TO 'drop_tests';
step s1-begin: BEGIN; step s1-begin: BEGIN;
step s2-begin: BEGIN; step s2-begin: BEGIN;
step s1-ddl-add-column: ALTER TABLE drop_hash ADD new_column int DEFAULT 0; step s1-ddl-add-column: ALTER TABLE drop_hash ADD new_column int DEFAULT 0;
@ -339,13 +397,14 @@ restore_isolation_tester_func
(1 row) (1 row)
starting permutation: s1-initialize s1-ddl-add-column s1-begin s2-begin s1-ddl-drop-column s2-drop s1-commit s2-commit s1-select-count s1-show-columns starting permutation: s1-initialize s2-initialize s1-ddl-add-column s1-begin s2-begin s1-ddl-drop-column s2-drop s1-commit s2-commit s1-select-count s1-show-columns
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV;
step s2-initialize: SET search_path TO 'drop_tests';
step s1-ddl-add-column: ALTER TABLE drop_hash ADD new_column int DEFAULT 0; step s1-ddl-add-column: ALTER TABLE drop_hash ADD new_column int DEFAULT 0;
step s1-begin: BEGIN; step s1-begin: BEGIN;
step s2-begin: BEGIN; step s2-begin: BEGIN;
@ -369,13 +428,14 @@ restore_isolation_tester_func
(1 row) (1 row)
starting permutation: s1-initialize s1-begin s2-begin s1-ddl-rename-column s2-drop s1-commit s2-commit s1-select-count s1-show-columns starting permutation: s1-initialize s2-initialize s1-begin s2-begin s1-ddl-rename-column s2-drop s1-commit s2-commit s1-select-count s1-show-columns
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV;
step s2-initialize: SET search_path TO 'drop_tests';
step s1-begin: BEGIN; step s1-begin: BEGIN;
step s2-begin: BEGIN; step s2-begin: BEGIN;
step s1-ddl-rename-column: ALTER TABLE drop_hash RENAME data TO new_column; step s1-ddl-rename-column: ALTER TABLE drop_hash RENAME data TO new_column;
@ -398,13 +458,14 @@ restore_isolation_tester_func
(1 row) (1 row)
starting permutation: s1-initialize s1-begin s2-begin s1-table-size s2-drop s1-commit s2-commit s1-select-count starting permutation: s1-initialize s2-initialize s1-begin s2-begin s1-table-size s2-drop s1-commit s2-commit s1-select-count
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV;
step s2-initialize: SET search_path TO 'drop_tests';
step s1-begin: BEGIN; step s1-begin: BEGIN;
step s2-begin: BEGIN; step s2-begin: BEGIN;
step s1-table-size: SELECT citus_total_relation_size('drop_hash'); step s1-table-size: SELECT citus_total_relation_size('drop_hash');
@ -424,7 +485,7 @@ restore_isolation_tester_func
(1 row) (1 row)
starting permutation: s1-drop s1-create-non-distributed-table s1-initialize s1-begin s2-begin s1-distribute-table s2-drop s1-commit s2-commit s1-select-count starting permutation: s1-drop s1-create-non-distributed-table s1-initialize s2-initialize s1-begin s2-begin s1-distribute-table s2-drop s1-commit s2-commit s1-select-count
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -432,7 +493,8 @@ create_distributed_table
step s1-drop: DROP TABLE drop_hash; step s1-drop: DROP TABLE drop_hash;
step s1-create-non-distributed-table: CREATE TABLE drop_hash(id integer, data text); COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; step s1-create-non-distributed-table: CREATE TABLE drop_hash(id integer, data text); COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV;
step s1-initialize: COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; step s1-initialize: SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV;
step s2-initialize: SET search_path TO 'drop_tests';
step s1-begin: BEGIN; step s1-begin: BEGIN;
step s2-begin: BEGIN; step s2-begin: BEGIN;
step s1-distribute-table: SELECT create_distributed_table('drop_hash', 'id'); step s1-distribute-table: SELECT create_distributed_table('drop_hash', 'id');

View File

@ -49,23 +49,32 @@ count
(1 row) (1 row)
step s3-as-user-1: step s3-as-user-1:
-- User should only be able to see its own transactions -- Eventhough we change the user via SET ROLE, the backends' (e.g., s1/2-begin-insert)
-- userId (e.g., PG_PROC->userId) does not change, and hence none of the
-- transactions show up because here we are using test_user_1. This is a
-- limitation of isolation tester, we should be able to re-connect with
-- test_user_1 on s1/2-begin-insert to show that test_user_1 sees only its own processes
SET ROLE test_user_1; SET ROLE test_user_1;
SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0; SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0;
SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0; SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0;
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 0
(1 row) (1 row)
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
2 1
(1 row) (1 row)
step s3-as-readonly: step s3-as-readonly:
-- Other user should not see transactions -- Eventhough we change the user via SET ROLE, the backends' (e.g., s1/2-begin-insert)
-- userId (e.g., PG_PROC->userId) does not change, and hence none of the
-- transactions show up because here we are using test_readonly. This is a
-- limitation of isolation tester, we should be able to re-connect with
-- test_readonly on s1/2-begin-insert to show that test_readonly sees only
-- its own processes
SET ROLE test_readonly; SET ROLE test_readonly;
SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0; SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0;
SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0; SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0;

View File

@ -33,7 +33,7 @@ blocked_statement |current_statement_in_blockin
--------------------------------------------------------------------- ---------------------------------------------------------------------
UPDATE ref_table SET value_1 = 12 WHERE user_id = 1| UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|
UPDATE ref_table SET value_1 = 15; UPDATE ref_table SET value_1 = 15;
|localhost |coordinator_host | 57638| 57636 |coordinator_host |coordinator_host | 57636| 57636
(1 row) (1 row)
step s1-commit: step s1-commit:
@ -116,7 +116,7 @@ step s3-select-distributed-waiting-queries:
blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port
--------------------------------------------------------------------- ---------------------------------------------------------------------
UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|localhost |localhost | 57638| 57637 UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|coordinator_host |coordinator_host | 57636| 57636
(1 row) (1 row)
step s1-commit-worker: step s1-commit-worker:
@ -212,7 +212,7 @@ step s3-select-distributed-waiting-queries:
blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port
--------------------------------------------------------------------- ---------------------------------------------------------------------
UPDATE tt1 SET value_1 = 5|UPDATE tt1 SET value_1 = 4 |localhost |localhost | 57638| 57637 UPDATE tt1 SET value_1 = 5|UPDATE tt1 SET value_1 = 4 |coordinator_host |coordinator_host | 57636| 57636
(1 row) (1 row)
step s1-commit-worker: step s1-commit-worker:
@ -308,7 +308,7 @@ step s3-select-distributed-waiting-queries:
blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port
--------------------------------------------------------------------- ---------------------------------------------------------------------
UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|DELETE FROM ref_table WHERE user_id = 1|localhost |localhost | 57638| 57637 UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|DELETE FROM ref_table WHERE user_id = 1|coordinator_host |coordinator_host | 57636| 57636
(1 row) (1 row)
step s1-commit-worker: step s1-commit-worker:
@ -404,7 +404,7 @@ step s3-select-distributed-waiting-queries:
blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port
--------------------------------------------------------------------- ---------------------------------------------------------------------
UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|INSERT INTO ref_table VALUES(8,81),(9,91)|localhost |localhost | 57638| 57637 UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|INSERT INTO ref_table VALUES(8,81),(9,91)|coordinator_host |coordinator_host | 57636| 57636
(1 row) (1 row)
step s1-commit-worker: step s1-commit-worker:
@ -594,7 +594,7 @@ step s3-select-distributed-waiting-queries:
blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port
--------------------------------------------------------------------- ---------------------------------------------------------------------
UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|COPY ref_table FROM PROGRAM 'echo 10, 101 && echo 11, 111' WITH CSV|localhost |localhost | 57638| 57637 UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|COPY ref_table FROM PROGRAM 'echo 10, 101 && echo 11, 111' WITH CSV|coordinator_host |coordinator_host | 57636| 57636
(1 row) (1 row)
step s1-commit-worker: step s1-commit-worker:
@ -878,7 +878,7 @@ step s3-select-distributed-waiting-queries:
blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port
--------------------------------------------------------------------- ---------------------------------------------------------------------
UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|SELECT * FROM ref_table FOR UPDATE |localhost |localhost | 57638| 57637 UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|SELECT * FROM ref_table FOR UPDATE |coordinator_host |coordinator_host | 57636| 57636
(1 row) (1 row)
step s1-commit-worker: step s1-commit-worker:
@ -963,7 +963,7 @@ blocked_statement |current_s
--------------------------------------------------------------------- ---------------------------------------------------------------------
ALTER TABLE ref_table ADD CONSTRAINT rf_p_key PRIMARY KEY(user_id); ALTER TABLE ref_table ADD CONSTRAINT rf_p_key PRIMARY KEY(user_id);
|INSERT INTO ref_table VALUES(8,81),(9,91)|coordinator_host |localhost | 57636| 57638 |INSERT INTO ref_table VALUES(8,81),(9,91)|coordinator_host |coordinator_host | 57636| 57636
(1 row) (1 row)
step s2-commit-worker: step s2-commit-worker:
@ -1073,7 +1073,7 @@ step s3-select-distributed-waiting-queries:
blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port
--------------------------------------------------------------------- ---------------------------------------------------------------------
UPDATE tt1 SET value_1 = 5|UPDATE tt1 SET value_1 = 4 |localhost |localhost | 57637| 57637 UPDATE tt1 SET value_1 = 5|UPDATE tt1 SET value_1 = 4 |coordinator_host |coordinator_host | 57636| 57636
(1 row) (1 row)
step s1-commit-worker: step s1-commit-worker:
@ -1161,7 +1161,7 @@ step s3-select-distributed-waiting-queries:
blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port
--------------------------------------------------------------------- ---------------------------------------------------------------------
UPDATE tt1 SET value_1 = 4 WHERE user_id = 1|UPDATE tt1 SET value_1 = 4 WHERE user_id = 1|localhost |localhost | 57638| 57637 UPDATE tt1 SET value_1 = 4 WHERE user_id = 1|UPDATE tt1 SET value_1 = 4 WHERE user_id = 1|coordinator_host |coordinator_host | 57636| 57636
(1 row) (1 row)
step s1-commit-worker: step s1-commit-worker:
@ -1225,7 +1225,7 @@ blocked_statement |current_statement_in_blockin
--------------------------------------------------------------------- ---------------------------------------------------------------------
UPDATE ref_table SET value_1 = 12 WHERE user_id = 1| UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|
UPDATE ref_table SET value_1 = 15; UPDATE ref_table SET value_1 = 15;
|localhost |coordinator_host | 57638| 57636 |coordinator_host |coordinator_host | 57636| 57636
(1 row) (1 row)
step s1-commit: step s1-commit:

View File

@ -237,6 +237,126 @@ t
(1 row) (1 row)
starting permutation: s1-begin s2-begin s1-start-metadata-sync s2-create-schema s1-commit s2-commit s3-compare-snapshot s2-drop-schema
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1-start-metadata-sync:
SELECT start_metadata_sync_to_node('localhost', 57638);
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
step s2-create-schema:
CREATE SCHEMA dist_schema
CREATE TABLE dist_table_in_schema(id int, data int);
SELECT create_distributed_table('dist_schema.dist_table_in_schema', 'id');
<waiting ...>
step s1-commit:
COMMIT;
step s2-create-schema: <... completed>
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s2-commit:
COMMIT;
step s3-compare-snapshot:
SELECT count(*) = 0 AS same_metadata_in_workers
FROM
(
(
SELECT unnest(activate_node_snapshot())
EXCEPT
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
)
UNION
(
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
EXCEPT
SELECT unnest(activate_node_snapshot())
)
) AS foo;
same_metadata_in_workers
---------------------------------------------------------------------
t
(1 row)
step s2-drop-schema:
DROP SCHEMA dist_schema CASCADE;
starting permutation: s2-create-schema s1-begin s2-begin s1-start-metadata-sync s2-drop-schema s1-commit s2-commit s3-compare-snapshot
step s2-create-schema:
CREATE SCHEMA dist_schema
CREATE TABLE dist_table_in_schema(id int, data int);
SELECT create_distributed_table('dist_schema.dist_table_in_schema', 'id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1-start-metadata-sync:
SELECT start_metadata_sync_to_node('localhost', 57638);
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
step s2-drop-schema:
DROP SCHEMA dist_schema CASCADE;
<waiting ...>
step s1-commit:
COMMIT;
step s2-drop-schema: <... completed>
step s2-commit:
COMMIT;
step s3-compare-snapshot:
SELECT count(*) = 0 AS same_metadata_in_workers
FROM
(
(
SELECT unnest(activate_node_snapshot())
EXCEPT
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
)
UNION
(
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
EXCEPT
SELECT unnest(activate_node_snapshot())
)
) AS foo;
same_metadata_in_workers
---------------------------------------------------------------------
t
(1 row)
starting permutation: s1-begin s2-begin s1-start-metadata-sync s2-create-dist-table s1-commit s2-commit s3-compare-snapshot starting permutation: s1-begin s2-begin s1-start-metadata-sync s2-create-dist-table s1-commit s2-commit s3-compare-snapshot
step s1-begin: step s1-begin:
BEGIN; BEGIN;
@ -608,6 +728,415 @@ t
(1 row) (1 row)
starting permutation: s2-create-type s1-begin s2-begin s1-start-metadata-sync s2-drop-type s1-commit s2-commit s3-compare-snapshot
step s2-create-type:
CREATE TYPE my_type AS (a int, b int);
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1-start-metadata-sync:
SELECT start_metadata_sync_to_node('localhost', 57638);
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
step s2-drop-type:
DROP TYPE my_type;
<waiting ...>
step s1-commit:
COMMIT;
step s2-drop-type: <... completed>
step s2-commit:
COMMIT;
step s3-compare-snapshot:
SELECT count(*) = 0 AS same_metadata_in_workers
FROM
(
(
SELECT unnest(activate_node_snapshot())
EXCEPT
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
)
UNION
(
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
EXCEPT
SELECT unnest(activate_node_snapshot())
)
) AS foo;
same_metadata_in_workers
---------------------------------------------------------------------
t
(1 row)
starting permutation: s2-create-dist-func s1-begin s2-begin s1-start-metadata-sync s2-drop-dist-func s1-commit s2-commit s3-compare-snapshot
step s2-create-dist-func:
CREATE FUNCTION squares(int) RETURNS SETOF RECORD
AS $$ SELECT i, i * i FROM generate_series(1, $1) i $$
LANGUAGE SQL;
SELECT create_distributed_function('squares(int)');
create_distributed_function
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1-start-metadata-sync:
SELECT start_metadata_sync_to_node('localhost', 57638);
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
step s2-drop-dist-func:
DROP FUNCTION squares(int);
<waiting ...>
step s1-commit:
COMMIT;
step s2-drop-dist-func: <... completed>
step s2-commit:
COMMIT;
step s3-compare-snapshot:
SELECT count(*) = 0 AS same_metadata_in_workers
FROM
(
(
SELECT unnest(activate_node_snapshot())
EXCEPT
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
)
UNION
(
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
EXCEPT
SELECT unnest(activate_node_snapshot())
)
) AS foo;
same_metadata_in_workers
---------------------------------------------------------------------
t
(1 row)
starting permutation: s2-create-type s1-begin s1-start-metadata-sync s2-alter-type s1-commit s3-compare-snapshot s3-compare-type-definition
step s2-create-type:
CREATE TYPE my_type AS (a int, b int);
step s1-begin:
BEGIN;
step s1-start-metadata-sync:
SELECT start_metadata_sync_to_node('localhost', 57638);
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
step s2-alter-type:
ALTER TYPE my_type ADD ATTRIBUTE x int;
<waiting ...>
step s1-commit:
COMMIT;
step s2-alter-type: <... completed>
step s3-compare-snapshot:
SELECT count(*) = 0 AS same_metadata_in_workers
FROM
(
(
SELECT unnest(activate_node_snapshot())
EXCEPT
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
)
UNION
(
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
EXCEPT
SELECT unnest(activate_node_snapshot())
)
) AS foo;
same_metadata_in_workers
---------------------------------------------------------------------
t
(1 row)
step s3-compare-type-definition:
SELECT run_command_on_workers($$SELECT '(1,1,1)'::my_type$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"(1,1,1)")
(localhost,57638,t,"(1,1,1)")
(2 rows)
starting permutation: s1-begin s2-begin s2-create-dist-table s1-start-metadata-sync s2-commit s1-commit s3-compare-snapshot
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-create-dist-table:
CREATE TABLE new_dist_table(id int, data int);
SELECT create_distributed_table('new_dist_table', 'id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-start-metadata-sync:
SELECT start_metadata_sync_to_node('localhost', 57638);
<waiting ...>
step s2-commit:
COMMIT;
step s1-start-metadata-sync: <... completed>
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
step s1-commit:
COMMIT;
step s3-compare-snapshot:
SELECT count(*) = 0 AS same_metadata_in_workers
FROM
(
(
SELECT unnest(activate_node_snapshot())
EXCEPT
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
)
UNION
(
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
EXCEPT
SELECT unnest(activate_node_snapshot())
)
) AS foo;
same_metadata_in_workers
---------------------------------------------------------------------
t
(1 row)
starting permutation: s2-create-dist-func s1-begin s2-begin s2-drop-dist-func s1-start-metadata-sync s2-commit s1-commit s3-compare-snapshot
step s2-create-dist-func:
CREATE FUNCTION squares(int) RETURNS SETOF RECORD
AS $$ SELECT i, i * i FROM generate_series(1, $1) i $$
LANGUAGE SQL;
SELECT create_distributed_function('squares(int)');
create_distributed_function
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-drop-dist-func:
DROP FUNCTION squares(int);
step s1-start-metadata-sync:
SELECT start_metadata_sync_to_node('localhost', 57638);
<waiting ...>
step s2-commit:
COMMIT;
step s1-start-metadata-sync: <... completed>
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
step s1-commit:
COMMIT;
step s3-compare-snapshot:
SELECT count(*) = 0 AS same_metadata_in_workers
FROM
(
(
SELECT unnest(activate_node_snapshot())
EXCEPT
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
)
UNION
(
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
EXCEPT
SELECT unnest(activate_node_snapshot())
)
) AS foo;
same_metadata_in_workers
---------------------------------------------------------------------
t
(1 row)
starting permutation: s2-create-schema s1-begin s2-begin s2-drop-schema s1-start-metadata-sync s2-commit s1-commit s3-compare-snapshot
step s2-create-schema:
CREATE SCHEMA dist_schema
CREATE TABLE dist_table_in_schema(id int, data int);
SELECT create_distributed_table('dist_schema.dist_table_in_schema', 'id');
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-drop-schema:
DROP SCHEMA dist_schema CASCADE;
step s1-start-metadata-sync:
SELECT start_metadata_sync_to_node('localhost', 57638);
<waiting ...>
step s2-commit:
COMMIT;
step s1-start-metadata-sync: <... completed>
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
step s1-commit:
COMMIT;
step s3-compare-snapshot:
SELECT count(*) = 0 AS same_metadata_in_workers
FROM
(
(
SELECT unnest(activate_node_snapshot())
EXCEPT
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
)
UNION
(
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
EXCEPT
SELECT unnest(activate_node_snapshot())
)
) AS foo;
same_metadata_in_workers
---------------------------------------------------------------------
t
(1 row)
starting permutation: s2-create-type s1-begin s2-begin s2-alter-type s1-start-metadata-sync s2-commit s1-commit s3-compare-snapshot s3-compare-type-definition
step s2-create-type:
CREATE TYPE my_type AS (a int, b int);
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-alter-type:
ALTER TYPE my_type ADD ATTRIBUTE x int;
step s1-start-metadata-sync:
SELECT start_metadata_sync_to_node('localhost', 57638);
<waiting ...>
step s2-commit:
COMMIT;
step s1-start-metadata-sync: <... completed>
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
step s1-commit:
COMMIT;
step s3-compare-snapshot:
SELECT count(*) = 0 AS same_metadata_in_workers
FROM
(
(
SELECT unnest(activate_node_snapshot())
EXCEPT
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
)
UNION
(
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
EXCEPT
SELECT unnest(activate_node_snapshot())
)
) AS foo;
same_metadata_in_workers
---------------------------------------------------------------------
t
(1 row)
step s3-compare-type-definition:
SELECT run_command_on_workers($$SELECT '(1,1,1)'::my_type$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"(1,1,1)")
(localhost,57638,t,"(1,1,1)")
(2 rows)
starting permutation: s1-begin s2-begin s1-start-metadata-sync s2-create-type s1-commit s2-commit s3-compare-snapshot starting permutation: s1-begin s2-begin s1-start-metadata-sync s2-create-type s1-commit s2-commit s3-compare-snapshot
step s1-begin: step s1-begin:
BEGIN; BEGIN;

View File

@ -1,11 +1,19 @@
Parsed test spec with 3 sessions Parsed test spec with 4 sessions
starting permutation: s1-begin s2-begin s1-update-dist-table s2-lock-ref-table-placement-on-coordinator s1-lock-ref-table-placement-on-coordinator s2-update-dist-table deadlock-checker-call s1-end s2-end starting permutation: add-node s1-begin s2-begin s1-update-dist-table s2-lock-ref-table-placement-on-coordinator s1-lock-ref-table-placement-on-coordinator s2-update-dist-table deadlock-checker-call s1-end s2-end
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
step add-node:
SELECT 1 FROM master_add_node('localhost', 57636, groupid => 0);
?column?
---------------------------------------------------------------------
1
(1 row)
step s1-begin: step s1-begin:
BEGIN; BEGIN;
@ -59,12 +67,20 @@ master_remove_node
(1 row) (1 row)
starting permutation: s1-begin s2-begin s1-update-ref-table s2-sleep s2-view-dist s2-view-worker s2-end s1-end starting permutation: add-node s1-begin s2-begin s1-update-ref-table s2-sleep s2-view-dist s2-view-worker s2-end s1-end
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
step add-node:
SELECT 1 FROM master_add_node('localhost', 57636, groupid => 0);
?column?
---------------------------------------------------------------------
1
(1 row)
step s1-begin: step s1-begin:
BEGIN; BEGIN;
@ -83,7 +99,7 @@ pg_sleep
(1 row) (1 row)
step s2-view-dist: step s2-view-dist:
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND query NOT ILIKE '%pg_isolation_test_session_is_blocked%' AND query NOT ILIKE '%BEGIN%' ORDER BY query DESC; SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND query NOT ILIKE '%pg_isolation_test_session_is_blocked%' AND query NOT ILIKE '%BEGIN%' AND query NOT ILIKE '%add_node%' ORDER BY query DESC;
query |query_hostname |query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname query |query_hostname |query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -92,7 +108,7 @@ query |query_hostname |query_hostport|distri
|coordinator_host| 57636| | 0|idle |Client |ClientRead|postgres|regression |coordinator_host| 57636| | 0|idle |Client |ClientRead|postgres|regression
update ref_table set a = a + 1; update ref_table set a = a + 1;
|coordinator_host| 57636|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression |coordinator_host| 57636| | 0|idle in transaction|Client |ClientRead|postgres|regression
(2 rows) (2 rows)
step s2-view-worker: step s2-view-worker:
@ -102,13 +118,15 @@ step s2-view-worker:
WHERE query NOT ILIKE '%pg_prepared_xacts%' AND WHERE query NOT ILIKE '%pg_prepared_xacts%' AND
query NOT ILIKE '%COMMIT%' AND query NOT ILIKE '%COMMIT%' AND
query NOT ILIKE '%dump_local_%' AND query NOT ILIKE '%dump_local_%' AND
query NOT ILIKE '%citus_internal_local_blocked_processes%' query NOT ILIKE '%citus_internal_local_blocked_processes%' AND
query NOT ILIKE '%add_node%' AND
backend_type = 'client backend'
ORDER BY query, query_hostport DESC; ORDER BY query, query_hostport DESC;
query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname
--------------------------------------------------------------------- ---------------------------------------------------------------------
UPDATE public.ref_table_1500777 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57638|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression UPDATE public.ref_table_1500777 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57638| | 0|idle in transaction|Client |ClientRead|postgres|regression
UPDATE public.ref_table_1500777 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57637|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression UPDATE public.ref_table_1500777 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57637| | 0|idle in transaction|Client |ClientRead|postgres|regression
(2 rows) (2 rows)
step s2-end: step s2-end:
@ -123,12 +141,20 @@ master_remove_node
(1 row) (1 row)
starting permutation: s1-begin s2-begin s1-update-ref-table s2-active-transactions s1-end s2-end starting permutation: add-node s1-begin s2-begin s1-update-ref-table s2-active-transactions s1-end s2-end
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
step add-node:
SELECT 1 FROM master_add_node('localhost', 57636, groupid => 0);
?column?
---------------------------------------------------------------------
1
(1 row)
step s1-begin: step s1-begin:
BEGIN; BEGIN;

View File

@ -91,6 +91,7 @@ ALTER TABLE abcd DROP COLUMN a;
-- connection worker and get ready for the tests -- connection worker and get ready for the tests
\c - - - :worker_1_port \c - - - :worker_1_port
SET search_path TO local_shard_execution; SET search_path TO local_shard_execution;
SET citus.enable_unique_job_ids TO off;
-- returns true of the distribution key filter -- returns true of the distribution key filter
-- on the distributed tables (e.g., WHERE key = 1), we'll hit a shard -- on the distributed tables (e.g., WHERE key = 1), we'll hit a shard
-- placement which is local to this not -- placement which is local to this not
@ -733,9 +734,51 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shar
(1 row) (1 row)
SELECT count(*) FROM distributed_table d1 join distributed_table d2 using(age); SELECT count(*) FROM distributed_table d1 join distributed_table d2 using(age);
ERROR: cannot execute command because a local execution has accessed a placement in the transaction NOTICE: executing the command locally: SELECT partition_index, 'repartition_65_1' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_65_1','SELECT age AS column1 FROM local_shard_execution.distributed_table_1470001 d1 WHERE true',0,'hash','{-2147483648,-1073741824,0,1073741824}'::text[],'{-1073741825,-1,1073741823,2147483647}'::text[],true,true,true) WHERE rows_written > 0
DETAIL: Some parallel commands cannot be executed if a previous command has already been executed locally NOTICE: executing the command locally: SELECT partition_index, 'repartition_65_3' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_65_3','SELECT age AS column1 FROM local_shard_execution.distributed_table_1470003 d1 WHERE true',0,'hash','{-2147483648,-1073741824,0,1073741824}'::text[],'{-1073741825,-1,1073741823,2147483647}'::text[],true,true,true) WHERE rows_written > 0
HINT: Try re-running the transaction with "SET LOCAL citus.enable_local_execution TO OFF;" NOTICE: executing the command locally: SELECT partition_index, 'repartition_66_1' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_66_1','SELECT age AS column1 FROM local_shard_execution.distributed_table_1470001 d2 WHERE true',0,'hash','{-2147483648,-1073741824,0,1073741824}'::text[],'{-1073741825,-1,1073741823,2147483647}'::text[],true,true,true) WHERE rows_written > 0
NOTICE: executing the command locally: SELECT partition_index, 'repartition_66_3' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_66_3','SELECT age AS column1 FROM local_shard_execution.distributed_table_1470003 d2 WHERE true',0,'hash','{-2147483648,-1073741824,0,1073741824}'::text[],'{-1073741825,-1,1073741823,2147483647}'::text[],true,true,true) WHERE rows_written > 0
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_1_0']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_2_0']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_3_0']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_4_0']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_1_0']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_2_0']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_3_0']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_4_0']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_1_1']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_2_1']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_3_1']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_4_1']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_1_1']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_2_1']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_3_1']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_4_1']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_1_2']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_2_2']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_3_2']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_4_2']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_1_2']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_2_2']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_3_2']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_4_2']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_1_3']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_2_3']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_3_3']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_4_3']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_1_3']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_2_3']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_3_3']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_66_4_3']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_65_1_0,repartition_65_2_0,repartition_65_3_0,repartition_65_4_0}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 bigint) JOIN read_intermediate_results('{repartition_66_1_0,repartition_66_2_0,repartition_66_3_0,repartition_66_4_0}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 bigint) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true
NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_65_1_1,repartition_65_2_1,repartition_65_3_1,repartition_65_4_1}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 bigint) JOIN read_intermediate_results('{repartition_66_1_1,repartition_66_2_1,repartition_66_3_1,repartition_66_4_1}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 bigint) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true
NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_65_1_2,repartition_65_2_2,repartition_65_3_2,repartition_65_4_2}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 bigint) JOIN read_intermediate_results('{repartition_66_1_2,repartition_66_2_2,repartition_66_3_2,repartition_66_4_2}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 bigint) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true
NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_65_1_3,repartition_65_2_3,repartition_65_3_3,repartition_65_4_3}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 bigint) JOIN read_intermediate_results('{repartition_66_1_3,repartition_66_2_3,repartition_66_3_3,repartition_66_4_3}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 bigint) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true
count
---------------------------------------------------------------------
2
(1 row)
ROLLBACK; ROLLBACK;
-- a local query is followed by an INSERT..SELECT with re-partitioning -- a local query is followed by an INSERT..SELECT with re-partitioning
BEGIN; BEGIN;

View File

@ -682,6 +682,7 @@ NOTICE: executing the copy locally for shard xxxxx
ROLLBACK; ROLLBACK;
BEGIN; BEGIN;
SET citus.enable_repartition_joins TO ON; SET citus.enable_repartition_joins TO ON;
SET citus.enable_unique_job_ids TO off;
SELECT count(*) FROM distributed_table; SELECT count(*) FROM distributed_table;
NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE true NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500001 distributed_table WHERE true
NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table WHERE true NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution_replicated.distributed_table_1500002 distributed_table WHERE true
@ -693,9 +694,51 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shar
(1 row) (1 row)
SELECT count(*) FROM distributed_table d1 join distributed_table d2 using(age); SELECT count(*) FROM distributed_table d1 join distributed_table d2 using(age);
ERROR: cannot execute command because a local execution has accessed a placement in the transaction NOTICE: executing the command locally: SELECT partition_index, 'repartition_64_1' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_64_1','SELECT age AS column1 FROM local_shard_execution_replicated.distributed_table_1500001 d1 WHERE true',0,'hash','{-2147483648,-1073741824,0,1073741824}'::text[],'{-1073741825,-1,1073741823,2147483647}'::text[],true,true,true) WHERE rows_written > 0
DETAIL: Some parallel commands cannot be executed if a previous command has already been executed locally NOTICE: executing the command locally: SELECT partition_index, 'repartition_64_3' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_64_3','SELECT age AS column1 FROM local_shard_execution_replicated.distributed_table_1500003 d1 WHERE true',0,'hash','{-2147483648,-1073741824,0,1073741824}'::text[],'{-1073741825,-1,1073741823,2147483647}'::text[],true,true,true) WHERE rows_written > 0
HINT: Try re-running the transaction with "SET LOCAL citus.enable_local_execution TO OFF;" NOTICE: executing the command locally: SELECT partition_index, 'repartition_65_1' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_65_1','SELECT age AS column1 FROM local_shard_execution_replicated.distributed_table_1500001 d2 WHERE true',0,'hash','{-2147483648,-1073741824,0,1073741824}'::text[],'{-1073741825,-1,1073741823,2147483647}'::text[],true,true,true) WHERE rows_written > 0
NOTICE: executing the command locally: SELECT partition_index, 'repartition_65_3' || '_' || partition_index::text , rows_written FROM pg_catalog.worker_partition_query_result('repartition_65_3','SELECT age AS column1 FROM local_shard_execution_replicated.distributed_table_1500003 d2 WHERE true',0,'hash','{-2147483648,-1073741824,0,1073741824}'::text[],'{-1073741825,-1,1073741823,2147483647}'::text[],true,true,true) WHERE rows_written > 0
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_1_0']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_2_0']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_3_0']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_4_0']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_1_0']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_2_0']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_3_0']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_4_0']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_1_1']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_2_1']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_3_1']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_4_1']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_1_1']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_2_1']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_3_1']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_4_1']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_1_2']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_2_2']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_3_2']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_4_2']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_1_2']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_2_2']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_3_2']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_4_2']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_1_3']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_2_3']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_3_3']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_64_4_3']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_1_3']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_2_3']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_3_3']::text[],'localhost',57637) bytes
NOTICE: executing the command locally: SELECT bytes FROM fetch_intermediate_results(ARRAY['repartition_65_4_3']::text[],'localhost',57638) bytes
NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_64_1_0,repartition_64_2_0,repartition_64_3_0,repartition_64_4_0}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 bigint) JOIN read_intermediate_results('{repartition_65_1_0,repartition_65_2_0,repartition_65_3_0,repartition_65_4_0}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 bigint) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true
NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_64_1_1,repartition_64_2_1,repartition_64_3_1,repartition_64_4_1}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 bigint) JOIN read_intermediate_results('{repartition_65_1_1,repartition_65_2_1,repartition_65_3_1,repartition_65_4_1}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 bigint) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true
NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_64_1_2,repartition_64_2_2,repartition_64_3_2,repartition_64_4_2}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 bigint) JOIN read_intermediate_results('{repartition_65_1_2,repartition_65_2_2,repartition_65_3_2,repartition_65_4_2}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 bigint) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true
NOTICE: executing the command locally: SELECT count(*) AS count FROM (read_intermediate_results('{repartition_64_1_3,repartition_64_2_3,repartition_64_3_3,repartition_64_4_3}'::text[], 'binary'::citus_copy_format) intermediate_result(column1 bigint) JOIN read_intermediate_results('{repartition_65_1_3,repartition_65_2_3,repartition_65_3_3,repartition_65_4_3}'::text[], 'binary'::citus_copy_format) intermediate_result_1(column1 bigint) ON ((intermediate_result.column1 OPERATOR(pg_catalog.=) intermediate_result_1.column1))) WHERE true
count
---------------------------------------------------------------------
2
(1 row)
ROLLBACK; ROLLBACK;
-- a local query is followed by an INSERT..SELECT with re-partitioning -- a local query is followed by an INSERT..SELECT with re-partitioning
BEGIN; BEGIN;

View File

@ -966,7 +966,7 @@ DELETE FROM pg_dist_shard WHERE shardid = 1;
CREATE TABLE e_transactions(order_id varchar(255) NULL, transaction_id int) PARTITION BY LIST(transaction_id); CREATE TABLE e_transactions(order_id varchar(255) NULL, transaction_id int) PARTITION BY LIST(transaction_id);
CREATE TABLE orders_2020_07_01 CREATE TABLE orders_2020_07_01
PARTITION OF e_transactions FOR VALUES IN (1,2,3); PARTITION OF e_transactions FOR VALUES IN (1,2,3);
INSERT INTO pg_dist_partition VALUES ('e_transactions'::regclass,'h', NULL, 7, 's'); INSERT INTO pg_dist_partition VALUES ('e_transactions'::regclass,'h', '{VAR :varno 1 :varattno 1 :vartype 1043 :vartypmod 259 :varcollid 100 :varlevelsup 0 :varnoold 1 :varoattno 1 :location -1}', 7, 's');
SELECT SELECT
(metadata->>'partitioned_citus_table_exists_pre_11')::boolean as partitioned_citus_table_exists_pre_11, (metadata->>'partitioned_citus_table_exists_pre_11')::boolean as partitioned_citus_table_exists_pre_11,
(metadata->>'partitioned_citus_table_exists_pre_11') IS NULL as is_null (metadata->>'partitioned_citus_table_exists_pre_11') IS NULL as is_null
@ -1008,9 +1008,11 @@ SELECT * FROM multi_extension.print_extension_changes();
function master_append_table_to_shard(bigint,text,text,integer) real | function master_append_table_to_shard(bigint,text,text,integer) real |
function master_apply_delete_command(text) integer | function master_apply_delete_command(text) integer |
function master_get_table_metadata(text) record | function master_get_table_metadata(text) record |
function worker_partition_query_result(text,text,integer,citus.distribution_type,text[],text[],boolean) SETOF record |
| function citus_check_cluster_node_health() SETOF record | function citus_check_cluster_node_health() SETOF record
| function citus_check_connection_to_node(text,integer) boolean | function citus_check_connection_to_node(text,integer) boolean
| function citus_disable_node(text,integer,boolean) void | function citus_disable_node(text,integer,boolean) void
| function citus_finalize_upgrade_to_citus11(boolean) boolean
| function citus_internal_add_object_metadata(text,text[],text[],integer,integer,boolean) void | function citus_internal_add_object_metadata(text,text[],text[],integer,integer,boolean) void
| function citus_internal_global_blocked_processes() SETOF record | function citus_internal_global_blocked_processes() SETOF record
| function citus_internal_local_blocked_processes() SETOF record | function citus_internal_local_blocked_processes() SETOF record
@ -1023,7 +1025,8 @@ SELECT * FROM multi_extension.print_extension_changes();
| function worker_create_or_replace_object(text[]) boolean | function worker_create_or_replace_object(text[]) boolean
| function worker_drop_sequence_dependency(text) void | function worker_drop_sequence_dependency(text) void
| function worker_drop_shell_table(text) void | function worker_drop_shell_table(text) void
(20 rows) | function worker_partition_query_result(text,text,integer,citus.distribution_type,text[],text[],boolean,boolean,boolean) SETOF record
(23 rows)
DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff; DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff;
-- show running version -- show running version
@ -1113,8 +1116,7 @@ HINT: Use DROP FUNCTION relation_is_a_known_shard(regclass) first.
SET citus.enable_version_checks TO 'false'; SET citus.enable_version_checks TO 'false';
SET columnar.enable_version_checks TO 'false'; SET columnar.enable_version_checks TO 'false';
-- This will fail because of previous function declaration -- This will fail because of previous function declaration
ALTER EXTENSION citus UPDATE TO '8.1-1'; ALTER EXTENSION citus UPDATE TO '9.1-1';
NOTICE: version "8.1-1" of extension "citus" is already installed
-- We can DROP problematic function and continue ALTER EXTENSION even when version checks are on -- We can DROP problematic function and continue ALTER EXTENSION even when version checks are on
SET citus.enable_version_checks TO 'true'; SET citus.enable_version_checks TO 'true';
SET columnar.enable_version_checks TO 'true'; SET columnar.enable_version_checks TO 'true';
@ -1123,8 +1125,8 @@ ERROR: cannot drop function relation_is_a_known_shard(regclass) because extensi
HINT: You can drop extension citus instead. HINT: You can drop extension citus instead.
SET citus.enable_version_checks TO 'false'; SET citus.enable_version_checks TO 'false';
SET columnar.enable_version_checks TO 'false'; SET columnar.enable_version_checks TO 'false';
ALTER EXTENSION citus UPDATE TO '8.1-1'; ALTER EXTENSION citus UPDATE TO '9.1-1';
NOTICE: version "8.1-1" of extension "citus" is already installed NOTICE: version "9.1-1" of extension "citus" is already installed
-- Test updating to the latest version without specifying the version number -- Test updating to the latest version without specifying the version number
ALTER EXTENSION citus UPDATE; ALTER EXTENSION citus UPDATE;
-- re-create in newest version -- re-create in newest version

View File

@ -117,12 +117,18 @@ order by s_i_id;
SET citus.enable_repartition_joins TO ON; SET citus.enable_repartition_joins TO ON;
SELECT count(*) FROM the_table t1 JOIN the_table t2 USING(b); SELECT count(*) FROM the_table t1 JOIN the_table t2 USING(b);
ERROR: writing to worker nodes is not currently allowed count
DETAIL: the database is read-only ---------------------------------------------------------------------
2
(1 row)
SET citus.enable_single_hash_repartition_joins TO ON; SET citus.enable_single_hash_repartition_joins TO ON;
SELECT count(*) FROM the_table t1 , the_table t2 WHERE t1.a = t2.b; SELECT count(*) FROM the_table t1 , the_table t2 WHERE t1.a = t2.b;
ERROR: writing to worker nodes is not currently allowed count
DETAIL: the database is read-only ---------------------------------------------------------------------
2
(1 row)
SELECT SELECT
node_name, node_port node_name, node_port
FROM FROM

View File

@ -27,11 +27,17 @@ AS 'SELECT hashtext( ($1.i + $1.i2)::text);'
LANGUAGE SQL LANGUAGE SQL
IMMUTABLE IMMUTABLE
RETURNS NULL ON NULL INPUT; RETURNS NULL ON NULL INPUT;
CREATE FUNCTION test_udt_cmp(test_udt,test_udt) RETURNS int
AS 'SELECT CASE WHEN $1.i < $2.i THEN -1 ELSE CASE WHEN $1.i > $2.i THEN 1 ELSE CASE WHEN $1.i2 < $2.i2 THEN -1 ELSE CASE WHEN $1.i2 > $2.i2 THEN 1 ELSE 0 END END END END'
LANGUAGE SQL
IMMUTABLE
RETURNS NULL ON NULL INPUT;
-- We need to define two different operator classes for the composite types -- We need to define two different operator classes for the composite types
-- One uses BTREE the other uses HASH -- One uses BTREE the other uses HASH
CREATE OPERATOR CLASS tudt_op_fam_clas3 CREATE OPERATOR CLASS tudt_op_fam_clas3
DEFAULT FOR TYPE test_udt USING BTREE AS DEFAULT FOR TYPE test_udt USING BTREE AS
OPERATOR 3 = (test_udt, test_udt); OPERATOR 3 = (test_udt, test_udt),
FUNCTION 1 test_udt_cmp(test_udt, test_udt);
CREATE OPERATOR CLASS tudt_op_fam_class CREATE OPERATOR CLASS tudt_op_fam_class
DEFAULT FOR TYPE test_udt USING HASH AS DEFAULT FOR TYPE test_udt USING HASH AS
OPERATOR 1 = (test_udt, test_udt), OPERATOR 1 = (test_udt, test_udt),
@ -66,7 +72,8 @@ CREATE OPERATOR FAMILY tudt_op_fam USING hash;
-- One uses BTREE the other uses HASH -- One uses BTREE the other uses HASH
CREATE OPERATOR CLASS tudt_op_fam_clas3 CREATE OPERATOR CLASS tudt_op_fam_clas3
DEFAULT FOR TYPE test_udt USING BTREE AS DEFAULT FOR TYPE test_udt USING BTREE AS
OPERATOR 3 = (test_udt, test_udt); OPERATOR 3 = (test_udt, test_udt),
FUNCTION 1 test_udt_cmp(test_udt, test_udt);
CREATE OPERATOR CLASS tudt_op_fam_class CREATE OPERATOR CLASS tudt_op_fam_class
DEFAULT FOR TYPE test_udt USING HASH AS DEFAULT FOR TYPE test_udt USING HASH AS
OPERATOR 1 = (test_udt, test_udt), OPERATOR 1 = (test_udt, test_udt),
@ -88,7 +95,8 @@ CREATE OPERATOR FAMILY tudt_op_fam USING hash;
-- One uses BTREE the other uses HASH -- One uses BTREE the other uses HASH
CREATE OPERATOR CLASS tudt_op_fam_clas3 CREATE OPERATOR CLASS tudt_op_fam_clas3
DEFAULT FOR TYPE test_udt USING BTREE AS DEFAULT FOR TYPE test_udt USING BTREE AS
OPERATOR 3 = (test_udt, test_udt); OPERATOR 3 = (test_udt, test_udt),
FUNCTION 1 test_udt_cmp(test_udt, test_udt);
CREATE OPERATOR CLASS tudt_op_fam_class CREATE OPERATOR CLASS tudt_op_fam_class
DEFAULT FOR TYPE test_udt USING HASH AS DEFAULT FOR TYPE test_udt USING HASH AS
OPERATOR 1 = (test_udt, test_udt), OPERATOR 1 = (test_udt, test_udt),

View File

@ -28,11 +28,17 @@ AS 'SELECT hashtext( ($1.i + $1.i2)::text);'
LANGUAGE SQL LANGUAGE SQL
IMMUTABLE IMMUTABLE
RETURNS NULL ON NULL INPUT; RETURNS NULL ON NULL INPUT;
CREATE FUNCTION test_udt_cmp(test_udt,test_udt) RETURNS int
AS 'SELECT CASE WHEN $1.i < $2.i THEN -1 ELSE CASE WHEN $1.i > $2.i THEN 1 ELSE CASE WHEN $1.i2 < $2.i2 THEN -1 ELSE CASE WHEN $1.i2 > $2.i2 THEN 1 ELSE 0 END END END END'
LANGUAGE SQL
IMMUTABLE
RETURNS NULL ON NULL INPUT;
-- We need to define two different operator classes for the composite types -- We need to define two different operator classes for the composite types
-- One uses BTREE the other uses HASH -- One uses BTREE the other uses HASH
CREATE OPERATOR CLASS tudt_op_fam_clas3 CREATE OPERATOR CLASS tudt_op_fam_clas3
DEFAULT FOR TYPE test_udt USING BTREE AS DEFAULT FOR TYPE test_udt USING BTREE AS
OPERATOR 3 = (test_udt, test_udt); OPERATOR 3 = (test_udt, test_udt),
FUNCTION 1 test_udt_cmp(test_udt, test_udt);
CREATE OPERATOR CLASS tudt_op_fam_class CREATE OPERATOR CLASS tudt_op_fam_class
DEFAULT FOR TYPE test_udt USING HASH AS DEFAULT FOR TYPE test_udt USING HASH AS
OPERATOR 1 = (test_udt, test_udt), OPERATOR 1 = (test_udt, test_udt),
@ -67,7 +73,8 @@ CREATE OPERATOR FAMILY tudt_op_fam USING hash;
-- One uses BTREE the other uses HASH -- One uses BTREE the other uses HASH
CREATE OPERATOR CLASS tudt_op_fam_clas3 CREATE OPERATOR CLASS tudt_op_fam_clas3
DEFAULT FOR TYPE test_udt USING BTREE AS DEFAULT FOR TYPE test_udt USING BTREE AS
OPERATOR 3 = (test_udt, test_udt); OPERATOR 3 = (test_udt, test_udt),
FUNCTION 1 test_udt_cmp(test_udt, test_udt);
CREATE OPERATOR CLASS tudt_op_fam_class CREATE OPERATOR CLASS tudt_op_fam_class
DEFAULT FOR TYPE test_udt USING HASH AS DEFAULT FOR TYPE test_udt USING HASH AS
OPERATOR 1 = (test_udt, test_udt), OPERATOR 1 = (test_udt, test_udt),
@ -89,7 +96,8 @@ CREATE OPERATOR FAMILY tudt_op_fam USING hash;
-- One uses BTREE the other uses HASH -- One uses BTREE the other uses HASH
CREATE OPERATOR CLASS tudt_op_fam_clas3 CREATE OPERATOR CLASS tudt_op_fam_clas3
DEFAULT FOR TYPE test_udt USING BTREE AS DEFAULT FOR TYPE test_udt USING BTREE AS
OPERATOR 3 = (test_udt, test_udt); OPERATOR 3 = (test_udt, test_udt),
FUNCTION 1 test_udt_cmp(test_udt, test_udt);
CREATE OPERATOR CLASS tudt_op_fam_class CREATE OPERATOR CLASS tudt_op_fam_class
DEFAULT FOR TYPE test_udt USING HASH AS DEFAULT FOR TYPE test_udt USING HASH AS
OPERATOR 1 = (test_udt, test_udt), OPERATOR 1 = (test_udt, test_udt),

View File

@ -24,3 +24,25 @@ BEGIN;
(1 row) (1 row)
ROLLBACK; ROLLBACK;
PREPARE xact_repartitioned_prepared AS
SELECT count(*) FROM repartition_prepared_test t1 JOIN repartition_prepared_test t2 USING (b);
BEGIN;
-- Prepared re-partition join in a transaction block after a write
INSERT INTO repartition_prepared_test VALUES (1,2);
EXECUTE xact_repartitioned_prepared;
count
---------------------------------------------------------------------
226
(1 row)
ROLLBACK;
BEGIN;
-- Prepared re-partition join in a transaction block before a write
EXECUTE xact_repartitioned_prepared;
count
---------------------------------------------------------------------
209
(1 row)
INSERT INTO repartition_prepared_test VALUES (1,2);
ROLLBACK;

View File

@ -7,6 +7,15 @@ SELECT citus_remove_node('localhost', :worker_1_port);
(1 row) (1 row)
-- not related, but added here to test propagation of aggregates
-- to newly added nodes
CREATE AGGREGATE array_agg (anynonarray)
(
sfunc = array_agg_transfn,
stype = internal,
finalfunc = array_agg_finalfn,
finalfunc_extra
);
-- create schema, extension and foreign server while the worker is removed -- create schema, extension and foreign server while the worker is removed
SET citus.enable_ddl_propagation TO OFF; SET citus.enable_ddl_propagation TO OFF;
CREATE SCHEMA test_dependent_schema; CREATE SCHEMA test_dependent_schema;
@ -28,6 +37,20 @@ NOTICE: localhost:xxxxx is the coordinator and already contains metadata, skipp
1 1
(1 row) (1 row)
-- verify that the aggregate is propagated to the new node
SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%propagate_foreign_server.array_agg%';$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57638,t,propagate_foreign_server.array_agg)
(1 row)
-- verify that the aggregate is added top pg_dist_object on the new node
SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'propagate_foreign_server.array_agg'::regproc;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57638,t,1)
(1 row)
SELECT citus_add_local_table_to_metadata('foreign_table'); SELECT citus_add_local_table_to_metadata('foreign_table');
citus_add_local_table_to_metadata citus_add_local_table_to_metadata
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -163,4 +186,6 @@ ERROR: operation is not allowed on this node
HINT: Connect to the coordinator and run it again. HINT: Connect to the coordinator and run it again.
\c - - - :master_port \c - - - :master_port
DROP SCHEMA propagate_foreign_server CASCADE; DROP SCHEMA propagate_foreign_server CASCADE;
NOTICE: drop cascades to extension postgres_fdw NOTICE: drop cascades to 2 other objects
DETAIL: drop cascades to function propagate_foreign_server.array_agg(anynonarray)
drop cascades to extension postgres_fdw

View File

@ -2380,3 +2380,64 @@ SELECT rebalance_table_shards();
CALL citus_cleanup_orphaned_shards(); CALL citus_cleanup_orphaned_shards();
DROP TABLE test_rebalance_with_index CASCADE; DROP TABLE test_rebalance_with_index CASCADE;
-- Test rebalancer with disabled worker
SET citus.next_shard_id TO 433500;
SET citus.shard_replication_factor TO 2;
DROP TABLE IF EXISTS test_rebalance_with_disabled_worker;
CREATE TABLE test_rebalance_with_disabled_worker (a int);
SELECT create_distributed_table('test_rebalance_with_disabled_worker', 'a', colocate_with:='none');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT citus_disable_node('localhost', :worker_2_port);
citus_disable_node
---------------------------------------------------------------------
(1 row)
SELECT public.wait_until_metadata_sync(30000);
wait_until_metadata_sync
---------------------------------------------------------------------
(1 row)
SELECT rebalance_table_shards('test_rebalance_with_disabled_worker');
rebalance_table_shards
---------------------------------------------------------------------
(1 row)
SELECT citus_activate_node('localhost', :worker_2_port);
citus_activate_node
---------------------------------------------------------------------
35
(1 row)
DROP TABLE test_rebalance_with_disabled_worker;
-- Test rebalance with all shards excluded
DROP TABLE IF EXISTS test_with_all_shards_excluded;
CREATE TABLE test_with_all_shards_excluded(a int PRIMARY KEY);
SELECT create_distributed_table('test_with_all_shards_excluded', 'a', colocate_with:='none', shard_count:=4);
create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT shardid FROM pg_dist_shard;
shardid
---------------------------------------------------------------------
433504
433505
433506
433507
(4 rows)
SELECT rebalance_table_shards('test_with_all_shards_excluded', excluded_shard_list:='{102073, 102074, 102075, 102076}');
rebalance_table_shards
---------------------------------------------------------------------
(1 row)
DROP TABLE test_with_all_shards_excluded;

View File

@ -484,6 +484,26 @@ SELECT create_distributed_table('t5', 'name');
(1 row) (1 row)
-- make sure partial indices propagate their dependencies
-- first have a TEXT SEARCH CONFIGURATION that is not distributed
SET citus.enable_ddl_propagation TO off;
CREATE TEXT SEARCH CONFIGURATION partial_index_test_config ( parser = default );
RESET citus.enable_ddl_propagation;
CREATE TABLE sensors(
measureid integer,
eventdatetime date,
measure_data jsonb,
name text,
PRIMARY KEY (measureid, eventdatetime, measure_data)
) PARTITION BY RANGE(eventdatetime);
CREATE TABLE sensors_a_partition PARTITION OF sensors FOR VALUES FROM ('2000-01-01') TO ('2020-01-01');
CREATE INDEX sensors_search_name ON sensors USING gin (to_tsvector('partial_index_test_config'::regconfig, (COALESCE(name, ''::character varying))::text));
SELECT create_distributed_table('sensors', 'measureid');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SET client_min_messages TO 'warning'; SET client_min_messages TO 'warning';
DROP SCHEMA text_search, text_search2, "Text Search Requiring Quote's" CASCADE; DROP SCHEMA text_search, text_search2, "Text Search Requiring Quote's" CASCADE;
DROP ROLE text_search_owner; DROP ROLE text_search_owner;

View File

@ -57,6 +57,7 @@ ORDER BY 1;
function citus_drop_trigger() function citus_drop_trigger()
function citus_executor_name(integer) function citus_executor_name(integer)
function citus_extradata_container(internal) function citus_extradata_container(internal)
function citus_finalize_upgrade_to_citus11(boolean)
function citus_finish_pg_upgrade() function citus_finish_pg_upgrade()
function citus_get_active_worker_nodes() function citus_get_active_worker_nodes()
function citus_internal.columnar_ensure_am_depends_catalog() function citus_internal.columnar_ensure_am_depends_catalog()
@ -225,7 +226,7 @@ ORDER BY 1;
function worker_partial_agg(oid,anyelement) function worker_partial_agg(oid,anyelement)
function worker_partial_agg_ffunc(internal) function worker_partial_agg_ffunc(internal)
function worker_partial_agg_sfunc(internal,oid,anyelement) function worker_partial_agg_sfunc(internal,oid,anyelement)
function worker_partition_query_result(text,text,integer,citus.distribution_type,text[],text[],boolean) function worker_partition_query_result(text,text,integer,citus.distribution_type,text[],text[],boolean,boolean,boolean)
function worker_partitioned_relation_size(regclass) function worker_partitioned_relation_size(regclass)
function worker_partitioned_relation_total_size(regclass) function worker_partitioned_relation_total_size(regclass)
function worker_partitioned_table_size(regclass) function worker_partitioned_table_size(regclass)
@ -272,5 +273,5 @@ ORDER BY 1;
view citus_worker_stat_activity view citus_worker_stat_activity
view pg_dist_shard_placement view pg_dist_shard_placement
view time_partitions view time_partitions
(256 rows) (257 rows)

View File

@ -39,12 +39,38 @@ drop cascades to table upgrade_basic.t_range
SELECT i.* FROM citus.pg_dist_object, pg_identify_object_as_address(classid, objid, objsubid) i ORDER BY 1, 2, 3; SELECT i.* FROM citus.pg_dist_object, pg_identify_object_as_address(classid, objid, objsubid) i ORDER BY 1, 2, 3;
type | object_names | object_args type | object_names | object_args
--------------------------------------------------------------------- ---------------------------------------------------------------------
collation | {post_11_upgrade,german_phonebook_unpropagated} | {}
database | {postgres} | {} database | {postgres} | {}
extension | {isn} | {} extension | {isn} | {}
extension | {plpgsql} | {}
function | {post_11_upgrade,func_in_transaction_def} | {}
role | {postgres} | {} role | {postgres} | {}
schema | {fooschema} | {} schema | {fooschema} | {}
schema | {new_schema} | {} schema | {new_schema} | {}
schema | {post_11_upgrade} | {}
schema | {public} | {} schema | {public} | {}
table | {fooschema,footable} | {}
table | {new_schema,another_dist_table} | {}
table | {post_11_upgrade,colocated_dist_table} | {}
table | {post_11_upgrade,colocated_partitioned_table} | {}
table | {post_11_upgrade,colocated_partitioned_table_2020_01_01} | {}
table | {post_11_upgrade,dist} | {}
table | {post_11_upgrade,index_backed_rep_identity} | {}
table | {post_11_upgrade,part_table} | {}
table | {post_11_upgrade,part_table_p202008} | {}
table | {post_11_upgrade,part_table_p202009} | {}
table | {post_11_upgrade,reference_table} | {}
table | {post_11_upgrade,sensors} | {}
table | {post_11_upgrade,sensors_2020_01_01} | {}
table | {post_11_upgrade,sensors_news} | {}
table | {post_11_upgrade,sensors_old} | {}
table | {post_11_upgrade,sensors_parser} | {}
table | {post_11_upgrade,sensors_parser_a_partition} | {}
table | {post_11_upgrade,test_propagate_collate} | {}
table | {public,dist_table} | {}
table | {public,isn_dist_table} | {}
text search configuration | {post_11_upgrade,partial_index_test_config} | {}
type | {fooschema.footype} | {} type | {fooschema.footype} | {}
(7 rows) type | {post_11_upgrade.my_type} | {}
(33 rows)

View File

@ -0,0 +1,54 @@
-- run this test only when old citus version is 9.0
\set upgrade_test_old_citus_version `echo "$CITUS_OLD_VERSION"`
SELECT substring(:'upgrade_test_old_citus_version', 'v(\d+)\.\d+\.\d+')::int = 9 AND
substring(:'upgrade_test_old_citus_version', 'v\d+\.(\d+)\.\d+')::int = 0
AS upgrade_test_old_citus_version_e_9_0;
upgrade_test_old_citus_version_e_9_0
---------------------------------------------------------------------
t
(1 row)
\gset
\if :upgrade_test_old_citus_version_e_9_0
\else
\q
\endif
-- drop objects from previous test (uprade_basic_after.sql) for a clean test
-- drop upgrade_basic schema and switch back to public schema
SET search_path to public;
DROP SCHEMA upgrade_basic CASCADE;
NOTICE: drop cascades to 7 other objects
DETAIL: drop cascades to table upgrade_basic.t
drop cascades to table upgrade_basic.tp
drop cascades to table upgrade_basic.t_ab
drop cascades to table upgrade_basic.t2
drop cascades to table upgrade_basic.r
drop cascades to table upgrade_basic.tr
drop cascades to table upgrade_basic.t_range
-- as we updated citus to available version,
-- "isn" extension
-- "new_schema" schema
-- "public" schema
-- "fooschema" schema
-- "footype" type (under schema 'fooschema')
-- will now be marked as distributed
-- but,
-- "seg" extension
-- will not be marked as distributed
-- see underlying objects
SELECT i.* FROM citus.pg_dist_object, pg_identify_object_as_address(classid, objid, objsubid) i ORDER BY 1, 2, 3;
type | object_names | object_args
---------------------------------------------------------------------
database | {postgres} | {}
extension | {isn} | {}
role | {postgres} | {}
schema | {fooschema} | {}
schema | {new_schema} | {}
schema | {public} | {}
table | {fooschema,footable} | {}
table | {new_schema,another_dist_table} | {}
table | {public,dist_table} | {}
table | {public,isn_dist_table} | {}
type | {fooschema.footype} | {}
(11 rows)

View File

@ -0,0 +1,76 @@
SET search_path = post_11_upgrade;
-- make sure that we always (re)sync the metadata
UPDATE pg_dist_node_metadata SET metadata=jsonb_set(metadata, '{partitioned_citus_table_exists_pre_11}', to_jsonb('true'::bool), true);
SELECT citus_finalize_upgrade_to_citus11(enforce_version_check:=false);
NOTICE: Preparing all the existing partitioned table indexes
NOTICE: Preparing to sync the metadata to all nodes
citus_finalize_upgrade_to_citus11
---------------------------------------------------------------------
t
(1 row)
-- tables are objects with Citus 11+
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object WHERE objid IN ('post_11_upgrade'::regnamespace, 'post_11_upgrade.part_table'::regclass, 'post_11_upgrade.sensors'::regclass, 'post_11_upgrade.func_in_transaction_def'::regproc, 'post_11_upgrade.partial_index_test_config'::regconfig, 'post_11_upgrade.my_type'::regtype) ORDER BY 1;
pg_identify_object_as_address
---------------------------------------------------------------------
(function,"{post_11_upgrade,func_in_transaction_def}",{})
(schema,{post_11_upgrade},{})
(table,"{post_11_upgrade,part_table}",{})
(table,"{post_11_upgrade,sensors}",{})
("text search configuration","{post_11_upgrade,partial_index_test_config}",{})
(type,{post_11_upgrade.my_type},{})
(6 rows)
-- on all nodes
SELECT run_command_on_workers($$SELECT array_agg(pg_identify_object_as_address(classid, objid, objsubid)) FROM citus.pg_dist_object WHERE objid IN ('post_11_upgrade'::regnamespace, 'post_11_upgrade.part_table'::regclass, 'post_11_upgrade.sensors'::regclass, 'post_11_upgrade.func_in_transaction_def'::regproc, 'post_11_upgrade.partial_index_test_config'::regconfig, 'post_11_upgrade.my_type'::regtype) ORDER BY 1;$$) ORDER BY 1;
run_command_on_workers
---------------------------------------------------------------------
(localhost,57636,t,"{""(type,{post_11_upgrade.my_type},{})"",""(function,\\""{post_11_upgrade,func_in_transaction_def}\\"",{})"",""(table,\\""{post_11_upgrade,part_table}\\"",{})"",""(table,\\""{post_11_upgrade,sensors}\\"",{})"",""(schema,{post_11_upgrade},{})"",""(\\""text search configuration\\"",\\""{post_11_upgrade,partial_index_test_config}\\"",{})""}")
(localhost,57637,t,"{""(type,{post_11_upgrade.my_type},{})"",""(function,\\""{post_11_upgrade,func_in_transaction_def}\\"",{})"",""(table,\\""{post_11_upgrade,part_table}\\"",{})"",""(table,\\""{post_11_upgrade,sensors}\\"",{})"",""(schema,{post_11_upgrade},{})"",""(\\""text search configuration\\"",\\""{post_11_upgrade,partial_index_test_config}\\"",{})""}")
(2 rows)
-- Create the necessary test utility function
CREATE OR REPLACE FUNCTION activate_node_snapshot()
RETURNS text[]
LANGUAGE C STRICT
AS 'citus';
-- make sure that workers and the coordinator has the same datesyle
SET datestyle = "ISO, YMD";
SELECT 1 FROM run_command_on_workers($$ALTER SYSTEM SET datestyle = "ISO, YMD";$$);
?column?
---------------------------------------------------------------------
1
1
(2 rows)
SELECT 1 FROM run_command_on_workers($$SELECT pg_reload_conf()$$);
?column?
---------------------------------------------------------------------
1
1
(2 rows)
-- make sure that the metadata is consistent across all nodes
-- we exclude the distributed_object_data as they are
-- not sorted in the same order (as OIDs differ on the nodes)
SELECT count(*) = 0 AS same_metadata_in_workers FROM
(
(
SELECT unnest(activate_node_snapshot()) as command
EXCEPT
SELECT unnest(result::text[]) AS command
FROM run_command_on_workers($$SELECT post_11_upgrade.activate_node_snapshot()$$)
)
UNION
(
SELECT unnest(result::text[]) AS command
FROM run_command_on_workers($$SELECT post_11_upgrade.activate_node_snapshot()$$)
EXCEPT
SELECT unnest(activate_node_snapshot()) as command
)
) AS foo WHERE command NOT ILIKE '%distributed_object_data%';
same_metadata_in_workers
---------------------------------------------------------------------
t
(1 row)

View File

@ -0,0 +1,184 @@
-- test cases for #3970
SET citus.shard_count TO 32;
SET citus.shard_replication_factor TO 1;
CREATE SCHEMA post_11_upgrade;
SET search_path = post_11_upgrade;
--1. create a partitioned table, and a vanilla table that will be colocated with this table
CREATE TABLE part_table (
work_ymdt timestamp without time zone NOT NULL,
seq bigint NOT NULL,
my_seq bigint NOT NULL,
work_memo character varying(150),
CONSTRAINT work_memo_check CHECK ((octet_length((work_memo)::text) <= 150)),
PRIMARY KEY(seq, work_ymdt)
)
PARTITION BY RANGE (work_ymdt);
CREATE TABLE dist(seq bigint UNIQUE);
--2. perform create_distributed_table
SELECT create_distributed_table('part_table', 'seq');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SELECT create_distributed_table('dist','seq');
create_distributed_table
---------------------------------------------------------------------
(1 row)
--3. add a partitions
CREATE TABLE part_table_p202008 PARTITION OF part_table FOR VALUES FROM ('2020-08-01 00:00:00') TO ('2020-09-01 00:00:00');
CREATE TABLE part_table_p202009 PARTITION OF part_table FOR VALUES FROM ('2020-09-01 00:00:00') TO ('2020-10-01 00:00:00');
--3. create indexes
CREATE INDEX i_part_1 ON part_table(seq);
CREATE INDEX i_part_2 ON part_table(my_seq, seq);
CREATE INDEX i_part_3 ON part_table(work_memo, seq);
CREATE TABLE sensors(
measureid integer,
eventdatetime date,
measure_data jsonb,
PRIMARY KEY (measureid, eventdatetime, measure_data))
PARTITION BY RANGE(eventdatetime);
CREATE TABLE sensors_old PARTITION OF sensors FOR VALUES FROM ('2000-01-01') TO ('2020-01-01');
CREATE TABLE sensors_2020_01_01 PARTITION OF sensors FOR VALUES FROM ('2020-01-01') TO ('2020-02-01');
CREATE TABLE sensors_news PARTITION OF sensors FOR VALUES FROM ('2020-05-01') TO ('2025-01-01');
CREATE INDEX index_on_parent ON sensors(lower(measureid::text));
CREATE INDEX index_on_child ON sensors_2020_01_01(lower(measure_data::text));
CREATE INDEX hash_index ON sensors USING HASH((measure_data->'IsFailed'));
CREATE INDEX index_with_include ON sensors ((measure_data->'IsFailed')) INCLUDE (measure_data, eventdatetime);
CREATE STATISTICS s1 (dependencies) ON measureid, eventdatetime FROM sensors;
CREATE STATISTICS s2 (dependencies) ON measureid, eventdatetime FROM sensors_2020_01_01;
ALTER INDEX index_on_parent ALTER COLUMN 1 SET STATISTICS 1000;
ALTER INDEX index_on_child ALTER COLUMN 1 SET STATISTICS 1000;
CLUSTER sensors_2020_01_01 USING index_on_child;
SELECT create_distributed_table('sensors', 'measureid');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- create a colocated distributed tables and create foreign keys FROM/TO
-- the partitions
CREATE TABLE colocated_dist_table (measureid integer PRIMARY KEY);
SELECT create_distributed_table('colocated_dist_table', 'measureid');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CLUSTER colocated_dist_table USING colocated_dist_table_pkey;
WARNING: not propagating CLUSTER command to worker nodes
CREATE TABLE colocated_partitioned_table(
measureid integer,
eventdatetime date,
PRIMARY KEY (measureid, eventdatetime))
PARTITION BY RANGE(eventdatetime);
CREATE TABLE colocated_partitioned_table_2020_01_01 PARTITION OF colocated_partitioned_table FOR VALUES FROM ('2020-01-01') TO ('2020-02-01');
SELECT create_distributed_table('colocated_partitioned_table', 'measureid');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CLUSTER colocated_partitioned_table_2020_01_01 USING colocated_partitioned_table_2020_01_01_pkey;
WARNING: not propagating CLUSTER command to worker nodes
CREATE TABLE reference_table (measureid integer PRIMARY KEY);
SELECT create_reference_table('reference_table');
create_reference_table
---------------------------------------------------------------------
(1 row)
-- this table is used to make sure that index backed
-- replica identites can have clustered indexes
-- and no index statistics
CREATE TABLE index_backed_rep_identity(key int NOT NULL);
CREATE UNIQUE INDEX uqx ON index_backed_rep_identity(key);
ALTER TABLE index_backed_rep_identity REPLICA IDENTITY USING INDEX uqx;
CLUSTER index_backed_rep_identity USING uqx;
SELECT create_distributed_table('index_backed_rep_identity', 'key');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- from parent to regular dist
ALTER TABLE sensors ADD CONSTRAINT fkey_from_parent_to_dist FOREIGN KEY (measureid) REFERENCES colocated_dist_table(measureid);
-- from parent to parent
ALTER TABLE sensors ADD CONSTRAINT fkey_from_parent_to_parent FOREIGN KEY (measureid, eventdatetime) REFERENCES colocated_partitioned_table(measureid, eventdatetime);
-- from parent to child
ALTER TABLE sensors ADD CONSTRAINT fkey_from_parent_to_child FOREIGN KEY (measureid, eventdatetime) REFERENCES colocated_partitioned_table_2020_01_01(measureid, eventdatetime);
-- load some data
INSERT INTO reference_table SELECT i FROM generate_series(0,1000)i;
INSERT INTO colocated_dist_table SELECT i FROM generate_series(0,1000)i;
INSERT INTO colocated_partitioned_table SELECT i, '2020-01-05' FROM generate_series(0,1000)i;
INSERT INTO sensors SELECT i, '2020-01-05', '{}' FROM generate_series(0,1000)i;
SET citus.enable_ddl_propagation TO off;
CREATE TEXT SEARCH CONFIGURATION post_11_upgrade.partial_index_test_config ( parser = default );
SELECT 1 FROM run_command_on_workers($$CREATE TEXT SEARCH CONFIGURATION post_11_upgrade.partial_index_test_config ( parser = default );$$);
?column?
---------------------------------------------------------------------
1
1
(2 rows)
CREATE OR REPLACE FUNCTION post_11_upgrade.func_in_transaction_def()
RETURNS int
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;
SELECT run_command_on_workers('SET citus.enable_ddl_propagation TO off;
CREATE OR REPLACE FUNCTION post_11_upgrade.func_in_transaction_def()
RETURNS int
LANGUAGE plpgsql AS
$$
BEGIN
return 1;
END;
$$;');
run_command_on_workers
---------------------------------------------------------------------
(localhost,57636,t,SET)
(localhost,57637,t,SET)
(2 rows)
CREATE TYPE post_11_upgrade.my_type AS (a int);
RESET citus.enable_ddl_propagation;
CREATE TABLE sensors_parser(
measureid integer,
eventdatetime date,
measure_data jsonb,
name text,
col_with_def int DEFAULT post_11_upgrade.func_in_transaction_def(),
col_with_type post_11_upgrade.my_type,
PRIMARY KEY (measureid, eventdatetime, measure_data)
) PARTITION BY RANGE(eventdatetime);
CREATE TABLE sensors_parser_a_partition PARTITION OF sensors_parser FOR VALUES FROM ('2000-01-01') TO ('2020-01-01');
CREATE INDEX sensors_parser_search_name ON sensors_parser USING gin (to_tsvector('partial_index_test_config'::regconfig, (COALESCE(name, ''::character varying))::text));
SELECT create_distributed_table('sensors_parser', 'measureid');
create_distributed_table
---------------------------------------------------------------------
(1 row)
SET citus.enable_ddl_propagation TO off;
CREATE COLLATION post_11_upgrade.german_phonebook_unpropagated (provider = icu, locale = 'de-u-co-phonebk');
SELECT 1 FROM run_command_on_workers($$CREATE COLLATION post_11_upgrade.german_phonebook_unpropagated (provider = icu, locale = 'de-u-co-phonebk');$$);
?column?
---------------------------------------------------------------------
1
1
(2 rows)
SET citus.enable_ddl_propagation TO on;
CREATE TABLE test_propagate_collate(id int, t2 text COLLATE german_phonebook_unpropagated);
SELECT create_distributed_table('test_propagate_collate', 'id');
create_distributed_table
---------------------------------------------------------------------
(1 row)

View File

@ -636,7 +636,7 @@ INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::re
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'second_dustbunnies'::regclass::oid, 0); INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'second_dustbunnies'::regclass::oid, 0);
SELECT 1 FROM master_activate_node('localhost', :worker_1_port); SELECT 1 FROM master_activate_node('localhost', :worker_1_port);
NOTICE: Replicating postgres objects to node localhost:57637 NOTICE: Replicating postgres objects to node localhost:57637
DETAIL: There are 114 objects to replicate, depending on your environment this might take a while DETAIL: There are 115 objects to replicate, depending on your environment this might take a while
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1

View File

@ -89,7 +89,7 @@ step "s3-rollback"
step "s3-view-worker" step "s3-view-worker"
{ {
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC; SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND backend_type = 'client backend' ORDER BY query DESC;
} }
// we prefer to sleep before "s2-view-dist" so that we can ensure // we prefer to sleep before "s2-view-dist" so that we can ensure

View File

@ -49,11 +49,11 @@ step "s1-verify-current-xact-is-on-worker"
{ {
SELECT SELECT
remote.nodeport, remote.nodeport,
remote.result = row(xact.initiator_node_identifier, xact.transaction_number)::text AS xact_exists remote.result = row(xact.transaction_number)::text AS xact_exists
FROM FROM
get_current_transaction_id() as xact, get_current_transaction_id() as xact,
run_command_on_workers($$ run_command_on_workers($$
SELECT row(initiator_node_identifier, transaction_number) SELECT row(transaction_number)
FROM get_all_active_transactions() FROM get_all_active_transactions()
WHERE transaction_number != 0; WHERE transaction_number != 0;
$$) as remote $$) as remote

View File

@ -2,30 +2,37 @@
// How we organize this isolation test spec, is explained at README.md file in this directory. // How we organize this isolation test spec, is explained at README.md file in this directory.
// //
// create range distributed table to test behavior of DROP in concurrent operations // create distributed table to test behavior of DROP in concurrent operations
setup setup
{ {
SELECT citus_internal.replace_isolation_tester_func(); SELECT citus_internal.replace_isolation_tester_func();
SELECT citus_internal.refresh_isolation_tester_prepared_statement(); SELECT citus_internal.refresh_isolation_tester_prepared_statement();
SET citus.shard_replication_factor TO 1; SET citus.shard_replication_factor TO 1;
CREATE SCHEMA drop_tests
CREATE TABLE drop_hash(id integer, data text); CREATE TABLE drop_hash(id integer, data text);
SELECT create_distributed_table('drop_hash', 'id'); SELECT create_distributed_table('drop_tests.drop_hash', 'id');
CREATE SCHEMA drop_tests_2
CREATE TABLE drop_hash_2(id integer, data text);
SELECT create_distributed_table('drop_tests_2.drop_hash_2', 'id');
} }
// drop distributed table // drop distributed table
teardown teardown
{ {
DROP TABLE IF EXISTS drop_hash CASCADE; DROP TABLE IF EXISTS drop_tests.drop_hash, drop_tests_2.drop_hash_2 CASCADE;
DROP SCHEMA IF EXISTS drop_tests, drop_tests_2 CASCADE;
SELECT citus_internal.restore_isolation_tester_func(); SELECT citus_internal.restore_isolation_tester_func();
} }
// session 1 // session 1
session "s1" session "s1"
step "s1-initialize" { COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV; } step "s1-initialize" { SET search_path TO 'drop_tests'; COPY drop_hash FROM PROGRAM 'echo 0, a && echo 1, b && echo 2, c && echo 3, d && echo 4, e' WITH CSV;}
step "s1-begin" { BEGIN; } step "s1-begin" { BEGIN; }
step "s1-drop" { DROP TABLE drop_hash; } step "s1-drop" { DROP TABLE drop_hash; }
step "s1-drop-schema" { DROP SCHEMA drop_tests CASCADE; }
step "s1-ddl-create-index" { CREATE INDEX drop_hash_index ON drop_hash(id); } step "s1-ddl-create-index" { CREATE INDEX drop_hash_index ON drop_hash(id); }
step "s1-ddl-drop-index" { DROP INDEX drop_hash_index; } step "s1-ddl-drop-index" { DROP INDEX drop_hash_index; }
step "s1-ddl-add-column" { ALTER TABLE drop_hash ADD new_column int DEFAULT 0; } step "s1-ddl-add-column" { ALTER TABLE drop_hash ADD new_column int DEFAULT 0; }
@ -41,8 +48,11 @@ step "s1-commit" { COMMIT; }
// session 2 // session 2
session "s2" session "s2"
step "s2-initialize" { SET search_path TO 'drop_tests'; }
step "s2-begin" { BEGIN; } step "s2-begin" { BEGIN; }
step "s2-drop" { DROP TABLE drop_hash; } step "s2-drop" { DROP TABLE drop_hash; }
step "s2-drop-schema" { DROP SCHEMA drop_tests CASCADE; }
step "s2-drop-schema-2" { DROP SCHEMA drop_tests_2 CASCADE; }
step "s2-ddl-create-index" { CREATE INDEX drop_hash_index ON drop_hash(id); } step "s2-ddl-create-index" { CREATE INDEX drop_hash_index ON drop_hash(id); }
step "s2-ddl-drop-index" { DROP INDEX drop_hash_index; } step "s2-ddl-drop-index" { DROP INDEX drop_hash_index; }
step "s2-ddl-create-index-concurrently" { CREATE INDEX CONCURRENTLY drop_hash_index ON drop_hash(id); } step "s2-ddl-create-index-concurrently" { CREATE INDEX CONCURRENTLY drop_hash_index ON drop_hash(id); }
@ -54,23 +64,25 @@ step "s2-distribute-table" { SELECT create_distributed_table('drop_hash', 'id');
step "s2-commit" { COMMIT; } step "s2-commit" { COMMIT; }
// permutations - DROP vs DROP // permutations - DROP vs DROP
permutation "s1-initialize" "s1-begin" "s2-begin" "s1-drop" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" permutation "s1-initialize" "s2-initialize" "s1-begin" "s2-begin" "s1-drop" "s2-drop" "s1-commit" "s2-commit" "s1-select-count"
permutation "s1-initialize" "s2-initialize" "s1-begin" "s2-begin" "s1-drop-schema" "s2-drop-schema" "s1-commit" "s2-commit" "s1-select-count"
permutation "s1-initialize" "s2-initialize" "s1-begin" "s2-begin" "s1-drop-schema" "s2-drop-schema-2" "s1-commit" "s2-commit" "s1-select-count"
// permutations - DROP first // permutations - DROP first
permutation "s1-initialize" "s1-begin" "s2-begin" "s1-drop" "s2-ddl-create-index" "s1-commit" "s2-commit" "s1-select-count" "s1-show-indexes" permutation "s1-initialize" "s2-initialize" "s1-begin" "s2-begin" "s1-drop" "s2-ddl-create-index" "s1-commit" "s2-commit" "s1-select-count" "s1-show-indexes"
permutation "s1-initialize" "s1-ddl-create-index" "s1-begin" "s2-begin" "s1-drop" "s2-ddl-drop-index" "s1-commit" "s2-commit" "s1-select-count" "s1-show-indexes" permutation "s1-initialize" "s2-initialize" "s1-ddl-create-index" "s1-begin" "s2-begin" "s1-drop" "s2-ddl-drop-index" "s1-commit" "s2-commit" "s1-select-count" "s1-show-indexes"
permutation "s1-initialize" "s1-begin" "s1-drop" "s2-ddl-create-index-concurrently" "s1-commit" "s1-select-count" "s1-show-indexes" permutation "s1-initialize" "s2-initialize" "s1-begin" "s1-drop" "s2-ddl-create-index-concurrently" "s1-commit" "s1-select-count" "s1-show-indexes"
permutation "s1-initialize" "s1-begin" "s2-begin" "s1-drop" "s2-ddl-add-column" "s1-commit" "s2-commit" "s1-select-count" "s1-show-columns" permutation "s1-initialize" "s2-initialize" "s1-begin" "s2-begin" "s1-drop" "s2-ddl-add-column" "s1-commit" "s2-commit" "s1-select-count" "s1-show-columns"
permutation "s1-initialize" "s1-ddl-add-column" "s1-begin" "s2-begin" "s1-drop" "s2-ddl-drop-column" "s1-commit" "s2-commit" "s1-select-count" "s1-show-columns" permutation "s1-initialize" "s2-initialize" "s1-ddl-add-column" "s1-begin" "s2-begin" "s1-drop" "s2-ddl-drop-column" "s1-commit" "s2-commit" "s1-select-count" "s1-show-columns"
permutation "s1-initialize" "s1-begin" "s2-begin" "s1-drop" "s2-ddl-rename-column" "s1-commit" "s2-commit" "s1-select-count" "s1-show-columns" permutation "s1-initialize" "s2-initialize" "s1-begin" "s2-begin" "s1-drop" "s2-ddl-rename-column" "s1-commit" "s2-commit" "s1-select-count" "s1-show-columns"
permutation "s1-initialize" "s1-begin" "s2-begin" "s1-drop" "s2-table-size" "s1-commit" "s2-commit" "s1-select-count" permutation "s1-initialize" "s2-initialize" "s1-begin" "s2-begin" "s1-drop" "s2-table-size" "s1-commit" "s2-commit" "s1-select-count"
permutation "s1-drop" "s1-create-non-distributed-table" "s1-initialize" "s1-begin" "s2-begin" "s1-drop" "s2-distribute-table" "s1-commit" "s2-commit" "s1-select-count" permutation "s1-drop" "s1-create-non-distributed-table" "s1-initialize" "s2-initialize" "s1-begin" "s2-begin" "s1-drop" "s2-distribute-table" "s1-commit" "s2-commit" "s1-select-count"
// permutations - DROP second // permutations - DROP second
permutation "s1-initialize" "s1-begin" "s2-begin" "s1-ddl-create-index" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" "s1-show-indexes" permutation "s1-initialize" "s2-initialize" "s1-begin" "s2-begin" "s1-ddl-create-index" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" "s1-show-indexes"
permutation "s1-initialize" "s1-ddl-create-index" "s1-begin" "s2-begin" "s1-ddl-drop-index" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" "s1-show-indexes" permutation "s1-initialize" "s2-initialize" "s1-ddl-create-index" "s1-begin" "s2-begin" "s1-ddl-drop-index" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" "s1-show-indexes"
permutation "s1-initialize" "s1-begin" "s2-begin" "s1-ddl-add-column" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" "s1-show-columns" permutation "s1-initialize" "s2-initialize" "s1-begin" "s2-begin" "s1-ddl-add-column" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" "s1-show-columns"
permutation "s1-initialize" "s1-ddl-add-column" "s1-begin" "s2-begin" "s1-ddl-drop-column" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" "s1-show-columns" permutation "s1-initialize" "s2-initialize" "s1-ddl-add-column" "s1-begin" "s2-begin" "s1-ddl-drop-column" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" "s1-show-columns"
permutation "s1-initialize" "s1-begin" "s2-begin" "s1-ddl-rename-column" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" "s1-show-columns" permutation "s1-initialize" "s2-initialize" "s1-begin" "s2-begin" "s1-ddl-rename-column" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" "s1-show-columns"
permutation "s1-initialize" "s1-begin" "s2-begin" "s1-table-size" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" permutation "s1-initialize" "s2-initialize" "s1-begin" "s2-begin" "s1-table-size" "s2-drop" "s1-commit" "s2-commit" "s1-select-count"
permutation "s1-drop" "s1-create-non-distributed-table" "s1-initialize" "s1-begin" "s2-begin" "s1-distribute-table" "s2-drop" "s1-commit" "s2-commit" "s1-select-count" permutation "s1-drop" "s1-create-non-distributed-table" "s1-initialize" "s2-initialize" "s1-begin" "s2-begin" "s1-distribute-table" "s2-drop" "s1-commit" "s2-commit" "s1-select-count"

View File

@ -77,7 +77,11 @@ step "s3-as-admin"
step "s3-as-user-1" step "s3-as-user-1"
{ {
-- User should only be able to see its own transactions -- Even though we change the user via SET ROLE, the backends' (e.g., s1/2-begin-insert)
-- userId (e.g., PG_PROC->userId) does not change, and hence none of the
-- transactions show up because here we are using test_user_1. This is a
-- limitation of isolation tester, we should be able to re-connect with
-- test_user_1 on s1/2-begin-insert to show that test_user_1 sees only its own processes
SET ROLE test_user_1; SET ROLE test_user_1;
SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0; SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0;
SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0; SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0;
@ -85,7 +89,12 @@ step "s3-as-user-1"
step "s3-as-readonly" step "s3-as-readonly"
{ {
-- Other user should not see transactions -- Even though we change the user via SET ROLE, the backends' (e.g., s1/2-begin-insert)
-- userId (e.g., PG_PROC->userId) does not change, and hence none of the
-- transactions show up because here we are using test_readonly. This is a
-- limitation of isolation tester, we should be able to re-connect with
-- test_readonly on s1/2-begin-insert to show that test_readonly sees only
-- its own processes
SET ROLE test_readonly; SET ROLE test_readonly;
SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0; SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0;
SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0; SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0;

View File

@ -29,7 +29,6 @@ setup
teardown teardown
{ {
// drop all distributed tables
DROP TABLE IF EXISTS ref_table, DROP TABLE IF EXISTS ref_table,
dist_table, dist_table,
dist_partitioned_table, dist_partitioned_table,
@ -39,7 +38,6 @@ teardown
new_ref_table; new_ref_table;
// drop all distributed objects
DROP FUNCTION activate_node_snapshot(); DROP FUNCTION activate_node_snapshot();
DROP FUNCTION IF EXISTS squares(int); DROP FUNCTION IF EXISTS squares(int);
DROP TYPE IF EXISTS my_type; DROP TYPE IF EXISTS my_type;
@ -110,6 +108,19 @@ step "s2-create-dist-table"
SELECT create_distributed_table('new_dist_table', 'id'); SELECT create_distributed_table('new_dist_table', 'id');
} }
step "s2-create-schema"
{
CREATE SCHEMA dist_schema
CREATE TABLE dist_table_in_schema(id int, data int);
SELECT create_distributed_table('dist_schema.dist_table_in_schema', 'id');
}
step "s2-drop-schema"
{
DROP SCHEMA dist_schema CASCADE;
}
step "s2-create-ref-table" step "s2-create-ref-table"
{ {
CREATE TABLE new_ref_table(id int, data int); CREATE TABLE new_ref_table(id int, data int);
@ -136,6 +147,16 @@ step "s2-create-type"
CREATE TYPE my_type AS (a int, b int); CREATE TYPE my_type AS (a int, b int);
} }
step "s2-drop-type"
{
DROP TYPE my_type;
}
step "s2-alter-type"
{
ALTER TYPE my_type ADD ATTRIBUTE x int;
}
step "s2-create-dist-func" step "s2-create-dist-func"
{ {
CREATE FUNCTION squares(int) RETURNS SETOF RECORD CREATE FUNCTION squares(int) RETURNS SETOF RECORD
@ -145,6 +166,11 @@ step "s2-create-dist-func"
SELECT create_distributed_function('squares(int)'); SELECT create_distributed_function('squares(int)');
} }
step "s2-drop-dist-func"
{
DROP FUNCTION squares(int);
}
session "s3" session "s3"
step "s3-compare-snapshot" step "s3-compare-snapshot"
@ -168,6 +194,11 @@ step "s3-compare-snapshot"
) AS foo; ) AS foo;
} }
step "s3-compare-type-definition"
{
SELECT run_command_on_workers($$SELECT '(1,1,1)'::my_type$$);
}
step "s3-debug" step "s3-debug"
{ {
SELECT unnest(activate_node_snapshot()); SELECT unnest(activate_node_snapshot());
@ -186,6 +217,8 @@ permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-start-metadata-sy
// the following operations get blocked when a concurrent metadata sync is in progress // the following operations get blocked when a concurrent metadata sync is in progress
permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-alter-table" "s1-commit" "s2-commit" "s3-compare-snapshot" permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-alter-table" "s1-commit" "s2-commit" "s3-compare-snapshot"
permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-drop-table" "s1-commit" "s2-commit" "s3-compare-snapshot" permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-drop-table" "s1-commit" "s2-commit" "s3-compare-snapshot"
permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-create-schema" "s1-commit" "s2-commit" "s3-compare-snapshot" "s2-drop-schema"
permutation "s2-create-schema" "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-drop-schema" "s1-commit" "s2-commit" "s3-compare-snapshot"
permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-create-dist-table" "s1-commit" "s2-commit" "s3-compare-snapshot" permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-create-dist-table" "s1-commit" "s2-commit" "s3-compare-snapshot"
permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-create-ref-table" "s1-commit" "s2-commit" "s3-compare-snapshot" permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-create-ref-table" "s1-commit" "s2-commit" "s3-compare-snapshot"
permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-attach-partition" "s1-commit" "s2-commit" "s3-compare-snapshot" permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-attach-partition" "s1-commit" "s2-commit" "s3-compare-snapshot"
@ -193,6 +226,16 @@ permutation "s2-attach-partition" "s1-begin" "s2-begin" "s1-start-metadata-sync"
permutation "s2-attach-partition" "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-create-partition-of" "s1-commit" "s2-commit" "s3-compare-snapshot" permutation "s2-attach-partition" "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-create-partition-of" "s1-commit" "s2-commit" "s3-compare-snapshot"
permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-add-fk" "s1-commit" "s2-commit" "s3-compare-snapshot" permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-add-fk" "s1-commit" "s2-commit" "s3-compare-snapshot"
permutation "s2-add-fk" "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-drop-fk" "s1-commit" "s2-commit" "s3-compare-snapshot" permutation "s2-add-fk" "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-drop-fk" "s1-commit" "s2-commit" "s3-compare-snapshot"
permutation "s2-create-type" "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-drop-type" "s1-commit" "s2-commit" "s3-compare-snapshot"
permutation "s2-create-dist-func" "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-drop-dist-func" "s1-commit" "s2-commit" "s3-compare-snapshot"
permutation "s2-create-type" "s1-begin" "s1-start-metadata-sync" "s2-alter-type" "s1-commit" "s3-compare-snapshot" "s3-compare-type-definition"
// the following operations block concurrent metadata sync calls
permutation "s1-begin" "s2-begin" "s2-create-dist-table" "s1-start-metadata-sync" "s2-commit" "s1-commit" "s3-compare-snapshot"
permutation "s2-create-dist-func" "s1-begin" "s2-begin" "s2-drop-dist-func" "s1-start-metadata-sync" "s2-commit" "s1-commit" "s3-compare-snapshot"
permutation "s2-create-schema" "s1-begin" "s2-begin" "s2-drop-schema" "s1-start-metadata-sync" "s2-commit" "s1-commit" "s3-compare-snapshot"
permutation "s2-create-type" "s1-begin" "s2-begin" "s2-alter-type" "s1-start-metadata-sync" "s2-commit" "s1-commit" "s3-compare-snapshot" "s3-compare-type-definition"
// the following operations do not get blocked // the following operations do not get blocked
permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-create-type" "s1-commit" "s2-commit" "s3-compare-snapshot" permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-create-type" "s1-commit" "s2-commit" "s3-compare-snapshot"

View File

@ -3,8 +3,6 @@ setup
SELECT citus_internal.replace_isolation_tester_func(); SELECT citus_internal.replace_isolation_tester_func();
SELECT citus_internal.refresh_isolation_tester_prepared_statement(); SELECT citus_internal.refresh_isolation_tester_prepared_statement();
SELECT master_add_node('localhost', 57636, groupid => 0);
CREATE TABLE ref_table(a int primary key); CREATE TABLE ref_table(a int primary key);
SELECT create_reference_table('ref_table'); SELECT create_reference_table('ref_table');
INSERT INTO ref_table VALUES (1), (3), (5), (7); INSERT INTO ref_table VALUES (1), (3), (5), (7);
@ -83,7 +81,7 @@ step "s2-lock-ref-table-placement-on-coordinator"
step "s2-view-dist" step "s2-view-dist"
{ {
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND query NOT ILIKE '%pg_isolation_test_session_is_blocked%' AND query NOT ILIKE '%BEGIN%' ORDER BY query DESC; SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND query NOT ILIKE '%pg_isolation_test_session_is_blocked%' AND query NOT ILIKE '%BEGIN%' AND query NOT ILIKE '%add_node%' ORDER BY query DESC;
} }
step "s2-view-worker" step "s2-view-worker"
@ -94,7 +92,9 @@ step "s2-view-worker"
WHERE query NOT ILIKE '%pg_prepared_xacts%' AND WHERE query NOT ILIKE '%pg_prepared_xacts%' AND
query NOT ILIKE '%COMMIT%' AND query NOT ILIKE '%COMMIT%' AND
query NOT ILIKE '%dump_local_%' AND query NOT ILIKE '%dump_local_%' AND
query NOT ILIKE '%citus_internal_local_blocked_processes%' query NOT ILIKE '%citus_internal_local_blocked_processes%' AND
query NOT ILIKE '%add_node%' AND
backend_type = 'client backend'
ORDER BY query, query_hostport DESC; ORDER BY query, query_hostport DESC;
} }
@ -123,14 +123,25 @@ step "deadlock-checker-call"
SELECT check_distributed_deadlocks(); SELECT check_distributed_deadlocks();
} }
// adding node in setup stage prevents getting a gpid with proper nodeid
session "add-node"
// we issue the checker not only when there are deadlocks to ensure that we never cancel
// backend inappropriately
step "add-node"
{
SELECT 1 FROM master_add_node('localhost', 57636, groupid => 0);
}
// verify that locks on the placement of the reference table on the coordinator is // verify that locks on the placement of the reference table on the coordinator is
// taken into account when looking for distributed deadlocks // taken into account when looking for distributed deadlocks
permutation "s1-begin" "s2-begin" "s1-update-dist-table" "s2-lock-ref-table-placement-on-coordinator" "s1-lock-ref-table-placement-on-coordinator" "s2-update-dist-table" "deadlock-checker-call" "s1-end" "s2-end" permutation "add-node" "s1-begin" "s2-begin" "s1-update-dist-table" "s2-lock-ref-table-placement-on-coordinator" "s1-lock-ref-table-placement-on-coordinator" "s2-update-dist-table" "deadlock-checker-call" "s1-end" "s2-end"
// verify that *_dist_stat_activity() functions return the correct result when query // verify that *_dist_stat_activity() functions return the correct result when query
// has a task on the coordinator. // has a task on the coordinator.
permutation "s1-begin" "s2-begin" "s1-update-ref-table" "s2-sleep" "s2-view-dist" "s2-view-worker" "s2-end" "s1-end" permutation "add-node" "s1-begin" "s2-begin" "s1-update-ref-table" "s2-sleep" "s2-view-dist" "s2-view-worker" "s2-end" "s1-end"
// verify that get_*_active_transactions() functions return the correct result when // verify that get_*_active_transactions() functions return the correct result when
// the query has a task on the coordinator. // the query has a task on the coordinator.
permutation "s1-begin" "s2-begin" "s1-update-ref-table" "s2-active-transactions" "s1-end" "s2-end" permutation "add-node" "s1-begin" "s2-begin" "s1-update-ref-table" "s2-active-transactions" "s1-end" "s2-end"

View File

@ -24,7 +24,6 @@ ROLLBACK;
BEGIN; BEGIN;
INSERT INTO ab values(1, 2); INSERT INTO ab values(1, 2);
-- DDL happened before repartition query in a transaction block, so this should error.
SELECT count(*) FROM (SELECT k.a FROM ab k, ab l WHERE k.a = l.b) first, (SELECT * FROM ab) second WHERE first.a = second.b; SELECT count(*) FROM (SELECT k.a FROM ab k, ab l WHERE k.a = l.b) first, (SELECT * FROM ab) second WHERE first.a = second.b;
ROLLBACK; ROLLBACK;

View File

@ -160,6 +160,10 @@ create aggregate binstragg(text, text)(
combinefunc=binstragg_combinefunc, combinefunc=binstragg_combinefunc,
stype=text stype=text
); );
-- verify that the aggregate is added into pg_dist_object, on each worker
SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.binstragg'::regproc;$$);
SELECT run_command_on_workers($$select count(*) from pg_aggregate where aggfnoid::text like '%binstragg%';$$);
select create_distributed_function('binstragg(text,text)'); select create_distributed_function('binstragg(text,text)');
@ -313,7 +317,6 @@ SELECT id%5, first(val ORDER BY key), last(val ORDER BY key)
FROM aggdata GROUP BY id%5 ORDER BY id%5; FROM aggdata GROUP BY id%5 ORDER BY id%5;
-- test aggregate with stype which is not a by-value datum -- test aggregate with stype which is not a by-value datum
-- also test our handling of the aggregate not existing on workers
create function sumstring_sfunc(state text, x text) create function sumstring_sfunc(state text, x text)
returns text immutable language plpgsql as $$ returns text immutable language plpgsql as $$
begin return (state::float8 + x::float8)::text; begin return (state::float8 + x::float8)::text;
@ -326,8 +329,10 @@ create aggregate sumstring(text) (
combinefunc = sumstring_sfunc, combinefunc = sumstring_sfunc,
initcond = '0' initcond = '0'
); );
-- verify that the aggregate is propagated
select aggfnoid from pg_aggregate where aggfnoid::text like '%sumstring%';
SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%sumstring%';$$);
select sumstring(valf::text) from aggdata where valf is not null;
select create_distributed_function('sumstring(text)'); select create_distributed_function('sumstring(text)');
select sumstring(valf::text) from aggdata where valf is not null; select sumstring(valf::text) from aggdata where valf is not null;
@ -533,5 +538,79 @@ SELECT floor(AVG(COALESCE(agg_col, 10)))
FROM dist_table FROM dist_table
LEFT JOIN ref_table ON TRUE; LEFT JOIN ref_table ON TRUE;
-- try createing aggregate having non-distributable dependency type
create table dummy_tbl (a int);
create function dummy_fnc(a dummy_tbl, d double precision) RETURNS dummy_tbl
AS $$SELECT 1;$$ LANGUAGE sql;
-- should give warning and create aggregate local only
create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc);
-- clear and try again with distributed table
DROP TABLE dummy_tbl CASCADE;
create table dummy_tbl (a int);
SELECT create_distributed_table('dummy_tbl','a');
create function dummy_fnc(a dummy_tbl, d double precision) RETURNS dummy_tbl
AS $$SELECT 1;$$ LANGUAGE sql;
-- test in tx block
-- shouldn't distribute, as citus.create_object_propagation is set to deferred
BEGIN;
create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc);
COMMIT;
-- verify not distributed
SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$);
drop aggregate dependent_agg ( double precision);
-- now try with create_object_propagation = immediate
SET citus.create_object_propagation TO immediate;
-- should distribute, as citus.create_object_propagation is set to immediate
-- will switch to sequential mode
BEGIN;
create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc);
COMMIT;
-- verify distributed
SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$);
drop aggregate dependent_agg ( double precision);
-- now try with create_object_propagation = automatic
SET citus.create_object_propagation TO automatic;
-- should distribute, as citus.create_object_propagation is set to automatic
-- will switch to sequential mode
BEGIN;
create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc);
COMMIT;
-- verify distributed
SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$);
-- verify that the aggregate is added into pg_dist_object, on each worker
SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$);
RESET citus.create_object_propagation;
-- drop and test outside of tx block
drop aggregate dependent_agg (float8);
-- verify that the aggregate is removed from pg_dist_object, on each worker
SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$);
create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc);
--verify
SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$);
DROP TABLE dummy_tbl CASCADE;
SET citus.create_object_propagation TO automatic;
begin;
create type typ1 as (a int);
create or replace function fnagg(a typ1, d double precision) RETURNS typ1 AS $$SELECT 1;$$LANGUAGE sql;
create aggregate dependent_agg (float8) (stype=typ1, sfunc=fnagg);
commit;
RESET citus.create_object_propagation;
SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$);
set client_min_messages to error; set client_min_messages to error;
drop schema aggregate_support cascade; drop schema aggregate_support cascade;

View File

@ -112,7 +112,7 @@ SET citus.enable_repartition_joins TO ON;
SELECT count(*) FROM test t1, test t2 WHERE t1.x = t2.y; SELECT count(*) FROM test t1, test t2 WHERE t1.x = t2.y;
BEGIN; BEGIN;
SET citus.enable_repartition_joins TO ON; SET citus.enable_unique_job_ids TO off;
SELECT count(*) FROM test t1, test t2 WHERE t1.x = t2.y; SELECT count(*) FROM test t1, test t2 WHERE t1.x = t2.y;
END; END;

Some files were not shown because too many files have changed in this diff Show More