mirror of https://github.com/citusdata/citus.git
Merge branch 'master' into fix/subscript-jsonb
commit
6fa965cd17
|
@ -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);
|
||||
}
|
|
@ -31,6 +31,7 @@
|
|||
#include "distributed/commands/sequence.h"
|
||||
#include "distributed/commands/utility_hook.h"
|
||||
#include "distributed/metadata/distobject.h"
|
||||
#include "distributed/metadata/dependency.h"
|
||||
#include "distributed/foreign_key_relationship.h"
|
||||
#include "distributed/listutils.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
|
||||
* in the MX case.
|
||||
*/
|
||||
EnsureRelationDependenciesCanBeDistributed(&tableAddress);
|
||||
EnsureDependenciesExistOnAllNodes(&tableAddress);
|
||||
|
||||
/*
|
||||
|
|
|
@ -87,10 +87,11 @@ CitusSignalBackend(uint64 globalPID, uint64 timeout, int sig)
|
|||
}
|
||||
#endif
|
||||
|
||||
int nodeId = ExtractNodeIdFromGlobalPID(globalPID);
|
||||
bool missingOk = false;
|
||||
int nodeId = ExtractNodeIdFromGlobalPID(globalPID, missingOk);
|
||||
int processId = ExtractProcessIdFromGlobalPID(globalPID);
|
||||
|
||||
WorkerNode *workerNode = FindNodeWithNodeId(nodeId);
|
||||
WorkerNode *workerNode = FindNodeWithNodeId(nodeId, missingOk);
|
||||
|
||||
StringInfo cancelQuery = makeStringInfo();
|
||||
|
||||
|
|
|
@ -530,11 +530,14 @@ PreprocessDefineCollationStmt(Node *node, const char *queryString,
|
|||
{
|
||||
Assert(castNode(DefineStmt, node)->kind == OBJECT_COLLATION);
|
||||
|
||||
if (ShouldPropagateDefineCollationStmt())
|
||||
if (!ShouldPropagateDefineCollationStmt())
|
||||
{
|
||||
EnsureCoordinator();
|
||||
return NIL;
|
||||
}
|
||||
|
||||
EnsureCoordinator();
|
||||
EnsureSequentialMode(OBJECT_COLLATION);
|
||||
|
||||
return NIL;
|
||||
}
|
||||
|
||||
|
@ -575,8 +578,7 @@ PostprocessDefineCollationStmt(Node *node, const char *queryString)
|
|||
* ShouldPropagateDefineCollationStmt checks if collation define
|
||||
* statement should be propagated. Don't propagate if:
|
||||
* - metadata syncing if off
|
||||
* - statement is part of a multi stmt transaction and the multi shard connection
|
||||
* type is not sequential
|
||||
* - create statement should be propagated according the the ddl propagation policy
|
||||
*/
|
||||
static bool
|
||||
ShouldPropagateDefineCollationStmt()
|
||||
|
@ -586,8 +588,7 @@ ShouldPropagateDefineCollationStmt()
|
|||
return false;
|
||||
}
|
||||
|
||||
if (IsMultiStatementTransaction() &&
|
||||
MultiShardConnectionType != SEQUENTIAL_CONNECTION)
|
||||
if (!ShouldPropagateCreateInCoordinatedTransction())
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
|
|
@ -443,6 +443,8 @@ CreateDistributedTable(Oid relationId, char *distributionColumnName,
|
|||
*/
|
||||
ObjectAddress tableAddress = { 0 };
|
||||
ObjectAddressSet(tableAddress, RelationRelationId, relationId);
|
||||
|
||||
EnsureRelationDependenciesCanBeDistributed(&tableAddress);
|
||||
EnsureDependenciesExistOnAllNodes(&tableAddress);
|
||||
|
||||
char replicationModel = DecideReplicationModel(distributionMethod,
|
||||
|
|
|
@ -20,6 +20,8 @@
|
|||
#include "distributed/metadata/dependency.h"
|
||||
#include "distributed/metadata/distobject.h"
|
||||
#include "distributed/metadata_sync.h"
|
||||
#include "distributed/multi_executor.h"
|
||||
#include "distributed/relation_access_tracking.h"
|
||||
#include "distributed/remote_commands.h"
|
||||
#include "distributed/worker_manager.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
|
||||
* creation commands.
|
||||
*/
|
||||
if (relKind == RELKIND_INDEX)
|
||||
if (relKind == RELKIND_INDEX ||
|
||||
relKind == RELKIND_PARTITIONED_INDEX)
|
||||
{
|
||||
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
|
||||
* on their object address.
|
||||
|
|
|
@ -43,11 +43,11 @@ static DistributeObjectOps Aggregate_AlterOwner = {
|
|||
};
|
||||
static DistributeObjectOps Aggregate_Define = {
|
||||
.deparse = NULL,
|
||||
.qualify = NULL,
|
||||
.qualify = QualifyDefineAggregateStmt,
|
||||
.preprocess = NULL,
|
||||
.postprocess = NULL,
|
||||
.postprocess = PostprocessDefineAggregateStmt,
|
||||
.address = DefineAggregateStmtObjectAddress,
|
||||
.markDistributed = false,
|
||||
.markDistributed = true,
|
||||
};
|
||||
static DistributeObjectOps Aggregate_Drop = {
|
||||
.deparse = DeparseDropFunctionStmt,
|
||||
|
|
|
@ -11,11 +11,13 @@
|
|||
#include "postgres.h"
|
||||
#include "miscadmin.h"
|
||||
|
||||
|
||||
#include "distributed/commands/utility_hook.h"
|
||||
#include "distributed/commands.h"
|
||||
#include "distributed/metadata_utility.h"
|
||||
#include "distributed/coordinator_protocol.h"
|
||||
#include "distributed/metadata_sync.h"
|
||||
#include "distributed/multi_partitioning_utils.h"
|
||||
#include "distributed/worker_transaction.h"
|
||||
#include "utils/builtins.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
|
||||
* have any workers with metadata. Also, the function errors out if called
|
||||
* 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
|
||||
MasterRemoveDistributedTableMetadataFromWorkers(Oid relationId, char *schemaName,
|
||||
|
@ -146,6 +152,16 @@ MasterRemoveDistributedTableMetadataFromWorkers(Oid relationId, char *schemaName
|
|||
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 */
|
||||
char *deleteDistributionCommand = DistributionDeleteCommand(schemaName, tableName);
|
||||
SendCommandToWorkersWithMetadata(deleteDistributionCommand);
|
||||
|
|
|
@ -136,11 +136,8 @@ PostprocessCreateExtensionStmt(Node *node, const char *queryString)
|
|||
return NIL;
|
||||
}
|
||||
|
||||
/*
|
||||
* If the extension command is a part of a multi-statement transaction,
|
||||
* do not propagate it
|
||||
*/
|
||||
if (IsMultiStatementTransaction())
|
||||
/* check creation against multi-statement transaction policy */
|
||||
if (!ShouldPropagateCreateInCoordinatedTransction())
|
||||
{
|
||||
return NIL;
|
||||
}
|
||||
|
@ -148,16 +145,6 @@ PostprocessCreateExtensionStmt(Node *node, const char *queryString)
|
|||
/* extension management can only be done via coordinator node */
|
||||
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,
|
||||
* 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 */
|
||||
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,
|
||||
* 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 */
|
||||
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,
|
||||
* 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 */
|
||||
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,
|
||||
* or can still safely be put in sequential mode
|
||||
|
@ -555,6 +513,16 @@ MarkExistingObjectDependenciesDistributedIfSupported()
|
|||
ObjectAddress tableAddress = { 0 };
|
||||
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 =
|
||||
GetDistributableDependenciesForObject(&tableAddress);
|
||||
|
||||
|
@ -578,11 +546,22 @@ MarkExistingObjectDependenciesDistributedIfSupported()
|
|||
/* remove duplicates from object addresses list for efficiency */
|
||||
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;
|
||||
foreach_ptr(objectAddress, uniqueObjectAddresses)
|
||||
{
|
||||
MarkObjectDistributed(objectAddress);
|
||||
}
|
||||
|
||||
SetLocalEnableMetadataSync(prevMetadataSyncValue);
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
#include "distributed/listutils.h"
|
||||
#include "distributed/metadata/distobject.h"
|
||||
#include "distributed/metadata_sync.h"
|
||||
#include "distributed/multi_executor.h"
|
||||
#include "distributed/worker_transaction.h"
|
||||
#include "foreign/foreign.h"
|
||||
#include "nodes/makefuncs.h"
|
||||
|
@ -41,7 +42,14 @@ PreprocessCreateForeignServerStmt(Node *node, const char *queryString,
|
|||
return NIL;
|
||||
}
|
||||
|
||||
/* check creation against multi-statement transaction policy */
|
||||
if (!ShouldPropagateCreateInCoordinatedTransction())
|
||||
{
|
||||
return NIL;
|
||||
}
|
||||
|
||||
EnsureCoordinator();
|
||||
EnsureSequentialMode(OBJECT_FOREIGN_SERVER);
|
||||
|
||||
char *sql = DeparseTreeNode(node);
|
||||
|
||||
|
@ -209,7 +217,18 @@ PreprocessDropForeignServerStmt(Node *node, const char *queryString,
|
|||
List *
|
||||
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);
|
||||
EnsureDependenciesExistOnAllNodes(&address);
|
||||
|
||||
|
@ -224,8 +243,14 @@ PostprocessCreateForeignServerStmt(Node *node, const char *queryString)
|
|||
List *
|
||||
PostprocessAlterForeignServerOwnerStmt(Node *node, const char *queryString)
|
||||
{
|
||||
bool missingOk = false;
|
||||
const bool missingOk = false;
|
||||
ObjectAddress address = GetObjectAddressFromParseTree(node, missingOk);
|
||||
|
||||
if (!ShouldPropagateObject(&address))
|
||||
{
|
||||
return NIL;
|
||||
}
|
||||
|
||||
EnsureDependenciesExistOnAllNodes(&address);
|
||||
|
||||
return NIL;
|
||||
|
|
|
@ -69,6 +69,10 @@
|
|||
(strncmp(arg, prefix, strlen(prefix)) == 0)
|
||||
|
||||
/* forward declaration for helper functions*/
|
||||
static bool RecreateSameNonColocatedFunction(ObjectAddress functionAddress,
|
||||
char *distributionArgumentName,
|
||||
bool colocateWithTableNameDefault,
|
||||
bool *forceDelegationAddress);
|
||||
static void ErrorIfAnyNodeDoesNotHaveMetadata(void);
|
||||
static char * GetAggregateDDLCommand(const RegProcedure funcOid, bool useCreateOrReplace);
|
||||
static char * GetFunctionAlterOwnerCommand(const RegProcedure funcOid);
|
||||
|
@ -82,7 +86,6 @@ static void EnsureFunctionCanBeColocatedWithTable(Oid functionOid, Oid
|
|||
static bool ShouldPropagateCreateFunction(CreateFunctionStmt *stmt);
|
||||
static bool ShouldPropagateAlterFunction(const ObjectAddress *address);
|
||||
static bool ShouldAddFunctionSignature(FunctionParameterMode mode);
|
||||
static ObjectAddress * GetUndistributableDependency(ObjectAddress *functionAddress);
|
||||
static ObjectAddress FunctionToObjectAddress(ObjectType objectType,
|
||||
ObjectWithArgs *objectWithArgs,
|
||||
bool missing_ok);
|
||||
|
@ -129,6 +132,7 @@ create_distributed_function(PG_FUNCTION_ARGS)
|
|||
|
||||
char *distributionArgumentName = NULL;
|
||||
char *colocateWithTableName = NULL;
|
||||
bool colocateWithTableNameDefault = false;
|
||||
bool *forceDelegationAddress = NULL;
|
||||
bool forceDelegation = false;
|
||||
ObjectAddress extensionAddress = { 0 };
|
||||
|
@ -168,8 +172,13 @@ create_distributed_function(PG_FUNCTION_ARGS)
|
|||
colocateWithText = PG_GETARG_TEXT_P(2);
|
||||
colocateWithTableName = text_to_cstring(colocateWithText);
|
||||
|
||||
if (pg_strncasecmp(colocateWithTableName, "default", NAMEDATALEN) == 0)
|
||||
{
|
||||
colocateWithTableNameDefault = true;
|
||||
}
|
||||
|
||||
/* check if the colocation belongs to a reference table */
|
||||
if (pg_strncasecmp(colocateWithTableName, "default", NAMEDATALEN) != 0)
|
||||
if (!colocateWithTableNameDefault)
|
||||
{
|
||||
Oid colocationRelationId = ResolveRelationId(colocateWithText, false);
|
||||
colocatedWithReferenceTable = IsCitusTableType(colocationRelationId,
|
||||
|
@ -193,6 +202,20 @@ create_distributed_function(PG_FUNCTION_ARGS)
|
|||
|
||||
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
|
||||
* 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
|
||||
* 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
|
||||
* 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->objname = stmt->defnames;
|
||||
|
||||
FunctionParameter *funcParam = NULL;
|
||||
foreach_ptr(funcParam, linitial(stmt->args))
|
||||
if (stmt->args != NIL)
|
||||
{
|
||||
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);
|
||||
|
@ -2019,10 +2054,10 @@ ShouldAddFunctionSignature(FunctionParameterMode mode)
|
|||
|
||||
|
||||
/*
|
||||
* FunctionToObjectAddress returns the ObjectAddress of a Function or Procedure based on
|
||||
* its type and ObjectWithArgs describing the Function/Procedure. If missing_ok is set to
|
||||
* false an error will be raised by postgres explaining the Function/Procedure could not
|
||||
* be found.
|
||||
* FunctionToObjectAddress returns the ObjectAddress of a Function, Procedure or
|
||||
* Aggregate based on its type and ObjectWithArgs describing the
|
||||
* Function/Procedure/Aggregate. If missing_ok is set to false an error will be
|
||||
* raised by postgres explaining the Function/Procedure could not be found.
|
||||
*/
|
||||
static ObjectAddress
|
||||
FunctionToObjectAddress(ObjectType objectType, ObjectWithArgs *objectWithArgs,
|
||||
|
|
|
@ -140,13 +140,6 @@ PostprocessAlterRoleStmt(Node *node, const char *queryString)
|
|||
|
||||
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;
|
||||
foreach_ptr(option, stmt->options)
|
||||
{
|
||||
|
|
|
@ -367,8 +367,8 @@ ShouldPropagateCreateSchemaStmt()
|
|||
return false;
|
||||
}
|
||||
|
||||
if (IsMultiStatementTransaction() &&
|
||||
MultiShardConnectionType != SEQUENTIAL_CONNECTION)
|
||||
/* check creation against multi-statement transaction policy */
|
||||
if (!ShouldPropagateCreateInCoordinatedTransction())
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
|
|
@ -1955,6 +1955,7 @@ PostprocessAlterTableStmt(AlterTableStmt *alterTableStatement)
|
|||
/* changing a relation could introduce new dependencies */
|
||||
ObjectAddress tableAddress = { 0 };
|
||||
ObjectAddressSet(tableAddress, RelationRelationId, relationId);
|
||||
EnsureRelationDependenciesCanBeDistributed(&tableAddress);
|
||||
EnsureDependenciesExistOnAllNodes(&tableAddress);
|
||||
}
|
||||
|
||||
|
|
|
@ -73,16 +73,10 @@ PostprocessCreateTextSearchConfigurationStmt(Node *node, const char *queryString
|
|||
return NIL;
|
||||
}
|
||||
|
||||
/*
|
||||
* If the create command is a part of a multi-statement transaction that is not in
|
||||
* sequential mode, don't propagate. Instead we will rely on back filling.
|
||||
*/
|
||||
if (IsMultiStatementTransaction())
|
||||
/* check creation against multi-statement transaction policy */
|
||||
if (!ShouldPropagateCreateInCoordinatedTransction())
|
||||
{
|
||||
if (MultiShardConnectionType != SEQUENTIAL_CONNECTION)
|
||||
{
|
||||
return NIL;
|
||||
}
|
||||
return NIL;
|
||||
}
|
||||
|
||||
EnsureCoordinator();
|
||||
|
|
|
@ -129,16 +129,6 @@ PreprocessCompositeTypeStmt(Node *node, const char *queryString,
|
|||
*/
|
||||
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 */
|
||||
QualifyTreeNode(node);
|
||||
|
||||
|
@ -1161,7 +1151,7 @@ ShouldPropagateTypeCreate()
|
|||
* this type will be used as a column in a table that will be created and distributed
|
||||
* in this same transaction.
|
||||
*/
|
||||
if (IsMultiStatementTransaction())
|
||||
if (!ShouldPropagateCreateInCoordinatedTransction())
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
|
|
@ -78,6 +78,7 @@
|
|||
#include "utils/syscache.h"
|
||||
|
||||
bool EnableDDLPropagation = true; /* ddl propagation is enabled */
|
||||
int CreateObjectPropagationMode = CREATE_OBJECT_PROPAGATION_DEFERRED;
|
||||
PropSetCmdBehavior PropagateSetCommands = PROPSETCMD_NONE; /* SET prop off */
|
||||
static bool shouldInvalidateForeignKeyGraph = false;
|
||||
static int activeAlterTables = 0;
|
||||
|
@ -258,7 +259,7 @@ multi_ProcessUtility(PlannedStmt *pstmt,
|
|||
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.
|
||||
*/
|
||||
DoBlockLevel += 1;
|
||||
|
@ -1563,7 +1564,8 @@ DDLTaskList(Oid relationId, const char *commandString)
|
|||
List *
|
||||
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)
|
||||
{
|
||||
|
|
|
@ -106,6 +106,9 @@ int MaxSharedPoolSize = 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 */
|
||||
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
|
||||
* via a GUC.
|
||||
|
@ -204,7 +226,7 @@ GetMaxSharedPoolSize(void)
|
|||
{
|
||||
if (MaxSharedPoolSize == ADJUST_POOLSIZE_AUTOMATICALLY)
|
||||
{
|
||||
return MaxConnections;
|
||||
return GetMaxClientConnections();
|
||||
}
|
||||
|
||||
return MaxSharedPoolSize;
|
||||
|
@ -223,7 +245,7 @@ GetLocalSharedPoolSize(void)
|
|||
{
|
||||
if (LocalSharedPoolSize == ADJUST_POOLSIZE_AUTOMATICALLY)
|
||||
{
|
||||
return MaxConnections * 0.5;
|
||||
return GetMaxClientConnections() * 0.5;
|
||||
}
|
||||
|
||||
return LocalSharedPoolSize;
|
||||
|
@ -318,7 +340,7 @@ TryToIncrementSharedConnectionCounter(const char *hostname, int port)
|
|||
return false;
|
||||
}
|
||||
|
||||
activeBackendCount = GetAllActiveClientBackendCount();
|
||||
activeBackendCount = GetExternalClientBackendCount();
|
||||
}
|
||||
|
||||
LockConnectionSharedMemory(LW_EXCLUSIVE);
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
}
|
|
@ -819,6 +819,9 @@ AdaptiveExecutor(CitusScanState *scanState)
|
|||
bool hasDependentJobs = HasDependentJobs(job);
|
||||
if (hasDependentJobs)
|
||||
{
|
||||
/* jobs use intermediate results, which require a distributed transaction */
|
||||
UseCoordinatedTransaction();
|
||||
|
||||
jobIdList = ExecuteDependentTasks(taskList, job);
|
||||
}
|
||||
|
||||
|
@ -828,9 +831,10 @@ AdaptiveExecutor(CitusScanState *scanState)
|
|||
targetPoolSize = 1;
|
||||
}
|
||||
|
||||
bool excludeFromXact = false;
|
||||
|
||||
TransactionProperties xactProperties = DecideTransactionPropertiesForTaskList(
|
||||
distributedPlan->modLevel, taskList,
|
||||
hasDependentJobs);
|
||||
distributedPlan->modLevel, taskList, excludeFromXact);
|
||||
|
||||
bool localExecutionSupported = true;
|
||||
DistributedExecution *execution = CreateDistributedExecution(
|
||||
|
@ -873,11 +877,6 @@ AdaptiveExecutor(CitusScanState *scanState)
|
|||
|
||||
FinishDistributedExecution(execution);
|
||||
|
||||
if (hasDependentJobs)
|
||||
{
|
||||
DoRepartitionCleanup(jobIdList);
|
||||
}
|
||||
|
||||
if (SortReturning && distributedPlan->expectResults && commandType != CMD_SELECT)
|
||||
{
|
||||
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
|
||||
* with defaults for some of the arguments.
|
||||
|
@ -2639,12 +2658,6 @@ RunDistributedExecution(DistributedExecution *execution)
|
|||
*/
|
||||
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)
|
||||
{
|
||||
FreeWaitEventSet(execution->waitEventSet);
|
||||
|
|
|
@ -150,8 +150,6 @@ RegisterCitusCustomScanMethods(void)
|
|||
static void
|
||||
CitusBeginScan(CustomScanState *node, EState *estate, int eflags)
|
||||
{
|
||||
MarkCitusInitiatedCoordinatorBackend();
|
||||
|
||||
CitusScanState *scanState = (CitusScanState *) node;
|
||||
|
||||
/*
|
||||
|
|
|
@ -39,6 +39,7 @@ static HTAB * CreateTaskHashTable(void);
|
|||
static bool IsAllDependencyCompleted(Task *task, HTAB *completedTasks);
|
||||
static void AddCompletedTasks(List *curCompletedTasks, 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 uint32 TaskHash(const void *key, Size keysize);
|
||||
static bool IsTaskAlreadyCompleted(Task *task, HTAB *completedTasks);
|
||||
|
@ -64,8 +65,13 @@ ExecuteTasksInDependencyOrder(List *allTasks, List *excludedTasks, List *jobIds)
|
|||
{
|
||||
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);
|
||||
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.
|
||||
*/
|
||||
|
|
|
@ -59,28 +59,6 @@ typedef struct 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 */
|
||||
static List * WrapTasksForPartitioning(const char *resultIdPrefix,
|
||||
List *selectTaskList,
|
||||
|
@ -98,9 +76,6 @@ static TupleDesc PartitioningTupleDestTupleDescForQuery(TupleDestination *self,
|
|||
queryNumber);
|
||||
static ArrayType * CreateArrayFromDatums(Datum *datumArray, bool *nullsArray, int
|
||||
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 DistributedResultFragment * TupleToDistributedResultFragment(HeapTuple heapTuple,
|
||||
TupleDesc tupleDesc,
|
||||
|
@ -115,8 +90,6 @@ static List ** ColocateFragmentsWithRelation(List *fragmentList,
|
|||
static List * ColocationTransfers(List *fragmentList,
|
||||
CitusTableCacheEntry *targetRelation);
|
||||
static List * FragmentTransferTaskList(List *fragmentListTransfers);
|
||||
static char * QueryStringForFragmentsTransfer(
|
||||
NodeToNodeFragmentsTransfer *fragmentsTransfer);
|
||||
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
|
||||
* intervals. Returned arrays are text arrays.
|
||||
*/
|
||||
static void
|
||||
void
|
||||
ShardMinMaxValueArrays(ShardInterval **shardIntervalArray, int shardCount,
|
||||
Oid intervalTypeOutFunc, ArrayType **minValueArray,
|
||||
ArrayType **maxValueArray)
|
||||
|
@ -632,7 +605,7 @@ FragmentTransferTaskList(List *fragmentListTransfers)
|
|||
* result fragments from source node to target node. See the structure of
|
||||
* NodeToNodeFragmentsTransfer for details of how these are decided.
|
||||
*/
|
||||
static char *
|
||||
char *
|
||||
QueryStringForFragmentsTransfer(NodeToNodeFragmentsTransfer *fragmentsTransfer)
|
||||
{
|
||||
StringInfo queryString = makeStringInfo();
|
||||
|
@ -667,7 +640,7 @@ QueryStringForFragmentsTransfer(NodeToNodeFragmentsTransfer *fragmentsTransfer)
|
|||
quote_literal_cstr(sourceNode->workerName),
|
||||
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)));
|
||||
|
||||
return queryString->data;
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
#include "pgstat.h"
|
||||
|
||||
#include "catalog/pg_enum.h"
|
||||
#include "catalog/pg_type.h"
|
||||
#include "commands/copy.h"
|
||||
#include "distributed/commands/multi_copy.h"
|
||||
#include "distributed/connection_management.h"
|
||||
|
@ -916,6 +917,8 @@ fetch_intermediate_results(PG_FUNCTION_ARGS)
|
|||
StringInfo beginAndSetXactId = BeginAndSetDistributedTransactionIdCommand();
|
||||
ExecuteCriticalRemoteCommand(connection, beginAndSetXactId->data);
|
||||
|
||||
CreateIntermediateResultsDirectory();
|
||||
|
||||
for (resultIndex = 0; resultIndex < resultCount; resultIndex++)
|
||||
{
|
||||
char *resultId = TextDatumGetCString(resultIdArray[resultIndex]);
|
||||
|
@ -938,6 +941,19 @@ fetch_intermediate_results(PG_FUNCTION_ARGS)
|
|||
static uint64
|
||||
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;
|
||||
|
||||
StringInfo copyCommand = makeStringInfo();
|
||||
|
@ -948,8 +964,6 @@ FetchRemoteIntermediateResult(MultiConnection *connection, char *resultId)
|
|||
int socket = PQsocket(pgConn);
|
||||
bool raiseErrors = true;
|
||||
|
||||
CreateIntermediateResultsDirectory();
|
||||
|
||||
appendStringInfo(copyCommand, "COPY \"%s\" TO STDOUT WITH (format result)",
|
||||
resultId);
|
||||
|
||||
|
@ -966,7 +980,6 @@ FetchRemoteIntermediateResult(MultiConnection *connection, char *resultId)
|
|||
|
||||
PQclear(result);
|
||||
|
||||
char *localPath = QueryResultFileName(resultId);
|
||||
File fileDesc = FileOpenForTransmit(localPath, fileFlags, fileMode);
|
||||
FileCompat fileCompat = FileCompatFromFileStart(fileDesc);
|
||||
|
||||
|
|
|
@ -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
|
||||
* on any shard has already been executed in the same transaction.
|
||||
|
|
|
@ -748,6 +748,11 @@ GetObjectTypeString(ObjectType objType)
|
|||
{
|
||||
switch (objType)
|
||||
{
|
||||
case OBJECT_AGGREGATE:
|
||||
{
|
||||
return "aggregate";
|
||||
}
|
||||
|
||||
case OBJECT_COLLATION:
|
||||
{
|
||||
return "collation";
|
||||
|
@ -763,6 +768,11 @@ GetObjectTypeString(ObjectType objType)
|
|||
return "extension";
|
||||
}
|
||||
|
||||
case OBJECT_FOREIGN_SERVER:
|
||||
{
|
||||
return "foreign server";
|
||||
}
|
||||
|
||||
case OBJECT_FUNCTION:
|
||||
{
|
||||
return "function";
|
||||
|
|
|
@ -86,6 +86,9 @@ typedef struct PartitionedResultDestReceiver
|
|||
|
||||
/* keeping track of which partitionDestReceivers have been started */
|
||||
Bitmapset *startedDestReceivers;
|
||||
|
||||
/* whether NULL partition column values are allowed */
|
||||
bool allowNullPartitionColumnValues;
|
||||
} PartitionedResultDestReceiver;
|
||||
|
||||
static Portal StartPortalForQueryExecution(const char *queryString);
|
||||
|
@ -99,7 +102,8 @@ static DestReceiver * CreatePartitionedResultDestReceiver(int partitionColumnInd
|
|||
shardSearchInfo,
|
||||
DestReceiver **
|
||||
partitionedDestReceivers,
|
||||
bool lazyStartup);
|
||||
bool lazyStartup,
|
||||
bool allowNullPartitionValues);
|
||||
static void PartitionedResultDestReceiverStartup(DestReceiver *dest, int operation,
|
||||
TupleDesc inputTupleDescriptor);
|
||||
static bool PartitionedResultDestReceiverReceive(TupleTableSlot *slot,
|
||||
|
@ -148,6 +152,8 @@ worker_partition_query_result(PG_FUNCTION_ARGS)
|
|||
int32 maxValuesCount = ArrayObjectCount(maxValuesArray);
|
||||
|
||||
bool binaryCopy = PG_GETARG_BOOL(6);
|
||||
bool allowNullPartitionColumnValues = PG_GETARG_BOOL(7);
|
||||
bool generateEmptyResults = PG_GETARG_BOOL(8);
|
||||
|
||||
if (!IsMultiStatementTransaction())
|
||||
{
|
||||
|
@ -226,13 +232,21 @@ worker_partition_query_result(PG_FUNCTION_ARGS)
|
|||
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(
|
||||
partitionColumnIndex,
|
||||
partitionCount,
|
||||
shardSearchInfo,
|
||||
dests,
|
||||
lazyStartup);
|
||||
lazyStartup,
|
||||
allowNullPartitionColumnValues);
|
||||
|
||||
/* execute the query */
|
||||
PortalRun(portal, FETCH_ALL, false, true, dest, dest, NULL);
|
||||
|
@ -390,7 +404,8 @@ CreatePartitionedResultDestReceiver(int partitionColumnIndex,
|
|||
int partitionCount,
|
||||
CitusTableCacheEntry *shardSearchInfo,
|
||||
DestReceiver **partitionedDestReceivers,
|
||||
bool lazyStartup)
|
||||
bool lazyStartup,
|
||||
bool allowNullPartitionColumnValues)
|
||||
{
|
||||
PartitionedResultDestReceiver *resultDest =
|
||||
palloc0(sizeof(PartitionedResultDestReceiver));
|
||||
|
@ -409,6 +424,7 @@ CreatePartitionedResultDestReceiver(int partitionColumnIndex,
|
|||
resultDest->partitionDestReceivers = partitionedDestReceivers;
|
||||
resultDest->startedDestReceivers = NULL;
|
||||
resultDest->lazyStartup = lazyStartup;
|
||||
resultDest->allowNullPartitionColumnValues = allowNullPartitionColumnValues;
|
||||
|
||||
return (DestReceiver *) resultDest;
|
||||
}
|
||||
|
@ -458,23 +474,40 @@ PartitionedResultDestReceiverReceive(TupleTableSlot *slot, DestReceiver *dest)
|
|||
Datum *columnValues = slot->tts_values;
|
||||
bool *columnNulls = slot->tts_isnull;
|
||||
|
||||
int partitionIndex;
|
||||
|
||||
if (columnNulls[self->partitionColumnIndex])
|
||||
{
|
||||
ereport(ERROR, (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
|
||||
errmsg("the partition column value cannot be NULL")));
|
||||
if (self->allowNullPartitionColumnValues)
|
||||
{
|
||||
/*
|
||||
* NULL values go into the first partition for both hash- and range-
|
||||
* partitioning, since that is the only way to guarantee that there is
|
||||
* always a partition for NULL and that it is always the same partition.
|
||||
*/
|
||||
partitionIndex = 0;
|
||||
}
|
||||
else
|
||||
{
|
||||
ereport(ERROR, (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
|
||||
errmsg("the partition column value cannot be NULL")));
|
||||
}
|
||||
}
|
||||
|
||||
Datum partitionColumnValue = columnValues[self->partitionColumnIndex];
|
||||
ShardInterval *shardInterval = FindShardInterval(partitionColumnValue,
|
||||
self->shardSearchInfo);
|
||||
if (shardInterval == NULL)
|
||||
else
|
||||
{
|
||||
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||
errmsg("could not find shard for partition column "
|
||||
"value")));
|
||||
Datum partitionColumnValue = columnValues[self->partitionColumnIndex];
|
||||
ShardInterval *shardInterval = FindShardInterval(partitionColumnValue,
|
||||
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];
|
||||
|
||||
/* check if this partitionDestReceiver has been started before, start if not */
|
||||
|
|
|
@ -44,12 +44,8 @@
|
|||
#include "distributed/worker_transaction.h"
|
||||
|
||||
|
||||
static List * CreateTemporarySchemasForMergeTasks(Job *topLevelJob);
|
||||
static List * ExtractJobsInJobTree(Job *job);
|
||||
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 *
|
||||
ExecuteDependentTasks(List *topLevelTasks, Job *topLevelJob)
|
||||
{
|
||||
EnsureNoModificationsHaveBeenDone();
|
||||
|
||||
List *allTasks = CreateTaskListForJobTree(topLevelTasks);
|
||||
|
||||
EnsureCompatibleLocalExecutionState(allTasks);
|
||||
|
||||
List *jobIds = CreateTemporarySchemasForMergeTasks(topLevelJob);
|
||||
List *jobIds = ExtractJobsInJobTree(topLevelJob);
|
||||
|
||||
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
|
||||
* where the given job is root.
|
||||
|
@ -139,67 +96,3 @@ TraverseJobTree(Job *curJob, List **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);
|
||||
}
|
||||
|
|
|
@ -122,6 +122,7 @@ typedef struct ViewDependencyNode
|
|||
|
||||
|
||||
static List * GetRelationSequenceDependencyList(Oid relationId);
|
||||
static List * GetRelationFunctionDependencyList(Oid relationId);
|
||||
static List * GetRelationTriggerFunctionDependencyList(Oid relationId);
|
||||
static List * GetRelationStatsSchemaDependencyList(Oid relationId);
|
||||
static List * GetRelationIndicesDependencyList(Oid relationId);
|
||||
|
@ -722,7 +723,8 @@ SupportedDependencyByCitus(const ObjectAddress *address)
|
|||
relKind == RELKIND_PARTITIONED_TABLE ||
|
||||
relKind == RELKIND_FOREIGN_TABLE ||
|
||||
relKind == RELKIND_SEQUENCE ||
|
||||
relKind == RELKIND_INDEX)
|
||||
relKind == RELKIND_INDEX ||
|
||||
relKind == RELKIND_PARTITIONED_INDEX)
|
||||
{
|
||||
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
|
||||
* owned by an extension.
|
||||
|
@ -1090,9 +1228,15 @@ ExpandCitusSupportedTypes(ObjectAddressCollector *collector, ObjectAddress targe
|
|||
* with them.
|
||||
*/
|
||||
List *sequenceDependencyList = GetRelationSequenceDependencyList(relationId);
|
||||
|
||||
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
|
||||
* 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
|
||||
* objects for the schemas that statistics' of the relation with relationId depends.
|
||||
|
|
|
@ -209,8 +209,9 @@ MarkObjectDistributedLocally(const ObjectAddress *distAddress)
|
|||
ObjectIdGetDatum(distAddress->objectId),
|
||||
Int32GetDatum(distAddress->objectSubId)
|
||||
};
|
||||
char *insertQuery = "INSERT INTO citus.pg_dist_object (classid, objid, objsubid) "
|
||||
"VALUES ($1, $2, $3) ON CONFLICT DO NOTHING";
|
||||
char *insertQuery =
|
||||
"INSERT INTO pg_catalog.pg_dist_object (classid, objid, objsubid) "
|
||||
"VALUES ($1, $2, $3) ON CONFLICT DO NOTHING";
|
||||
int spiStatus = ExecuteCommandAsSuperuser(insertQuery, paramCount, paramTypes,
|
||||
paramValues);
|
||||
if (spiStatus < 0)
|
||||
|
@ -324,7 +325,7 @@ UnmarkObjectDistributed(const ObjectAddress *address)
|
|||
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";
|
||||
|
||||
int spiStatus = ExecuteCommandAsSuperuser(deleteQuery, paramCount, paramTypes,
|
||||
|
|
|
@ -247,8 +247,13 @@ static void GetPartitionTypeInputInfo(char *partitionKeyString, char partitionMe
|
|||
Oid *intervalTypeId, int32 *intervalTypeMod);
|
||||
static void CachedNamespaceLookup(const char *nspname, 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,
|
||||
Oid *cachedOid);
|
||||
static void CachedRelationNamespaceLookupExtended(const char *relationName,
|
||||
Oid renamespace, Oid *cachedOid,
|
||||
bool missing_ok);
|
||||
static ShardPlacement * ResolveGroupShardPlacement(
|
||||
GroupShardPlacement *groupShardPlacement, CitusTableCacheEntry *tableEntry,
|
||||
int shardIndex);
|
||||
|
@ -2321,8 +2326,37 @@ CitusCatalogNamespaceId(void)
|
|||
Oid
|
||||
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;
|
||||
}
|
||||
|
@ -2332,9 +2366,38 @@ DistObjectRelationId(void)
|
|||
Oid
|
||||
DistObjectPrimaryKeyIndexId(void)
|
||||
{
|
||||
CachedRelationNamespaceLookup("pg_dist_object_pkey",
|
||||
CitusCatalogNamespaceId(),
|
||||
&MetadataCache.distObjectPrimaryKeyIndexId);
|
||||
/*
|
||||
* 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_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;
|
||||
}
|
||||
|
@ -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
|
||||
CachedRelationNamespaceLookup(const char *relationName, Oid relnamespace,
|
||||
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 */
|
||||
InitializeCaches();
|
||||
|
@ -4602,7 +4686,7 @@ CachedRelationNamespaceLookup(const char *relationName, Oid relnamespace,
|
|||
{
|
||||
*cachedOid = get_relname_relid(relationName, relnamespace);
|
||||
|
||||
if (*cachedOid == InvalidOid)
|
||||
if (*cachedOid == InvalidOid && !missing_ok)
|
||||
{
|
||||
ereport(ERROR, (errmsg(
|
||||
"cache lookup failed for %s, called too early?",
|
||||
|
|
|
@ -28,9 +28,12 @@
|
|||
#include "catalog/indexing.h"
|
||||
#include "catalog/pg_am.h"
|
||||
#include "catalog/pg_attrdef.h"
|
||||
#include "catalog/pg_collation.h"
|
||||
#include "catalog/pg_constraint.h"
|
||||
#include "catalog/pg_depend.h"
|
||||
#include "catalog/pg_foreign_server.h"
|
||||
#include "catalog/pg_namespace.h"
|
||||
#include "catalog/pg_proc.h"
|
||||
#include "catalog/pg_type.h"
|
||||
#include "commands/async.h"
|
||||
#include "distributed/argutils.h"
|
||||
|
@ -46,12 +49,14 @@
|
|||
#include "distributed/maintenanced.h"
|
||||
#include "distributed/metadata_cache.h"
|
||||
#include "distributed/metadata_sync.h"
|
||||
#include "distributed/metadata_utility.h"
|
||||
#include "distributed/metadata/distobject.h"
|
||||
#include "distributed/metadata/pg_dist_object.h"
|
||||
#include "distributed/multi_executor.h"
|
||||
#include "distributed/multi_join_order.h"
|
||||
#include "distributed/multi_partitioning_utils.h"
|
||||
#include "distributed/multi_physical_planner.h"
|
||||
#include "distributed/pg_dist_colocation.h"
|
||||
#include "distributed/pg_dist_node.h"
|
||||
#include "distributed/pg_dist_shard.h"
|
||||
#include "distributed/relation_access_tracking.h"
|
||||
|
@ -85,6 +90,7 @@ char *EnableManualMetadataChangesForUser = "";
|
|||
|
||||
static void EnsureObjectMetadataIsSane(int distributionArgumentIndex,
|
||||
int colocationId);
|
||||
static List * GetFunctionDependenciesForObjects(ObjectAddress *objectAddress);
|
||||
static char * SchemaOwnerName(Oid objectId);
|
||||
static bool HasMetadataWorkers(void);
|
||||
static void CreateShellTableOnWorkers(Oid relationId);
|
||||
|
@ -104,7 +110,6 @@ static List * GetObjectsForGrantStmt(ObjectType objectType, Oid objectId);
|
|||
static AccessPriv * GetAccessPrivObjectForGrantStmt(char *permission);
|
||||
static List * GenerateGrantOnSchemaQueriesFromAclItem(Oid schemaOid,
|
||||
AclItem *aclItem);
|
||||
static void SetLocalEnableMetadataSync(bool state);
|
||||
static void SetLocalReplicateReferenceTablesOnActivate(bool state);
|
||||
static char * GenerateSetRoleQuery(Oid roleOid);
|
||||
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,
|
||||
int64 placementId, int32 shardState,
|
||||
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(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_update_relation_colocation);
|
||||
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;
|
||||
|
@ -556,6 +571,7 @@ DropMetadataSnapshotOnNode(WorkerNode *workerNode)
|
|||
dropMetadataCommandList = lappend(dropMetadataCommandList, DELETE_ALL_PLACEMENTS);
|
||||
dropMetadataCommandList = lappend(dropMetadataCommandList,
|
||||
DELETE_ALL_DISTRIBUTED_OBJECTS);
|
||||
dropMetadataCommandList = lappend(dropMetadataCommandList, DELETE_ALL_COLOCATION);
|
||||
|
||||
Assert(superuser());
|
||||
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
|
||||
* 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
|
||||
*/
|
||||
static void
|
||||
void
|
||||
SetLocalEnableMetadataSync(bool state)
|
||||
{
|
||||
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();
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* 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);
|
||||
}
|
||||
|
|
|
@ -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
|
||||
* that are sitting on group with groupId for given shardId.
|
||||
|
@ -1323,53 +1369,39 @@ ActiveShardPlacementListOnGroup(uint64 shardId, int32 groupId)
|
|||
List *
|
||||
ActiveShardPlacementList(uint64 shardId)
|
||||
{
|
||||
List *activePlacementList = NIL;
|
||||
List *shardPlacementList =
|
||||
ShardPlacementListIncludingOrphanedPlacements(shardId);
|
||||
|
||||
ShardPlacement *shardPlacement = NULL;
|
||||
foreach_ptr(shardPlacement, shardPlacementList)
|
||||
{
|
||||
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);
|
||||
}
|
||||
}
|
||||
List *activePlacementList = FilterShardPlacementList(shardPlacementList,
|
||||
IsActiveShardPlacement);
|
||||
|
||||
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
|
||||
* the ones that are orphaned, because they are marked to be deleted at a later
|
||||
* point (shardstate = 4).
|
||||
* the ones that are orphaned.
|
||||
*/
|
||||
List *
|
||||
ShardPlacementListWithoutOrphanedPlacements(uint64 shardId)
|
||||
{
|
||||
List *activePlacementList = NIL;
|
||||
List *shardPlacementList =
|
||||
ShardPlacementListIncludingOrphanedPlacements(shardId);
|
||||
|
||||
ShardPlacement *shardPlacement = NULL;
|
||||
foreach_ptr(shardPlacement, shardPlacementList)
|
||||
{
|
||||
if (shardPlacement->shardState != SHARD_STATE_TO_DELETE)
|
||||
{
|
||||
activePlacementList = lappend(activePlacementList, shardPlacement);
|
||||
}
|
||||
}
|
||||
List *activePlacementList = FilterShardPlacementList(shardPlacementList,
|
||||
IsShardPlacementNotOrphaned);
|
||||
|
||||
return SortList(activePlacementList, CompareShardPlacementsByWorker);
|
||||
}
|
||||
|
|
|
@ -146,6 +146,8 @@ PG_FUNCTION_INFO_V1(master_activate_node);
|
|||
PG_FUNCTION_INFO_V1(citus_update_node);
|
||||
PG_FUNCTION_INFO_V1(master_update_node);
|
||||
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);
|
||||
nodeMetadata.nodeCluster = NameStr(*nodeClusterName);
|
||||
|
||||
/* prevent concurrent modification */
|
||||
LockRelationOid(DistNodeRelationId(), RowShareLock);
|
||||
|
||||
bool isCoordinatorInMetadata = false;
|
||||
WorkerNode *coordinatorNode = PrimaryNodeForGroup(COORDINATOR_GROUP_ID,
|
||||
&isCoordinatorInMetadata);
|
||||
|
@ -656,6 +655,8 @@ PgDistTableMetadataSyncCommandList(void)
|
|||
DELETE_ALL_PLACEMENTS);
|
||||
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
|
||||
DELETE_ALL_DISTRIBUTED_OBJECTS);
|
||||
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
|
||||
DELETE_ALL_COLOCATION);
|
||||
|
||||
/* create pg_dist_partition, pg_dist_shard and pg_dist_placement entries */
|
||||
foreach_ptr(cacheEntry, propagatedTableList)
|
||||
|
@ -667,6 +668,11 @@ PgDistTableMetadataSyncCommandList(void)
|
|||
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 */
|
||||
Assert(ShouldPropagate());
|
||||
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
|
||||
* 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.
|
||||
*/
|
||||
WorkerNode *
|
||||
FindNodeWithNodeId(int nodeId)
|
||||
FindNodeWithNodeId(int nodeId, bool missingOk)
|
||||
{
|
||||
List *workerList = ActiveReadableNodeList();
|
||||
WorkerNode *workerNode = NULL;
|
||||
List *nodeList = ActiveReadableNodeList();
|
||||
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 */
|
||||
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;
|
||||
}
|
||||
|
@ -1780,12 +1833,6 @@ AddNodeMetadata(char *nodeName, int32 nodePort,
|
|||
|
||||
*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);
|
||||
if (workerNode != NULL)
|
||||
{
|
||||
|
|
|
@ -34,6 +34,7 @@
|
|||
#include "distributed/lock_graph.h"
|
||||
#include "distributed/coordinator_protocol.h"
|
||||
#include "distributed/metadata_cache.h"
|
||||
#include "distributed/metadata_utility.h"
|
||||
#include "distributed/multi_client_executor.h"
|
||||
#include "distributed/multi_progress.h"
|
||||
#include "distributed/multi_server_executor.h"
|
||||
|
@ -190,7 +191,7 @@ static void UpdateShardPlacement(PlacementUpdateEvent *placementUpdateEvent,
|
|||
List *responsiveNodeList, Oid shardReplicationModeOid);
|
||||
|
||||
/* 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,
|
||||
WorkerNode *workerNode);
|
||||
static void PlacementsHashEnter(HTAB *placementsHash, uint64 shardId,
|
||||
|
@ -396,6 +397,7 @@ FullShardPlacementList(Oid relationId, ArrayType *excludedShardArray)
|
|||
placement->shardId = groupPlacement->shardId;
|
||||
placement->shardLength = groupPlacement->shardLength;
|
||||
placement->shardState = groupPlacement->shardState;
|
||||
placement->nodeId = worker->nodeId;
|
||||
placement->nodeName = pstrdup(worker->workerName);
|
||||
placement->nodePort = worker->workerPort;
|
||||
placement->placementId = groupPlacement->placementId;
|
||||
|
@ -446,14 +448,17 @@ GetRebalanceSteps(RebalanceOptions *options)
|
|||
|
||||
/* sort the lists to make the function more deterministic */
|
||||
List *activeWorkerList = SortedActiveWorkers();
|
||||
List *shardPlacementListList = NIL;
|
||||
List *activeShardPlacementListList = NIL;
|
||||
|
||||
Oid relationId = InvalidOid;
|
||||
foreach_oid(relationId, options->relationIdList)
|
||||
{
|
||||
List *shardPlacementList = FullShardPlacementList(relationId,
|
||||
options->excludedShardArray);
|
||||
shardPlacementListList = lappend(shardPlacementListList, shardPlacementList);
|
||||
List *activeShardPlacementListForRelation =
|
||||
FilterShardPlacementList(shardPlacementList, IsActiveShardPlacement);
|
||||
activeShardPlacementListList =
|
||||
lappend(activeShardPlacementListList, activeShardPlacementListForRelation);
|
||||
}
|
||||
|
||||
if (options->threshold < options->rebalanceStrategy->minimumThreshold)
|
||||
|
@ -471,7 +476,7 @@ GetRebalanceSteps(RebalanceOptions *options)
|
|||
}
|
||||
|
||||
return RebalancePlacementUpdates(activeWorkerList,
|
||||
shardPlacementListList,
|
||||
activeShardPlacementListList,
|
||||
options->threshold,
|
||||
options->maxShardMoves,
|
||||
options->drainOnly,
|
||||
|
@ -795,7 +800,6 @@ rebalance_table_shards(PG_FUNCTION_ARGS)
|
|||
{
|
||||
Oid relationId = PG_GETARG_OID(0);
|
||||
ErrorIfMoveUnsupportedTableType(relationId);
|
||||
|
||||
relationIdList = list_make1_oid(relationId);
|
||||
}
|
||||
else
|
||||
|
@ -951,9 +955,11 @@ replicate_table_shards(PG_FUNCTION_ARGS)
|
|||
|
||||
List *activeWorkerList = SortedActiveWorkers();
|
||||
List *shardPlacementList = FullShardPlacementList(relationId, excludedShardArray);
|
||||
List *activeShardPlacementList = FilterShardPlacementList(shardPlacementList,
|
||||
IsActiveShardPlacement);
|
||||
|
||||
List *placementUpdateList = ReplicationPlacementUpdates(activeWorkerList,
|
||||
shardPlacementList,
|
||||
activeShardPlacementList,
|
||||
shardReplicationFactor);
|
||||
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
|
||||
* 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
|
||||
* make sure different colocation groups are balanced separately, so each list
|
||||
* contains the placements of a colocation group.
|
||||
*/
|
||||
List *
|
||||
RebalancePlacementUpdates(List *workerNodeList, List *shardPlacementListList,
|
||||
RebalancePlacementUpdates(List *workerNodeList, List *activeShardPlacementListList,
|
||||
double threshold,
|
||||
int32 maxShardMoves,
|
||||
bool drainOnly,
|
||||
|
@ -1755,7 +1761,7 @@ RebalancePlacementUpdates(List *workerNodeList, List *shardPlacementListList,
|
|||
List *shardPlacementList = NIL;
|
||||
List *placementUpdateList = NIL;
|
||||
|
||||
foreach_ptr(shardPlacementList, shardPlacementListList)
|
||||
foreach_ptr(shardPlacementList, activeShardPlacementListList)
|
||||
{
|
||||
state = InitRebalanceState(workerNodeList, shardPlacementList,
|
||||
functions);
|
||||
|
@ -1861,7 +1867,7 @@ InitRebalanceState(List *workerNodeList, List *shardPlacementList,
|
|||
|
||||
RebalanceState *state = palloc0(sizeof(RebalanceState));
|
||||
state->functions = functions;
|
||||
state->placementsHash = ActivePlacementsHash(shardPlacementList);
|
||||
state->placementsHash = ShardPlacementsListToHash(shardPlacementList);
|
||||
|
||||
/* create empty fill state for all of the worker nodes */
|
||||
foreach_ptr(workerNode, workerNodeList)
|
||||
|
@ -2413,29 +2419,25 @@ FindAndMoveShardCost(float4 utilizationLowerBound,
|
|||
/*
|
||||
* ReplicationPlacementUpdates returns a list of placement updates which
|
||||
* 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
|
||||
* smallest number of shards as the target node.
|
||||
*/
|
||||
List *
|
||||
ReplicationPlacementUpdates(List *workerNodeList, List *shardPlacementList,
|
||||
ReplicationPlacementUpdates(List *workerNodeList, List *activeShardPlacementList,
|
||||
int shardReplicationFactor)
|
||||
{
|
||||
List *placementUpdateList = NIL;
|
||||
ListCell *shardPlacementCell = NULL;
|
||||
uint32 workerNodeIndex = 0;
|
||||
HTAB *placementsHash = ActivePlacementsHash(shardPlacementList);
|
||||
HTAB *placementsHash = ShardPlacementsListToHash(activeShardPlacementList);
|
||||
uint32 workerNodeCount = list_length(workerNodeList);
|
||||
|
||||
/* get number of shards per node */
|
||||
uint32 *shardCountArray = palloc0(workerNodeCount * sizeof(uint32));
|
||||
foreach(shardPlacementCell, shardPlacementList)
|
||||
foreach(shardPlacementCell, activeShardPlacementList)
|
||||
{
|
||||
ShardPlacement *placement = lfirst(shardPlacementCell);
|
||||
if (placement->shardState != SHARD_STATE_ACTIVE)
|
||||
{
|
||||
continue;
|
||||
}
|
||||
|
||||
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 *targetNode = NULL;
|
||||
|
@ -2586,11 +2588,11 @@ ShardActivePlacementCount(HTAB *activePlacementsHash, uint64 shardId,
|
|||
|
||||
|
||||
/*
|
||||
* ActivePlacementsHash creates and returns a hash set for the placements in
|
||||
* the given list of shard placements which are in active state.
|
||||
* ShardPlacementsListToHash creates and returns a hash set from a shard
|
||||
* placement list.
|
||||
*/
|
||||
static HTAB *
|
||||
ActivePlacementsHash(List *shardPlacementList)
|
||||
ShardPlacementsListToHash(List *shardPlacementList)
|
||||
{
|
||||
ListCell *shardPlacementCell = NULL;
|
||||
HASHCTL info;
|
||||
|
@ -2609,11 +2611,8 @@ ActivePlacementsHash(List *shardPlacementList)
|
|||
foreach(shardPlacementCell, shardPlacementList)
|
||||
{
|
||||
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;
|
||||
|
|
|
@ -40,6 +40,7 @@
|
|||
#include "distributed/colocation_utils.h"
|
||||
#include "distributed/deparse_shard_query.h"
|
||||
#include "distributed/coordinator_protocol.h"
|
||||
#include "distributed/intermediate_results.h"
|
||||
#include "distributed/metadata_cache.h"
|
||||
#include "distributed/multi_router_planner.h"
|
||||
#include "distributed/multi_join_order.h"
|
||||
|
@ -52,15 +53,16 @@
|
|||
#include "distributed/pg_dist_shard.h"
|
||||
#include "distributed/query_pushdown_planning.h"
|
||||
#include "distributed/query_utils.h"
|
||||
#include "distributed/recursive_planning.h"
|
||||
#include "distributed/shardinterval_utils.h"
|
||||
#include "distributed/shard_pruning.h"
|
||||
#include "distributed/string_utils.h"
|
||||
|
||||
#include "distributed/worker_manager.h"
|
||||
#include "distributed/worker_protocol.h"
|
||||
#include "distributed/version_compat.h"
|
||||
#include "nodes/makefuncs.h"
|
||||
#include "nodes/nodeFuncs.h"
|
||||
#include "nodes/print.h"
|
||||
#include "optimizer/clauses.h"
|
||||
#include "nodes/pathnodes.h"
|
||||
#include "optimizer/optimizer.h"
|
||||
|
@ -157,8 +159,6 @@ static MapMergeJob * BuildMapMergeJob(Query *jobQuery, List *dependentJobList,
|
|||
Oid baseRelationId,
|
||||
BoundaryNodeJobType boundaryNodeJobType);
|
||||
static uint32 HashPartitionCount(void);
|
||||
static ArrayType * SplitPointObject(ShardInterval **shardIntervalArray,
|
||||
uint32 shardIntervalCount);
|
||||
|
||||
/* Local functions forward declarations for task list creation and helper functions */
|
||||
static Job * BuildJobTreeTaskList(Job *jobTree,
|
||||
|
@ -195,11 +195,11 @@ static bool JoinPrunable(RangeTableFragment *leftFragment,
|
|||
static ShardInterval * FragmentInterval(RangeTableFragment *fragment);
|
||||
static StringInfo FragmentIntervalString(ShardInterval *fragmentInterval);
|
||||
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 void UpdateRangeTableAlias(List *rangeTableList, List *fragmentList);
|
||||
static Alias * FragmentAlias(RangeTblEntry *rangeTableEntry,
|
||||
RangeTableFragment *fragment);
|
||||
static List * FetchTaskResultNameList(List *mapOutputFetchTaskList);
|
||||
static uint64 AnchorShardId(List *fragmentList, uint32 anchorRangeTableId);
|
||||
static List * PruneSqlTaskDependencies(List *sqlTaskList);
|
||||
static List * AssignTaskList(List *sqlTaskList);
|
||||
|
@ -218,11 +218,13 @@ static void AssignDataFetchDependencies(List *taskList);
|
|||
static uint32 TaskListHighestTaskId(List *taskList);
|
||||
static List * MapTaskList(MapMergeJob *mapMergeJob, List *filterTaskList);
|
||||
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,
|
||||
uint32 taskIdIndex);
|
||||
static StringInfo ColumnNameArrayString(uint32 columnCount, uint64 generatingJobId);
|
||||
static StringInfo ColumnTypeArrayString(List *targetEntryList);
|
||||
|
||||
static List * FetchEqualityAttrNumsForRTEOpExpr(OpExpr *opExpr);
|
||||
static List * FetchEqualityAttrNumsForRTEBoolExpr(BoolExpr *boolExpr);
|
||||
|
@ -853,10 +855,14 @@ TargetEntryList(List *expressionList)
|
|||
foreach(expressionCell, expressionList)
|
||||
{
|
||||
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);
|
||||
}
|
||||
|
||||
|
@ -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
|
||||
* ------------------------------------------------------------
|
||||
|
@ -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,
|
||||
* and returns the created task.
|
||||
|
@ -4234,19 +4173,26 @@ FragmentAlias(RangeTblEntry *rangeTableEntry, RangeTableFragment *fragment)
|
|||
else if (fragmentType == CITUS_RTE_REMOTE_QUERY)
|
||||
{
|
||||
Task *mergeTask = (Task *) fragment->fragmentReference;
|
||||
uint64 jobId = mergeTask->jobId;
|
||||
uint32 taskId = mergeTask->taskId;
|
||||
List *mapOutputFetchTaskList = mergeTask->dependentTaskList;
|
||||
List *resultNameList = FetchTaskResultNameList(mapOutputFetchTaskList);
|
||||
List *mapJobTargetList = mergeTask->mapJobTargetList;
|
||||
|
||||
StringInfo jobSchemaName = JobSchemaName(jobId);
|
||||
StringInfo taskTableName = TaskTableName(taskId);
|
||||
/* determine whether all types have binary input/output functions */
|
||||
bool useBinaryFormat = CanUseBinaryCopyFormatForTargetList(mapJobTargetList);
|
||||
|
||||
StringInfo aliasNameString = makeStringInfo();
|
||||
appendStringInfo(aliasNameString, "%s.%s",
|
||||
jobSchemaName->data, taskTableName->data);
|
||||
/* generate the query on the intermediate result */
|
||||
Query *fragmentSetQuery = BuildReadIntermediateResultsArrayQuery(mapJobTargetList,
|
||||
NIL,
|
||||
resultNameList,
|
||||
useBinaryFormat);
|
||||
|
||||
aliasName = aliasNameString->data;
|
||||
fragmentName = taskTableName->data;
|
||||
schemaName = jobSchemaName->data;
|
||||
/* we only really care about the function RTE */
|
||||
RangeTblEntry *readIntermediateResultsRTE = linitial(fragmentSetQuery->rtable);
|
||||
|
||||
/* 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
|
||||
* fragment that corresponds to the given anchor range tableId, and returns this
|
||||
|
@ -4377,17 +4347,28 @@ MapTaskList(MapMergeJob *mapMergeJob, List *filterTaskList)
|
|||
filterQuery->targetList);
|
||||
}
|
||||
|
||||
/* determine whether all types have binary input/output functions */
|
||||
bool useBinaryFormat = CanUseBinaryCopyFormatForTargetList(filterQuery->targetList);
|
||||
|
||||
foreach(filterTaskCell, filterTaskList)
|
||||
{
|
||||
Task *filterTask = (Task *) lfirst(filterTaskCell);
|
||||
StringInfo mapQueryString = CreateMapQueryString(mapMergeJob, filterTask,
|
||||
partitionColumnResNo);
|
||||
partitionColumnResNo,
|
||||
useBinaryFormat);
|
||||
|
||||
/* convert filter query task into map task */
|
||||
Task *mapTask = filterTask;
|
||||
SetTaskQueryString(mapTask, mapQueryString->data);
|
||||
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);
|
||||
}
|
||||
|
||||
|
@ -4428,21 +4409,19 @@ PartitionColumnIndex(Var *targetVar, List *targetList)
|
|||
*/
|
||||
static StringInfo
|
||||
CreateMapQueryString(MapMergeJob *mapMergeJob, Task *filterTask,
|
||||
uint32 partitionColumnIndex)
|
||||
uint32 partitionColumnIndex, bool useBinaryFormat)
|
||||
{
|
||||
uint64 jobId = filterTask->jobId;
|
||||
uint32 taskId = filterTask->taskId;
|
||||
char *resultNamePrefix = PartitionResultNamePrefix(jobId, taskId);
|
||||
|
||||
/* wrap repartition query string around filter query string */
|
||||
StringInfo mapQueryString = makeStringInfo();
|
||||
char *filterQueryString = TaskQueryString(filterTask);
|
||||
char *filterQueryEscapedText = quote_literal_cstr(filterQueryString);
|
||||
PartitionType partitionType = mapMergeJob->partitionType;
|
||||
|
||||
Var *partitionColumn = mapMergeJob->partitionColumn;
|
||||
Oid partitionColumnType = partitionColumn->vartype;
|
||||
char *partitionColumnTypeFullName = format_type_be_qualified(partitionColumnType);
|
||||
int32 partitionColumnTypeMod = partitionColumn->vartypmod;
|
||||
|
||||
ShardInterval **intervalArray = mapMergeJob->sortedShardIntervalArray;
|
||||
uint32 intervalCount = mapMergeJob->partitionCount;
|
||||
|
@ -4450,38 +4429,101 @@ CreateMapQueryString(MapMergeJob *mapMergeJob, Task *filterTask,
|
|||
if (partitionType == DUAL_HASH_PARTITION_TYPE)
|
||||
{
|
||||
partitionColumnType = INT4OID;
|
||||
partitionColumnTypeMod = get_typmodin(INT4OID);
|
||||
intervalArray = GenerateSyntheticShardIntervalArray(intervalCount);
|
||||
}
|
||||
else if (partitionType == SINGLE_HASH_PARTITION_TYPE)
|
||||
{
|
||||
partitionColumnType = INT4OID;
|
||||
partitionColumnTypeMod = get_typmodin(INT4OID);
|
||||
}
|
||||
|
||||
ArrayType *splitPointObject = SplitPointObject(intervalArray, intervalCount);
|
||||
StringInfo splitPointString = ArrayObjectToString(splitPointObject,
|
||||
partitionColumnType,
|
||||
partitionColumnTypeMod);
|
||||
|
||||
char *partitionCommand = NULL;
|
||||
if (partitionType == RANGE_PARTITION_TYPE)
|
||||
else if (partitionType == RANGE_PARTITION_TYPE)
|
||||
{
|
||||
partitionCommand = RANGE_PARTITION_COMMAND;
|
||||
}
|
||||
else
|
||||
{
|
||||
partitionCommand = HASH_PARTITION_COMMAND;
|
||||
/* add a partition for NULL values at index 0 */
|
||||
intervalArray = RangeIntervalArrayWithNullBucket(intervalArray, intervalCount);
|
||||
intervalCount++;
|
||||
}
|
||||
|
||||
char *partitionColumnIndextText = ConvertIntToString(partitionColumnIndex);
|
||||
appendStringInfo(mapQueryString, partitionCommand, jobId, taskId,
|
||||
filterQueryEscapedText, partitionColumnIndextText,
|
||||
partitionColumnTypeFullName, splitPointString->data);
|
||||
Oid intervalTypeOutFunc = InvalidOid;
|
||||
bool intervalTypeVarlena = false;
|
||||
ArrayType *minValueArray = NULL;
|
||||
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;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* 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
|
||||
* 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 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->minValueExists = true;
|
||||
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
|
||||
*/
|
||||
|
@ -4598,7 +4674,7 @@ ArrayObjectToString(ArrayType *arrayObject, Oid columnType, int32 columnTypeMod)
|
|||
char *arrayOutputEscapedText = quote_literal_cstr(arrayOutputText);
|
||||
|
||||
/* 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();
|
||||
appendStringInfo(arrayString, "%s::%s",
|
||||
|
@ -4660,17 +4736,9 @@ MergeTaskList(MapMergeJob *mapMergeJob, List *mapTaskList, uint32 taskIdIndex)
|
|||
Query *reduceQuery = mapMergeJob->reduceQuery;
|
||||
if (reduceQuery == NULL)
|
||||
{
|
||||
uint32 columnCount = (uint32) list_length(targetEntryList);
|
||||
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 */
|
||||
/* create logical merge task (not executed, but useful for bookkeeping) */
|
||||
mergeTask = CreateBasicTask(jobId, mergeTaskId, MERGE_TASK,
|
||||
mergeQueryString->data);
|
||||
"<merge>");
|
||||
}
|
||||
mergeTask->partitionId = partitionId;
|
||||
taskIdIndex++;
|
||||
|
@ -4682,26 +4750,35 @@ MergeTaskList(MapMergeJob *mapMergeJob, List *mapTaskList, uint32 taskIdIndex)
|
|||
|
||||
/* find the node name/port for map task's execution */
|
||||
List *mapTaskPlacementList = mapTask->taskPlacementList;
|
||||
|
||||
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
|
||||
* When replication factor is greater than 1 and there
|
||||
* is a connection problem to the node that has done the map task, we will get
|
||||
* an error in fetch task execution.
|
||||
* Target node is not yet decided, and not necessary for
|
||||
* QueryStringForFragmentsTransfer.
|
||||
*/
|
||||
StringInfo mapFetchQueryString = makeStringInfo();
|
||||
appendStringInfo(mapFetchQueryString, MAP_OUTPUT_FETCH_COMMAND,
|
||||
mapTask->jobId, mapTask->taskId, partitionId,
|
||||
mergeTaskId, /* fetch results to merge task */
|
||||
mapTaskNodeName, mapTaskNodePort);
|
||||
fragmentsTransfer.nodes.targetNodeId = -1;
|
||||
|
||||
fragmentsTransfer.fragmentList = list_make1(&singleFragmentTransfer);
|
||||
|
||||
char *fetchQueryString = QueryStringForFragmentsTransfer(&fragmentsTransfer);
|
||||
|
||||
Task *mapOutputFetchTask = CreateBasicTask(jobId, taskIdIndex,
|
||||
MAP_OUTPUT_FETCH_TASK,
|
||||
mapFetchQueryString->data);
|
||||
fetchQueryString);
|
||||
mapOutputFetchTask->partitionId = partitionId;
|
||||
mapOutputFetchTask->upstreamTaskId = mergeTaskId;
|
||||
mapOutputFetchTask->dependentTaskList = list_make1(mapTask);
|
||||
|
@ -4712,6 +4789,7 @@ MergeTaskList(MapMergeJob *mapMergeJob, List *mapTaskList, uint32 taskIdIndex)
|
|||
|
||||
/* merge task depends on completion of fetch tasks */
|
||||
mergeTask->dependentTaskList = mapOutputFetchTaskList;
|
||||
mergeTask->mapJobTargetList = targetEntryList;
|
||||
|
||||
/* if single repartitioned, each merge task represents an interval */
|
||||
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
|
||||
* 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 */
|
||||
activeShardPlacementList = SortList(activeShardPlacementList,
|
||||
CompareShardPlacements);
|
||||
|
||||
shardPlacementLists = lappend(shardPlacementLists, activeShardPlacementList);
|
||||
}
|
||||
|
||||
|
|
|
@ -23,6 +23,7 @@
|
|||
|
||||
#include "safe_lib.h"
|
||||
|
||||
#include "catalog/pg_authid.h"
|
||||
#include "citus_version.h"
|
||||
#include "commands/explain.h"
|
||||
#include "common/string.h"
|
||||
|
@ -84,12 +85,14 @@
|
|||
#include "libpq/auth.h"
|
||||
#include "port/atomics.h"
|
||||
#include "postmaster/postmaster.h"
|
||||
#include "replication/walsender.h"
|
||||
#include "storage/ipc.h"
|
||||
#include "optimizer/planner.h"
|
||||
#include "optimizer/paths.h"
|
||||
#include "tcop/tcopprot.h"
|
||||
#include "utils/guc.h"
|
||||
#include "utils/guc_tables.h"
|
||||
#include "utils/syscache.h"
|
||||
#include "utils/varlena.h"
|
||||
|
||||
#include "columnar/mod.h"
|
||||
|
@ -113,9 +116,9 @@ static void DoInitialCleanup(void);
|
|||
static void ResizeStackToMaximumDepth(void);
|
||||
static void multi_log_hook(ErrorData *edata);
|
||||
static void RegisterConnectionCleanup(void);
|
||||
static void RegisterClientBackendCounterDecrement(void);
|
||||
static void RegisterExternalClientBackendCounterDecrement(void);
|
||||
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 RegisterCitusConfigVariables(void);
|
||||
static void OverridePostgresConfigAssignHooks(void);
|
||||
|
@ -135,6 +138,7 @@ static const char * LocalPoolSizeGucShowHook(void);
|
|||
static bool StatisticsCollectionGucCheckHook(bool *newval, void **extra, GucSource
|
||||
source);
|
||||
static void CitusAuthHook(Port *port, int status);
|
||||
static bool IsSuperuser(char *userName);
|
||||
|
||||
|
||||
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 }
|
||||
};
|
||||
|
||||
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* */
|
||||
|
||||
|
||||
|
@ -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
|
||||
*/
|
||||
static void
|
||||
RegisterClientBackendCounterDecrement(void)
|
||||
RegisterExternalClientBackendCounterDecrement(void)
|
||||
{
|
||||
static bool registeredCleanup = false;
|
||||
if (registeredCleanup == false)
|
||||
{
|
||||
before_shmem_exit(DecrementClientBackendCounterAtExit, 0);
|
||||
before_shmem_exit(DecrementExternalClientBackendCounterAtExit, 0);
|
||||
|
||||
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
|
||||
*/
|
||||
static void
|
||||
DecrementClientBackendCounterAtExit(int code, Datum arg)
|
||||
DecrementExternalClientBackendCounterAtExit(int code, Datum arg)
|
||||
{
|
||||
DecrementClientBackendCounter();
|
||||
DecrementExternalClientBackendCounter();
|
||||
}
|
||||
|
||||
|
||||
|
@ -603,7 +614,7 @@ RegisterCitusConfigVariables(void)
|
|||
false,
|
||||
#endif
|
||||
PGC_SIGHUP,
|
||||
GUC_STANDARD,
|
||||
GUC_NO_SHOW_ALL,
|
||||
NULL, NULL, NULL);
|
||||
|
||||
DefineCustomBoolVariable(
|
||||
|
@ -669,6 +680,24 @@ RegisterCitusConfigVariables(void)
|
|||
GUC_STANDARD,
|
||||
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(
|
||||
"citus.defer_drop_after_shard_move",
|
||||
gettext_noop("When enabled a shard move will mark the original shards "
|
||||
|
@ -1315,6 +1344,23 @@ RegisterCitusConfigVariables(void)
|
|||
GUC_STANDARD,
|
||||
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(
|
||||
"citus.max_intermediate_result_size",
|
||||
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
|
||||
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 */
|
||||
if (original_client_auth_hook)
|
||||
{
|
||||
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;
|
||||
}
|
||||
|
|
|
@ -10,6 +10,8 @@
|
|||
#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_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/worker_drop_sequence_dependency/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_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/citus_isolation_test_session_is_blocked/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
|
||||
SELECT * FROM pg_catalog.citus_worker_stat_activity();
|
||||
ALTER VIEW citus.citus_worker_stat_activity SET SCHEMA pg_catalog;
|
||||
GRANT SELECT ON pg_catalog.citus_worker_stat_activity TO PUBLIC;
|
||||
|
||||
DROP FUNCTION IF EXISTS pg_catalog.citus_dist_stat_activity() CASCADE;
|
||||
DROP FUNCTION IF EXISTS pg_catalog.citus_worker_stat_activity() CASCADE;
|
||||
#include "udfs/citus_dist_stat_activity/11.0-1.sql"
|
||||
|
||||
CREATE VIEW citus.citus_dist_stat_activity AS
|
||||
SELECT * FROM pg_catalog.citus_dist_stat_activity();
|
||||
ALTER VIEW citus.citus_dist_stat_activity SET SCHEMA pg_catalog;
|
||||
GRANT SELECT ON pg_catalog.citus_dist_stat_activity TO PUBLIC;
|
||||
-- a very simple helper function defined for citus_lock_waits
|
||||
CREATE OR REPLACE FUNCTION get_nodeid_for_groupid(groupIdInput int) RETURNS int AS $$
|
||||
DECLARE
|
||||
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/pg_cancel_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_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);
|
||||
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"
|
||||
|
|
|
@ -1,6 +1,12 @@
|
|||
-- 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.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)
|
||||
RETURNS integer
|
||||
LANGUAGE C STRICT
|
||||
|
@ -21,6 +27,7 @@ CREATE FUNCTION pg_catalog.master_get_table_metadata(
|
|||
AS 'MODULE_PATHNAME', $$master_get_table_metadata$$;
|
||||
COMMENT ON FUNCTION master_get_table_metadata(relation_name text)
|
||||
IS 'fetch metadata values for the table';
|
||||
|
||||
ALTER TABLE pg_catalog.pg_dist_partition DROP COLUMN autoconverted;
|
||||
|
||||
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_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.worker_drop_sequence_dependency(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"
|
||||
ALTER TABLE citus.pg_dist_object DROP COLUMN force_delegation;
|
||||
|
||||
|
||||
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;
|
||||
|
||||
DROP FUNCTION citus_internal_local_blocked_processes CASCADE;
|
||||
DROP FUNCTION citus_internal_global_blocked_processes CASCADE;
|
||||
DROP VIEW pg_catalog.citus_lock_waits;
|
||||
|
||||
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,
|
||||
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;
|
||||
|
||||
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,
|
||||
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[]);
|
||||
#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_terminate_backend(bigint, bigint) CASCADE;
|
||||
DROP FUNCTION IF EXISTS pg_catalog.pg_cancel_backend(bigint);
|
||||
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(
|
||||
OUT waiting_pid 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()
|
||||
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(
|
||||
OUT waiting_pid int4,
|
||||
OUT waiting_node_id int4,
|
||||
|
@ -342,4 +354,19 @@ JOIN
|
|||
ALTER VIEW citus.citus_lock_waits SET SCHEMA pg_catalog;
|
||||
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;
|
||||
|
||||
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);
|
||||
|
|
|
@ -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;
|
|
@ -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;
|
|
@ -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;
|
|
@ -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;
|
|
@ -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,
|
||||
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_dist_stat_activity$$;
|
||||
CREATE OR REPLACE VIEW citus.citus_dist_stat_activity AS
|
||||
SELECT * FROM citus_stat_activity
|
||||
WHERE is_worker_query = false;
|
||||
|
||||
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,
|
||||
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 distributed tables';
|
||||
ALTER VIEW citus.citus_dist_stat_activity SET SCHEMA pg_catalog;
|
||||
GRANT SELECT ON pg_catalog.citus_dist_stat_activity TO PUBLIC;
|
||||
|
|
|
@ -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,
|
||||
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_dist_stat_activity$$;
|
||||
CREATE OR REPLACE VIEW citus.citus_dist_stat_activity AS
|
||||
SELECT * FROM citus_stat_activity
|
||||
WHERE is_worker_query = false;
|
||||
|
||||
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,
|
||||
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 distributed tables';
|
||||
ALTER VIEW citus.citus_dist_stat_activity SET SCHEMA pg_catalog;
|
||||
GRANT SELECT ON pg_catalog.citus_dist_stat_activity TO PUBLIC;
|
||||
|
|
224
src/backend/distributed/sql/udfs/citus_finalize_upgrade_to_citus11/11.0-1.sql
generated
Normal file
224
src/backend/distributed/sql/udfs/citus_finalize_upgrade_to_citus11/11.0-1.sql
generated
Normal 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;
|
|
@ -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;
|
|
@ -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';
|
|
@ -12,15 +12,22 @@ 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;
|
||||
|
||||
|
@ -124,8 +131,8 @@ BEGIN
|
|||
PERFORM citus_internal.columnar_ensure_am_depends_catalog();
|
||||
|
||||
-- restore pg_dist_object from the stable identifiers
|
||||
TRUNCATE citus.pg_dist_object;
|
||||
INSERT INTO citus.pg_dist_object (classid, objid, objsubid, distribution_argument_index, colocationid)
|
||||
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,
|
||||
|
|
13
src/backend/distributed/sql/udfs/citus_internal_add_colocation_metadata/11.0-1.sql
generated
Normal file
13
src/backend/distributed/sql/udfs/citus_internal_add_colocation_metadata/11.0-1.sql
generated
Normal 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';
|
|
@ -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';
|
9
src/backend/distributed/sql/udfs/citus_internal_delete_colocation_metadata/11.0-1.sql
generated
Normal file
9
src/backend/distributed/sql/udfs/citus_internal_delete_colocation_metadata/11.0-1.sql
generated
Normal 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';
|
|
@ -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';
|
|
@ -2,43 +2,44 @@ SET search_path = 'pg_catalog';
|
|||
|
||||
CREATE VIEW citus.citus_lock_waits AS
|
||||
WITH
|
||||
citus_dist_stat_activity AS
|
||||
(
|
||||
SELECT * FROM citus_dist_stat_activity
|
||||
unique_global_wait_edges_with_calculated_gpids AS (
|
||||
SELECT
|
||||
-- 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
|
||||
(
|
||||
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
|
||||
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
|
||||
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
|
||||
-- if global_pid is NULL, it is most likely that a backend is blocked on a DDL
|
||||
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
|
||||
)
|
||||
SELECT
|
||||
waiting.global_pid as waiting_gpid,
|
||||
blocking.global_pid as blocking_gpid,
|
||||
waiting.pid AS waiting_pid,
|
||||
blocking.pid AS blocking_pid,
|
||||
waiting.query AS blocked_statement,
|
||||
blocking.query AS current_statement_in_blocking_process,
|
||||
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
|
||||
waiting.global_pid as waiting_gpid,
|
||||
blocking.global_pid as blocking_gpid,
|
||||
waiting.query AS blocked_statement,
|
||||
blocking.query AS current_statement_in_blocking_process,
|
||||
waiting.nodeid AS waiting_nodeid,
|
||||
blocking.nodeid AS blocking_nodeid
|
||||
FROM
|
||||
unique_global_wait_edges
|
||||
JOIN
|
||||
citus_dist_stat_activity_with_node_id waiting ON (unique_global_wait_edges.waiting_global_pid = waiting.global_pid)
|
||||
JOIN
|
||||
citus_dist_stat_activity_with_node_id blocking ON (unique_global_wait_edges.blocking_global_pid = blocking.global_pid);
|
||||
unique_global_wait_edges
|
||||
JOIN
|
||||
citus_dist_stat_activity_with_calculated_gpids waiting ON (unique_global_wait_edges.waiting_global_pid = waiting.global_pid)
|
||||
JOIN
|
||||
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;
|
||||
GRANT SELECT ON pg_catalog.citus_lock_waits TO PUBLIC;
|
||||
|
|
|
@ -2,43 +2,44 @@ SET search_path = 'pg_catalog';
|
|||
|
||||
CREATE VIEW citus.citus_lock_waits AS
|
||||
WITH
|
||||
citus_dist_stat_activity AS
|
||||
(
|
||||
SELECT * FROM citus_dist_stat_activity
|
||||
unique_global_wait_edges_with_calculated_gpids AS (
|
||||
SELECT
|
||||
-- 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
|
||||
(
|
||||
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
|
||||
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
|
||||
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
|
||||
-- if global_pid is NULL, it is most likely that a backend is blocked on a DDL
|
||||
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
|
||||
)
|
||||
SELECT
|
||||
waiting.global_pid as waiting_gpid,
|
||||
blocking.global_pid as blocking_gpid,
|
||||
waiting.pid AS waiting_pid,
|
||||
blocking.pid AS blocking_pid,
|
||||
waiting.query AS blocked_statement,
|
||||
blocking.query AS current_statement_in_blocking_process,
|
||||
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
|
||||
waiting.global_pid as waiting_gpid,
|
||||
blocking.global_pid as blocking_gpid,
|
||||
waiting.query AS blocked_statement,
|
||||
blocking.query AS current_statement_in_blocking_process,
|
||||
waiting.nodeid AS waiting_nodeid,
|
||||
blocking.nodeid AS blocking_nodeid
|
||||
FROM
|
||||
unique_global_wait_edges
|
||||
JOIN
|
||||
citus_dist_stat_activity_with_node_id waiting ON (unique_global_wait_edges.waiting_global_pid = waiting.global_pid)
|
||||
JOIN
|
||||
citus_dist_stat_activity_with_node_id blocking ON (unique_global_wait_edges.blocking_global_pid = blocking.global_pid);
|
||||
unique_global_wait_edges
|
||||
JOIN
|
||||
citus_dist_stat_activity_with_calculated_gpids waiting ON (unique_global_wait_edges.waiting_global_pid = waiting.global_pid)
|
||||
JOIN
|
||||
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;
|
||||
GRANT SELECT ON pg_catalog.citus_lock_waits TO PUBLIC;
|
||||
|
|
|
@ -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';
|
|
@ -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';
|
|
@ -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';
|
|
@ -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';
|
|
@ -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';
|
|
@ -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';
|
|
@ -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';
|
|
@ -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';
|
|
@ -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';
|
|
@ -65,7 +65,7 @@ BEGIN
|
|||
address.object_args,
|
||||
objects.distribution_argument_index,
|
||||
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;
|
||||
END;
|
||||
$cppu$;
|
||||
|
|
|
@ -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;
|
|
@ -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;
|
|
@ -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';
|
|
@ -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';
|
|
@ -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$;
|
|
@ -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$;
|
|
@ -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$$;
|
||||
|
||||
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';
|
||||
|
|
|
@ -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$$;
|
||||
|
||||
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';
|
||||
|
|
|
@ -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';
|
|
@ -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(
|
||||
result_prefix text,
|
||||
query text,
|
||||
|
@ -5,12 +7,14 @@ CREATE OR REPLACE FUNCTION pg_catalog.worker_partition_query_result(
|
|||
partition_method citus.distribution_type,
|
||||
partition_min_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 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)
|
||||
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';
|
||||
|
|
|
@ -27,5 +27,5 @@ PG_FUNCTION_INFO_V1(get_all_active_client_backend_count);
|
|||
Datum
|
||||
get_all_active_client_backend_count(PG_FUNCTION_ARGS)
|
||||
{
|
||||
PG_RETURN_UINT32(GetAllActiveClientBackendCount());
|
||||
PG_RETURN_UINT32(GetExternalClientBackendCount());
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
|
@ -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();
|
||||
}
|
|
@ -188,7 +188,7 @@ run_commands_on_session_level_connection_to_node(PG_FUNCTION_ARGS)
|
|||
|
||||
/*
|
||||
* override_backend_data_command_originator is a wrapper around
|
||||
* OverrideBackendDataDistributedCommandOriginator().
|
||||
* SetBackendDataDistributedCommandOriginator().
|
||||
*/
|
||||
Datum
|
||||
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);
|
||||
|
||||
OverrideBackendDataDistributedCommandOriginator(distributedCommandOriginator);
|
||||
SetBackendDataDistributedCommandOriginator(distributedCommandOriginator);
|
||||
|
||||
PG_RETURN_VOID();
|
||||
}
|
||||
|
|
|
@ -20,9 +20,11 @@
|
|||
#include "distributed/citus_ruleutils.h"
|
||||
#include "distributed/connection_management.h"
|
||||
#include "distributed/listutils.h"
|
||||
#include "distributed/metadata_utility.h"
|
||||
#include "distributed/multi_physical_planner.h"
|
||||
#include "distributed/shard_cleaner.h"
|
||||
#include "distributed/shard_rebalancer.h"
|
||||
#include "distributed/relay_utility.h"
|
||||
#include "funcapi.h"
|
||||
#include "miscadmin.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
|
||||
* 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)
|
||||
{
|
||||
shardPlacementList = SortList(shardPlacementList, CompareShardPlacements);
|
||||
shardPlacementListList = lappend(shardPlacementListList, shardPlacementList);
|
||||
shardPlacementListList = lappend(shardPlacementListList,
|
||||
FilterShardPlacementList(shardPlacementList,
|
||||
IsActiveTestShardPlacement));
|
||||
shardPlacementList = NIL;
|
||||
}
|
||||
shardPlacementList = lappend(shardPlacementList,
|
||||
|
@ -290,12 +306,15 @@ shard_placement_replication_array(PG_FUNCTION_ARGS)
|
|||
shardPlacementTestInfo->placement);
|
||||
}
|
||||
|
||||
List *activeShardPlacementList = FilterShardPlacementList(shardPlacementList,
|
||||
IsActiveTestShardPlacement);
|
||||
|
||||
/* sort the lists to make the function more deterministic */
|
||||
workerNodeList = SortList(workerNodeList, CompareWorkerNodes);
|
||||
shardPlacementList = SortList(shardPlacementList, CompareShardPlacements);
|
||||
activeShardPlacementList = SortList(activeShardPlacementList, CompareShardPlacements);
|
||||
|
||||
List *placementUpdateList = ReplicationPlacementUpdates(workerNodeList,
|
||||
shardPlacementList,
|
||||
activeShardPlacementList,
|
||||
shardReplicationFactor);
|
||||
ArrayType *placementUpdateJsonArray = PlacementUpdateListToJsonArray(
|
||||
placementUpdateList);
|
||||
|
@ -426,6 +445,9 @@ JsonArrayToWorkerTestInfoList(ArrayType *workerNodeJsonArrayObject)
|
|||
workerTestInfo->capacity = JsonFieldValueUInt64Default(workerNodeJson,
|
||||
"capacity", 1);
|
||||
|
||||
workerNode->isActive = JsonFieldValueBoolDefault(workerNodeJson,
|
||||
"isActive", true);
|
||||
|
||||
workerTestInfoList = lappend(workerTestInfoList, workerTestInfo);
|
||||
char *disallowedShardsString = JsonFieldValueString(
|
||||
workerNodeJson, "disallowed_shards");
|
||||
|
|
|
@ -68,14 +68,12 @@ typedef struct BackendManagementShmemData
|
|||
pg_atomic_uint64 nextTransactionNumber;
|
||||
|
||||
/*
|
||||
* Total number of client backends that are authenticated.
|
||||
* We only care about activeClientBackendCounter when adaptive
|
||||
* connection management is enabled, otherwise always zero.
|
||||
* Total number of external client backends that are authenticated.
|
||||
*
|
||||
* 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];
|
||||
} BackendManagementShmemData;
|
||||
|
@ -83,6 +81,8 @@ typedef struct BackendManagementShmemData
|
|||
|
||||
static void StoreAllActiveTransactions(Tuplestorestate *tupleStore, TupleDesc
|
||||
tupleDescriptor);
|
||||
static bool UserHasPermissionToViewStatsOf(Oid currentUserId, Oid backendOwnedId);
|
||||
static uint64 CalculateGlobalPID(int32 nodeId, pid_t pid);
|
||||
static uint64 GenerateGlobalPID(void);
|
||||
|
||||
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_global_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);
|
||||
|
||||
Oid userId = GetUserId();
|
||||
|
||||
/* prepare data before acquiring spinlock to protect against errors */
|
||||
int32 initiatorNodeIdentifier = PG_GETARG_INT32(0);
|
||||
uint64 transactionNumber = PG_GETARG_INT64(1);
|
||||
|
@ -144,17 +146,11 @@ assign_distributed_transaction_id(PG_FUNCTION_ARGS)
|
|||
"transaction id")));
|
||||
}
|
||||
|
||||
MyBackendData->databaseId = MyDatabaseId;
|
||||
MyBackendData->userId = userId;
|
||||
|
||||
MyBackendData->transactionId.initiatorNodeIdentifier = initiatorNodeIdentifier;
|
||||
MyBackendData->transactionId.transactionNumber = transactionNumber;
|
||||
MyBackendData->transactionId.timestamp = timestamp;
|
||||
MyBackendData->transactionId.transactionOriginator = false;
|
||||
|
||||
MyBackendData->citusBackend.initiatorNodeIdentifier =
|
||||
MyBackendData->transactionId.initiatorNodeIdentifier;
|
||||
|
||||
SpinLockRelease(&MyBackendData->mutex);
|
||||
|
||||
PG_RETURN_VOID();
|
||||
|
@ -360,55 +356,48 @@ StoreAllActiveTransactions(Tuplestorestate *tupleStore, TupleDesc tupleDescripto
|
|||
{
|
||||
Datum values[ACTIVE_TRANSACTION_COLUMN_COUNT];
|
||||
bool isNulls[ACTIVE_TRANSACTION_COLUMN_COUNT];
|
||||
bool showAllTransactions = superuser();
|
||||
bool showAllBackends = superuser();
|
||||
const Oid userId = GetUserId();
|
||||
|
||||
/*
|
||||
* 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))
|
||||
if (!showAllBackends && is_member_of_role(userId, ROLE_PG_MONITOR))
|
||||
{
|
||||
showAllTransactions = true;
|
||||
showAllBackends = true;
|
||||
}
|
||||
|
||||
/* we're reading all distributed transactions, prevent new backends */
|
||||
LockBackendSharedMemory(LW_SHARED);
|
||||
|
||||
for (int backendIndex = 0; backendIndex < MaxBackends; ++backendIndex)
|
||||
for (int backendIndex = 0; backendIndex < TotalProcCount(); ++backendIndex)
|
||||
{
|
||||
bool showCurrentBackendDetails = showAllBackends;
|
||||
BackendData *currentBackend =
|
||||
&backendManagementShmemData->backends[backendIndex];
|
||||
PGPROC *currentProc = &ProcGlobal->allProcs[backendIndex];
|
||||
|
||||
/* to work on data after releasing g spinlock to protect against errors */
|
||||
int initiatorNodeIdentifier = -1;
|
||||
uint64 transactionNumber = 0;
|
||||
|
||||
SpinLockAcquire(¤tBackend->mutex);
|
||||
|
||||
if (currentBackend->globalPID == INVALID_CITUS_INTERNAL_BACKEND_GPID)
|
||||
if (currentProc->pid == 0)
|
||||
{
|
||||
/* unused PGPROC slot */
|
||||
SpinLockRelease(¤tBackend->mutex);
|
||||
continue;
|
||||
}
|
||||
|
||||
/*
|
||||
* 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(¤tBackend->mutex);
|
||||
continue;
|
||||
showCurrentBackendDetails = true;
|
||||
}
|
||||
|
||||
Oid databaseId = currentBackend->databaseId;
|
||||
int backendPid = ProcGlobal->allProcs[backendIndex].pid;
|
||||
initiatorNodeIdentifier = currentBackend->citusBackend.initiatorNodeIdentifier;
|
||||
|
||||
/*
|
||||
* We prefer to use worker_query instead of distributedCommandOriginator in
|
||||
|
@ -423,13 +412,42 @@ StoreAllActiveTransactions(Tuplestorestate *tupleStore, TupleDesc tupleDescripto
|
|||
|
||||
SpinLockRelease(¤tBackend->mutex);
|
||||
|
||||
values[0] = ObjectIdGetDatum(databaseId);
|
||||
values[1] = Int32GetDatum(backendPid);
|
||||
values[2] = Int32GetDatum(initiatorNodeIdentifier);
|
||||
values[3] = !distributedCommandOriginator;
|
||||
values[4] = UInt64GetDatum(transactionNumber);
|
||||
values[5] = TimestampTzGetDatum(transactionIdTimestamp);
|
||||
values[6] = UInt64GetDatum(currentBackend->globalPID);
|
||||
memset(values, 0, sizeof(values));
|
||||
memset(isNulls, false, sizeof(isNulls));
|
||||
|
||||
/*
|
||||
* We imitate pg_stat_activity such that if a user doesn't have enough
|
||||
* privileges, we only show the minimal information including the pid,
|
||||
* global pid and distributedCommandOriginator.
|
||||
*
|
||||
* pid is already can be found in pg_stat_activity for any process, and
|
||||
* the rest doesn't reveal anything critial for under priviledge users
|
||||
* but still could be useful for monitoring purposes of Citus.
|
||||
*/
|
||||
if (showCurrentBackendDetails)
|
||||
{
|
||||
bool missingOk = true;
|
||||
int initiatorNodeId =
|
||||
ExtractNodeIdFromGlobalPID(currentBackend->globalPID, missingOk);
|
||||
|
||||
values[0] = ObjectIdGetDatum(databaseId);
|
||||
values[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);
|
||||
|
||||
|
@ -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
|
||||
* from Postgres and sets up the shared memory startup hook.
|
||||
|
@ -504,7 +551,7 @@ BackendManagementShmemInit(void)
|
|||
pg_atomic_init_u64(&backendManagementShmemData->nextTransactionNumber, 1);
|
||||
|
||||
/* 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
|
||||
|
@ -520,7 +567,6 @@ BackendManagementShmemInit(void)
|
|||
{
|
||||
BackendData *backendData =
|
||||
&backendManagementShmemData->backends[backendIndex];
|
||||
backendData->citusBackend.initiatorNodeIdentifier = -1;
|
||||
SpinLockInit(&backendData->mutex);
|
||||
}
|
||||
}
|
||||
|
@ -652,16 +698,12 @@ UnSetDistributedTransactionId(void)
|
|||
{
|
||||
SpinLockAcquire(&MyBackendData->mutex);
|
||||
|
||||
MyBackendData->databaseId = 0;
|
||||
MyBackendData->userId = 0;
|
||||
MyBackendData->cancelledDueToDeadlock = false;
|
||||
MyBackendData->transactionId.initiatorNodeIdentifier = 0;
|
||||
MyBackendData->transactionId.transactionOriginator = false;
|
||||
MyBackendData->transactionId.transactionNumber = 0;
|
||||
MyBackendData->transactionId.timestamp = 0;
|
||||
|
||||
MyBackendData->citusBackend.initiatorNodeIdentifier = -1;
|
||||
|
||||
SpinLockRelease(&MyBackendData->mutex);
|
||||
}
|
||||
}
|
||||
|
@ -679,6 +721,9 @@ UnSetGlobalPID(void)
|
|||
SpinLockAcquire(&MyBackendData->mutex);
|
||||
|
||||
MyBackendData->globalPID = 0;
|
||||
MyBackendData->databaseId = 0;
|
||||
MyBackendData->userId = 0;
|
||||
MyBackendData->distributedCommandOriginator = false;
|
||||
|
||||
SpinLockRelease(&MyBackendData->mutex);
|
||||
}
|
||||
|
@ -760,41 +805,14 @@ AssignDistributedTransactionId(void)
|
|||
uint64 nextTransactionNumber = pg_atomic_fetch_add_u64(transactionNumberSequence, 1);
|
||||
int32 localGroupId = GetLocalGroupId();
|
||||
TimestampTz currentTimestamp = GetCurrentTimestamp();
|
||||
Oid userId = GetUserId();
|
||||
|
||||
SpinLockAcquire(&MyBackendData->mutex);
|
||||
|
||||
MyBackendData->databaseId = MyDatabaseId;
|
||||
MyBackendData->userId = userId;
|
||||
|
||||
MyBackendData->transactionId.initiatorNodeIdentifier = localGroupId;
|
||||
MyBackendData->transactionId.transactionOriginator = true;
|
||||
MyBackendData->transactionId.transactionNumber = nextTransactionNumber;
|
||||
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);
|
||||
}
|
||||
|
||||
|
@ -821,20 +839,30 @@ AssignGlobalPID(void)
|
|||
globalPID = ExtractGlobalPID(application_name);
|
||||
}
|
||||
|
||||
Oid userId = GetUserId();
|
||||
|
||||
SpinLockAcquire(&MyBackendData->mutex);
|
||||
|
||||
MyBackendData->globalPID = globalPID;
|
||||
MyBackendData->distributedCommandOriginator = distributedCommandOriginator;
|
||||
MyBackendData->databaseId = MyDatabaseId;
|
||||
MyBackendData->userId = userId;
|
||||
|
||||
SpinLockRelease(&MyBackendData->mutex);
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* OverrideBackendDataDistributedCommandOriginator should only be used for isolation testing.
|
||||
* See how it is used in the relevant functions.
|
||||
* SetBackendDataDistributedCommandOriginator is used to set the distributedCommandOriginator
|
||||
* field on MyBackendData.
|
||||
*/
|
||||
void
|
||||
OverrideBackendDataDistributedCommandOriginator(bool distributedCommandOriginator)
|
||||
SetBackendDataDistributedCommandOriginator(bool distributedCommandOriginator)
|
||||
{
|
||||
if (!MyBackendData)
|
||||
{
|
||||
return;
|
||||
}
|
||||
SpinLockAcquire(&MyBackendData->mutex);
|
||||
MyBackendData->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
|
||||
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.
|
||||
|
@ -876,7 +921,60 @@ GenerateGlobalPID(void)
|
|||
* 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.
|
||||
*/
|
||||
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.
|
||||
*/
|
||||
int
|
||||
ExtractNodeIdFromGlobalPID(uint64 globalPID)
|
||||
ExtractNodeIdFromGlobalPID(uint64 globalPID, bool missingOk)
|
||||
{
|
||||
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 "
|
||||
"%lu is not in Citus' metadata", globalPID),
|
||||
|
@ -1145,36 +1244,37 @@ GetMyProcLocalTransactionId(void)
|
|||
|
||||
|
||||
/*
|
||||
* GetAllActiveClientBackendCount returns activeClientBackendCounter in
|
||||
* GetExternalClientBackendCount returns externalClientBackendCounter in
|
||||
* the shared memory.
|
||||
*/
|
||||
int
|
||||
GetAllActiveClientBackendCount(void)
|
||||
GetExternalClientBackendCount(void)
|
||||
{
|
||||
uint32 activeBackendCount =
|
||||
pg_atomic_read_u32(&backendManagementShmemData->activeClientBackendCounter);
|
||||
pg_atomic_read_u32(&backendManagementShmemData->externalClientBackendCounter);
|
||||
|
||||
return activeBackendCount;
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* IncrementClientBackendCounter increments activeClientBackendCounter in
|
||||
* IncrementExternalClientBackendCounter increments externalClientBackendCounter in
|
||||
* the shared memory by one.
|
||||
*/
|
||||
void
|
||||
IncrementClientBackendCounter(void)
|
||||
uint32
|
||||
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.
|
||||
*/
|
||||
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
|
@ -583,6 +583,25 @@ CreateColocationGroup(int shardCount, int replicationFactor, Oid distributionCol
|
|||
Oid distributionColumnCollation)
|
||||
{
|
||||
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];
|
||||
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 */
|
||||
CommandCounterIncrement();
|
||||
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
|
||||
DeleteColocationGroup(uint32 colocationId)
|
||||
{
|
||||
DeleteColocationGroupLocally(colocationId);
|
||||
SyncDeleteColocationGroupToNodes(colocationId);
|
||||
}
|
||||
|
||||
|
||||
/*
|
||||
* DeleteColocationGroupLocally deletes the colocation group from pg_dist_colocation.
|
||||
*/
|
||||
void
|
||||
DeleteColocationGroupLocally(uint32 colocationId)
|
||||
{
|
||||
int scanKeyCount = 1;
|
||||
ScanKeyData scanKey[1];
|
||||
|
|
|
@ -29,6 +29,7 @@
|
|||
#include "distributed/coordinator_protocol.h"
|
||||
#include "distributed/metadata_cache.h"
|
||||
#include "distributed/metadata/distobject.h"
|
||||
#include "distributed/multi_partitioning_utils.h"
|
||||
#include "foreign/foreign.h"
|
||||
#include "utils/builtins.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_sequence_dependency);
|
||||
|
||||
|
||||
static void WorkerDropDistributedTable(Oid relationId);
|
||||
#if PG_VERSION_NUM < PG_VERSION_13
|
||||
static long deleteDependencyRecordsForSpecific(Oid classId, Oid objectId, char deptype,
|
||||
Oid refclassId, Oid refobjectId);
|
||||
#endif
|
||||
|
||||
|
||||
/*
|
||||
* 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
|
||||
|
@ -64,8 +66,6 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS)
|
|||
text *relationName = PG_GETARG_TEXT_P(0);
|
||||
Oid relationId = ResolveRelationId(relationName, true);
|
||||
|
||||
ObjectAddress distributedTableObject = { InvalidOid, InvalidOid, 0 };
|
||||
|
||||
if (!OidIsValid(relationId))
|
||||
{
|
||||
ereport(NOTICE, (errmsg("relation %s does not exist, skipping",
|
||||
|
@ -75,8 +75,45 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS)
|
|||
|
||||
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 */
|
||||
Relation distributedRelation = relation_open(relationId, AccessShareLock);
|
||||
|
||||
|
@ -86,9 +123,7 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS)
|
|||
relation_close(distributedRelation, AccessShareLock);
|
||||
|
||||
/* prepare distributedTableObject for dropping the table */
|
||||
distributedTableObject.classId = RelationRelationId;
|
||||
distributedTableObject.objectId = relationId;
|
||||
distributedTableObject.objectSubId = 0;
|
||||
ObjectAddress distributedTableObject = { RelationRelationId, relationId, 0 };
|
||||
|
||||
/* Drop dependent sequences from pg_dist_object */
|
||||
#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 */
|
||||
List *shardList = LoadShardList(relationId);
|
||||
uint64 *shardIdPointer = NULL;
|
||||
foreach_ptr(shardIdPointer, shardList)
|
||||
{
|
||||
|
@ -140,8 +176,6 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS)
|
|||
|
||||
/* delete the row from pg_dist_partition */
|
||||
DeletePartitionRow(relationId);
|
||||
|
||||
PG_RETURN_VOID();
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -15,6 +15,7 @@ extern bool EnableCostBasedConnectionEstablishment;
|
|||
extern bool PreventIncompleteConnectionEstablishment;
|
||||
|
||||
extern bool ShouldRunTasksSequentially(List *taskList);
|
||||
extern uint64 ExecuteTaskList(RowModifyLevel modLevel, List *taskList);
|
||||
extern uint64 ExecuteUtilityTaskList(List *utilityTaskList, bool localExecutionSupported);
|
||||
extern uint64 ExecuteUtilityTaskListExtended(List *utilityTaskList, int poolSize,
|
||||
bool localExecutionSupported);
|
||||
|
|
|
@ -22,16 +22,6 @@
|
|||
#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
|
||||
* BackendData in shared memory.
|
||||
|
@ -51,7 +41,6 @@ typedef struct BackendData
|
|||
bool cancelledDueToDeadlock;
|
||||
uint64 globalPID;
|
||||
bool distributedCommandOriginator;
|
||||
CitusInitiatedBackend citusBackend;
|
||||
DistributedTransactionId transactionId;
|
||||
} BackendData;
|
||||
|
||||
|
@ -64,13 +53,12 @@ extern void UnlockBackendSharedMemory(void);
|
|||
extern void UnSetDistributedTransactionId(void);
|
||||
extern void UnSetGlobalPID(void);
|
||||
extern void AssignDistributedTransactionId(void);
|
||||
extern void MarkCitusInitiatedCoordinatorBackend(void);
|
||||
extern void AssignGlobalPID(void);
|
||||
extern uint64 GetGlobalPID(void);
|
||||
extern void OverrideBackendDataDistributedCommandOriginator(bool
|
||||
distributedCommandOriginator);
|
||||
extern void SetBackendDataDistributedCommandOriginator(bool
|
||||
distributedCommandOriginator);
|
||||
extern uint64 ExtractGlobalPID(char *applicationName);
|
||||
extern int ExtractNodeIdFromGlobalPID(uint64 globalPID);
|
||||
extern int ExtractNodeIdFromGlobalPID(uint64 globalPID, bool missingOk);
|
||||
extern int ExtractProcessIdFromGlobalPID(uint64 globalPID);
|
||||
extern void GetBackendDataForProc(PGPROC *proc, BackendData *result);
|
||||
extern void CancelTransactionDueToDeadlock(PGPROC *proc);
|
||||
|
@ -78,9 +66,9 @@ extern bool MyBackendGotCancelledDueToDeadlock(bool clearState);
|
|||
extern bool MyBackendIsInDisributedTransaction(void);
|
||||
extern List * ActiveDistributedTransactionNumbers(void);
|
||||
extern LocalTransactionId GetMyProcLocalTransactionId(void);
|
||||
extern int GetAllActiveClientBackendCount(void);
|
||||
extern void IncrementClientBackendCounter(void);
|
||||
extern void DecrementClientBackendCounter(void);
|
||||
extern int GetExternalClientBackendCount(void);
|
||||
extern uint32 IncrementExternalClientBackendCounter(void);
|
||||
extern void DecrementExternalClientBackendCounter(void);
|
||||
|
||||
extern bool ExecuteRemoteQueryOrCommand(char *nodeName, uint32 nodePort,
|
||||
char *queryString, StringInfo queryResultString,
|
||||
|
|
|
@ -31,6 +31,10 @@ uint32 ColocationId(int shardCount, int replicationFactor, Oid distributionColum
|
|||
extern uint32 CreateColocationGroup(int shardCount, int replicationFactor,
|
||||
Oid distributionColumnType,
|
||||
Oid distributionColumnCollation);
|
||||
extern void InsertColocationGroupLocally(uint32 colocationId, int shardCount,
|
||||
int replicationFactor,
|
||||
Oid distributionColumnType,
|
||||
Oid distributionColumnCollation);
|
||||
extern bool IsColocateWithNone(char *colocateWithTableName);
|
||||
extern uint32 GetNextColocationId(void);
|
||||
extern void ErrorIfShardPlacementsNotColocated(Oid leftRelationId, Oid rightRelationId);
|
||||
|
@ -43,5 +47,6 @@ extern void UpdateRelationColocationGroup(Oid distributedRelationId, uint32 colo
|
|||
bool localOnly);
|
||||
extern void DeleteColocationGroupIfNoTablesBelong(uint32 colocationId);
|
||||
extern List * ColocationGroupTableList(uint32 colocationId, uint32 count);
|
||||
extern void DeleteColocationGroupLocally(uint32 colocationId);
|
||||
|
||||
#endif /* COLOCATION_UTILS_H_ */
|
||||
|
|
|
@ -122,6 +122,9 @@ typedef enum SearchForeignKeyColumnFlags
|
|||
} SearchForeignKeyColumnFlags;
|
||||
|
||||
|
||||
/* aggregate.c - forward declarations */
|
||||
extern List * PostprocessDefineAggregateStmt(Node *node, const char *queryString);
|
||||
|
||||
/* cluster.c - forward declarations */
|
||||
extern List * PreprocessClusterStmt(Node *node, const char *clusterCommand,
|
||||
ProcessUtilityContext processUtilityContext);
|
||||
|
@ -264,6 +267,7 @@ extern List * PreprocessCreateFunctionStmt(Node *stmt, const char *queryString,
|
|||
ProcessUtilityContext processUtilityContext);
|
||||
extern List * PostprocessCreateFunctionStmt(Node *stmt,
|
||||
const char *queryString);
|
||||
extern ObjectAddress * GetUndistributableDependency(ObjectAddress *functionAddress);
|
||||
extern ObjectAddress CreateFunctionStmtObjectAddress(Node *stmt,
|
||||
bool missing_ok);
|
||||
extern ObjectAddress DefineAggregateStmtObjectAddress(Node *stmt,
|
||||
|
|
|
@ -22,6 +22,13 @@
|
|||
#include "distributed/version_compat.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
|
||||
{
|
||||
PROPSETCMD_INVALID = -1,
|
||||
|
@ -32,6 +39,7 @@ typedef enum
|
|||
} PropSetCmdBehavior;
|
||||
extern PropSetCmdBehavior PropagateSetCommands;
|
||||
extern bool EnableDDLPropagation;
|
||||
extern int CreateObjectPropagationMode;
|
||||
extern bool EnableCreateTypePropagation;
|
||||
extern bool EnableAlterRolePropagation;
|
||||
extern bool EnableAlterRoleSetPropagation;
|
||||
|
|
|
@ -33,6 +33,9 @@ extern void QualifyTreeNode(Node *stmt);
|
|||
extern char * DeparseTreeNode(Node *stmt);
|
||||
extern List * DeparseTreeNodes(List *stmts);
|
||||
|
||||
/* forward declarations for qualify_aggregate_stmts.c */
|
||||
extern void QualifyDefineAggregateStmt(Node *node);
|
||||
|
||||
/* forward declarations for deparse_attribute_stmts.c */
|
||||
extern char * DeparseRenameAttributeStmt(Node *);
|
||||
|
||||
|
|
|
@ -48,6 +48,28 @@ typedef struct 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 */
|
||||
extern DestReceiver * CreateRemoteFileDestReceiver(const char *resultId,
|
||||
EState *executorState,
|
||||
|
@ -72,5 +94,10 @@ extern List * PartitionTasklistResults(const char *resultIdPrefix, List *selectT
|
|||
int partitionColumnIndex,
|
||||
CitusTableCacheEntry *distributionScheme,
|
||||
bool binaryFormat);
|
||||
extern char * QueryStringForFragmentsTransfer(
|
||||
NodeToNodeFragmentsTransfer *fragmentsTransfer);
|
||||
extern void ShardMinMaxValueArrays(ShardInterval **shardIntervalArray, int shardCount,
|
||||
Oid intervalTypeId, ArrayType **minValueArray,
|
||||
ArrayType **maxValueArray);
|
||||
|
||||
#endif /* INTERMEDIATE_RESULTS_H */
|
||||
|
|
|
@ -43,6 +43,7 @@ extern void ExecuteUtilityCommand(const char *utilityCommand);
|
|||
extern bool ShouldExecuteTasksLocally(List *taskList);
|
||||
extern bool AnyTaskAccessesLocalNode(List *taskList);
|
||||
extern bool TaskAccessesLocalNode(Task *task);
|
||||
extern void EnsureCompatibleLocalExecutionState(List *taskList);
|
||||
extern void ErrorIfTransactionAccessedPlacementsLocally(void);
|
||||
extern void DisableLocalExecution(void);
|
||||
extern void SetLocalExecutionStatus(LocalExecutionStatus newStatus);
|
||||
|
|
|
@ -21,6 +21,8 @@ extern List * GetUniqueDependenciesList(List *objectAddressesList);
|
|||
extern List * GetDependenciesForObject(const ObjectAddress *target);
|
||||
extern List * GetAllSupportedDependenciesForObject(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 bool SupportedDependencyByCitus(const ObjectAddress *address);
|
||||
extern List * GetPgDependTuplesForDependingObjects(Oid targetObjectClassId,
|
||||
|
|
|
@ -38,6 +38,7 @@ extern bool ShouldSyncTableMetadata(Oid relationId);
|
|||
extern bool ShouldSyncTableMetadataViaCatalog(Oid relationId);
|
||||
extern List * NodeMetadataCreateCommands(void);
|
||||
extern List * DistributedObjectMetadataSyncCommandList(void);
|
||||
extern List * ColocationGroupCreateCommandList(void);
|
||||
extern List * CitusTableMetadataCreateCommandList(Oid relationId);
|
||||
extern List * NodeMetadataDropCommands(void);
|
||||
extern char * MarkObjectsDistributedCreateCommand(List *addresses,
|
||||
|
@ -73,13 +74,21 @@ extern List * GetSequencesFromAttrDef(Oid attrdefOid);
|
|||
extern void GetDependentSequencesWithRelation(Oid relationId, List **attnumList,
|
||||
List **dependentSequenceList, AttrNumber
|
||||
attnum);
|
||||
extern List * GetDependentFunctionsWithRelation(Oid relationId);
|
||||
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_PLACEMENTS "DELETE FROM pg_dist_placement"
|
||||
#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_COLOCATION "DELETE FROM pg_catalog.pg_dist_colocation"
|
||||
#define REMOVE_ALL_SHELL_TABLES_COMMAND \
|
||||
"SELECT worker_drop_shell_table(logicalrelid::regclass::text) FROM pg_dist_partition"
|
||||
#define REMOVE_ALL_CITUS_TABLES_COMMAND \
|
||||
|
|
|
@ -209,6 +209,9 @@ extern ShardInterval * CopyShardInterval(ShardInterval *srcInterval);
|
|||
extern uint64 ShardLength(uint64 shardId);
|
||||
extern bool NodeGroupHasShardPlacements(int32 groupId,
|
||||
bool onlyConsiderActivePlacements);
|
||||
extern bool IsActiveShardPlacement(ShardPlacement *ShardPlacement);
|
||||
extern List * FilterShardPlacementList(List *shardPlacementList, bool (*filter)(
|
||||
ShardPlacement *));
|
||||
extern List * ActiveShardPlacementListOnGroup(uint64 shardId, int32 groupId);
|
||||
extern List * ActiveShardPlacementList(uint64 shardId);
|
||||
extern List * ShardPlacementListWithoutOrphanedPlacements(uint64 shardId);
|
||||
|
@ -248,6 +251,7 @@ extern TableConversionReturn * UndistributeTable(TableConversionParameters *para
|
|||
extern void EnsureDependenciesExistOnAllNodes(const ObjectAddress *target);
|
||||
extern List * GetDistributableDependenciesForObject(const ObjectAddress *target);
|
||||
extern bool ShouldPropagate(void);
|
||||
extern bool ShouldPropagateCreateInCoordinatedTransction(void);
|
||||
extern bool ShouldPropagateObject(const ObjectAddress *address);
|
||||
extern List * ReplicateAllObjectsToNodeCommandList(const char *nodeName, int nodePort);
|
||||
|
||||
|
|
|
@ -36,15 +36,6 @@
|
|||
/* Definitions local to the physical planner */
|
||||
#define NON_PRUNABLE_JOIN -1
|
||||
#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;
|
||||
|
||||
|
@ -262,6 +253,10 @@ typedef struct Task
|
|||
uint32 upstreamTaskId; /* only applies to data fetch tasks */
|
||||
ShardInterval *shardInterval; /* 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 */
|
||||
|
||||
/*
|
||||
|
|
|
@ -13,8 +13,6 @@
|
|||
#include "nodes/pg_list.h"
|
||||
|
||||
extern List * ExecuteDependentTasks(List *taskList, Job *topLevelJob);
|
||||
extern void EnsureCompatibleLocalExecutionState(List *taskList);
|
||||
extern void DoRepartitionCleanup(List *jobIds);
|
||||
|
||||
|
||||
#endif /* REPARTITION_JOIN_EXECUTION_H */
|
||||
|
|
|
@ -182,7 +182,8 @@ extern Datum init_rebalance_monitor(PG_FUNCTION_ARGS);
|
|||
extern Datum finalize_rebalance_monitor(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,
|
||||
int32 maxShardMoves,
|
||||
bool drainOnly,
|
||||
|
|
|
@ -14,15 +14,18 @@
|
|||
#define ADJUST_POOLSIZE_AUTOMATICALLY 0
|
||||
#define DISABLE_CONNECTION_THROTTLING -1
|
||||
#define DISABLE_REMOTE_CONNECTIONS_FOR_LOCAL_QUERIES -1
|
||||
#define ALLOW_ALL_EXTERNAL_CONNECTIONS -1
|
||||
|
||||
|
||||
extern int MaxSharedPoolSize;
|
||||
extern int LocalSharedPoolSize;
|
||||
extern int MaxClientConnections;
|
||||
|
||||
|
||||
extern void InitializeSharedConnectionStats(void);
|
||||
extern void WaitForSharedConnection(void);
|
||||
extern void WakeupWaiterBackendsForSharedConnection(void);
|
||||
extern int GetMaxClientConnections(void);
|
||||
extern int GetMaxSharedPoolSize(void);
|
||||
extern int GetLocalSharedPoolSize(void);
|
||||
extern bool TryToIncrementSharedConnectionCounter(const char *hostname, int port);
|
||||
|
|
|
@ -86,7 +86,7 @@ extern List * ActiveReadableNodeList(void);
|
|||
extern WorkerNode * FindWorkerNode(const char *nodeName, int32 nodePort);
|
||||
extern WorkerNode * FindWorkerNodeOrError(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 void EnsureCoordinator(void);
|
||||
extern void InsertCoordinatorIfClusterEmpty(void);
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue