Merge branch 'master' into fix/subscript-jsonb

pull/5692/head
Onur Tirtir 2022-03-09 14:08:17 +03:00 committed by GitHub
commit 6fa965cd17
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
252 changed files with 9034 additions and 3003 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

@ -31,6 +31,7 @@
#include "distributed/commands/sequence.h" #include "distributed/commands/sequence.h"
#include "distributed/commands/utility_hook.h" #include "distributed/commands/utility_hook.h"
#include "distributed/metadata/distobject.h" #include "distributed/metadata/distobject.h"
#include "distributed/metadata/dependency.h"
#include "distributed/foreign_key_relationship.h" #include "distributed/foreign_key_relationship.h"
#include "distributed/listutils.h" #include "distributed/listutils.h"
#include "distributed/local_executor.h" #include "distributed/local_executor.h"
@ -317,6 +318,7 @@ CreateCitusLocalTable(Oid relationId, bool cascadeViaForeignKeys, bool autoConve
* Ensure dependencies exist as we will create shell table on the other nodes * Ensure dependencies exist as we will create shell table on the other nodes
* in the MX case. * in the MX case.
*/ */
EnsureRelationDependenciesCanBeDistributed(&tableAddress);
EnsureDependenciesExistOnAllNodes(&tableAddress); EnsureDependenciesExistOnAllNodes(&tableAddress);
/* /*

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

@ -443,6 +443,8 @@ CreateDistributedTable(Oid relationId, char *distributionColumnName,
*/ */
ObjectAddress tableAddress = { 0 }; ObjectAddress tableAddress = { 0 };
ObjectAddressSet(tableAddress, RelationRelationId, relationId); ObjectAddressSet(tableAddress, RelationRelationId, relationId);
EnsureRelationDependenciesCanBeDistributed(&tableAddress);
EnsureDependenciesExistOnAllNodes(&tableAddress); EnsureDependenciesExistOnAllNodes(&tableAddress);
char replicationModel = DecideReplicationModel(distributionMethod, char replicationModel = DecideReplicationModel(distributionMethod,

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

@ -11,11 +11,13 @@
#include "postgres.h" #include "postgres.h"
#include "miscadmin.h" #include "miscadmin.h"
#include "distributed/commands/utility_hook.h" #include "distributed/commands/utility_hook.h"
#include "distributed/commands.h" #include "distributed/commands.h"
#include "distributed/metadata_utility.h" #include "distributed/metadata_utility.h"
#include "distributed/coordinator_protocol.h" #include "distributed/coordinator_protocol.h"
#include "distributed/metadata_sync.h" #include "distributed/metadata_sync.h"
#include "distributed/multi_partitioning_utils.h"
#include "distributed/worker_transaction.h" #include "distributed/worker_transaction.h"
#include "utils/builtins.h" #include "utils/builtins.h"
#include "utils/lsyscache.h" #include "utils/lsyscache.h"
@ -123,6 +125,10 @@ master_remove_distributed_table_metadata_from_workers(PG_FUNCTION_ARGS)
* The function is a no-op for non-distributed tables and clusters that don't * The function is a no-op for non-distributed tables and clusters that don't
* have any workers with metadata. Also, the function errors out if called * have any workers with metadata. Also, the function errors out if called
* from a worker node. * from a worker node.
*
* This function assumed that it is called via a trigger. But we cannot do the
* typical CALLED_AS_TRIGGER check because this is called via another trigger,
* which CALLED_AS_TRIGGER does not cover.
*/ */
static void static void
MasterRemoveDistributedTableMetadataFromWorkers(Oid relationId, char *schemaName, MasterRemoveDistributedTableMetadataFromWorkers(Oid relationId, char *schemaName,
@ -146,6 +152,16 @@ MasterRemoveDistributedTableMetadataFromWorkers(Oid relationId, char *schemaName
return; return;
} }
if (PartitionTable(relationId))
{
/*
* MasterRemoveDistributedTableMetadataFromWorkers is only called from drop trigger.
* When parent is dropped in a drop trigger, we remove all the corresponding
* partitions via the parent, mostly for performance reasons.
*/
return;
}
/* drop the distributed table metadata on the workers */ /* drop the distributed table metadata on the workers */
char *deleteDistributionCommand = DistributionDeleteCommand(schemaName, tableName); char *deleteDistributionCommand = DistributionDeleteCommand(schemaName, tableName);
SendCommandToWorkersWithMetadata(deleteDistributionCommand); SendCommandToWorkersWithMetadata(deleteDistributionCommand);

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

@ -69,6 +69,10 @@
(strncmp(arg, prefix, strlen(prefix)) == 0) (strncmp(arg, prefix, strlen(prefix)) == 0)
/* forward declaration for helper functions*/ /* forward declaration for helper functions*/
static bool RecreateSameNonColocatedFunction(ObjectAddress functionAddress,
char *distributionArgumentName,
bool colocateWithTableNameDefault,
bool *forceDelegationAddress);
static void ErrorIfAnyNodeDoesNotHaveMetadata(void); static void ErrorIfAnyNodeDoesNotHaveMetadata(void);
static char * GetAggregateDDLCommand(const RegProcedure funcOid, bool useCreateOrReplace); static char * GetAggregateDDLCommand(const RegProcedure funcOid, bool useCreateOrReplace);
static char * GetFunctionAlterOwnerCommand(const RegProcedure funcOid); static char * GetFunctionAlterOwnerCommand(const RegProcedure funcOid);
@ -82,7 +86,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);
@ -129,6 +132,7 @@ create_distributed_function(PG_FUNCTION_ARGS)
char *distributionArgumentName = NULL; char *distributionArgumentName = NULL;
char *colocateWithTableName = NULL; char *colocateWithTableName = NULL;
bool colocateWithTableNameDefault = false;
bool *forceDelegationAddress = NULL; bool *forceDelegationAddress = NULL;
bool forceDelegation = false; bool forceDelegation = false;
ObjectAddress extensionAddress = { 0 }; ObjectAddress extensionAddress = { 0 };
@ -168,8 +172,13 @@ create_distributed_function(PG_FUNCTION_ARGS)
colocateWithText = PG_GETARG_TEXT_P(2); colocateWithText = PG_GETARG_TEXT_P(2);
colocateWithTableName = text_to_cstring(colocateWithText); colocateWithTableName = text_to_cstring(colocateWithText);
if (pg_strncasecmp(colocateWithTableName, "default", NAMEDATALEN) == 0)
{
colocateWithTableNameDefault = true;
}
/* check if the colocation belongs to a reference table */ /* check if the colocation belongs to a reference table */
if (pg_strncasecmp(colocateWithTableName, "default", NAMEDATALEN) != 0) if (!colocateWithTableNameDefault)
{ {
Oid colocationRelationId = ResolveRelationId(colocateWithText, false); Oid colocationRelationId = ResolveRelationId(colocateWithText, false);
colocatedWithReferenceTable = IsCitusTableType(colocationRelationId, colocatedWithReferenceTable = IsCitusTableType(colocationRelationId,
@ -193,6 +202,20 @@ create_distributed_function(PG_FUNCTION_ARGS)
ObjectAddressSet(functionAddress, ProcedureRelationId, funcOid); ObjectAddressSet(functionAddress, ProcedureRelationId, funcOid);
if (RecreateSameNonColocatedFunction(functionAddress,
distributionArgumentName,
colocateWithTableNameDefault,
forceDelegationAddress))
{
char *schemaName = get_namespace_name(get_func_namespace(funcOid));
char *functionName = get_func_name(funcOid);
char *qualifiedName = quote_qualified_identifier(schemaName, functionName);
ereport(NOTICE, (errmsg("procedure %s is already distributed", qualifiedName),
errdetail("Citus distributes procedures with CREATE "
"[PROCEDURE|FUNCTION|AGGREGATE] commands")));
PG_RETURN_VOID();
}
/* /*
* If the function is owned by an extension, only update the * If the function is owned by an extension, only update the
* pg_dist_object, and not propagate the CREATE FUNCTION. Function * pg_dist_object, and not propagate the CREATE FUNCTION. Function
@ -260,6 +283,55 @@ create_distributed_function(PG_FUNCTION_ARGS)
} }
/*
* RecreateSameNonColocatedFunction returns true if the given parameters of
* create_distributed_function will not change anything on the given function.
* Returns false otherwise.
*/
static bool
RecreateSameNonColocatedFunction(ObjectAddress functionAddress,
char *distributionArgumentName,
bool colocateWithTableNameDefault,
bool *forceDelegationAddress)
{
DistObjectCacheEntry *cacheEntry =
LookupDistObjectCacheEntry(ProcedureRelationId,
functionAddress.objectId,
InvalidOid);
if (cacheEntry == NULL || !cacheEntry->isValid || !cacheEntry->isDistributed)
{
return false;
}
/*
* If the colocationId, forceDelegation and distributionArgIndex fields of a
* pg_dist_object entry of a distributed function are all set to zero, it means
* that function is either automatically distributed by ddl propagation, without
* calling create_distributed_function. Or, it could be distributed via
* create_distributed_function, but with no parameters.
*
* For these cases, calling create_distributed_function for that function,
* without parameters would be idempotent. Hence we can simply early return here,
* by providing a notice message to the user.
*/
/* are pg_dist_object fields set to zero? */
bool functionDistributedWithoutParams =
cacheEntry->colocationId == 0 &&
cacheEntry->forceDelegation == 0 &&
cacheEntry->distributionArgIndex == 0;
/* called create_distributed_function without parameters? */
bool distributingAgainWithNoParams =
distributionArgumentName == NULL &&
colocateWithTableNameDefault &&
forceDelegationAddress == NULL;
return functionDistributedWithoutParams && distributingAgainWithNoParams;
}
/* /*
* ErrorIfAnyNodeDoesNotHaveMetadata throws error if any * ErrorIfAnyNodeDoesNotHaveMetadata throws error if any
* of the worker nodes does not have the metadata. * of the worker nodes does not have the metadata.
@ -1348,51 +1420,6 @@ PostprocessCreateFunctionStmt(Node *node, const char *queryString)
} }
/*
* GetUndistributableDependency checks whether object has any non-distributable
* dependency. If any one found, it will be returned.
*/
static ObjectAddress *
GetUndistributableDependency(ObjectAddress *objectAddress)
{
List *dependencies = GetAllDependenciesForObject(objectAddress);
ObjectAddress *dependency = NULL;
foreach_ptr(dependency, dependencies)
{
if (IsObjectDistributed(dependency))
{
continue;
}
if (!SupportedDependencyByCitus(dependency))
{
/*
* Since roles should be handled manually with Citus community, skip them.
*/
if (getObjectClass(dependency) != OCLASS_ROLE)
{
return dependency;
}
}
if (getObjectClass(dependency) == OCLASS_CLASS)
{
/*
* Citus can only distribute dependent non-distributed sequence
* and composite types.
*/
char relKind = get_rel_relkind(dependency->objectId);
if (relKind != RELKIND_SEQUENCE && relKind != RELKIND_COMPOSITE_TYPE)
{
return dependency;
}
}
}
return NULL;
}
/* /*
* CreateFunctionStmtObjectAddress returns the ObjectAddress for the subject of the * CreateFunctionStmtObjectAddress returns the ObjectAddress for the subject of the
* CREATE [OR REPLACE] FUNCTION statement. If missing_ok is false it will error with the * CREATE [OR REPLACE] FUNCTION statement. If missing_ok is false it will error with the
@ -1443,10 +1470,18 @@ DefineAggregateStmtObjectAddress(Node *node, bool missing_ok)
ObjectWithArgs *objectWithArgs = makeNode(ObjectWithArgs); ObjectWithArgs *objectWithArgs = makeNode(ObjectWithArgs);
objectWithArgs->objname = stmt->defnames; objectWithArgs->objname = stmt->defnames;
FunctionParameter *funcParam = NULL; if (stmt->args != NIL)
foreach_ptr(funcParam, linitial(stmt->args))
{ {
objectWithArgs->objargs = lappend(objectWithArgs->objargs, funcParam->argType); FunctionParameter *funcParam = NULL;
foreach_ptr(funcParam, linitial(stmt->args))
{
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 +2054,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

@ -1955,6 +1955,7 @@ PostprocessAlterTableStmt(AlterTableStmt *alterTableStatement)
/* changing a relation could introduce new dependencies */ /* changing a relation could introduce new dependencies */
ObjectAddress tableAddress = { 0 }; ObjectAddress tableAddress = { 0 };
ObjectAddressSet(tableAddress, RelationRelationId, relationId); ObjectAddressSet(tableAddress, RelationRelationId, relationId);
EnsureRelationDependenciesCanBeDistributed(&tableAddress);
EnsureDependenciesExistOnAllNodes(&tableAddress); EnsureDependenciesExistOnAllNodes(&tableAddress);
} }

View File

@ -73,16 +73,10 @@ 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();

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

@ -106,6 +106,9 @@ int MaxSharedPoolSize = 0;
*/ */
int LocalSharedPoolSize = 0; int LocalSharedPoolSize = 0;
/* number of connections reserved for Citus */
int MaxClientConnections = ALLOW_ALL_EXTERNAL_CONNECTIONS;
/* the following two structs are used for accessing shared memory */ /* the following two structs are used for accessing shared memory */
static HTAB *SharedConnStatsHash = NULL; static HTAB *SharedConnStatsHash = NULL;
@ -192,6 +195,25 @@ StoreAllRemoteConnectionStats(Tuplestorestate *tupleStore, TupleDesc tupleDescri
} }
/*
* GetMaxClientConnections returns the value of citus.max_client_connections,
* or max_connections when it is -1 or when connecting as superuser.
*
* The latter is done because citus.max_client_connections does not apply to
* superuser.
*/
int
GetMaxClientConnections(void)
{
if (MaxClientConnections == ALLOW_ALL_EXTERNAL_CONNECTIONS || superuser())
{
return MaxConnections;
}
return MaxClientConnections;
}
/* /*
* GetMaxSharedPoolSize is a wrapper around MaxSharedPoolSize which is controlled * GetMaxSharedPoolSize is a wrapper around MaxSharedPoolSize which is controlled
* via a GUC. * via a GUC.
@ -204,7 +226,7 @@ GetMaxSharedPoolSize(void)
{ {
if (MaxSharedPoolSize == ADJUST_POOLSIZE_AUTOMATICALLY) if (MaxSharedPoolSize == ADJUST_POOLSIZE_AUTOMATICALLY)
{ {
return MaxConnections; return GetMaxClientConnections();
} }
return MaxSharedPoolSize; return MaxSharedPoolSize;
@ -223,7 +245,7 @@ GetLocalSharedPoolSize(void)
{ {
if (LocalSharedPoolSize == ADJUST_POOLSIZE_AUTOMATICALLY) if (LocalSharedPoolSize == ADJUST_POOLSIZE_AUTOMATICALLY)
{ {
return MaxConnections * 0.5; return GetMaxClientConnections() * 0.5;
} }
return LocalSharedPoolSize; return LocalSharedPoolSize;
@ -318,7 +340,7 @@ TryToIncrementSharedConnectionCounter(const char *hostname, int port)
return false; return false;
} }
activeBackendCount = GetAllActiveClientBackendCount(); activeBackendCount = GetExternalClientBackendCount();
} }
LockConnectionSharedMemory(LW_EXCLUSIVE); LockConnectionSharedMemory(LW_EXCLUSIVE);

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,23 +474,40 @@ 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])
{ {
ereport(ERROR, (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED), if (self->allowNullPartitionColumnValues)
errmsg("the partition column value cannot be NULL"))); {
/*
* 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),
errmsg("the partition column value cannot be NULL")));
}
} }
else
Datum partitionColumnValue = columnValues[self->partitionColumnIndex];
ShardInterval *shardInterval = FindShardInterval(partitionColumnValue,
self->shardSearchInfo);
if (shardInterval == NULL)
{ {
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE), Datum partitionColumnValue = columnValues[self->partitionColumnIndex];
errmsg("could not find shard for partition column " ShardInterval *shardInterval = FindShardInterval(partitionColumnValue,
"value"))); self->shardSearchInfo);
if (shardInterval == NULL)
{
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("could not find shard for partition column "
"value")));
}
partitionIndex = shardInterval->shardIndex;
} }
int 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;
} }
@ -739,6 +741,142 @@ SupportedDependencyByCitus(const ObjectAddress *address)
} }
/*
* EnsureRelationDependenciesCanBeDistributed ensures all dependencies of the relation
* can be distributed.
*/
void
EnsureRelationDependenciesCanBeDistributed(ObjectAddress *relationAddress)
{
ObjectAddress *undistributableDependency =
GetUndistributableDependency(relationAddress);
if (undistributableDependency != NULL)
{
char *tableName = get_rel_name(relationAddress->objectId);
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
*
* For now the relations are the only type that is supported by Citus
* but can not be distributed as dependency, though we've added an
* explicit check below as well to not to break the logic here in case
* GetUndistributableDependency changes.
*/
if (getObjectClass(undistributableDependency) == OCLASS_CLASS)
{
char *dependentRelationName = get_rel_name(
undistributableDependency->objectId);
ereport(ERROR, (errmsg("Relation \"%s\" has dependency to a table"
" \"%s\" that is not in Citus' metadata",
tableName, dependentRelationName),
errhint("Distribute dependent relation first.")));
}
}
char *objectType = NULL;
#if PG_VERSION_NUM >= PG_VERSION_14
objectType = getObjectDescription(undistributableDependency, false);
#else
objectType = getObjectDescription(undistributableDependency);
#endif
ereport(ERROR, (errmsg("Relation \"%s\" has dependency on unsupported "
"object \"%s\"", tableName, objectType)));
}
}
/*
* GetUndistributableDependency checks whether object has any non-distributable
* dependency. If any one found, it will be returned.
*/
ObjectAddress *
GetUndistributableDependency(ObjectAddress *objectAddress)
{
List *dependencies = GetAllDependenciesForObject(objectAddress);
ObjectAddress *dependency = NULL;
/*
* Users can disable metadata sync by their own risk. If it is disabled, Citus
* doesn't propagate dependencies. So, if it is disabled, there is no undistributable
* dependency.
*/
if (!EnableMetadataSync)
{
return NULL;
}
foreach_ptr(dependency, dependencies)
{
/*
* Objects with the id smaller than FirstNormalObjectId should be created within
* initdb. Citus needs to have such objects as distributed, so we can not add
* such check to dependency resolution logic. Though, Citus shouldn't error
* out if such dependency is not supported. So, skip them here.
*/
if (dependency->objectId < FirstNormalObjectId)
{
continue;
}
/*
* If object is distributed already, ignore it.
*/
if (IsObjectDistributed(dependency))
{
continue;
}
/*
* If the dependency is not supported with Citus, return the dependency.
*/
if (!SupportedDependencyByCitus(dependency))
{
/*
* Since roles should be handled manually with Citus community, skip them.
*/
if (getObjectClass(dependency) != OCLASS_ROLE)
{
return dependency;
}
}
if (getObjectClass(dependency) == OCLASS_CLASS)
{
char relKind = get_rel_relkind(dependency->objectId);
if (relKind == RELKIND_SEQUENCE || relKind == RELKIND_COMPOSITE_TYPE)
{
/* citus knows how to auto-distribute these dependencies */
continue;
}
else if (relKind == RELKIND_INDEX || relKind == RELKIND_PARTITIONED_INDEX)
{
/*
* Indexes are only qualified for distributed objects for dependency
* tracking purposes, so we can ignore those.
*/
continue;
}
else
{
/*
* Citus doesn't know how to auto-distribute the rest of the RELKINDs
* via dependency resolution
*/
return dependency;
}
}
}
return NULL;
}
/* /*
* IsTableOwnedByExtension returns whether the table with the given relation ID is * IsTableOwnedByExtension returns whether the table with the given relation ID is
* owned by an extension. * owned by an extension.
@ -1090,9 +1228,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 +1277,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

@ -209,8 +209,9 @@ MarkObjectDistributedLocally(const ObjectAddress *distAddress)
ObjectIdGetDatum(distAddress->objectId), ObjectIdGetDatum(distAddress->objectId),
Int32GetDatum(distAddress->objectSubId) Int32GetDatum(distAddress->objectSubId)
}; };
char *insertQuery = "INSERT INTO citus.pg_dist_object (classid, objid, objsubid) " char *insertQuery =
"VALUES ($1, $2, $3) ON CONFLICT DO NOTHING"; "INSERT INTO pg_catalog.pg_dist_object (classid, objid, objsubid) "
"VALUES ($1, $2, $3) ON CONFLICT DO NOTHING";
int spiStatus = ExecuteCommandAsSuperuser(insertQuery, paramCount, paramTypes, int spiStatus = ExecuteCommandAsSuperuser(insertQuery, paramCount, paramTypes,
paramValues); paramValues);
if (spiStatus < 0) if (spiStatus < 0)
@ -324,7 +325,7 @@ UnmarkObjectDistributed(const ObjectAddress *address)
Int32GetDatum(address->objectSubId) Int32GetDatum(address->objectSubId)
}; };
char *deleteQuery = "DELETE FROM citus.pg_dist_object WHERE classid = $1 AND " char *deleteQuery = "DELETE FROM pg_catalog.pg_dist_object WHERE classid = $1 AND "
"objid = $2 AND objsubid = $3"; "objid = $2 AND objsubid = $3";
int spiStatus = ExecuteCommandAsSuperuser(deleteQuery, paramCount, paramTypes, int spiStatus = ExecuteCommandAsSuperuser(deleteQuery, paramCount, paramTypes,

View File

@ -247,8 +247,13 @@ static void GetPartitionTypeInputInfo(char *partitionKeyString, char partitionMe
Oid *intervalTypeId, int32 *intervalTypeMod); Oid *intervalTypeId, int32 *intervalTypeMod);
static void CachedNamespaceLookup(const char *nspname, Oid *cachedOid); static void CachedNamespaceLookup(const char *nspname, Oid *cachedOid);
static void CachedRelationLookup(const char *relationName, Oid *cachedOid); static void CachedRelationLookup(const char *relationName, Oid *cachedOid);
static void CachedRelationLookupExtended(const char *relationName, Oid *cachedOid,
bool missing_ok);
static void CachedRelationNamespaceLookup(const char *relationName, Oid relnamespace, static void CachedRelationNamespaceLookup(const char *relationName, Oid relnamespace,
Oid *cachedOid); Oid *cachedOid);
static void CachedRelationNamespaceLookupExtended(const char *relationName,
Oid renamespace, Oid *cachedOid,
bool missing_ok);
static ShardPlacement * ResolveGroupShardPlacement( static ShardPlacement * ResolveGroupShardPlacement(
GroupShardPlacement *groupShardPlacement, CitusTableCacheEntry *tableEntry, GroupShardPlacement *groupShardPlacement, CitusTableCacheEntry *tableEntry,
int shardIndex); int shardIndex);
@ -2321,8 +2326,37 @@ CitusCatalogNamespaceId(void)
Oid Oid
DistObjectRelationId(void) DistObjectRelationId(void)
{ {
CachedRelationNamespaceLookup("pg_dist_object", CitusCatalogNamespaceId(), /*
&MetadataCache.distObjectRelationId); * In older versions pg_dist_object was living in the `citus` namespace, With Citus 11
* this has been moved to pg_dist_catalog.
*
* During upgrades it could therefore be that we simply need to look in the old
* catalog. Since we expect to find it most of the time in the pg_catalog schema from
* now on we will start there.
*
* even after the table has been moved, the oid's stay the same, so we don't have to
* invalidate the cache after a move
*
* Note: during testing we also up/downgrade the extension, and sometimes interact
* with the database when the schema and the binary are not in sync. Hance we always
* allow the catalog to be missing on our first lookup. The error message might
* therefore become misleading as it will complain about citus.pg_dist_object not
* being found when called too early.
*/
CachedRelationLookupExtended("pg_dist_object",
&MetadataCache.distObjectRelationId,
true);
if (!OidIsValid(MetadataCache.distObjectRelationId))
{
/*
* We can only ever reach here while we are creating/altering our extension before
* the table is moved to pg_catalog.
*/
CachedRelationNamespaceLookupExtended("pg_dist_object",
CitusCatalogNamespaceId(),
&MetadataCache.distObjectRelationId,
false);
}
return MetadataCache.distObjectRelationId; return MetadataCache.distObjectRelationId;
} }
@ -2332,9 +2366,38 @@ DistObjectRelationId(void)
Oid Oid
DistObjectPrimaryKeyIndexId(void) DistObjectPrimaryKeyIndexId(void)
{ {
CachedRelationNamespaceLookup("pg_dist_object_pkey", /*
CitusCatalogNamespaceId(), * In older versions pg_dist_object was living in the `citus` namespace, With Citus 11
&MetadataCache.distObjectPrimaryKeyIndexId); * this has been moved to pg_dist_catalog.
*
* During upgrades it could therefore be that we simply need to look in the old
* catalog. Since we expect to find it most of the time in the pg_catalog schema from
* now on we will start there.
*
* even after the table has been moved, the oid's stay the same, so we don't have to
* invalidate the cache after a move
*
* Note: during testing we also up/downgrade the extension, and sometimes interact
* with the database when the schema and the binary are not in sync. Hance we always
* allow the catalog to be missing on our first lookup. The error message might
* therefore become misleading as it will complain about citus.pg_dist_object not
* being found when called too early.
*/
CachedRelationLookupExtended("pg_dist_object_pkey",
&MetadataCache.distObjectPrimaryKeyIndexId,
true);
if (!OidIsValid(MetadataCache.distObjectPrimaryKeyIndexId))
{
/*
* We can only ever reach here while we are creating/altering our extension before
* the table is moved to pg_catalog.
*/
CachedRelationNamespaceLookupExtended("pg_dist_object_pkey",
CitusCatalogNamespaceId(),
&MetadataCache.distObjectPrimaryKeyIndexId,
false);
}
return MetadataCache.distObjectPrimaryKeyIndexId; return MetadataCache.distObjectPrimaryKeyIndexId;
} }
@ -4591,9 +4654,30 @@ CachedRelationLookup(const char *relationName, Oid *cachedOid)
} }
/*
* CachedRelationLookupExtended performs a cached lookup for the relation
* relationName, with the result cached in *cachedOid. Will _not_ throw an error when
* missing_ok is set to true.
*/
static void
CachedRelationLookupExtended(const char *relationName, Oid *cachedOid, bool missing_ok)
{
CachedRelationNamespaceLookupExtended(relationName, PG_CATALOG_NAMESPACE, cachedOid,
missing_ok);
}
static void static void
CachedRelationNamespaceLookup(const char *relationName, Oid relnamespace, CachedRelationNamespaceLookup(const char *relationName, Oid relnamespace,
Oid *cachedOid) Oid *cachedOid)
{
CachedRelationNamespaceLookupExtended(relationName, relnamespace, cachedOid, false);
}
static void
CachedRelationNamespaceLookupExtended(const char *relationName, Oid relnamespace,
Oid *cachedOid, bool missing_ok)
{ {
/* force callbacks to be registered, so we always get notified upon changes */ /* force callbacks to be registered, so we always get notified upon changes */
InitializeCaches(); InitializeCaches();
@ -4602,7 +4686,7 @@ CachedRelationNamespaceLookup(const char *relationName, Oid relnamespace,
{ {
*cachedOid = get_relname_relid(relationName, relnamespace); *cachedOid = get_relname_relid(relationName, relnamespace);
if (*cachedOid == InvalidOid) if (*cachedOid == InvalidOid && !missing_ok)
{ {
ereport(ERROR, (errmsg( ereport(ERROR, (errmsg(
"cache lookup failed for %s, called too early?", "cache lookup failed for %s, called too early?",

View File

@ -28,9 +28,12 @@
#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_collation.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"
@ -46,12 +49,14 @@
#include "distributed/maintenanced.h" #include "distributed/maintenanced.h"
#include "distributed/metadata_cache.h" #include "distributed/metadata_cache.h"
#include "distributed/metadata_sync.h" #include "distributed/metadata_sync.h"
#include "distributed/metadata_utility.h"
#include "distributed/metadata/distobject.h" #include "distributed/metadata/distobject.h"
#include "distributed/metadata/pg_dist_object.h" #include "distributed/metadata/pg_dist_object.h"
#include "distributed/multi_executor.h" #include "distributed/multi_executor.h"
#include "distributed/multi_join_order.h" #include "distributed/multi_join_order.h"
#include "distributed/multi_partitioning_utils.h" #include "distributed/multi_partitioning_utils.h"
#include "distributed/multi_physical_planner.h" #include "distributed/multi_physical_planner.h"
#include "distributed/pg_dist_colocation.h"
#include "distributed/pg_dist_node.h" #include "distributed/pg_dist_node.h"
#include "distributed/pg_dist_shard.h" #include "distributed/pg_dist_shard.h"
#include "distributed/relation_access_tracking.h" #include "distributed/relation_access_tracking.h"
@ -85,6 +90,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 +110,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);
@ -122,6 +127,14 @@ static void EnsureShardMetadataIsSane(Oid relationId, int64 shardId, char storag
static void EnsureShardPlacementMetadataIsSane(Oid relationId, int64 shardId, static void EnsureShardPlacementMetadataIsSane(Oid relationId, int64 shardId,
int64 placementId, int32 shardState, int64 placementId, int32 shardState,
int64 shardLength, int32 groupId); int64 shardLength, int32 groupId);
static char * ColocationGroupCreateCommand(uint32 colocationId, int shardCount,
int replicationFactor,
Oid distributionColumnType,
Oid distributionColumnCollation);
static char * ColocationGroupDeleteCommand(uint32 colocationId);
static char * RemoteTypeIdExpression(Oid typeId);
static char * RemoteCollationIdExpression(Oid colocationId);
PG_FUNCTION_INFO_V1(start_metadata_sync_to_node); PG_FUNCTION_INFO_V1(start_metadata_sync_to_node);
PG_FUNCTION_INFO_V1(stop_metadata_sync_to_node); PG_FUNCTION_INFO_V1(stop_metadata_sync_to_node);
@ -140,6 +153,8 @@ PG_FUNCTION_INFO_V1(citus_internal_update_placement_metadata);
PG_FUNCTION_INFO_V1(citus_internal_delete_shard_metadata); PG_FUNCTION_INFO_V1(citus_internal_delete_shard_metadata);
PG_FUNCTION_INFO_V1(citus_internal_update_relation_colocation); PG_FUNCTION_INFO_V1(citus_internal_update_relation_colocation);
PG_FUNCTION_INFO_V1(citus_internal_add_object_metadata); PG_FUNCTION_INFO_V1(citus_internal_add_object_metadata);
PG_FUNCTION_INFO_V1(citus_internal_add_colocation_metadata);
PG_FUNCTION_INFO_V1(citus_internal_delete_colocation_metadata);
static bool got_SIGTERM = false; static bool got_SIGTERM = false;
@ -556,6 +571,7 @@ DropMetadataSnapshotOnNode(WorkerNode *workerNode)
dropMetadataCommandList = lappend(dropMetadataCommandList, DELETE_ALL_PLACEMENTS); dropMetadataCommandList = lappend(dropMetadataCommandList, DELETE_ALL_PLACEMENTS);
dropMetadataCommandList = lappend(dropMetadataCommandList, dropMetadataCommandList = lappend(dropMetadataCommandList,
DELETE_ALL_DISTRIBUTED_OBJECTS); DELETE_ALL_DISTRIBUTED_OBJECTS);
dropMetadataCommandList = lappend(dropMetadataCommandList, DELETE_ALL_COLOCATION);
Assert(superuser()); Assert(superuser());
SendOptionalMetadataCommandListToWorkerInCoordinatedTransaction( SendOptionalMetadataCommandListToWorkerInCoordinatedTransaction(
@ -1544,6 +1560,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 +1961,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",
@ -3046,3 +3175,311 @@ citus_internal_update_relation_colocation(PG_FUNCTION_ARGS)
PG_RETURN_VOID(); PG_RETURN_VOID();
} }
/*
* citus_internal_add_colocation_metadata is an internal UDF to
* add a row to pg_dist_colocation.
*/
Datum
citus_internal_add_colocation_metadata(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
EnsureSuperUser();
int colocationId = PG_GETARG_INT32(0);
int shardCount = PG_GETARG_INT32(1);
int replicationFactor = PG_GETARG_INT32(2);
Oid distributionColumnType = PG_GETARG_INT32(3);
Oid distributionColumnCollation = PG_GETARG_INT32(4);
if (!ShouldSkipMetadataChecks())
{
/* this UDF is not allowed allowed for executing as a separate command */
EnsureCoordinatorInitiatedOperation();
}
InsertColocationGroupLocally(colocationId, shardCount, replicationFactor,
distributionColumnType, distributionColumnCollation);
PG_RETURN_VOID();
}
/*
* citus_internal_delete_colocation_metadata is an internal UDF to
* delte row from pg_dist_colocation.
*/
Datum
citus_internal_delete_colocation_metadata(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
EnsureSuperUser();
int colocationId = PG_GETARG_INT32(0);
if (!ShouldSkipMetadataChecks())
{
/* this UDF is not allowed allowed for executing as a separate command */
EnsureCoordinatorInitiatedOperation();
}
DeleteColocationGroupLocally(colocationId);
PG_RETURN_VOID();
}
/*
* SyncNewColocationGroup synchronizes a new pg_dist_colocation entry to a worker.
*/
void
SyncNewColocationGroupToNodes(uint32 colocationId, int shardCount, int replicationFactor,
Oid distributionColumnType, Oid distributionColumnCollation)
{
char *command = ColocationGroupCreateCommand(colocationId, shardCount,
replicationFactor,
distributionColumnType,
distributionColumnCollation);
/*
* We require superuser for all pg_dist_colocation operations because we have
* no reasonable way of restricting access.
*/
SendCommandToWorkersWithMetadataViaSuperUser(command);
}
/*
* ColocationGroupCreateCommand returns a command for creating a colocation group.
*/
static char *
ColocationGroupCreateCommand(uint32 colocationId, int shardCount, int replicationFactor,
Oid distributionColumnType, Oid distributionColumnCollation)
{
StringInfo insertColocationCommand = makeStringInfo();
appendStringInfo(insertColocationCommand,
"SELECT pg_catalog.citus_internal_add_colocation_metadata("
"%d, %d, %d, %s, %s)",
colocationId,
shardCount,
replicationFactor,
RemoteTypeIdExpression(distributionColumnType),
RemoteCollationIdExpression(distributionColumnCollation));
return insertColocationCommand->data;
}
/*
* RemoteTypeIdExpression returns an expression in text form that can
* be used to obtain the OID of a type on a different node when included
* in a query string.
*/
static char *
RemoteTypeIdExpression(Oid typeId)
{
/* by default, use 0 (InvalidOid) */
char *expression = "0";
/* we also have pg_dist_colocation entries for reference tables */
if (typeId != InvalidOid)
{
char *typeName = format_type_extended(typeId, -1,
FORMAT_TYPE_FORCE_QUALIFY |
FORMAT_TYPE_ALLOW_INVALID);
/* format_type_extended returns ??? in case of an unknown type */
if (strcmp(typeName, "???") != 0)
{
StringInfo regtypeExpression = makeStringInfo();
appendStringInfo(regtypeExpression,
"%s::regtype",
quote_literal_cstr(typeName));
expression = regtypeExpression->data;
}
}
return expression;
}
/*
* RemoteCollationIdExpression returns an expression in text form that can
* be used to obtain the OID of a type on a different node when included
* in a query string. Currently this is a sublink because regcollation type
* is not available in PG12.
*/
static char *
RemoteCollationIdExpression(Oid colocationId)
{
/* by default, use 0 (InvalidOid) */
char *expression = "0";
if (colocationId != InvalidOid)
{
Datum collationIdDatum = ObjectIdGetDatum(colocationId);
HeapTuple collationTuple = SearchSysCache1(COLLOID, collationIdDatum);
if (HeapTupleIsValid(collationTuple))
{
Form_pg_collation collationform =
(Form_pg_collation) GETSTRUCT(collationTuple);
char *collationName = NameStr(collationform->collname);
char *collationSchemaName = get_namespace_name(collationform->collnamespace);
StringInfo colocationIdQuery = makeStringInfo();
appendStringInfo(colocationIdQuery,
"(select oid from pg_collation"
" where collname = %s"
" and collnamespace = %s::regnamespace)",
quote_literal_cstr(collationName),
quote_literal_cstr(collationSchemaName));
expression = colocationIdQuery->data;
}
ReleaseSysCache(collationTuple);
}
return expression;
}
/*
* SyncDeleteColocationGroupToNodes deletes a pg_dist_colocation record from workers.
*/
void
SyncDeleteColocationGroupToNodes(uint32 colocationId)
{
char *command = ColocationGroupDeleteCommand(colocationId);
/*
* We require superuser for all pg_dist_colocation operations because we have
* no reasonable way of restricting access.
*/
SendCommandToWorkersWithMetadataViaSuperUser(command);
}
/*
* ColocationGroupDeleteCommand returns a command for deleting a colocation group.
*/
static char *
ColocationGroupDeleteCommand(uint32 colocationId)
{
StringInfo deleteColocationCommand = makeStringInfo();
appendStringInfo(deleteColocationCommand,
"SELECT pg_catalog.citus_internal_delete_colocation_metadata(%d)",
colocationId);
return deleteColocationCommand->data;
}
/*
* ColocationGroupCreateCommandList returns the full list of commands for syncing
* pg_dist_colocation.
*/
List *
ColocationGroupCreateCommandList(void)
{
bool hasColocations = false;
StringInfo colocationGroupCreateCommand = makeStringInfo();
appendStringInfo(colocationGroupCreateCommand,
"WITH colocation_group_data (colocationid, shardcount, "
"replicationfactor, distributioncolumntype, "
"distributioncolumncollationname, "
"distributioncolumncollationschema) AS (VALUES ");
Relation pgDistColocation = table_open(DistColocationRelationId(), AccessShareLock);
bool indexOK = false;
SysScanDesc scanDescriptor = systable_beginscan(pgDistColocation, InvalidOid, indexOK,
NULL, 0, NULL);
HeapTuple colocationTuple = systable_getnext(scanDescriptor);
while (HeapTupleIsValid(colocationTuple))
{
if (hasColocations)
{
appendStringInfo(colocationGroupCreateCommand, ", ");
}
hasColocations = true;
Form_pg_dist_colocation colocationForm =
(Form_pg_dist_colocation) GETSTRUCT(colocationTuple);
appendStringInfo(colocationGroupCreateCommand,
"(%d, %d, %d, %s, ",
colocationForm->colocationid,
colocationForm->shardcount,
colocationForm->replicationfactor,
RemoteTypeIdExpression(colocationForm->distributioncolumntype));
/*
* For collations, include the names in the VALUES section and then
* join with pg_collation.
*/
Oid distributionColumCollation = colocationForm->distributioncolumncollation;
if (distributionColumCollation != InvalidOid)
{
Datum collationIdDatum = ObjectIdGetDatum(distributionColumCollation);
HeapTuple collationTuple = SearchSysCache1(COLLOID, collationIdDatum);
if (HeapTupleIsValid(collationTuple))
{
Form_pg_collation collationform =
(Form_pg_collation) GETSTRUCT(collationTuple);
char *collationName = NameStr(collationform->collname);
char *collationSchemaName = get_namespace_name(
collationform->collnamespace);
appendStringInfo(colocationGroupCreateCommand,
"%s, %s)",
quote_literal_cstr(collationName),
quote_literal_cstr(collationSchemaName));
ReleaseSysCache(collationTuple);
}
else
{
appendStringInfo(colocationGroupCreateCommand,
"NULL, NULL)");
}
}
else
{
appendStringInfo(colocationGroupCreateCommand,
"NULL, NULL)");
}
colocationTuple = systable_getnext(scanDescriptor);
}
systable_endscan(scanDescriptor);
table_close(pgDistColocation, AccessShareLock);
if (!hasColocations)
{
return NIL;
}
appendStringInfo(colocationGroupCreateCommand,
") SELECT pg_catalog.citus_internal_add_colocation_metadata("
"colocationid, shardcount, replicationfactor, "
"distributioncolumntype, coalesce(c.oid, 0)) "
"FROM colocation_group_data d LEFT JOIN pg_collation c "
"ON (d.distributioncolumncollationname = c.collname "
"AND d.distributioncolumncollationschema::regnamespace"
" = c.collnamespace)");
return list_make1(colocationGroupCreateCommand->data);
}

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

@ -146,6 +146,8 @@ PG_FUNCTION_INFO_V1(master_activate_node);
PG_FUNCTION_INFO_V1(citus_update_node); PG_FUNCTION_INFO_V1(citus_update_node);
PG_FUNCTION_INFO_V1(master_update_node); PG_FUNCTION_INFO_V1(master_update_node);
PG_FUNCTION_INFO_V1(get_shard_id_for_distribution_column); PG_FUNCTION_INFO_V1(get_shard_id_for_distribution_column);
PG_FUNCTION_INFO_V1(citus_nodename_for_nodeid);
PG_FUNCTION_INFO_V1(citus_nodeport_for_nodeid);
/* /*
@ -188,9 +190,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);
@ -656,6 +655,8 @@ PgDistTableMetadataSyncCommandList(void)
DELETE_ALL_PLACEMENTS); DELETE_ALL_PLACEMENTS);
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList, metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
DELETE_ALL_DISTRIBUTED_OBJECTS); DELETE_ALL_DISTRIBUTED_OBJECTS);
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
DELETE_ALL_COLOCATION);
/* create pg_dist_partition, pg_dist_shard and pg_dist_placement entries */ /* create pg_dist_partition, pg_dist_shard and pg_dist_placement entries */
foreach_ptr(cacheEntry, propagatedTableList) foreach_ptr(cacheEntry, propagatedTableList)
@ -667,6 +668,11 @@ PgDistTableMetadataSyncCommandList(void)
tableMetadataCreateCommandList); tableMetadataCreateCommandList);
} }
/* commands to insert pg_dist_colocation entries */
List *colocationGroupSyncCommandList = ColocationGroupCreateCommandList();
metadataSnapshotCommandList = list_concat(metadataSnapshotCommandList,
colocationGroupSyncCommandList);
/* As the last step, propagate the pg_dist_object entities */ /* As the last step, propagate the pg_dist_object entities */
Assert(ShouldPropagate()); Assert(ShouldPropagate());
List *distributedObjectSyncCommandList = DistributedObjectMetadataSyncCommandList(); List *distributedObjectSyncCommandList = DistributedObjectMetadataSyncCommandList();
@ -1469,6 +1475,50 @@ get_shard_id_for_distribution_column(PG_FUNCTION_ARGS)
} }
/*
* citus_nodename_for_nodeid returns the node name for the node with given node id
*/
Datum
citus_nodename_for_nodeid(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
int nodeId = PG_GETARG_INT32(0);
bool missingOk = true;
WorkerNode *node = FindNodeWithNodeId(nodeId, missingOk);
if (node == NULL)
{
PG_RETURN_NULL();
}
PG_RETURN_TEXT_P(cstring_to_text(node->workerName));
}
/*
* citus_nodeport_for_nodeid returns the node port for the node with given node id
*/
Datum
citus_nodeport_for_nodeid(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
int nodeId = PG_GETARG_INT32(0);
bool missingOk = true;
WorkerNode *node = FindNodeWithNodeId(nodeId, missingOk);
if (node == NULL)
{
PG_RETURN_NULL();
}
PG_RETURN_INT32(node->workerPort);
}
/* /*
* FindWorkerNode searches over the worker nodes and returns the workerNode * FindWorkerNode searches over the worker nodes and returns the workerNode
* if it already exists. Else, the function returns NULL. * if it already exists. Else, the function returns NULL.
@ -1544,21 +1594,24 @@ 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 *nodeList = ActiveReadableNodeList();
WorkerNode *workerNode = NULL; WorkerNode *node = NULL;
foreach_ptr(workerNode, workerList) foreach_ptr(node, nodeList)
{ {
if (workerNode->nodeId == nodeId) if (node->nodeId == nodeId)
{ {
return workerNode; return node;
} }
} }
/* there isn't any node with nodeId in pg_dist_node */ /* there isn't any node with nodeId in pg_dist_node */
elog(ERROR, "worker node with node id %d could not be found", nodeId); if (!missingOk)
{
elog(ERROR, "node with node id %d could not be found", nodeId);
}
return NULL; return NULL;
} }
@ -1780,12 +1833,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,11 +2611,8 @@ 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;
{ hash_search(shardPlacementsHash, hashKey, HASH_ENTER, NULL);
void *hashKey = (void *) shardPlacement;
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

@ -23,6 +23,7 @@
#include "safe_lib.h" #include "safe_lib.h"
#include "catalog/pg_authid.h"
#include "citus_version.h" #include "citus_version.h"
#include "commands/explain.h" #include "commands/explain.h"
#include "common/string.h" #include "common/string.h"
@ -84,12 +85,14 @@
#include "libpq/auth.h" #include "libpq/auth.h"
#include "port/atomics.h" #include "port/atomics.h"
#include "postmaster/postmaster.h" #include "postmaster/postmaster.h"
#include "replication/walsender.h"
#include "storage/ipc.h" #include "storage/ipc.h"
#include "optimizer/planner.h" #include "optimizer/planner.h"
#include "optimizer/paths.h" #include "optimizer/paths.h"
#include "tcop/tcopprot.h" #include "tcop/tcopprot.h"
#include "utils/guc.h" #include "utils/guc.h"
#include "utils/guc_tables.h" #include "utils/guc_tables.h"
#include "utils/syscache.h"
#include "utils/varlena.h" #include "utils/varlena.h"
#include "columnar/mod.h" #include "columnar/mod.h"
@ -113,9 +116,9 @@ static void DoInitialCleanup(void);
static void ResizeStackToMaximumDepth(void); static void ResizeStackToMaximumDepth(void);
static void multi_log_hook(ErrorData *edata); static void multi_log_hook(ErrorData *edata);
static void RegisterConnectionCleanup(void); static void RegisterConnectionCleanup(void);
static void RegisterClientBackendCounterDecrement(void); static void RegisterExternalClientBackendCounterDecrement(void);
static void CitusCleanupConnectionsAtExit(int code, Datum arg); static void CitusCleanupConnectionsAtExit(int code, Datum arg);
static void DecrementClientBackendCounterAtExit(int code, Datum arg); static void DecrementExternalClientBackendCounterAtExit(int code, Datum arg);
static void CreateRequiredDirectories(void); static void CreateRequiredDirectories(void);
static void RegisterCitusConfigVariables(void); static void RegisterCitusConfigVariables(void);
static void OverridePostgresConfigAssignHooks(void); static void OverridePostgresConfigAssignHooks(void);
@ -135,6 +138,7 @@ static const char * LocalPoolSizeGucShowHook(void);
static bool StatisticsCollectionGucCheckHook(bool *newval, void **extra, GucSource static bool StatisticsCollectionGucCheckHook(bool *newval, void **extra, GucSource
source); source);
static void CitusAuthHook(Port *port, int status); static void CitusAuthHook(Port *port, int status);
static bool IsSuperuser(char *userName);
static ClientAuthentication_hook_type original_client_auth_hook = NULL; static ClientAuthentication_hook_type original_client_auth_hook = NULL;
@ -218,6 +222,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* */
@ -481,16 +492,16 @@ RegisterConnectionCleanup(void)
/* /*
* RegisterClientBackendCounterDecrement is called when the backend terminates. * RegisterExternalClientBackendCounterDecrement is called when the backend terminates.
* For all client backends, we register a callback that will undo * For all client backends, we register a callback that will undo
*/ */
static void static void
RegisterClientBackendCounterDecrement(void) RegisterExternalClientBackendCounterDecrement(void)
{ {
static bool registeredCleanup = false; static bool registeredCleanup = false;
if (registeredCleanup == false) if (registeredCleanup == false)
{ {
before_shmem_exit(DecrementClientBackendCounterAtExit, 0); before_shmem_exit(DecrementExternalClientBackendCounterAtExit, 0);
registeredCleanup = true; registeredCleanup = true;
} }
@ -520,13 +531,13 @@ CitusCleanupConnectionsAtExit(int code, Datum arg)
/* /*
* DecrementClientBackendCounterAtExit is called before_shmem_exit() of the * DecrementExternalClientBackendCounterAtExit is called before_shmem_exit() of the
* backend for the purposes decrementing * backend for the purposes decrementing
*/ */
static void static void
DecrementClientBackendCounterAtExit(int code, Datum arg) DecrementExternalClientBackendCounterAtExit(int code, Datum arg)
{ {
DecrementClientBackendCounter(); DecrementExternalClientBackendCounter();
} }
@ -603,7 +614,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(
@ -669,6 +680,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 "
@ -1315,6 +1344,23 @@ RegisterCitusConfigVariables(void)
GUC_STANDARD, GUC_STANDARD,
NULL, NULL, NULL); NULL, NULL, NULL);
DefineCustomIntVariable(
"citus.max_client_connections",
gettext_noop("Sets the maximum number of connections regular clients can make"),
gettext_noop("To ensure that a Citus cluster has a sufficient number of "
"connection slots to serve queries internally, it can be "
"useful to reserve connection slots for Citus internal "
"connections. When max_client_connections is set to a value "
"below max_connections, the remaining connections are reserved "
"for connections between Citus nodes. This does not affect "
"superuser_reserved_connections. If set to -1, no connections "
"are reserved."),
&MaxClientConnections,
-1, -1, MaxConnections,
PGC_SUSET,
GUC_STANDARD,
NULL, NULL, NULL);
DefineCustomIntVariable( DefineCustomIntVariable(
"citus.max_intermediate_result_size", "citus.max_intermediate_result_size",
gettext_noop("Sets the maximum size of the intermediate results in KB for " gettext_noop("Sets the maximum size of the intermediate results in KB for "
@ -2146,12 +2192,86 @@ StatisticsCollectionGucCheckHook(bool *newval, void **extra, GucSource source)
static void static void
CitusAuthHook(Port *port, int status) CitusAuthHook(Port *port, int status)
{ {
uint64 gpid = ExtractGlobalPID(port->application_name);
/* external connections to not have a GPID immediately */
if (gpid == INVALID_CITUS_INTERNAL_BACKEND_GPID)
{
/*
* We raise the shared connection counter pre-emptively. As a result, we may
* have scenarios in which a few simultaneous connection attempts prevent
* each other from succeeding, but we avoid scenarios where we oversubscribe
* the system.
*
* By also calling RegisterExternalClientBackendCounterDecrement here, we
* immediately lower the counter if we throw a FATAL error below. The client
* connection counter may temporarily exceed maxClientConnections in between.
*/
RegisterExternalClientBackendCounterDecrement();
uint32 externalClientCount = IncrementExternalClientBackendCounter();
/*
* Limit non-superuser client connections if citus.max_client_connections
* is set.
*/
if (MaxClientConnections >= 0 &&
!IsSuperuser(port->user_name) &&
externalClientCount > MaxClientConnections)
{
ereport(FATAL, (errcode(ERRCODE_TOO_MANY_CONNECTIONS),
errmsg("remaining connection slots are reserved for "
"non-replication superuser connections"),
errdetail("the server is configured to accept up to %d "
"regular client connections",
MaxClientConnections)));
}
/*
* Right after this, before we assign global pid, this backend
* might get blocked by a DDL as that happens during parsing.
*
* That's why, lets mark the backend as an external backend
* which is likely to execute a distributed command.
*
* We do this so that this backend gets the chance to show
* up in citus_lock_waits.
*/
InitializeBackendData();
SetBackendDataDistributedCommandOriginator(true);
}
/* let other authentication hooks to kick in first */ /* let other authentication hooks to kick in first */
if (original_client_auth_hook) if (original_client_auth_hook)
{ {
original_client_auth_hook(port, status); original_client_auth_hook(port, status);
} }
}
RegisterClientBackendCounterDecrement();
IncrementClientBackendCounter();
/*
* IsSuperuser returns whether the role with the given name is superuser.
*/
static bool
IsSuperuser(char *roleName)
{
if (roleName == NULL)
{
return false;
}
HeapTuple roleTuple = SearchSysCache1(AUTHNAME, CStringGetDatum(roleName));
if (!HeapTupleIsValid(roleTuple))
{
ereport(FATAL,
(errcode(ERRCODE_INVALID_AUTHORIZATION_SPECIFICATION),
errmsg("role \"%s\" does not exist", roleName)));
}
Form_pg_authid rform = (Form_pg_authid) GETSTRUCT(roleTuple);
bool isSuperuser = rform->rolsuper;
ReleaseSysCache(roleTuple);
return isSuperuser;
} }

View File

@ -10,6 +10,8 @@
#include "udfs/citus_shard_indexes_on_worker/11.0-1.sql" #include "udfs/citus_shard_indexes_on_worker/11.0-1.sql"
#include "udfs/citus_internal_add_object_metadata/11.0-1.sql" #include "udfs/citus_internal_add_object_metadata/11.0-1.sql"
#include "udfs/citus_internal_add_colocation_metadata/11.0-1.sql"
#include "udfs/citus_internal_delete_colocation_metadata/11.0-1.sql"
#include "udfs/citus_run_local_command/11.0-1.sql" #include "udfs/citus_run_local_command/11.0-1.sql"
#include "udfs/worker_drop_sequence_dependency/11.0-1.sql" #include "udfs/worker_drop_sequence_dependency/11.0-1.sql"
#include "udfs/worker_drop_shell_table/11.0-1.sql" #include "udfs/worker_drop_shell_table/11.0-1.sql"
@ -20,30 +22,36 @@
#include "udfs/citus_internal_local_blocked_processes/11.0-1.sql" #include "udfs/citus_internal_local_blocked_processes/11.0-1.sql"
#include "udfs/citus_internal_global_blocked_processes/11.0-1.sql" #include "udfs/citus_internal_global_blocked_processes/11.0-1.sql"
#include "udfs/citus_worker_stat_activity/11.0-1.sql" #include "udfs/run_command_on_all_nodes/11.0-1.sql"
#include "udfs/citus_stat_activity/11.0-1.sql"
#include "udfs/worker_create_or_replace_object/11.0-1.sql" #include "udfs/worker_create_or_replace_object/11.0-1.sql"
#include "udfs/citus_isolation_test_session_is_blocked/11.0-1.sql" #include "udfs/citus_isolation_test_session_is_blocked/11.0-1.sql"
#include "udfs/citus_blocking_pids/11.0-1.sql" #include "udfs/citus_blocking_pids/11.0-1.sql"
#include "udfs/citus_calculate_gpid/11.0-1.sql"
#include "udfs/citus_backend_gpid/11.0-1.sql"
CREATE VIEW citus.citus_worker_stat_activity AS DROP FUNCTION IF EXISTS pg_catalog.citus_dist_stat_activity() CASCADE;
SELECT * FROM pg_catalog.citus_worker_stat_activity(); DROP FUNCTION IF EXISTS pg_catalog.citus_worker_stat_activity() CASCADE;
ALTER VIEW citus.citus_worker_stat_activity SET SCHEMA pg_catalog;
GRANT SELECT ON pg_catalog.citus_worker_stat_activity TO PUBLIC;
#include "udfs/citus_dist_stat_activity/11.0-1.sql" #include "udfs/citus_dist_stat_activity/11.0-1.sql"
CREATE VIEW citus.citus_dist_stat_activity AS -- a very simple helper function defined for citus_lock_waits
SELECT * FROM pg_catalog.citus_dist_stat_activity(); CREATE OR REPLACE FUNCTION get_nodeid_for_groupid(groupIdInput int) RETURNS int AS $$
ALTER VIEW citus.citus_dist_stat_activity SET SCHEMA pg_catalog; DECLARE
GRANT SELECT ON pg_catalog.citus_dist_stat_activity TO PUBLIC; returnNodeNodeId int := 0;
begin
SELECT nodeId into returnNodeNodeId FROM pg_dist_node WHERE groupid = groupIdInput and nodecluster = current_setting('citus.cluster_name');
RETURN returnNodeNodeId;
end
$$ LANGUAGE plpgsql;
-- we have to recreate this view because recreated citus_dist_stat_activity that this view depends
#include "udfs/citus_lock_waits/11.0-1.sql" #include "udfs/citus_lock_waits/11.0-1.sql"
#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 +87,16 @@ 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"
ALTER TABLE citus.pg_dist_object SET SCHEMA pg_catalog;
GRANT SELECT ON pg_catalog.pg_dist_object TO public;
#include "udfs/citus_prepare_pg_upgrade/11.0-1.sql"
#include "udfs/citus_finish_pg_upgrade/11.0-1.sql"
#include "udfs/citus_nodename_for_nodeid/11.0-1.sql"
#include "udfs/citus_nodeport_for_nodeid/11.0-1.sql"
#include "udfs/citus_nodeid_for_gpid/11.0-1.sql"
#include "udfs/citus_pid_for_gpid/11.0-1.sql"

View File

@ -1,6 +1,12 @@
-- citus--11.0-1--10.2-4 -- citus--11.0-1--10.2-4
REVOKE SELECT ON pg_catalog.pg_dist_object FROM public;
ALTER TABLE pg_catalog.pg_dist_object SET SCHEMA citus;
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 +27,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)
@ -45,6 +52,8 @@ DROP FUNCTION pg_catalog.citus_check_connection_to_node (text, integer);
DROP FUNCTION pg_catalog.citus_check_cluster_node_health (); DROP FUNCTION pg_catalog.citus_check_cluster_node_health ();
DROP FUNCTION pg_catalog.citus_internal_add_object_metadata(text, text[], text[], integer, integer, boolean); DROP FUNCTION pg_catalog.citus_internal_add_object_metadata(text, text[], text[], integer, integer, boolean);
DROP FUNCTION pg_catalog.citus_internal_add_colocation_metadata(int, int, int, regtype, oid);
DROP FUNCTION pg_catalog.citus_internal_delete_colocation_metadata(int);
DROP FUNCTION pg_catalog.citus_run_local_command(text); DROP FUNCTION pg_catalog.citus_run_local_command(text);
DROP FUNCTION pg_catalog.worker_drop_sequence_dependency(text); DROP FUNCTION pg_catalog.worker_drop_sequence_dependency(text);
DROP FUNCTION pg_catalog.worker_drop_shell_table(table_name text); DROP FUNCTION pg_catalog.worker_drop_shell_table(table_name text);
@ -85,7 +94,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';
@ -113,10 +121,13 @@ CREATE FUNCTION get_global_active_transactions(OUT datid oid, OUT process_id int
RESET search_path; RESET search_path;
DROP FUNCTION citus_internal_local_blocked_processes CASCADE; DROP VIEW pg_catalog.citus_lock_waits;
DROP FUNCTION citus_internal_global_blocked_processes CASCADE;
DROP FUNCTION pg_catalog.citus_dist_stat_activity CASCADE; DROP FUNCTION citus_internal_local_blocked_processes;
DROP FUNCTION citus_internal_global_blocked_processes;
DROP VIEW IF EXISTS pg_catalog.citus_dist_stat_activity;
DROP FUNCTION IF EXISTS pg_catalog.citus_dist_stat_activity;
CREATE OR REPLACE FUNCTION pg_catalog.citus_dist_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int, CREATE OR REPLACE FUNCTION pg_catalog.citus_dist_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int,
OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name, OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name,
@ -142,7 +153,8 @@ ALTER VIEW citus.citus_dist_stat_activity SET SCHEMA pg_catalog;
GRANT SELECT ON pg_catalog.citus_dist_stat_activity TO PUBLIC; GRANT SELECT ON pg_catalog.citus_dist_stat_activity TO PUBLIC;
SET search_path = 'pg_catalog'; SET search_path = 'pg_catalog';
DROP FUNCTION citus_worker_stat_activity CASCADE; DROP VIEW IF EXISTS citus_worker_stat_activity;
DROP FUNCTION IF EXISTS citus_worker_stat_activity;
CREATE OR REPLACE FUNCTION citus_worker_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int, CREATE OR REPLACE FUNCTION citus_worker_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int,
OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name, OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name,
@ -165,10 +177,10 @@ IS 'returns distributed transaction activity on shards of distributed tables';
DROP FUNCTION pg_catalog.worker_create_or_replace_object(text[]); DROP FUNCTION pg_catalog.worker_create_or_replace_object(text[]);
#include "../udfs/worker_create_or_replace_object/9.0-1.sql" #include "../udfs/worker_create_or_replace_object/9.0-1.sql"
DROP FUNCTION IF EXISTS pg_catalog.pg_cancel_backend(bigint) CASCADE; DROP FUNCTION IF EXISTS pg_catalog.pg_cancel_backend(bigint);
DROP FUNCTION IF EXISTS pg_catalog.pg_terminate_backend(bigint, bigint) CASCADE; DROP FUNCTION IF EXISTS pg_catalog.pg_terminate_backend(bigint, bigint);
DROP FUNCTION pg_catalog.dump_local_wait_edges CASCADE; DROP FUNCTION pg_catalog.dump_local_wait_edges;
CREATE FUNCTION pg_catalog.dump_local_wait_edges( CREATE FUNCTION pg_catalog.dump_local_wait_edges(
OUT waiting_pid int4, OUT waiting_pid int4,
OUT waiting_node_id int4, OUT waiting_node_id int4,
@ -185,7 +197,7 @@ AS $$MODULE_PATHNAME$$, $$dump_local_wait_edges$$;
COMMENT ON FUNCTION pg_catalog.dump_local_wait_edges() COMMENT ON FUNCTION pg_catalog.dump_local_wait_edges()
IS 'returns all local lock wait chains, that start from distributed transactions'; IS 'returns all local lock wait chains, that start from distributed transactions';
DROP FUNCTION pg_catalog.dump_global_wait_edges CASCADE; DROP FUNCTION pg_catalog.dump_global_wait_edges;
CREATE FUNCTION pg_catalog.dump_global_wait_edges( CREATE FUNCTION pg_catalog.dump_global_wait_edges(
OUT waiting_pid int4, OUT waiting_pid int4,
OUT waiting_node_id int4, OUT waiting_node_id int4,
@ -342,4 +354,19 @@ 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);
DROP FUNCTION pg_catalog.citus_calculate_gpid(integer,integer);
DROP FUNCTION pg_catalog.citus_backend_gpid();
DROP FUNCTION get_nodeid_for_groupid(integer);
RESET search_path; RESET search_path;
DROP VIEW pg_catalog.citus_stat_activity;
DROP FUNCTION pg_catalog.citus_stat_activity;
DROP FUNCTION pg_catalog.run_command_on_all_nodes;
DROP FUNCTION pg_catalog.citus_nodename_for_nodeid(integer);
DROP FUNCTION pg_catalog.citus_nodeport_for_nodeid(integer);
DROP FUNCTION pg_catalog.citus_nodeid_for_gpid(bigint);
DROP FUNCTION pg_catalog.citus_pid_for_gpid(bigint);

View File

@ -0,0 +1,8 @@
CREATE FUNCTION pg_catalog.citus_backend_gpid()
RETURNS BIGINT
LANGUAGE C STRICT
AS 'MODULE_PATHNAME',$$citus_backend_gpid$$;
COMMENT ON FUNCTION pg_catalog.citus_backend_gpid()
IS 'returns gpid of the current backend';
GRANT EXECUTE ON FUNCTION pg_catalog.citus_backend_gpid() TO PUBLIC;

View File

@ -0,0 +1,8 @@
CREATE FUNCTION pg_catalog.citus_backend_gpid()
RETURNS BIGINT
LANGUAGE C STRICT
AS 'MODULE_PATHNAME',$$citus_backend_gpid$$;
COMMENT ON FUNCTION pg_catalog.citus_backend_gpid()
IS 'returns gpid of the current backend';
GRANT EXECUTE ON FUNCTION pg_catalog.citus_backend_gpid() TO PUBLIC;

View File

@ -0,0 +1,9 @@
CREATE FUNCTION pg_catalog.citus_calculate_gpid(nodeid integer,
pid integer)
RETURNS BIGINT
LANGUAGE C STRICT
AS 'MODULE_PATHNAME',$$citus_calculate_gpid$$;
COMMENT ON FUNCTION pg_catalog.citus_calculate_gpid(nodeid integer, pid integer)
IS 'calculate gpid of a backend running on any node';
GRANT EXECUTE ON FUNCTION pg_catalog.citus_calculate_gpid(integer, integer) TO PUBLIC;

View File

@ -0,0 +1,9 @@
CREATE FUNCTION pg_catalog.citus_calculate_gpid(nodeid integer,
pid integer)
RETURNS BIGINT
LANGUAGE C STRICT
AS 'MODULE_PATHNAME',$$citus_calculate_gpid$$;
COMMENT ON FUNCTION pg_catalog.citus_calculate_gpid(nodeid integer, pid integer)
IS 'calculate gpid of a backend running on any node';
GRANT EXECUTE ON FUNCTION pg_catalog.citus_calculate_gpid(integer, integer) TO PUBLIC;

View File

@ -1,19 +1,8 @@
DROP FUNCTION IF EXISTS pg_catalog.citus_dist_stat_activity CASCADE; DROP VIEW IF EXISTS pg_catalog.citus_dist_stat_activity;
CREATE OR REPLACE FUNCTION pg_catalog.citus_dist_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int, CREATE OR REPLACE VIEW citus.citus_dist_stat_activity AS
OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name, SELECT * FROM citus_stat_activity
OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET, WHERE is_worker_query = false;
OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz,
OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text,
OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text, OUT global_pid int8)
RETURNS SETOF RECORD
LANGUAGE C STRICT AS 'MODULE_PATHNAME',
$$citus_dist_stat_activity$$;
COMMENT ON FUNCTION pg_catalog.citus_dist_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int, ALTER VIEW citus.citus_dist_stat_activity SET SCHEMA pg_catalog;
OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name, GRANT SELECT ON pg_catalog.citus_dist_stat_activity TO PUBLIC;
OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET,
OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz,
OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text,
OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text, OUT global_pid int8)
IS 'returns distributed transaction activity on distributed tables';

View File

@ -1,19 +1,8 @@
DROP FUNCTION IF EXISTS pg_catalog.citus_dist_stat_activity CASCADE; DROP VIEW IF EXISTS pg_catalog.citus_dist_stat_activity;
CREATE OR REPLACE FUNCTION pg_catalog.citus_dist_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int, CREATE OR REPLACE VIEW citus.citus_dist_stat_activity AS
OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name, SELECT * FROM citus_stat_activity
OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET, WHERE is_worker_query = false;
OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz,
OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text,
OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text, OUT global_pid int8)
RETURNS SETOF RECORD
LANGUAGE C STRICT AS 'MODULE_PATHNAME',
$$citus_dist_stat_activity$$;
COMMENT ON FUNCTION pg_catalog.citus_dist_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int, ALTER VIEW citus.citus_dist_stat_activity SET SCHEMA pg_catalog;
OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name, GRANT SELECT ON pg_catalog.citus_dist_stat_activity TO PUBLIC;
OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET,
OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz,
OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text,
OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text, OUT global_pid int8)
IS 'returns distributed transaction activity on distributed tables';

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

@ -0,0 +1,151 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_finish_pg_upgrade()
RETURNS void
LANGUAGE plpgsql
SET search_path = pg_catalog
AS $cppu$
DECLARE
table_name regclass;
command text;
trigger_name text;
BEGIN
IF substring(current_Setting('server_version'), '\d+')::int >= 14 THEN
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);
COMMENT ON AGGREGATE array_cat_agg(anycompatiblearray)
IS 'concatenate input arrays into a single array';
RESET citus.enable_ddl_propagation;
$cmd$;
ELSE
EXECUTE $cmd$
SET citus.enable_ddl_propagation TO OFF;
CREATE AGGREGATE array_cat_agg(anyarray) (SFUNC = array_cat, STYPE = anyarray);
COMMENT ON AGGREGATE array_cat_agg(anyarray)
IS 'concatenate input arrays into a single array';
RESET citus.enable_ddl_propagation;
$cmd$;
END IF;
--
-- Citus creates the array_cat_agg but because of a compatibility
-- issue between pg13-pg14, we drop and create it during upgrade.
-- And as Citus creates it, there needs to be a dependency to the
-- Citus extension, so we create that dependency here.
-- We are not using:
-- ALTER EXENSION citus DROP/CREATE AGGREGATE array_cat_agg
-- because we don't have an easy way to check if the aggregate
-- exists with anyarray type or anycompatiblearray type.
INSERT INTO pg_depend
SELECT
'pg_proc'::regclass::oid as classid,
(SELECT oid FROM pg_proc WHERE proname = 'array_cat_agg') as objid,
0 as objsubid,
'pg_extension'::regclass::oid as refclassid,
(select oid from pg_extension where extname = 'citus') as refobjid,
0 as refobjsubid ,
'e' as deptype;
--
-- restore citus catalog tables
--
INSERT INTO pg_catalog.pg_dist_partition SELECT * FROM public.pg_dist_partition;
INSERT INTO pg_catalog.pg_dist_shard SELECT * FROM public.pg_dist_shard;
INSERT INTO pg_catalog.pg_dist_placement SELECT * FROM public.pg_dist_placement;
INSERT INTO pg_catalog.pg_dist_node_metadata SELECT * FROM public.pg_dist_node_metadata;
INSERT INTO pg_catalog.pg_dist_node SELECT * FROM public.pg_dist_node;
INSERT INTO pg_catalog.pg_dist_local_group SELECT * FROM public.pg_dist_local_group;
INSERT INTO pg_catalog.pg_dist_transaction SELECT * FROM public.pg_dist_transaction;
INSERT INTO pg_catalog.pg_dist_colocation SELECT * FROM public.pg_dist_colocation;
-- enterprise catalog tables
INSERT INTO pg_catalog.pg_dist_authinfo SELECT * FROM public.pg_dist_authinfo;
INSERT INTO pg_catalog.pg_dist_poolinfo SELECT * FROM public.pg_dist_poolinfo;
INSERT INTO pg_catalog.pg_dist_rebalance_strategy SELECT
name,
default_strategy,
shard_cost_function::regprocedure::regproc,
node_capacity_function::regprocedure::regproc,
shard_allowed_on_node_function::regprocedure::regproc,
default_threshold,
minimum_threshold,
improvement_threshold
FROM public.pg_dist_rebalance_strategy;
--
-- drop backup tables
--
DROP TABLE public.pg_dist_authinfo;
DROP TABLE public.pg_dist_colocation;
DROP TABLE public.pg_dist_local_group;
DROP TABLE public.pg_dist_node;
DROP TABLE public.pg_dist_node_metadata;
DROP TABLE public.pg_dist_partition;
DROP TABLE public.pg_dist_placement;
DROP TABLE public.pg_dist_poolinfo;
DROP TABLE public.pg_dist_shard;
DROP TABLE public.pg_dist_transaction;
DROP TABLE public.pg_dist_rebalance_strategy;
--
-- reset sequences
--
PERFORM setval('pg_catalog.pg_dist_shardid_seq', (SELECT MAX(shardid)+1 AS max_shard_id FROM pg_dist_shard), false);
PERFORM setval('pg_catalog.pg_dist_placement_placementid_seq', (SELECT MAX(placementid)+1 AS max_placement_id FROM pg_dist_placement), false);
PERFORM setval('pg_catalog.pg_dist_groupid_seq', (SELECT MAX(groupid)+1 AS max_group_id FROM pg_dist_node), false);
PERFORM setval('pg_catalog.pg_dist_node_nodeid_seq', (SELECT MAX(nodeid)+1 AS max_node_id FROM pg_dist_node), false);
PERFORM setval('pg_catalog.pg_dist_colocationid_seq', (SELECT MAX(colocationid)+1 AS max_colocation_id FROM pg_dist_colocation), false);
--
-- register triggers
--
FOR table_name IN SELECT logicalrelid FROM pg_catalog.pg_dist_partition
LOOP
trigger_name := 'truncate_trigger_' || table_name::oid;
command := 'create trigger ' || trigger_name || ' after truncate on ' || table_name || ' execute procedure pg_catalog.citus_truncate_trigger()';
EXECUTE command;
command := 'update pg_trigger set tgisinternal = true where tgname = ' || quote_literal(trigger_name);
EXECUTE command;
END LOOP;
--
-- set dependencies
--
INSERT INTO pg_depend
SELECT
'pg_class'::regclass::oid as classid,
p.logicalrelid::regclass::oid as objid,
0 as objsubid,
'pg_extension'::regclass::oid as refclassid,
(select oid from pg_extension where extname = 'citus') as refobjid,
0 as refobjsubid ,
'n' as deptype
FROM pg_catalog.pg_dist_partition p;
-- set dependencies for columnar table access method
PERFORM citus_internal.columnar_ensure_am_depends_catalog();
-- restore pg_dist_object from the stable identifiers
TRUNCATE pg_catalog.pg_dist_object;
INSERT INTO pg_catalog.pg_dist_object (classid, objid, objsubid, distribution_argument_index, colocationid)
SELECT
address.classid,
address.objid,
address.objsubid,
naming.distribution_argument_index,
naming.colocationid
FROM
public.pg_dist_object naming,
pg_catalog.pg_get_object_address(naming.type, naming.object_names, naming.object_args) address;
DROP TABLE public.pg_dist_object;
END;
$cppu$;
COMMENT ON FUNCTION pg_catalog.citus_finish_pg_upgrade()
IS 'perform tasks to restore citus settings from a location that has been prepared before pg_upgrade';

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;
@ -124,8 +131,8 @@ BEGIN
PERFORM citus_internal.columnar_ensure_am_depends_catalog(); PERFORM citus_internal.columnar_ensure_am_depends_catalog();
-- restore pg_dist_object from the stable identifiers -- restore pg_dist_object from the stable identifiers
TRUNCATE citus.pg_dist_object; TRUNCATE pg_catalog.pg_dist_object;
INSERT INTO citus.pg_dist_object (classid, objid, objsubid, distribution_argument_index, colocationid) INSERT INTO pg_catalog.pg_dist_object (classid, objid, objsubid, distribution_argument_index, colocationid)
SELECT SELECT
address.classid, address.classid,
address.objid, address.objid,

View File

@ -0,0 +1,13 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_add_colocation_metadata(
colocation_id int,
shard_count int,
replication_factor int,
distribution_column_type regtype,
distribution_column_collation oid)
RETURNS void
LANGUAGE C
STRICT
AS 'MODULE_PATHNAME';
COMMENT ON FUNCTION pg_catalog.citus_internal_add_colocation_metadata(int,int,int,regtype,oid) IS
'Inserts a co-location group into pg_dist_colocation';

View File

@ -0,0 +1,13 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_add_colocation_metadata(
colocation_id int,
shard_count int,
replication_factor int,
distribution_column_type regtype,
distribution_column_collation oid)
RETURNS void
LANGUAGE C
STRICT
AS 'MODULE_PATHNAME';
COMMENT ON FUNCTION pg_catalog.citus_internal_add_colocation_metadata(int,int,int,regtype,oid) IS
'Inserts a co-location group into pg_dist_colocation';

View File

@ -0,0 +1,9 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_delete_colocation_metadata(
colocation_id int)
RETURNS void
LANGUAGE C
STRICT
AS 'MODULE_PATHNAME';
COMMENT ON FUNCTION pg_catalog.citus_internal_delete_colocation_metadata(int) IS
'deletes a co-location group from pg_dist_colocation';

View File

@ -0,0 +1,9 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_delete_colocation_metadata(
colocation_id int)
RETURNS void
LANGUAGE C
STRICT
AS 'MODULE_PATHNAME';
COMMENT ON FUNCTION pg_catalog.citus_internal_delete_colocation_metadata(int) IS
'deletes a co-location group from pg_dist_colocation';

View File

@ -2,43 +2,44 @@ SET search_path = 'pg_catalog';
CREATE VIEW citus.citus_lock_waits AS CREATE VIEW citus.citus_lock_waits AS
WITH WITH
citus_dist_stat_activity AS unique_global_wait_edges_with_calculated_gpids AS (
( SELECT
SELECT * FROM citus_dist_stat_activity -- if global_pid is NULL, it is most likely that a backend is blocked on a DDL
-- also for legacy reasons citus_internal_global_blocked_processes() returns groupId, we replace that with nodeIds
case WHEN waiting_global_pid !=0 THEN waiting_global_pid ELSE citus_calculate_gpid(get_nodeid_for_groupid(waiting_node_id), waiting_pid) END waiting_global_pid,
case WHEN blocking_global_pid !=0 THEN blocking_global_pid ELSE citus_calculate_gpid(get_nodeid_for_groupid(blocking_node_id), blocking_pid) END blocking_global_pid,
-- citus_internal_global_blocked_processes returns groupId, we replace it here with actual
-- nodeId to be consisten with the other views
get_nodeid_for_groupid(blocking_node_id) as blocking_node_id,
get_nodeid_for_groupid(waiting_node_id) as waiting_node_id,
blocking_transaction_waiting
FROM citus_internal_global_blocked_processes()
), ),
unique_global_wait_edges AS unique_global_wait_edges AS
( (
SELECT DISTINCT ON(waiting_global_pid, blocking_global_pid) * FROM citus_internal_global_blocked_processes() SELECT DISTINCT ON(waiting_global_pid, blocking_global_pid) * FROM unique_global_wait_edges_with_calculated_gpids
), ),
citus_dist_stat_activity_with_node_id AS citus_dist_stat_activity_with_calculated_gpids AS
( (
SELECT -- if global_pid is NULL, it is most likely that a backend is blocked on a DDL
citus_dist_stat_activity.*, (CASE citus_dist_stat_activity.distributed_query_host_name WHEN 'coordinator_host' THEN 0 ELSE pg_dist_node.nodeid END) as initiator_node_id SELECT CASE WHEN global_pid != 0 THEN global_pid ELSE citus_calculate_gpid(nodeid, pid) END global_pid, nodeid, pid, query FROM citus_dist_stat_activity
FROM
citus_dist_stat_activity LEFT JOIN pg_dist_node
ON
citus_dist_stat_activity.distributed_query_host_name = pg_dist_node.nodename AND
citus_dist_stat_activity.distributed_query_host_port = pg_dist_node.nodeport
) )
SELECT SELECT
waiting.global_pid as waiting_gpid, waiting.global_pid as waiting_gpid,
blocking.global_pid as blocking_gpid, blocking.global_pid as blocking_gpid,
waiting.pid AS waiting_pid, waiting.query AS blocked_statement,
blocking.pid AS blocking_pid, blocking.query AS current_statement_in_blocking_process,
waiting.query AS blocked_statement, waiting.nodeid AS waiting_nodeid,
blocking.query AS current_statement_in_blocking_process, blocking.nodeid AS blocking_nodeid
waiting.initiator_node_id AS waiting_node_id,
blocking.initiator_node_id AS blocking_node_id,
waiting.distributed_query_host_name AS waiting_node_name,
blocking.distributed_query_host_name AS blocking_node_name,
waiting.distributed_query_host_port AS waiting_node_port,
blocking.distributed_query_host_port AS blocking_node_port
FROM FROM
unique_global_wait_edges unique_global_wait_edges
JOIN JOIN
citus_dist_stat_activity_with_node_id waiting ON (unique_global_wait_edges.waiting_global_pid = waiting.global_pid) citus_dist_stat_activity_with_calculated_gpids waiting ON (unique_global_wait_edges.waiting_global_pid = waiting.global_pid)
JOIN JOIN
citus_dist_stat_activity_with_node_id blocking ON (unique_global_wait_edges.blocking_global_pid = blocking.global_pid); citus_dist_stat_activity_with_calculated_gpids blocking ON (unique_global_wait_edges.blocking_global_pid = blocking.global_pid);
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;

View File

@ -2,43 +2,44 @@ SET search_path = 'pg_catalog';
CREATE VIEW citus.citus_lock_waits AS CREATE VIEW citus.citus_lock_waits AS
WITH WITH
citus_dist_stat_activity AS unique_global_wait_edges_with_calculated_gpids AS (
( SELECT
SELECT * FROM citus_dist_stat_activity -- if global_pid is NULL, it is most likely that a backend is blocked on a DDL
-- also for legacy reasons citus_internal_global_blocked_processes() returns groupId, we replace that with nodeIds
case WHEN waiting_global_pid !=0 THEN waiting_global_pid ELSE citus_calculate_gpid(get_nodeid_for_groupid(waiting_node_id), waiting_pid) END waiting_global_pid,
case WHEN blocking_global_pid !=0 THEN blocking_global_pid ELSE citus_calculate_gpid(get_nodeid_for_groupid(blocking_node_id), blocking_pid) END blocking_global_pid,
-- citus_internal_global_blocked_processes returns groupId, we replace it here with actual
-- nodeId to be consisten with the other views
get_nodeid_for_groupid(blocking_node_id) as blocking_node_id,
get_nodeid_for_groupid(waiting_node_id) as waiting_node_id,
blocking_transaction_waiting
FROM citus_internal_global_blocked_processes()
), ),
unique_global_wait_edges AS unique_global_wait_edges AS
( (
SELECT DISTINCT ON(waiting_global_pid, blocking_global_pid) * FROM citus_internal_global_blocked_processes() SELECT DISTINCT ON(waiting_global_pid, blocking_global_pid) * FROM unique_global_wait_edges_with_calculated_gpids
), ),
citus_dist_stat_activity_with_node_id AS citus_dist_stat_activity_with_calculated_gpids AS
( (
SELECT -- if global_pid is NULL, it is most likely that a backend is blocked on a DDL
citus_dist_stat_activity.*, (CASE citus_dist_stat_activity.distributed_query_host_name WHEN 'coordinator_host' THEN 0 ELSE pg_dist_node.nodeid END) as initiator_node_id SELECT CASE WHEN global_pid != 0 THEN global_pid ELSE citus_calculate_gpid(nodeid, pid) END global_pid, nodeid, pid, query FROM citus_dist_stat_activity
FROM
citus_dist_stat_activity LEFT JOIN pg_dist_node
ON
citus_dist_stat_activity.distributed_query_host_name = pg_dist_node.nodename AND
citus_dist_stat_activity.distributed_query_host_port = pg_dist_node.nodeport
) )
SELECT SELECT
waiting.global_pid as waiting_gpid, waiting.global_pid as waiting_gpid,
blocking.global_pid as blocking_gpid, blocking.global_pid as blocking_gpid,
waiting.pid AS waiting_pid, waiting.query AS blocked_statement,
blocking.pid AS blocking_pid, blocking.query AS current_statement_in_blocking_process,
waiting.query AS blocked_statement, waiting.nodeid AS waiting_nodeid,
blocking.query AS current_statement_in_blocking_process, blocking.nodeid AS blocking_nodeid
waiting.initiator_node_id AS waiting_node_id,
blocking.initiator_node_id AS blocking_node_id,
waiting.distributed_query_host_name AS waiting_node_name,
blocking.distributed_query_host_name AS blocking_node_name,
waiting.distributed_query_host_port AS waiting_node_port,
blocking.distributed_query_host_port AS blocking_node_port
FROM FROM
unique_global_wait_edges unique_global_wait_edges
JOIN JOIN
citus_dist_stat_activity_with_node_id waiting ON (unique_global_wait_edges.waiting_global_pid = waiting.global_pid) citus_dist_stat_activity_with_calculated_gpids waiting ON (unique_global_wait_edges.waiting_global_pid = waiting.global_pid)
JOIN JOIN
citus_dist_stat_activity_with_node_id blocking ON (unique_global_wait_edges.blocking_global_pid = blocking.global_pid); citus_dist_stat_activity_with_calculated_gpids blocking ON (unique_global_wait_edges.blocking_global_pid = blocking.global_pid);
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;

View File

@ -0,0 +1,7 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_nodeid_for_gpid(global_pid bigint)
RETURNS integer
LANGUAGE C STABLE STRICT
AS 'MODULE_PATHNAME', $$citus_nodeid_for_gpid$$;
COMMENT ON FUNCTION pg_catalog.citus_nodeid_for_gpid(global_pid bigint)
IS 'returns node id for the global process with given global pid';

View File

@ -0,0 +1,7 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_nodeid_for_gpid(global_pid bigint)
RETURNS integer
LANGUAGE C STABLE STRICT
AS 'MODULE_PATHNAME', $$citus_nodeid_for_gpid$$;
COMMENT ON FUNCTION pg_catalog.citus_nodeid_for_gpid(global_pid bigint)
IS 'returns node id for the global process with given global pid';

View File

@ -0,0 +1,7 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_nodename_for_nodeid(nodeid integer)
RETURNS text
LANGUAGE C STABLE STRICT
AS 'MODULE_PATHNAME', $$citus_nodename_for_nodeid$$;
COMMENT ON FUNCTION pg_catalog.citus_nodename_for_nodeid(nodeid integer)
IS 'returns node name for the node with given node id';

View File

@ -0,0 +1,7 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_nodename_for_nodeid(nodeid integer)
RETURNS text
LANGUAGE C STABLE STRICT
AS 'MODULE_PATHNAME', $$citus_nodename_for_nodeid$$;
COMMENT ON FUNCTION pg_catalog.citus_nodename_for_nodeid(nodeid integer)
IS 'returns node name for the node with given node id';

View File

@ -0,0 +1,7 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_nodeport_for_nodeid(nodeid integer)
RETURNS integer
LANGUAGE C STABLE STRICT
AS 'MODULE_PATHNAME', $$citus_nodeport_for_nodeid$$;
COMMENT ON FUNCTION pg_catalog.citus_nodeport_for_nodeid(nodeid integer)
IS 'returns node port for the node with given node id';

View File

@ -0,0 +1,7 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_nodeport_for_nodeid(nodeid integer)
RETURNS integer
LANGUAGE C STABLE STRICT
AS 'MODULE_PATHNAME', $$citus_nodeport_for_nodeid$$;
COMMENT ON FUNCTION pg_catalog.citus_nodeport_for_nodeid(nodeid integer)
IS 'returns node port for the node with given node id';

View File

@ -0,0 +1,7 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_pid_for_gpid(global_pid bigint)
RETURNS integer
LANGUAGE C STABLE STRICT
AS 'MODULE_PATHNAME', $$citus_pid_for_gpid$$;
COMMENT ON FUNCTION pg_catalog.citus_pid_for_gpid(global_pid bigint)
IS 'returns process id for the global process with given global pid';

View File

@ -0,0 +1,7 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_pid_for_gpid(global_pid bigint)
RETURNS integer
LANGUAGE C STABLE STRICT
AS 'MODULE_PATHNAME', $$citus_pid_for_gpid$$;
COMMENT ON FUNCTION pg_catalog.citus_pid_for_gpid(global_pid bigint)
IS 'returns process id for the global process with given global pid';

View File

@ -0,0 +1,74 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_prepare_pg_upgrade()
RETURNS void
LANGUAGE plpgsql
SET search_path = pg_catalog
AS $cppu$
BEGIN
DELETE FROM pg_depend WHERE
objid IN (SELECT oid FROM pg_proc WHERE proname = 'array_cat_agg') AND
refobjid IN (select oid from pg_extension where extname = 'citus');
--
-- We are dropping the aggregates because postgres 14 changed
-- array_cat type from anyarray to anycompatiblearray. When
-- upgrading to pg14, specifically when running pg_restore on
-- array_cat_agg we would get an error. So we drop the aggregate
-- and create the right one on citus_finish_pg_upgrade.
DROP AGGREGATE IF EXISTS array_cat_agg(anyarray);
DROP AGGREGATE IF EXISTS array_cat_agg(anycompatiblearray);
--
-- Drop existing backup tables
--
DROP TABLE IF EXISTS public.pg_dist_partition;
DROP TABLE IF EXISTS public.pg_dist_shard;
DROP TABLE IF EXISTS public.pg_dist_placement;
DROP TABLE IF EXISTS public.pg_dist_node_metadata;
DROP TABLE IF EXISTS public.pg_dist_node;
DROP TABLE IF EXISTS public.pg_dist_local_group;
DROP TABLE IF EXISTS public.pg_dist_transaction;
DROP TABLE IF EXISTS public.pg_dist_colocation;
DROP TABLE IF EXISTS public.pg_dist_authinfo;
DROP TABLE IF EXISTS public.pg_dist_poolinfo;
DROP TABLE IF EXISTS public.pg_dist_rebalance_strategy;
DROP TABLE IF EXISTS public.pg_dist_object;
--
-- backup citus catalog tables
--
CREATE TABLE public.pg_dist_partition AS SELECT * FROM pg_catalog.pg_dist_partition;
CREATE TABLE public.pg_dist_shard AS SELECT * FROM pg_catalog.pg_dist_shard;
CREATE TABLE public.pg_dist_placement AS SELECT * FROM pg_catalog.pg_dist_placement;
CREATE TABLE public.pg_dist_node_metadata AS SELECT * FROM pg_catalog.pg_dist_node_metadata;
CREATE TABLE public.pg_dist_node AS SELECT * FROM pg_catalog.pg_dist_node;
CREATE TABLE public.pg_dist_local_group AS SELECT * FROM pg_catalog.pg_dist_local_group;
CREATE TABLE public.pg_dist_transaction AS SELECT * FROM pg_catalog.pg_dist_transaction;
CREATE TABLE public.pg_dist_colocation AS SELECT * FROM pg_catalog.pg_dist_colocation;
-- enterprise catalog tables
CREATE TABLE public.pg_dist_authinfo AS SELECT * FROM pg_catalog.pg_dist_authinfo;
CREATE TABLE public.pg_dist_poolinfo AS SELECT * FROM pg_catalog.pg_dist_poolinfo;
CREATE TABLE public.pg_dist_rebalance_strategy AS SELECT
name,
default_strategy,
shard_cost_function::regprocedure::text,
node_capacity_function::regprocedure::text,
shard_allowed_on_node_function::regprocedure::text,
default_threshold,
minimum_threshold,
improvement_threshold
FROM pg_catalog.pg_dist_rebalance_strategy;
-- store upgrade stable identifiers on pg_dist_object catalog
CREATE TABLE public.pg_dist_object AS SELECT
address.type,
address.object_names,
address.object_args,
objects.distribution_argument_index,
objects.colocationid
FROM pg_catalog.pg_dist_object objects,
pg_catalog.pg_identify_object_as_address(objects.classid, objects.objid, objects.objsubid) address;
END;
$cppu$;
COMMENT ON FUNCTION pg_catalog.citus_prepare_pg_upgrade()
IS 'perform tasks to copy citus settings to a location that could later be restored after pg_upgrade is done';

View File

@ -65,7 +65,7 @@ BEGIN
address.object_args, address.object_args,
objects.distribution_argument_index, objects.distribution_argument_index,
objects.colocationid objects.colocationid
FROM citus.pg_dist_object objects, FROM pg_catalog.pg_dist_object objects,
pg_catalog.pg_identify_object_as_address(objects.classid, objects.objid, objects.objsubid) address; pg_catalog.pg_identify_object_as_address(objects.classid, objects.objid, objects.objsubid) address;
END; END;
$cppu$; $cppu$;

View File

@ -0,0 +1,42 @@
-- citus_stat_activity combines the pg_stat_activity views from all nodes and adds global_pid, nodeid and is_worker_query columns.
-- The columns of citus_stat_activity don't change based on the Postgres version, however the pg_stat_activity's columns do.
-- Both Postgres 13 and 14 added one more column to pg_stat_activity (leader_pid and query_id).
-- citus_stat_activity has the most expansive column set, including the newly added columns.
-- If citus_stat_activity is queried in a Postgres version where pg_stat_activity doesn't have some columns citus_stat_activity has
-- the values for those columns will be NULL
CREATE OR REPLACE FUNCTION pg_catalog.citus_stat_activity(OUT global_pid bigint, OUT nodeid int, OUT is_worker_query boolean, OUT datid oid, OUT datname name, OUT pid integer,
OUT leader_pid integer, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr inet, OUT client_hostname text,
OUT client_port integer, OUT backend_start timestamp with time zone, OUT xact_start timestamp with time zone,
OUT query_start timestamp with time zone, OUT state_change timestamp with time zone, OUT wait_event_type text, OUT wait_event text,
OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query_id bigint, OUT query text, OUT backend_type text)
RETURNS SETOF record
LANGUAGE plpgsql
AS $function$
BEGIN
RETURN QUERY SELECT * FROM jsonb_to_recordset((
SELECT jsonb_agg(all_csa_rows_as_jsonb.csa_row_as_jsonb)::JSONB FROM (
SELECT jsonb_array_elements(run_command_on_all_nodes.result::JSONB)::JSONB || ('{"nodeid":' || run_command_on_all_nodes.nodeid || '}')::JSONB AS csa_row_as_jsonb
FROM run_command_on_all_nodes($$
SELECT coalesce(to_jsonb(array_agg(csa_from_one_node.*)), '[{}]'::JSONB)
FROM (
SELECT global_pid, worker_query AS is_worker_query, pg_stat_activity.* FROM
pg_stat_activity LEFT JOIN get_all_active_transactions() ON process_id = pid
) AS csa_from_one_node;
$$, parallel:=true, give_warning_for_connection_errors:=true)
WHERE success = 't'
) AS all_csa_rows_as_jsonb
))
AS (global_pid bigint, nodeid int, is_worker_query boolean, datid oid, datname name, pid integer,
leader_pid integer, usesysid oid, usename name, application_name text, client_addr inet, client_hostname text,
client_port integer, backend_start timestamp with time zone, xact_start timestamp with time zone,
query_start timestamp with time zone, state_change timestamp with time zone, wait_event_type text, wait_event text,
state text, backend_xid xid, backend_xmin xid, query_id bigint, query text, backend_type text);
END;
$function$;
CREATE OR REPLACE VIEW citus.citus_stat_activity AS
SELECT * FROM pg_catalog.citus_stat_activity();
ALTER VIEW citus.citus_stat_activity SET SCHEMA pg_catalog;
GRANT SELECT ON pg_catalog.citus_stat_activity TO PUBLIC;

View File

@ -0,0 +1,42 @@
-- citus_stat_activity combines the pg_stat_activity views from all nodes and adds global_pid, nodeid and is_worker_query columns.
-- The columns of citus_stat_activity don't change based on the Postgres version, however the pg_stat_activity's columns do.
-- Both Postgres 13 and 14 added one more column to pg_stat_activity (leader_pid and query_id).
-- citus_stat_activity has the most expansive column set, including the newly added columns.
-- If citus_stat_activity is queried in a Postgres version where pg_stat_activity doesn't have some columns citus_stat_activity has
-- the values for those columns will be NULL
CREATE OR REPLACE FUNCTION pg_catalog.citus_stat_activity(OUT global_pid bigint, OUT nodeid int, OUT is_worker_query boolean, OUT datid oid, OUT datname name, OUT pid integer,
OUT leader_pid integer, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr inet, OUT client_hostname text,
OUT client_port integer, OUT backend_start timestamp with time zone, OUT xact_start timestamp with time zone,
OUT query_start timestamp with time zone, OUT state_change timestamp with time zone, OUT wait_event_type text, OUT wait_event text,
OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query_id bigint, OUT query text, OUT backend_type text)
RETURNS SETOF record
LANGUAGE plpgsql
AS $function$
BEGIN
RETURN QUERY SELECT * FROM jsonb_to_recordset((
SELECT jsonb_agg(all_csa_rows_as_jsonb.csa_row_as_jsonb)::JSONB FROM (
SELECT jsonb_array_elements(run_command_on_all_nodes.result::JSONB)::JSONB || ('{"nodeid":' || run_command_on_all_nodes.nodeid || '}')::JSONB AS csa_row_as_jsonb
FROM run_command_on_all_nodes($$
SELECT coalesce(to_jsonb(array_agg(csa_from_one_node.*)), '[{}]'::JSONB)
FROM (
SELECT global_pid, worker_query AS is_worker_query, pg_stat_activity.* FROM
pg_stat_activity LEFT JOIN get_all_active_transactions() ON process_id = pid
) AS csa_from_one_node;
$$, parallel:=true, give_warning_for_connection_errors:=true)
WHERE success = 't'
) AS all_csa_rows_as_jsonb
))
AS (global_pid bigint, nodeid int, is_worker_query boolean, datid oid, datname name, pid integer,
leader_pid integer, usesysid oid, usename name, application_name text, client_addr inet, client_hostname text,
client_port integer, backend_start timestamp with time zone, xact_start timestamp with time zone,
query_start timestamp with time zone, state_change timestamp with time zone, wait_event_type text, wait_event text,
state text, backend_xid xid, backend_xmin xid, query_id bigint, query text, backend_type text);
END;
$function$;
CREATE OR REPLACE VIEW citus.citus_stat_activity AS
SELECT * FROM pg_catalog.citus_stat_activity();
ALTER VIEW citus.citus_stat_activity SET SCHEMA pg_catalog;
GRANT SELECT ON pg_catalog.citus_stat_activity TO PUBLIC;

View File

@ -1,19 +0,0 @@
DROP FUNCTION IF EXISTS pg_catalog.citus_worker_stat_activity CASCADE;
CREATE OR REPLACE FUNCTION pg_catalog.citus_worker_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int,
OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name,
OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET,
OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz,
OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text,
OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text, OUT global_pid int8)
RETURNS SETOF RECORD
LANGUAGE C STRICT AS 'MODULE_PATHNAME',
$$citus_worker_stat_activity$$;
COMMENT ON FUNCTION pg_catalog.citus_worker_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int,
OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name,
OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET,
OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz,
OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text,
OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text, OUT global_pid int8)
IS 'returns distributed transaction activity on shards of distributed tables';

View File

@ -1,19 +0,0 @@
DROP FUNCTION IF EXISTS pg_catalog.citus_worker_stat_activity CASCADE;
CREATE OR REPLACE FUNCTION pg_catalog.citus_worker_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int,
OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name,
OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET,
OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz,
OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text,
OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text, OUT global_pid int8)
RETURNS SETOF RECORD
LANGUAGE C STRICT AS 'MODULE_PATHNAME',
$$citus_worker_stat_activity$$;
COMMENT ON FUNCTION pg_catalog.citus_worker_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int,
OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name,
OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET,
OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz,
OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text,
OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text, OUT global_pid int8)
IS 'returns distributed transaction activity on shards of distributed tables';

View File

@ -0,0 +1,60 @@
DROP FUNCTION IF EXISTS pg_catalog.run_command_on_all_nodes;
CREATE FUNCTION pg_catalog.run_command_on_all_nodes(command text, parallel bool default true, give_warning_for_connection_errors bool default false,
OUT nodeid int, OUT success bool, OUT result text)
RETURNS SETOF record
LANGUAGE plpgsql
AS $function$
DECLARE
nodenames text[];
ports int[];
commands text[];
current_node_is_in_metadata boolean;
command_result_of_current_node text;
BEGIN
WITH citus_nodes AS (
SELECT * FROM pg_dist_node
WHERE isactive = 't' AND nodecluster = current_setting('citus.cluster_name')
AND (
(current_setting('citus.use_secondary_nodes') = 'never' AND noderole = 'primary')
OR
(current_setting('citus.use_secondary_nodes') = 'always' AND noderole = 'secondary')
)
ORDER BY nodename, nodeport
)
SELECT array_agg(citus_nodes.nodename), array_agg(citus_nodes.nodeport), array_agg(command)
INTO nodenames, ports, commands
FROM citus_nodes;
SELECT count(*) > 0 FROM pg_dist_node
WHERE isactive = 't'
AND nodecluster = current_setting('citus.cluster_name')
AND groupid IN (SELECT groupid FROM pg_dist_local_group)
INTO current_node_is_in_metadata;
-- This will happen when we call this function on coordinator and
-- the coordinator is not added to the metadata.
-- We'll manually add current node to the lists to actually run on all nodes.
-- But when the coordinator is not added to metadata and this function
-- is called from a worker node, this will not be enough and we'll
-- not be able run on all nodes.
IF NOT current_node_is_in_metadata THEN
SELECT
array_append(nodenames, current_setting('citus.local_hostname')),
array_append(ports, current_setting('port')::int),
array_append(commands, command)
INTO nodenames, ports, commands;
END IF;
FOR nodeid, success, result IN
SELECT coalesce(pg_dist_node.nodeid, 0) AS nodeid, mrow.success, mrow.result
FROM master_run_on_worker(nodenames, ports, commands, parallel) mrow
LEFT JOIN pg_dist_node ON mrow.node_name = pg_dist_node.nodename AND mrow.node_port = pg_dist_node.nodeport
LOOP
IF give_warning_for_connection_errors AND NOT success THEN
RAISE WARNING 'Error on node with node id %: %', nodeid, result;
END IF;
RETURN NEXT;
END LOOP;
END;
$function$;

View File

@ -0,0 +1,60 @@
DROP FUNCTION IF EXISTS pg_catalog.run_command_on_all_nodes;
CREATE FUNCTION pg_catalog.run_command_on_all_nodes(command text, parallel bool default true, give_warning_for_connection_errors bool default false,
OUT nodeid int, OUT success bool, OUT result text)
RETURNS SETOF record
LANGUAGE plpgsql
AS $function$
DECLARE
nodenames text[];
ports int[];
commands text[];
current_node_is_in_metadata boolean;
command_result_of_current_node text;
BEGIN
WITH citus_nodes AS (
SELECT * FROM pg_dist_node
WHERE isactive = 't' AND nodecluster = current_setting('citus.cluster_name')
AND (
(current_setting('citus.use_secondary_nodes') = 'never' AND noderole = 'primary')
OR
(current_setting('citus.use_secondary_nodes') = 'always' AND noderole = 'secondary')
)
ORDER BY nodename, nodeport
)
SELECT array_agg(citus_nodes.nodename), array_agg(citus_nodes.nodeport), array_agg(command)
INTO nodenames, ports, commands
FROM citus_nodes;
SELECT count(*) > 0 FROM pg_dist_node
WHERE isactive = 't'
AND nodecluster = current_setting('citus.cluster_name')
AND groupid IN (SELECT groupid FROM pg_dist_local_group)
INTO current_node_is_in_metadata;
-- This will happen when we call this function on coordinator and
-- the coordinator is not added to the metadata.
-- We'll manually add current node to the lists to actually run on all nodes.
-- But when the coordinator is not added to metadata and this function
-- is called from a worker node, this will not be enough and we'll
-- not be able run on all nodes.
IF NOT current_node_is_in_metadata THEN
SELECT
array_append(nodenames, current_setting('citus.local_hostname')),
array_append(ports, current_setting('port')::int),
array_append(commands, command)
INTO nodenames, ports, commands;
END IF;
FOR nodeid, success, result IN
SELECT coalesce(pg_dist_node.nodeid, 0) AS nodeid, mrow.success, mrow.result
FROM master_run_on_worker(nodenames, ports, commands, parallel) mrow
LEFT JOIN pg_dist_node ON mrow.node_name = pg_dist_node.nodename AND mrow.node_port = pg_dist_node.nodeport
LOOP
IF give_warning_for_connection_errors AND NOT success THEN
RAISE WARNING 'Error on node with node id %: %', nodeid, result;
END IF;
RETURN NEXT;
END LOOP;
END;
$function$;

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

@ -27,5 +27,5 @@ PG_FUNCTION_INFO_V1(get_all_active_client_backend_count);
Datum Datum
get_all_active_client_backend_count(PG_FUNCTION_ARGS) get_all_active_client_backend_count(PG_FUNCTION_ARGS)
{ {
PG_RETURN_UINT32(GetAllActiveClientBackendCount()); PG_RETURN_UINT32(GetExternalClientBackendCount());
} }

View File

@ -0,0 +1,33 @@
/*-------------------------------------------------------------------------
*
* test/src/global_pid.c
*
* This file contains functions to test the global pid.
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "fmgr.h"
#include "distributed/backend_data.h"
#include "distributed/metadata_cache.h"
PG_FUNCTION_INFO_V1(test_assign_global_pid);
/*
* test_assign_global_pid is the wrapper UDF for AssignGlobalPID and is only meant for use
* in tests.
*/
Datum
test_assign_global_pid(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
AssignGlobalPID();
PG_RETURN_VOID();
}

View File

@ -0,0 +1,67 @@
/*-------------------------------------------------------------------------
*
* test/src/make_external_connection.c
*
* This file contains UDF to connect to a node without using the Citus
* internal application_name.
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "miscadmin.h"
#include "libpq-fe.h"
#include "access/xact.h"
#include "distributed/connection_management.h"
#include "distributed/coordinator_protocol.h"
#include "distributed/function_utils.h"
#include "distributed/intermediate_result_pruning.h"
#include "distributed/lock_graph.h"
#include "distributed/metadata_cache.h"
#include "distributed/remote_commands.h"
#include "distributed/run_from_same_connection.h"
#include "distributed/version_compat.h"
#include "executor/spi.h"
#include "lib/stringinfo.h"
#include "postmaster/postmaster.h"
#include "utils/builtins.h"
#include "utils/memutils.h"
PG_FUNCTION_INFO_V1(make_external_connection_to_node);
/*
* make_external_connection_to_node opens a conneciton to a node
* and keeps it until the end of the session.
*/
Datum
make_external_connection_to_node(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
char *nodeName = text_to_cstring(PG_GETARG_TEXT_P(0));
uint32 nodePort = PG_GETARG_UINT32(1);
char *userName = text_to_cstring(PG_GETARG_TEXT_P(2));
char *databaseName = text_to_cstring(PG_GETARG_TEXT_P(3));
StringInfo connectionString = makeStringInfo();
appendStringInfo(connectionString,
"host=%s port=%d user=%s dbname=%s",
nodeName, nodePort, userName, databaseName);
PGconn *pgConn = PQconnectdb(connectionString->data);
if (PQstatus(pgConn) != CONNECTION_OK)
{
PQfinish(pgConn);
ereport(ERROR, (errmsg("connection failed")));
}
PG_RETURN_VOID();
}

View File

@ -188,7 +188,7 @@ run_commands_on_session_level_connection_to_node(PG_FUNCTION_ARGS)
/* /*
* override_backend_data_command_originator is a wrapper around * override_backend_data_command_originator is a wrapper around
* OverrideBackendDataDistributedCommandOriginator(). * SetBackendDataDistributedCommandOriginator().
*/ */
Datum Datum
override_backend_data_command_originator(PG_FUNCTION_ARGS) override_backend_data_command_originator(PG_FUNCTION_ARGS)
@ -197,7 +197,7 @@ override_backend_data_command_originator(PG_FUNCTION_ARGS)
bool distributedCommandOriginator = PG_GETARG_BOOL(0); bool distributedCommandOriginator = PG_GETARG_BOOL(0);
OverrideBackendDataDistributedCommandOriginator(distributedCommandOriginator); SetBackendDataDistributedCommandOriginator(distributedCommandOriginator);
PG_RETURN_VOID(); PG_RETURN_VOID();
} }

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

@ -68,14 +68,12 @@ typedef struct BackendManagementShmemData
pg_atomic_uint64 nextTransactionNumber; pg_atomic_uint64 nextTransactionNumber;
/* /*
* Total number of client backends that are authenticated. * Total number of external client backends that are authenticated.
* We only care about activeClientBackendCounter when adaptive
* connection management is enabled, otherwise always zero.
* *
* Note that the counter does not consider any background workers * Note that the counter does not consider any background workers
* or such, it only counts client_backends. * or such, and also exludes internal connections between nodes.
*/ */
pg_atomic_uint32 activeClientBackendCounter; pg_atomic_uint32 externalClientBackendCounter;
BackendData backends[FLEXIBLE_ARRAY_MEMBER]; BackendData backends[FLEXIBLE_ARRAY_MEMBER];
} BackendManagementShmemData; } BackendManagementShmemData;
@ -83,6 +81,8 @@ 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 CalculateGlobalPID(int32 nodeId, pid_t pid);
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;
@ -98,6 +98,10 @@ PG_FUNCTION_INFO_V1(assign_distributed_transaction_id);
PG_FUNCTION_INFO_V1(get_current_transaction_id); PG_FUNCTION_INFO_V1(get_current_transaction_id);
PG_FUNCTION_INFO_V1(get_global_active_transactions); PG_FUNCTION_INFO_V1(get_global_active_transactions);
PG_FUNCTION_INFO_V1(get_all_active_transactions); PG_FUNCTION_INFO_V1(get_all_active_transactions);
PG_FUNCTION_INFO_V1(citus_calculate_gpid);
PG_FUNCTION_INFO_V1(citus_backend_gpid);
PG_FUNCTION_INFO_V1(citus_nodeid_for_gpid);
PG_FUNCTION_INFO_V1(citus_pid_for_gpid);
/* /*
@ -114,8 +118,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 +146,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 +356,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 +412,42 @@ StoreAllActiveTransactions(Tuplestorestate *tupleStore, TupleDesc tupleDescripto
SpinLockRelease(&currentBackend->mutex); SpinLockRelease(&currentBackend->mutex);
values[0] = ObjectIdGetDatum(databaseId); memset(values, 0, sizeof(values));
values[1] = Int32GetDatum(backendPid); memset(isNulls, false, sizeof(isNulls));
values[2] = Int32GetDatum(initiatorNodeIdentifier);
values[3] = !distributedCommandOriginator; /*
values[4] = UInt64GetDatum(transactionNumber); * We imitate pg_stat_activity such that if a user doesn't have enough
values[5] = TimestampTzGetDatum(transactionIdTimestamp); * privileges, we only show the minimal information including the pid,
values[6] = UInt64GetDatum(currentBackend->globalPID); * 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[1] = Int32GetDatum(backendPid);
values[2] = Int32GetDatum(initiatorNodeId);
values[3] = !distributedCommandOriginator;
values[4] = UInt64GetDatum(transactionNumber);
values[5] = TimestampTzGetDatum(transactionIdTimestamp);
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 +464,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.
@ -504,7 +551,7 @@ BackendManagementShmemInit(void)
pg_atomic_init_u64(&backendManagementShmemData->nextTransactionNumber, 1); pg_atomic_init_u64(&backendManagementShmemData->nextTransactionNumber, 1);
/* there are no active backends yet, so start with zero */ /* there are no active backends yet, so start with zero */
pg_atomic_init_u32(&backendManagementShmemData->activeClientBackendCounter, 0); pg_atomic_init_u32(&backendManagementShmemData->externalClientBackendCounter, 0);
/* /*
* We need to init per backend's spinlock before any backend * We need to init per backend's spinlock before any backend
@ -520,7 +567,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 +698,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 +721,9 @@ UnSetGlobalPID(void)
SpinLockAcquire(&MyBackendData->mutex); SpinLockAcquire(&MyBackendData->mutex);
MyBackendData->globalPID = 0; MyBackendData->globalPID = 0;
MyBackendData->databaseId = 0;
MyBackendData->userId = 0;
MyBackendData->distributedCommandOriginator = false;
SpinLockRelease(&MyBackendData->mutex); SpinLockRelease(&MyBackendData->mutex);
} }
@ -760,41 +805,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,20 +839,30 @@ 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);
} }
/* /*
* OverrideBackendDataDistributedCommandOriginator should only be used for isolation testing. * SetBackendDataDistributedCommandOriginator is used to set the distributedCommandOriginator
* See how it is used in the relevant functions. * field on MyBackendData.
*/ */
void void
OverrideBackendDataDistributedCommandOriginator(bool distributedCommandOriginator) SetBackendDataDistributedCommandOriginator(bool distributedCommandOriginator)
{ {
if (!MyBackendData)
{
return;
}
SpinLockAcquire(&MyBackendData->mutex); SpinLockAcquire(&MyBackendData->mutex);
MyBackendData->distributedCommandOriginator = MyBackendData->distributedCommandOriginator =
distributedCommandOriginator; distributedCommandOriginator;
@ -862,10 +890,27 @@ GetGlobalPID(void)
/* /*
* GenerateGlobalPID generates the global process id for the current backend. * citus_calculate_gpid calculates the gpid for any given process on any
* given node.
*/
Datum
citus_calculate_gpid(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
int32 nodeId = PG_GETARG_INT32(0);
int32 pid = PG_GETARG_INT32(1);
PG_RETURN_UINT64(CalculateGlobalPID(nodeId, pid));
}
/*
* CalculateGlobalPID gets a nodeId and pid, and returns the global pid
* that can be assigned for a process with the given input.
*/ */
static uint64 static uint64
GenerateGlobalPID(void) CalculateGlobalPID(int32 nodeId, pid_t pid)
{ {
/* /*
* We try to create a human readable global pid that consists of node id and process id. * We try to create a human readable global pid that consists of node id and process id.
@ -876,7 +921,60 @@ GenerateGlobalPID(void)
* node ids might cause overflow. But even for the applications that scale around 50 nodes every * node ids might cause overflow. But even for the applications that scale around 50 nodes every
* day it'd take about 100K years. So we are not worried. * day it'd take about 100K years. So we are not worried.
*/ */
return (((uint64) GetLocalNodeId()) * GLOBAL_PID_NODE_ID_MULTIPLIER) + getpid(); return (((uint64) nodeId) * GLOBAL_PID_NODE_ID_MULTIPLIER) + pid;
}
/*
* GenerateGlobalPID generates the global process id for the current backend.
* See CalculateGlobalPID for the details.
*/
static uint64
GenerateGlobalPID(void)
{
return CalculateGlobalPID(GetLocalNodeId(), getpid());
}
/*
* citus_backend_gpid similar to pg_backend_pid, but returns Citus
* assigned gpid.
*/
Datum
citus_backend_gpid(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
PG_RETURN_UINT64(GetGlobalPID());
}
/*
* citus_nodeid_for_gpid returns node id for the global process with given global pid
*/
Datum
citus_nodeid_for_gpid(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
uint64 globalPID = PG_GETARG_INT64(0);
bool missingOk = false;
PG_RETURN_INT32(ExtractNodeIdFromGlobalPID(globalPID, missingOk));
}
/*
* citus_pid_for_gpid returns process id for the global process with given global pid
*/
Datum
citus_pid_for_gpid(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
uint64 globalPID = PG_GETARG_INT64(0);
PG_RETURN_INT32(ExtractProcessIdFromGlobalPID(globalPID));
} }
@ -926,11 +1024,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),
@ -1145,36 +1244,37 @@ GetMyProcLocalTransactionId(void)
/* /*
* GetAllActiveClientBackendCount returns activeClientBackendCounter in * GetExternalClientBackendCount returns externalClientBackendCounter in
* the shared memory. * the shared memory.
*/ */
int int
GetAllActiveClientBackendCount(void) GetExternalClientBackendCount(void)
{ {
uint32 activeBackendCount = uint32 activeBackendCount =
pg_atomic_read_u32(&backendManagementShmemData->activeClientBackendCounter); pg_atomic_read_u32(&backendManagementShmemData->externalClientBackendCounter);
return activeBackendCount; return activeBackendCount;
} }
/* /*
* IncrementClientBackendCounter increments activeClientBackendCounter in * IncrementExternalClientBackendCounter increments externalClientBackendCounter in
* the shared memory by one. * the shared memory by one.
*/ */
void uint32
IncrementClientBackendCounter(void) IncrementExternalClientBackendCounter(void)
{ {
pg_atomic_add_fetch_u32(&backendManagementShmemData->activeClientBackendCounter, 1); return pg_atomic_add_fetch_u32(
&backendManagementShmemData->externalClientBackendCounter, 1);
} }
/* /*
* DecrementClientBackendCounter decrements activeClientBackendCounter in * DecrementExternalClientBackendCounter decrements externalClientBackendCounter in
* the shared memory by one. * the shared memory by one.
*/ */
void void
DecrementClientBackendCounter(void) DecrementExternalClientBackendCounter(void)
{ {
pg_atomic_sub_fetch_u32(&backendManagementShmemData->activeClientBackendCounter, 1); pg_atomic_sub_fetch_u32(&backendManagementShmemData->externalClientBackendCounter, 1);
} }

File diff suppressed because it is too large Load Diff

View File

@ -583,6 +583,25 @@ CreateColocationGroup(int shardCount, int replicationFactor, Oid distributionCol
Oid distributionColumnCollation) Oid distributionColumnCollation)
{ {
uint32 colocationId = GetNextColocationId(); uint32 colocationId = GetNextColocationId();
InsertColocationGroupLocally(colocationId, shardCount, replicationFactor,
distributionColumnType, distributionColumnCollation);
SyncNewColocationGroupToNodes(colocationId, shardCount, replicationFactor,
distributionColumnType, distributionColumnCollation);
return colocationId;
}
/*
* InsertColocationGroupLocally inserts a record into pg_dist_colocation.
*/
void
InsertColocationGroupLocally(uint32 colocationId, int shardCount, int replicationFactor,
Oid distributionColumnType,
Oid distributionColumnCollation)
{
Datum values[Natts_pg_dist_colocation]; Datum values[Natts_pg_dist_colocation];
bool isNulls[Natts_pg_dist_colocation]; bool isNulls[Natts_pg_dist_colocation];
@ -610,8 +629,6 @@ CreateColocationGroup(int shardCount, int replicationFactor, Oid distributionCol
/* increment the counter so that next command can see the row */ /* increment the counter so that next command can see the row */
CommandCounterIncrement(); CommandCounterIncrement();
table_close(pgDistColocation, RowExclusiveLock); table_close(pgDistColocation, RowExclusiveLock);
return colocationId;
} }
@ -1215,10 +1232,22 @@ DeleteColocationGroupIfNoTablesBelong(uint32 colocationId)
/* /*
* DeleteColocationGroup deletes the colocation group from pg_dist_colocation. * DeleteColocationGroup deletes the colocation group from pg_dist_colocation
* throughout the cluster.
*/ */
static void static void
DeleteColocationGroup(uint32 colocationId) DeleteColocationGroup(uint32 colocationId)
{
DeleteColocationGroupLocally(colocationId);
SyncDeleteColocationGroupToNodes(colocationId);
}
/*
* DeleteColocationGroupLocally deletes the colocation group from pg_dist_colocation.
*/
void
DeleteColocationGroupLocally(uint32 colocationId)
{ {
int scanKeyCount = 1; int scanKeyCount = 1;
ScanKeyData scanKey[1]; ScanKeyData scanKey[1];

View File

@ -29,6 +29,7 @@
#include "distributed/coordinator_protocol.h" #include "distributed/coordinator_protocol.h"
#include "distributed/metadata_cache.h" #include "distributed/metadata_cache.h"
#include "distributed/metadata/distobject.h" #include "distributed/metadata/distobject.h"
#include "distributed/multi_partitioning_utils.h"
#include "foreign/foreign.h" #include "foreign/foreign.h"
#include "utils/builtins.h" #include "utils/builtins.h"
#include "utils/fmgroids.h" #include "utils/fmgroids.h"
@ -37,12 +38,13 @@ PG_FUNCTION_INFO_V1(worker_drop_distributed_table);
PG_FUNCTION_INFO_V1(worker_drop_shell_table); PG_FUNCTION_INFO_V1(worker_drop_shell_table);
PG_FUNCTION_INFO_V1(worker_drop_sequence_dependency); PG_FUNCTION_INFO_V1(worker_drop_sequence_dependency);
static void WorkerDropDistributedTable(Oid relationId);
#if PG_VERSION_NUM < PG_VERSION_13 #if PG_VERSION_NUM < PG_VERSION_13
static long deleteDependencyRecordsForSpecific(Oid classId, Oid objectId, char deptype, static long deleteDependencyRecordsForSpecific(Oid classId, Oid objectId, char deptype,
Oid refclassId, Oid refobjectId); Oid refclassId, Oid refobjectId);
#endif #endif
/* /*
* worker_drop_distributed_table drops the distributed table with the given oid, * worker_drop_distributed_table drops the distributed table with the given oid,
* then, removes the associated rows from pg_dist_partition, pg_dist_shard and * then, removes the associated rows from pg_dist_partition, pg_dist_shard and
@ -64,8 +66,6 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS)
text *relationName = PG_GETARG_TEXT_P(0); text *relationName = PG_GETARG_TEXT_P(0);
Oid relationId = ResolveRelationId(relationName, true); Oid relationId = ResolveRelationId(relationName, true);
ObjectAddress distributedTableObject = { InvalidOid, InvalidOid, 0 };
if (!OidIsValid(relationId)) if (!OidIsValid(relationId))
{ {
ereport(NOTICE, (errmsg("relation %s does not exist, skipping", ereport(NOTICE, (errmsg("relation %s does not exist, skipping",
@ -75,8 +75,45 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS)
EnsureTableOwner(relationId); EnsureTableOwner(relationId);
List *shardList = LoadShardList(relationId); if (PartitionedTable(relationId))
{
/*
* When "DROP SCHEMA .. CASCADE" happens, we rely on Postgres' drop trigger
* to send the individual DROP TABLE commands for tables.
*
* In case of partitioned tables, we have no control on the order of DROP
* commands that is sent to the extension. We can try to sort while processing
* on the coordinator, but we prefer to handle it in a more flexible manner.
*
* That's why, whenever we see a partitioned table, we drop all the corresponding
* partitions first. Otherwise, WorkerDropDistributedTable() would already drop
* the shell tables of the partitions (e.g., due to performDeletion(..CASCADE),
* and further WorkerDropDistributedTable() on the partitions would become no-op.
*
* If, say one partition has already been dropped earlier, that should also be fine
* because we read the existing partitions.
*/
List *partitionList = PartitionList(relationId);
Oid partitionOid = InvalidOid;
foreach_oid(partitionOid, partitionList)
{
WorkerDropDistributedTable(partitionOid);
}
}
WorkerDropDistributedTable(relationId);
PG_RETURN_VOID();
}
/*
* WorkerDropDistributedTable is a helper function for worker_drop_distributed_table, see
* tha function for the details.
*/
static void
WorkerDropDistributedTable(Oid relationId)
{
/* first check the relation type */ /* first check the relation type */
Relation distributedRelation = relation_open(relationId, AccessShareLock); Relation distributedRelation = relation_open(relationId, AccessShareLock);
@ -86,9 +123,7 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS)
relation_close(distributedRelation, AccessShareLock); relation_close(distributedRelation, AccessShareLock);
/* prepare distributedTableObject for dropping the table */ /* prepare distributedTableObject for dropping the table */
distributedTableObject.classId = RelationRelationId; ObjectAddress distributedTableObject = { RelationRelationId, relationId, 0 };
distributedTableObject.objectId = relationId;
distributedTableObject.objectSubId = 0;
/* Drop dependent sequences from pg_dist_object */ /* Drop dependent sequences from pg_dist_object */
#if PG_VERSION_NUM >= PG_VERSION_13 #if PG_VERSION_NUM >= PG_VERSION_13
@ -121,6 +156,7 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS)
} }
/* iterate over shardList to delete the corresponding rows */ /* iterate over shardList to delete the corresponding rows */
List *shardList = LoadShardList(relationId);
uint64 *shardIdPointer = NULL; uint64 *shardIdPointer = NULL;
foreach_ptr(shardIdPointer, shardList) foreach_ptr(shardIdPointer, shardList)
{ {
@ -140,8 +176,6 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS)
/* delete the row from pg_dist_partition */ /* delete the row from pg_dist_partition */
DeletePartitionRow(relationId); DeletePartitionRow(relationId);
PG_RETURN_VOID();
} }

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 SetBackendDataDistributedCommandOriginator(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);
@ -78,9 +66,9 @@ extern bool MyBackendGotCancelledDueToDeadlock(bool clearState);
extern bool MyBackendIsInDisributedTransaction(void); extern bool MyBackendIsInDisributedTransaction(void);
extern List * ActiveDistributedTransactionNumbers(void); extern List * ActiveDistributedTransactionNumbers(void);
extern LocalTransactionId GetMyProcLocalTransactionId(void); extern LocalTransactionId GetMyProcLocalTransactionId(void);
extern int GetAllActiveClientBackendCount(void); extern int GetExternalClientBackendCount(void);
extern void IncrementClientBackendCounter(void); extern uint32 IncrementExternalClientBackendCounter(void);
extern void DecrementClientBackendCounter(void); extern void DecrementExternalClientBackendCounter(void);
extern bool ExecuteRemoteQueryOrCommand(char *nodeName, uint32 nodePort, extern bool ExecuteRemoteQueryOrCommand(char *nodeName, uint32 nodePort,
char *queryString, StringInfo queryResultString, char *queryString, StringInfo queryResultString,

View File

@ -31,6 +31,10 @@ uint32 ColocationId(int shardCount, int replicationFactor, Oid distributionColum
extern uint32 CreateColocationGroup(int shardCount, int replicationFactor, extern uint32 CreateColocationGroup(int shardCount, int replicationFactor,
Oid distributionColumnType, Oid distributionColumnType,
Oid distributionColumnCollation); Oid distributionColumnCollation);
extern void InsertColocationGroupLocally(uint32 colocationId, int shardCount,
int replicationFactor,
Oid distributionColumnType,
Oid distributionColumnCollation);
extern bool IsColocateWithNone(char *colocateWithTableName); extern bool IsColocateWithNone(char *colocateWithTableName);
extern uint32 GetNextColocationId(void); extern uint32 GetNextColocationId(void);
extern void ErrorIfShardPlacementsNotColocated(Oid leftRelationId, Oid rightRelationId); extern void ErrorIfShardPlacementsNotColocated(Oid leftRelationId, Oid rightRelationId);
@ -43,5 +47,6 @@ extern void UpdateRelationColocationGroup(Oid distributedRelationId, uint32 colo
bool localOnly); bool localOnly);
extern void DeleteColocationGroupIfNoTablesBelong(uint32 colocationId); extern void DeleteColocationGroupIfNoTablesBelong(uint32 colocationId);
extern List * ColocationGroupTableList(uint32 colocationId, uint32 count); extern List * ColocationGroupTableList(uint32 colocationId, uint32 count);
extern void DeleteColocationGroupLocally(uint32 colocationId);
#endif /* COLOCATION_UTILS_H_ */ #endif /* COLOCATION_UTILS_H_ */

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

@ -21,6 +21,8 @@ extern List * GetUniqueDependenciesList(List *objectAddressesList);
extern List * GetDependenciesForObject(const ObjectAddress *target); extern List * GetDependenciesForObject(const ObjectAddress *target);
extern List * GetAllSupportedDependenciesForObject(const ObjectAddress *target); extern List * GetAllSupportedDependenciesForObject(const ObjectAddress *target);
extern List * GetAllDependenciesForObject(const ObjectAddress *target); extern List * GetAllDependenciesForObject(const ObjectAddress *target);
extern void EnsureRelationDependenciesCanBeDistributed(ObjectAddress *relationAddress);
extern ObjectAddress * GetUndistributableDependency(ObjectAddress *target);
extern List * OrderObjectAddressListInDependencyOrder(List *objectAddressList); extern List * OrderObjectAddressListInDependencyOrder(List *objectAddressList);
extern bool SupportedDependencyByCitus(const ObjectAddress *address); extern bool SupportedDependencyByCitus(const ObjectAddress *address);
extern List * GetPgDependTuplesForDependingObjects(Oid targetObjectClassId, extern List * GetPgDependTuplesForDependingObjects(Oid targetObjectClassId,

View File

@ -38,6 +38,7 @@ extern bool ShouldSyncTableMetadata(Oid relationId);
extern bool ShouldSyncTableMetadataViaCatalog(Oid relationId); extern bool ShouldSyncTableMetadataViaCatalog(Oid relationId);
extern List * NodeMetadataCreateCommands(void); extern List * NodeMetadataCreateCommands(void);
extern List * DistributedObjectMetadataSyncCommandList(void); extern List * DistributedObjectMetadataSyncCommandList(void);
extern List * ColocationGroupCreateCommandList(void);
extern List * CitusTableMetadataCreateCommandList(Oid relationId); extern List * CitusTableMetadataCreateCommandList(Oid relationId);
extern List * NodeMetadataDropCommands(void); extern List * NodeMetadataDropCommands(void);
extern char * MarkObjectsDistributedCreateCommand(List *addresses, extern char * MarkObjectsDistributedCreateCommand(List *addresses,
@ -73,13 +74,21 @@ 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);
extern void SyncNewColocationGroupToNodes(uint32 colocationId, int shardCount,
int replicationFactor,
Oid distributionColumType,
Oid distributionColumnCollation);
extern void SyncDeleteColocationGroupToNodes(uint32 colocationId);
#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"
#define DELETE_ALL_SHARDS "DELETE FROM pg_dist_shard" #define DELETE_ALL_SHARDS "DELETE FROM pg_dist_shard"
#define DELETE_ALL_DISTRIBUTED_OBJECTS "DELETE FROM citus.pg_dist_object" #define DELETE_ALL_DISTRIBUTED_OBJECTS "DELETE FROM pg_catalog.pg_dist_object"
#define DELETE_ALL_PARTITIONS "DELETE FROM pg_dist_partition" #define DELETE_ALL_PARTITIONS "DELETE FROM pg_dist_partition"
#define DELETE_ALL_COLOCATION "DELETE FROM pg_catalog.pg_dist_colocation"
#define REMOVE_ALL_SHELL_TABLES_COMMAND \ #define REMOVE_ALL_SHELL_TABLES_COMMAND \
"SELECT worker_drop_shell_table(logicalrelid::regclass::text) FROM pg_dist_partition" "SELECT worker_drop_shell_table(logicalrelid::regclass::text) FROM pg_dist_partition"
#define REMOVE_ALL_CITUS_TABLES_COMMAND \ #define REMOVE_ALL_CITUS_TABLES_COMMAND \

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

@ -14,15 +14,18 @@
#define ADJUST_POOLSIZE_AUTOMATICALLY 0 #define ADJUST_POOLSIZE_AUTOMATICALLY 0
#define DISABLE_CONNECTION_THROTTLING -1 #define DISABLE_CONNECTION_THROTTLING -1
#define DISABLE_REMOTE_CONNECTIONS_FOR_LOCAL_QUERIES -1 #define DISABLE_REMOTE_CONNECTIONS_FOR_LOCAL_QUERIES -1
#define ALLOW_ALL_EXTERNAL_CONNECTIONS -1
extern int MaxSharedPoolSize; extern int MaxSharedPoolSize;
extern int LocalSharedPoolSize; extern int LocalSharedPoolSize;
extern int MaxClientConnections;
extern void InitializeSharedConnectionStats(void); extern void InitializeSharedConnectionStats(void);
extern void WaitForSharedConnection(void); extern void WaitForSharedConnection(void);
extern void WakeupWaiterBackendsForSharedConnection(void); extern void WakeupWaiterBackendsForSharedConnection(void);
extern int GetMaxClientConnections(void);
extern int GetMaxSharedPoolSize(void); extern int GetMaxSharedPoolSize(void);
extern int GetLocalSharedPoolSize(void); extern int GetLocalSharedPoolSize(void);
extern bool TryToIncrementSharedConnectionCounter(const char *hostname, int port); extern bool TryToIncrementSharedConnectionCounter(const char *hostname, int port);

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);

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