Merge branch 'master' into update-contributing-md

pull/5695/head
gledis69 2022-02-11 14:55:11 +03:00
commit 98f7c6bc49
232 changed files with 5303 additions and 3507 deletions

View File

@ -1,3 +1,32 @@
### citus v10.2.4 (February 1, 2022) ###
* Adds support for operator class parameters in indexes
* Fixes a bug with distributed functions that have `OUT` parameters or
return `TABLE`
* Fixes a build error that happens when `lz4` is not installed
* Improves self-deadlock prevention for `CREATE INDEX` &
`REINDEX CONCURRENTLY` commands for builds using PG14 or higher
* Fixes a bug that causes commands to fail when `application_name` is set
### citus v10.1.4 (February 1, 2022) ###
* Adds missing version checks for columnar tables
* Fixes a bug that could break `DROP SCHEMA/EXTENSION` commands when there is
a columnar table
* Fixes a build error that happens when `lz4` is not installed
* Fixes a missing `FROM` clause entry error
* Reinstates optimisation for uniform shard interval ranges
* Fixes a bug that causes commands to fail when `application_name` is set
### citus v10.2.3 (November 29, 2021) ###
* Adds `fix_partition_shard_index_names` udf to fix currently broken

View File

@ -186,17 +186,17 @@ ColumnarStorageInit(SMgrRelation srel, uint64 storageId)
(char *) &metapage, sizeof(ColumnarMetapage));
phdr->pd_lower += sizeof(ColumnarMetapage);
PageSetChecksumInplace(page, COLUMNAR_METAPAGE_BLOCKNO);
log_newpage(&srel->smgr_rnode.node, MAIN_FORKNUM,
COLUMNAR_METAPAGE_BLOCKNO, page, true);
PageSetChecksumInplace(page, COLUMNAR_METAPAGE_BLOCKNO);
smgrextend(srel, MAIN_FORKNUM, COLUMNAR_METAPAGE_BLOCKNO, page, true);
/* write empty page */
PageInit(page, BLCKSZ, 0);
PageSetChecksumInplace(page, COLUMNAR_EMPTY_BLOCKNO);
log_newpage(&srel->smgr_rnode.node, MAIN_FORKNUM,
COLUMNAR_EMPTY_BLOCKNO, page, true);
PageSetChecksumInplace(page, COLUMNAR_EMPTY_BLOCKNO);
smgrextend(srel, MAIN_FORKNUM, COLUMNAR_EMPTY_BLOCKNO, page, true);
/*

View File

@ -1913,13 +1913,6 @@ columnar_tableam_init()
}
void
columnar_tableam_finish()
{
object_access_hook = PrevObjectAccessHook;
}
/*
* Get the number of chunks filtered out during the given scan.
*/
@ -2334,7 +2327,11 @@ alter_columnar_table_set(PG_FUNCTION_ARGS)
quote_identifier(RelationGetRelationName(rel)))));
}
EnsureTableOwner(relationId);
if (!pg_class_ownercheck(relationId, GetUserId()))
{
aclcheck_error(ACLCHECK_NOT_OWNER, OBJECT_TABLE,
get_rel_name(relationId));
}
ColumnarOptions options = { 0 };
if (!ReadColumnarOptions(relationId, &options))
@ -2454,7 +2451,11 @@ alter_columnar_table_reset(PG_FUNCTION_ARGS)
quote_identifier(RelationGetRelationName(rel)))));
}
EnsureTableOwner(relationId);
if (!pg_class_ownercheck(relationId, GetUserId()))
{
aclcheck_error(ACLCHECK_NOT_OWNER, OBJECT_TABLE,
get_rel_name(relationId));
}
ColumnarOptions options = { 0 };
if (!ReadColumnarOptions(relationId, &options))

View File

@ -28,10 +28,3 @@ columnar_init(void)
columnar_init_gucs();
columnar_tableam_init();
}
void
columnar_fini(void)
{
columnar_tableam_finish();
}

View File

@ -68,7 +68,7 @@ CallDistributedProcedureRemotely(CallStmt *callStmt, DestReceiver *dest)
return false;
}
if (IsCitusInitiatedRemoteBackend())
if (IsCitusInternalBackend())
{
/*
* We are in a citus-initiated backend handling a CALL to a distributed

View File

@ -30,6 +30,7 @@
#include "distributed/commands.h"
#include "distributed/commands/sequence.h"
#include "distributed/commands/utility_hook.h"
#include "distributed/metadata/distobject.h"
#include "distributed/foreign_key_relationship.h"
#include "distributed/listutils.h"
#include "distributed/local_executor.h"
@ -90,7 +91,7 @@ static void TransferSequenceOwnership(Oid ownedSequenceId, Oid targetRelationId,
char *columnName);
static void InsertMetadataForCitusLocalTable(Oid citusLocalTableId, uint64 shardId,
bool autoConverted);
static void FinalizeCitusLocalTableCreation(Oid relationId, List *dependentSequenceList);
static void FinalizeCitusLocalTableCreation(Oid relationId);
PG_FUNCTION_INFO_V1(citus_add_local_table_to_metadata);
@ -307,7 +308,13 @@ CreateCitusLocalTable(Oid relationId, bool cascadeViaForeignKeys, bool autoConve
ObjectAddressSet(tableAddress, RelationRelationId, relationId);
/*
* Ensure dependencies first as we will create shell table on the other nodes
* Ensure that the sequences used in column defaults of the table
* have proper types
*/
EnsureRelationHasCompatibleSequenceTypes(relationId);
/*
* Ensure dependencies exist as we will create shell table on the other nodes
* in the MX case.
*/
EnsureDependenciesExistOnAllNodes(&tableAddress);
@ -354,18 +361,7 @@ CreateCitusLocalTable(Oid relationId, bool cascadeViaForeignKeys, bool autoConve
InsertMetadataForCitusLocalTable(shellRelationId, shardId, autoConverted);
/*
* Ensure that the sequences used in column defaults of the table
* have proper types
*/
List *attnumList = NIL;
List *dependentSequenceList = NIL;
GetDependentSequencesWithRelation(shellRelationId, &attnumList,
&dependentSequenceList, 0);
EnsureDistributedSequencesHaveOneType(shellRelationId, dependentSequenceList,
attnumList);
FinalizeCitusLocalTableCreation(shellRelationId, dependentSequenceList);
FinalizeCitusLocalTableCreation(shellRelationId);
}
@ -657,8 +653,10 @@ GetShellTableDDLEventsForCitusLocalTable(Oid relationId)
*/
IncludeSequenceDefaults includeSequenceDefaults = NEXTVAL_SEQUENCE_DEFAULTS;
bool creatingShellTableOnRemoteNode = false;
List *tableDDLCommands = GetFullTableCreationCommands(relationId,
includeSequenceDefaults);
includeSequenceDefaults,
creatingShellTableOnRemoteNode);
List *shellTableDDLEvents = NIL;
TableDDLCommand *tableDDLCommand = NULL;
@ -1227,7 +1225,7 @@ InsertMetadataForCitusLocalTable(Oid citusLocalTableId, uint64 shardId,
* sequences dependent with the table.
*/
static void
FinalizeCitusLocalTableCreation(Oid relationId, List *dependentSequenceList)
FinalizeCitusLocalTableCreation(Oid relationId)
{
/*
* If it is a foreign table, then skip creating citus truncate trigger
@ -1240,16 +1238,7 @@ FinalizeCitusLocalTableCreation(Oid relationId, List *dependentSequenceList)
if (ShouldSyncTableMetadata(relationId))
{
if (ClusterHasKnownMetadataWorkers())
{
/*
* Ensure sequence dependencies and mark them as distributed
* before creating table metadata on workers
*/
MarkSequenceListDistributedAndPropagateWithDependencies(relationId,
dependentSequenceList);
}
CreateTableMetadataOnWorkers(relationId);
SyncCitusTableMetadata(relationId);
}
/*

View File

@ -113,8 +113,9 @@ static void EnsureLocalTableEmptyIfNecessary(Oid relationId, char distributionMe
static bool ShouldLocalTableBeEmpty(Oid relationId, char distributionMethod, bool
viaDeprecatedAPI);
static void EnsureCitusTableCanBeCreated(Oid relationOid);
static void EnsureSequenceExistOnMetadataWorkersForRelation(Oid relationId,
Oid sequenceOid);
static void EnsureDistributedSequencesHaveOneType(Oid relationId,
List *dependentSequenceList,
List *attnumList);
static List * GetFKeyCreationCommandsRelationInvolvedWithTableType(Oid relationId,
int tableTypeFlag);
static Oid DropFKeysAndUndistributeTable(Oid relationId);
@ -432,6 +433,12 @@ CreateDistributedTable(Oid relationId, Var *distributionColumn, char distributio
DropFKeysRelationInvolvedWithTableType(relationId, INCLUDE_LOCAL_TABLES);
}
/*
* Ensure that the sequences used in column defaults of the table
* have proper types
*/
EnsureRelationHasCompatibleSequenceTypes(relationId);
/*
* distributed tables might have dependencies on different objects, since we create
* shards for a distributed table via multiple sessions these objects will be created
@ -446,7 +453,6 @@ CreateDistributedTable(Oid relationId, Var *distributionColumn, char distributio
colocateWithTableName,
viaDeprecatedAPI);
/*
* Due to dropping columns, the parent's distribution key may not match the
* partition's distribution key. The input distributionColumn belongs to
@ -494,16 +500,6 @@ CreateDistributedTable(Oid relationId, Var *distributionColumn, char distributio
InsertIntoPgDistPartition(relationId, distributionMethod, distributionColumn,
colocationId, replicationModel, autoConverted);
/*
* Ensure that the sequences used in column defaults of the table
* have proper types
*/
List *attnumList = NIL;
List *dependentSequenceList = NIL;
GetDependentSequencesWithRelation(relationId, &attnumList, &dependentSequenceList, 0);
EnsureDistributedSequencesHaveOneType(relationId, dependentSequenceList,
attnumList);
/* foreign tables do not support TRUNCATE trigger */
if (RegularTable(relationId))
{
@ -537,17 +533,7 @@ CreateDistributedTable(Oid relationId, Var *distributionColumn, char distributio
if (ShouldSyncTableMetadata(relationId))
{
if (ClusterHasKnownMetadataWorkers())
{
/*
* Ensure both sequence and its' dependencies and mark them as distributed
* before creating table metadata on workers
*/
MarkSequenceListDistributedAndPropagateWithDependencies(relationId,
dependentSequenceList);
}
CreateTableMetadataOnWorkers(relationId);
SyncCitusTableMetadata(relationId);
}
/*
@ -605,11 +591,15 @@ CreateDistributedTable(Oid relationId, Var *distributionColumn, char distributio
* If any other distributed table uses the input sequence, it checks whether
* the types of the columns using the sequence match. If they don't, it errors out.
* Otherwise, the condition is ensured.
* Since the owner of the sequence may not distributed yet, it should be added
* explicitly.
*/
void
EnsureSequenceTypeSupported(Oid seqOid, Oid seqTypId)
EnsureSequenceTypeSupported(Oid seqOid, Oid seqTypId, Oid ownerRelationId)
{
List *citusTableIdList = CitusTableTypeIdList(ANY_CITUS_TABLE_TYPE);
citusTableIdList = list_append_unique_oid(citusTableIdList, ownerRelationId);
Oid citusTableId = InvalidOid;
foreach_oid(citusTableId, citusTableIdList)
{
@ -676,59 +666,18 @@ AlterSequenceType(Oid seqOid, Oid typeOid)
/*
* MarkSequenceListDistributedAndPropagateWithDependencies ensures sequences and their
* dependencies for the given sequence list exist on all nodes and marks them as distributed.
* EnsureRelationHasCompatibleSequenceTypes ensures that sequences used for columns
* of the table have compatible types both with the column type on that table and
* all other distributed tables' columns they have used for
*/
void
MarkSequenceListDistributedAndPropagateWithDependencies(Oid relationId,
List *sequenceList)
EnsureRelationHasCompatibleSequenceTypes(Oid relationId)
{
Oid sequenceOid = InvalidOid;
foreach_oid(sequenceOid, sequenceList)
{
MarkSequenceDistributedAndPropagateWithDependencies(relationId, sequenceOid);
}
}
List *attnumList = NIL;
List *dependentSequenceList = NIL;
/*
* MarkSequenceDistributedAndPropagateWithDependencies ensures sequence and its'
* dependencies for the given sequence exist on all nodes and marks them as distributed.
*/
void
MarkSequenceDistributedAndPropagateWithDependencies(Oid relationId, Oid sequenceOid)
{
/* get sequence address */
ObjectAddress sequenceAddress = { 0 };
ObjectAddressSet(sequenceAddress, RelationRelationId, sequenceOid);
EnsureDependenciesExistOnAllNodes(&sequenceAddress);
EnsureSequenceExistOnMetadataWorkersForRelation(relationId, sequenceOid);
MarkObjectDistributed(&sequenceAddress);
}
/*
* EnsureSequenceExistOnMetadataWorkersForRelation ensures sequence for the given relation
* exist on each worker node with metadata.
*/
static void
EnsureSequenceExistOnMetadataWorkersForRelation(Oid relationId, Oid sequenceOid)
{
Assert(ShouldSyncTableMetadata(relationId));
char *ownerName = TableOwner(relationId);
List *sequenceDDLList = DDLCommandsForSequence(sequenceOid, ownerName);
/* prevent recursive propagation */
SendCommandToWorkersWithMetadata(DISABLE_DDL_PROPAGATION);
const char *sequenceCommand = NULL;
foreach_ptr(sequenceCommand, sequenceDDLList)
{
SendCommandToWorkersWithMetadata(sequenceCommand);
}
SendCommandToWorkersWithMetadata(ENABLE_DDL_PROPAGATION);
GetDependentSequencesWithRelation(relationId, &attnumList, &dependentSequenceList, 0);
EnsureDistributedSequencesHaveOneType(relationId, dependentSequenceList, attnumList);
}
@ -737,7 +686,7 @@ EnsureSequenceExistOnMetadataWorkersForRelation(Oid relationId, Oid sequenceOid)
* in which the sequence is used as default is supported for each sequence in input
* dependentSequenceList, and then alters the sequence type if not the same with the column type.
*/
void
static void
EnsureDistributedSequencesHaveOneType(Oid relationId, List *dependentSequenceList,
List *attnumList)
{
@ -753,7 +702,7 @@ EnsureDistributedSequencesHaveOneType(Oid relationId, List *dependentSequenceLis
* that sequence is supported
*/
Oid seqTypId = GetAttributeTypeOid(relationId, attnum);
EnsureSequenceTypeSupported(sequenceOid, seqTypId);
EnsureSequenceTypeSupported(sequenceOid, seqTypId, relationId);
/*
* Alter the sequence's data type in the coordinator if needed.

View File

@ -34,8 +34,6 @@ static List * GetDependencyCreateDDLCommands(const ObjectAddress *dependency);
static List * FilterObjectAddressListByPredicate(List *objectAddressList,
AddressPredicate predicate);
bool EnableDependencyCreation = true;
/*
* EnsureDependenciesExistOnAllNodes finds all the dependencies that we support and makes
* sure these are available on all workers. If not available they will be created on the
@ -122,7 +120,15 @@ EnsureDependenciesExistOnAllNodes(const ObjectAddress *target)
*/
foreach_ptr(dependency, dependenciesWithCommands)
{
MarkObjectDistributed(dependency);
/*
* pg_dist_object entries must be propagated with the super user, since
* the owner of the target object may not own dependencies but we must
* propagate as we send objects itself with the superuser.
*
* Only dependent object's metadata should be propagated with super user.
* Metadata of the table itself must be propagated with the current user.
*/
MarkObjectDistributedViaSuperUser(dependency);
}
}
@ -224,15 +230,47 @@ GetDependencyCreateDDLCommands(const ObjectAddress *dependency)
{
case OCLASS_CLASS:
{
char relKind = get_rel_relkind(dependency->objectId);
/*
* types have an intermediate dependency on a relation (aka class), so we do
* support classes when the relkind is composite
*/
if (get_rel_relkind(dependency->objectId) == RELKIND_COMPOSITE_TYPE)
if (relKind == RELKIND_COMPOSITE_TYPE)
{
return NIL;
}
if (relKind == RELKIND_RELATION || relKind == RELKIND_PARTITIONED_TABLE ||
relKind == RELKIND_FOREIGN_TABLE)
{
Oid relationId = dependency->objectId;
List *commandList = NIL;
if (IsCitusTable(relationId))
{
bool creatingShellTableOnRemoteNode = true;
List *tableDDLCommands = GetFullTableCreationCommands(relationId,
WORKER_NEXTVAL_SEQUENCE_DEFAULTS,
creatingShellTableOnRemoteNode);
TableDDLCommand *tableDDLCommand = NULL;
foreach_ptr(tableDDLCommand, tableDDLCommands)
{
Assert(CitusIsA(tableDDLCommand, TableDDLCommand));
commandList = lappend(commandList, GetTableDDLCommand(
tableDDLCommand));
}
}
return commandList;
}
if (relKind == RELKIND_SEQUENCE)
{
char *sequenceOwnerName = TableOwner(dependency->objectId);
return DDLCommandsForSequence(dependency->objectId, sequenceOwnerName);
}
/* if this relation is not supported, break to the error at the end */
break;
}
@ -316,14 +354,15 @@ GetDependencyCreateDDLCommands(const ObjectAddress *dependency)
/*
* ReplicateAllDependenciesToNode replicate all previously marked objects to a worker
* node. The function also sets clusterHasDistributedFunction if there are any
* distributed functions.
* ReplicateAllObjectsToNodeCommandList returns commands to replicate all
* previously marked objects to a worker node. The function also sets
* clusterHasDistributedFunction if there are any distributed functions.
*/
void
ReplicateAllDependenciesToNode(const char *nodeName, int nodePort)
List *
ReplicateAllObjectsToNodeCommandList(const char *nodeName, int nodePort)
{
List *ddlCommands = NIL;
/* since we are executing ddl commands disable propagation first, primarily for mx */
List *ddlCommands = list_make1(DISABLE_DDL_PROPAGATION);
/*
* collect all dependencies in creation order and get their ddl commands
@ -331,7 +370,7 @@ ReplicateAllDependenciesToNode(const char *nodeName, int nodePort)
List *dependencies = GetDistributedObjectAddressList();
/*
* Depending on changes in the environment, such as the enable_object_propagation guc
* Depending on changes in the environment, such as the enable_metadata_sync guc
* there might be objects in the distributed object address list that should currently
* not be propagated by citus as they are 'not supported'.
*/
@ -357,24 +396,22 @@ ReplicateAllDependenciesToNode(const char *nodeName, int nodePort)
ObjectAddress *dependency = NULL;
foreach_ptr(dependency, dependencies)
{
if (IsObjectAddressOwnedByExtension(dependency, NULL))
{
/*
* we expect extension-owned objects to be created as a result
* of the extension being created.
*/
continue;
}
ddlCommands = list_concat(ddlCommands,
GetDependencyCreateDDLCommands(dependency));
}
if (list_length(ddlCommands) <= 0)
{
/* no commands to replicate dependencies to the new worker */
return;
}
/* since we are executing ddl commands lets disable propagation, primarily for mx */
ddlCommands = list_concat(list_make1(DISABLE_DDL_PROPAGATION), ddlCommands);
ddlCommands = lappend(ddlCommands, ENABLE_DDL_PROPAGATION);
/* send commands to new workers, the current user should a superuser */
Assert(superuser());
SendMetadataCommandListToWorkerInCoordinatedTransaction(nodeName,
nodePort,
CurrentUserName(),
ddlCommands);
return ddlCommands;
}
@ -393,7 +430,7 @@ ShouldPropagate(void)
return false;
}
if (!EnableDependencyCreation)
if (!EnableMetadataSync)
{
/*
* we are configured to disable object propagation, should not propagate anything

View File

@ -193,6 +193,14 @@ static DistributeObjectOps Any_CreateForeignServer = {
.address = CreateForeignServerStmtObjectAddress,
.markDistributed = true,
};
static DistributeObjectOps Any_CreateSchema = {
.deparse = DeparseCreateSchemaStmt,
.qualify = NULL,
.preprocess = PreprocessCreateSchemaStmt,
.postprocess = NULL,
.address = CreateSchemaStmtObjectAddress,
.markDistributed = true,
};
static DistributeObjectOps Any_CreateStatistics = {
.deparse = DeparseCreateStatisticsStmt,
.qualify = QualifyCreateStatisticsStmt,
@ -538,7 +546,7 @@ static DistributeObjectOps Routine_Rename = {
.markDistributed = false,
};
static DistributeObjectOps Schema_Drop = {
.deparse = NULL,
.deparse = DeparseDropSchemaStmt,
.qualify = NULL,
.preprocess = PreprocessDropSchemaStmt,
.postprocess = NULL,
@ -976,6 +984,11 @@ GetDistributeObjectOps(Node *node)
return &Any_CreatePolicy;
}
case T_CreateSchemaStmt:
{
return &Any_CreateSchema;
}
case T_CreateStatsStmt:
{
return &Any_CreateStatistics;

View File

@ -649,7 +649,7 @@ static bool
ShouldPropagateExtensionCommand(Node *parseTree)
{
/* if we disabled object propagation, then we should not propagate anything. */
if (!EnableDependencyCreation)
if (!EnableMetadataSync)
{
return false;
}

View File

@ -85,7 +85,6 @@ static ObjectAddress FunctionToObjectAddress(ObjectType objectType,
ObjectWithArgs *objectWithArgs,
bool missing_ok);
static void ErrorIfUnsupportedAlterFunctionStmt(AlterFunctionStmt *stmt);
static void ErrorIfFunctionDependsOnExtension(const ObjectAddress *functionAddress);
static char * quote_qualified_func_name(Oid funcOid);
static void DistributeFunctionWithDistributionArgument(RegProcedure funcOid,
char *distributionArgumentName,
@ -101,6 +100,9 @@ static void DistributeFunctionColocatedWithDistributedTable(RegProcedure funcOid
static void DistributeFunctionColocatedWithReferenceTable(const
ObjectAddress *functionAddress);
static void EnsureExtensionFunctionCanBeDistributed(const ObjectAddress functionAddress,
const ObjectAddress extensionAddress,
char *distributionArgumentName);
PG_FUNCTION_INFO_V1(create_distributed_function);
@ -127,6 +129,7 @@ create_distributed_function(PG_FUNCTION_ARGS)
char *colocateWithTableName = NULL;
bool *forceDelegationAddress = NULL;
bool forceDelegation = false;
ObjectAddress extensionAddress = { 0 };
/* if called on NULL input, error out */
if (funcOid == InvalidOid)
@ -187,22 +190,35 @@ create_distributed_function(PG_FUNCTION_ARGS)
EnsureFunctionOwner(funcOid);
ObjectAddressSet(functionAddress, ProcedureRelationId, funcOid);
ErrorIfFunctionDependsOnExtension(&functionAddress);
/*
* when we allow propagation within a transaction block we should make sure to only
* allow this in sequential mode
* If the function is owned by an extension, only update the
* pg_dist_object, and not propagate the CREATE FUNCTION. Function
* will be created by the virtue of the extension creation.
*/
EnsureSequentialModeForFunctionDDL();
if (IsObjectAddressOwnedByExtension(&functionAddress, &extensionAddress))
{
EnsureExtensionFunctionCanBeDistributed(functionAddress, extensionAddress,
distributionArgumentName);
}
else
{
/*
* when we allow propagation within a transaction block we should make sure
* to only allow this in sequential mode.
*/
EnsureSequentialModeForFunctionDDL();
EnsureDependenciesExistOnAllNodes(&functionAddress);
EnsureDependenciesExistOnAllNodes(&functionAddress);
const char *createFunctionSQL = GetFunctionDDLCommand(funcOid, true);
const char *alterFunctionOwnerSQL = GetFunctionAlterOwnerCommand(funcOid);
initStringInfo(&ddlCommand);
appendStringInfo(&ddlCommand, "%s;%s;%s;%s", DISABLE_OBJECT_PROPAGATION,
createFunctionSQL, alterFunctionOwnerSQL, ENABLE_OBJECT_PROPAGATION);
SendCommandToWorkersAsUser(NON_COORDINATOR_NODES, CurrentUserName(), ddlCommand.data);
const char *createFunctionSQL = GetFunctionDDLCommand(funcOid, true);
const char *alterFunctionOwnerSQL = GetFunctionAlterOwnerCommand(funcOid);
initStringInfo(&ddlCommand);
appendStringInfo(&ddlCommand, "%s;%s;%s;%s", DISABLE_METADATA_SYNC,
createFunctionSQL, alterFunctionOwnerSQL, ENABLE_METADATA_SYNC);
SendCommandToWorkersAsUser(NON_COORDINATOR_NODES, CurrentUserName(),
ddlCommand.data);
}
MarkObjectDistributed(&functionAddress);
@ -698,7 +714,7 @@ UpdateFunctionDistributionInfo(const ObjectAddress *distAddress,
table_close(pgDistObjectRel, NoLock);
if (EnableDependencyCreation)
if (EnableMetadataSync)
{
List *objectAddressList = list_make1((ObjectAddress *) distAddress);
List *distArgumentIndexList = NIL;
@ -1206,7 +1222,7 @@ ShouldPropagateCreateFunction(CreateFunctionStmt *stmt)
return false;
}
if (!EnableDependencyCreation)
if (!EnableMetadataSync)
{
/*
* we are configured to disable object propagation, should not propagate anything
@ -1254,7 +1270,7 @@ ShouldPropagateAlterFunction(const ObjectAddress *address)
return false;
}
if (!EnableDependencyCreation)
if (!EnableMetadataSync)
{
/*
* we are configured to disable object propagation, should not propagate anything
@ -1556,7 +1572,7 @@ PreprocessDropFunctionStmt(Node *node, const char *queryString,
return NIL;
}
if (!EnableDependencyCreation)
if (!EnableMetadataSync)
{
/*
* we are configured to disable object propagation, should not propagate anything
@ -1657,7 +1673,7 @@ PreprocessAlterFunctionDependsStmt(Node *node, const char *queryString,
return NIL;
}
if (!EnableDependencyCreation)
if (!EnableMetadataSync)
{
/*
* we are configured to disable object propagation, should not propagate anything
@ -2013,33 +2029,6 @@ ErrorIfUnsupportedAlterFunctionStmt(AlterFunctionStmt *stmt)
}
/*
* ErrorIfFunctionDependsOnExtension functions depending on extensions should raise an
* error informing the user why they can't be distributed.
*/
static void
ErrorIfFunctionDependsOnExtension(const ObjectAddress *functionAddress)
{
/* captures the extension address during lookup */
ObjectAddress extensionAddress = { 0 };
if (IsObjectAddressOwnedByExtension(functionAddress, &extensionAddress))
{
char *functionName =
getObjectIdentity_compat(functionAddress, /* missingOk: */ false);
char *extensionName =
getObjectIdentity_compat(&extensionAddress, /* missingOk: */ false);
ereport(ERROR, (errmsg("unable to create a distributed function from functions "
"owned by an extension"),
errdetail("Function \"%s\" has a dependency on extension \"%s\". "
"Functions depending on an extension cannot be "
"distributed. Create the function by creating the "
"extension on the workers.", functionName,
extensionName)));
}
}
/* returns the quoted qualified name of a given function oid */
static char *
quote_qualified_func_name(Oid funcOid)
@ -2048,3 +2037,54 @@ quote_qualified_func_name(Oid funcOid)
get_namespace_name(get_func_namespace(funcOid)),
get_func_name(funcOid));
}
/*
* EnsureExtensionFuncionCanBeCreated checks if the dependent objects
* (including extension) exists on all nodes, if not, creates them. In
* addition, it also checks if distribution argument is passed.
*/
static void
EnsureExtensionFunctionCanBeDistributed(const ObjectAddress functionAddress,
const ObjectAddress extensionAddress,
char *distributionArgumentName)
{
if (CitusExtensionObject(&extensionAddress))
{
/*
* Citus extension is a special case. It's the extension that
* provides the 'distributed capabilities' in the first place.
* Trying to distribute it's own function(s) doesn't make sense.
*/
ereport(ERROR, (errmsg("Citus extension functions(%s) "
"cannot be distributed.",
get_func_name(functionAddress.objectId))));
}
/*
* Distributing functions from extensions has the most benefit when
* distribution argument is specified.
*/
if (distributionArgumentName == NULL)
{
ereport(ERROR, (errmsg("Extension functions(%s) "
"without distribution argument "
"are not supported.",
get_func_name(functionAddress.objectId))));
}
/*
* Ensure corresponding extension is in pg_dist_object.
* Functions owned by an extension are depending internally on that extension,
* hence EnsureDependenciesExistOnAllNodes() creates the extension, which in
* turn creates the function, and thus we don't have to create it ourself like
* we do for non-extension functions.
*/
ereport(DEBUG1, (errmsg("Extension(%s) owning the "
"function(%s) is not distributed, "
"attempting to propogate the extension",
get_extension_name(extensionAddress.objectId),
get_func_name(functionAddress.objectId))));
EnsureDependenciesExistOnAllNodes(&functionAddress);
}

View File

@ -40,8 +40,40 @@
#include "utils/relcache.h"
static ObjectAddress GetObjectAddressBySchemaName(char *schemaName, bool missing_ok);
static List * FilterDistributedSchemas(List *schemas);
static void EnsureSequentialModeForSchemaDDL(void);
static bool SchemaHasDistributedTableWithFKey(char *schemaName);
static bool ShouldPropagateCreateSchemaStmt(void);
/*
* PreprocessCreateSchemaStmt is called during the planning phase for
* CREATE SCHEMA ..
*/
List *
PreprocessCreateSchemaStmt(Node *node, const char *queryString,
ProcessUtilityContext processUtilityContext)
{
if (!ShouldPropagateCreateSchemaStmt())
{
return NIL;
}
EnsureCoordinator();
EnsureSequentialModeForSchemaDDL();
/* deparse sql*/
const char *sql = DeparseTreeNode(node);
/* to prevent recursion with mx we disable ddl propagation */
List *commands = list_make3(DISABLE_DDL_PROPAGATION,
(void *) sql,
ENABLE_DDL_PROPAGATION);
return NodeDDLTaskList(NON_COORDINATOR_NODES, commands);
}
/*
@ -53,76 +85,54 @@ PreprocessDropSchemaStmt(Node *node, const char *queryString,
ProcessUtilityContext processUtilityContext)
{
DropStmt *dropStatement = castNode(DropStmt, node);
Relation pgClass = NULL;
HeapTuple heapTuple = NULL;
SysScanDesc scanDescriptor = NULL;
ScanKeyData scanKey[1];
int scanKeyCount = 1;
Oid scanIndexId = InvalidOid;
bool useIndex = false;
Assert(dropStatement->removeType == OBJECT_SCHEMA);
if (dropStatement->behavior != DROP_CASCADE)
if (!ShouldPropagate())
{
return NIL;
}
Value *schemaValue = NULL;
foreach_ptr(schemaValue, dropStatement->objects)
EnsureCoordinator();
List *distributedSchemas = FilterDistributedSchemas(dropStatement->objects);
if (list_length(distributedSchemas) < 1)
{
const char *schemaString = strVal(schemaValue);
Oid namespaceOid = get_namespace_oid(schemaString, true);
if (namespaceOid == InvalidOid)
{
continue;
}
pgClass = table_open(RelationRelationId, AccessShareLock);
ScanKeyInit(&scanKey[0], Anum_pg_class_relnamespace, BTEqualStrategyNumber,
F_OIDEQ, namespaceOid);
scanDescriptor = systable_beginscan(pgClass, scanIndexId, useIndex, NULL,
scanKeyCount, scanKey);
heapTuple = systable_getnext(scanDescriptor);
while (HeapTupleIsValid(heapTuple))
{
Form_pg_class relationForm = (Form_pg_class) GETSTRUCT(heapTuple);
char *relationName = NameStr(relationForm->relname);
Oid relationId = get_relname_relid(relationName, namespaceOid);
/* we're not interested in non-valid, non-distributed relations */
if (relationId == InvalidOid || !IsCitusTable(relationId))
{
heapTuple = systable_getnext(scanDescriptor);
continue;
}
if (IsCitusTableType(relationId, REFERENCE_TABLE))
{
/* prevent concurrent EnsureReferenceTablesExistOnAllNodes */
int colocationId = CreateReferenceTableColocationId();
LockColocationId(colocationId, ExclusiveLock);
}
/* invalidate foreign key cache if the table involved in any foreign key */
if (TableReferenced(relationId) || TableReferencing(relationId))
{
MarkInvalidateForeignKeyGraph();
systable_endscan(scanDescriptor);
table_close(pgClass, NoLock);
return NIL;
}
heapTuple = systable_getnext(scanDescriptor);
}
systable_endscan(scanDescriptor);
table_close(pgClass, NoLock);
return NIL;
}
return NIL;
EnsureSequentialModeForSchemaDDL();
Value *schemaVal = NULL;
foreach_ptr(schemaVal, distributedSchemas)
{
if (SchemaHasDistributedTableWithFKey(strVal(schemaVal)))
{
MarkInvalidateForeignKeyGraph();
break;
}
}
/*
* We swap around the schema's in the statement to only contain the distributed
* schemas before deparsing. We need to restore the original list as postgres
* will execute on this statement locally, which requires all original schemas
* from the user to be present.
*/
List *originalObjects = dropStatement->objects;
dropStatement->objects = distributedSchemas;
const char *sql = DeparseTreeNode(node);
dropStatement->objects = originalObjects;
/* to prevent recursion with mx we disable ddl propagation */
List *commands = list_make3(DISABLE_DDL_PROPAGATION,
(void *) sql,
ENABLE_DDL_PROPAGATION);
return NodeDDLTaskList(NON_COORDINATOR_NODES, commands);
}
@ -205,6 +215,19 @@ PreprocessAlterSchemaRenameStmt(Node *node, const char *queryString,
}
/*
* CreateSchemaStmtObjectAddress returns the ObjectAddress of the schema that is
* the object of the CreateSchemaStmt. Errors if missing_ok is false.
*/
ObjectAddress
CreateSchemaStmtObjectAddress(Node *node, bool missing_ok)
{
CreateSchemaStmt *stmt = castNode(CreateSchemaStmt, node);
return GetObjectAddressBySchemaName(stmt->schemaname, missing_ok);
}
/*
* AlterSchemaRenameStmtObjectAddress returns the ObjectAddress of the schema that is
* the object of the RenameStmt. Errors if missing_ok is false.
@ -215,7 +238,17 @@ AlterSchemaRenameStmtObjectAddress(Node *node, bool missing_ok)
RenameStmt *stmt = castNode(RenameStmt, node);
Assert(stmt->renameType == OBJECT_SCHEMA);
const char *schemaName = stmt->subname;
return GetObjectAddressBySchemaName(stmt->subname, missing_ok);
}
/*
* GetObjectAddressBySchemaName returns the ObjectAddress of the schema with the
* given name. Errors out if schema is not found and missing_ok is false.
*/
ObjectAddress
GetObjectAddressBySchemaName(char *schemaName, bool missing_ok)
{
Oid schemaOid = get_namespace_oid(schemaName, missing_ok);
ObjectAddress address = { 0 };
@ -282,9 +315,9 @@ EnsureSequentialModeForSchemaDDL(void)
ereport(ERROR, (errmsg("cannot create or modify schema because there was a "
"parallel operation on a distributed table in the "
"transaction"),
errdetail("When creating or altering a schema, Citus needs to "
"perform all operations over a single connection per "
"node to ensure consistency."),
errdetail("When creating, altering, or dropping a schema, Citus "
"needs to perform all operations over a single "
"connection per node to ensure consistency."),
errhint("Try re-running the transaction with "
"\"SET LOCAL citus.multi_shard_modify_mode TO "
"\'sequential\';\"")));
@ -296,3 +329,88 @@ EnsureSequentialModeForSchemaDDL(void)
"use only one connection for all future commands")));
SetLocalMultiShardModifyModeToSequential();
}
/*
* SchemaHasDistributedTableWithFKey takes a schema name and scans the relations within
* that schema. If any one of the relations has a foreign key relationship, it returns
* true. Returns false otherwise.
*/
static bool
SchemaHasDistributedTableWithFKey(char *schemaName)
{
ScanKeyData scanKey[1];
int scanKeyCount = 1;
Oid scanIndexId = InvalidOid;
bool useIndex = false;
Oid namespaceOid = get_namespace_oid(schemaName, true);
if (namespaceOid == InvalidOid)
{
return false;
}
Relation pgClass = table_open(RelationRelationId, AccessShareLock);
ScanKeyInit(&scanKey[0], Anum_pg_class_relnamespace, BTEqualStrategyNumber,
F_OIDEQ, namespaceOid);
SysScanDesc scanDescriptor = systable_beginscan(pgClass, scanIndexId, useIndex, NULL,
scanKeyCount, scanKey);
HeapTuple heapTuple = systable_getnext(scanDescriptor);
while (HeapTupleIsValid(heapTuple))
{
Form_pg_class relationForm = (Form_pg_class) GETSTRUCT(heapTuple);
char *relationName = NameStr(relationForm->relname);
Oid relationId = get_relname_relid(relationName, namespaceOid);
/* we're not interested in non-valid, non-distributed relations */
if (relationId == InvalidOid || !IsCitusTable(relationId))
{
heapTuple = systable_getnext(scanDescriptor);
continue;
}
/* invalidate foreign key cache if the table involved in any foreign key */
if (TableReferenced(relationId) || TableReferencing(relationId))
{
systable_endscan(scanDescriptor);
table_close(pgClass, NoLock);
return true;
}
heapTuple = systable_getnext(scanDescriptor);
}
systable_endscan(scanDescriptor);
table_close(pgClass, NoLock);
return false;
}
/*
* ShouldPropagateCreateSchemaStmt gets called only for CreateSchemaStmt's.
* This function wraps the ShouldPropagate function which is commonly used
* for all object types; additionally it checks whether there's a multi-statement
* transaction ongoing or not. For transaction blocks, we require sequential mode
* with this function, for CREATE SCHEMA statements. If Citus has not already
* switched to sequential mode, we don't propagate.
*/
static bool
ShouldPropagateCreateSchemaStmt()
{
if (!ShouldPropagate())
{
return false;
}
if (IsMultiStatementTransaction() &&
MultiShardConnectionType != SEQUENTIAL_CONNECTION)
{
return false;
}
return true;
}

View File

@ -11,6 +11,7 @@
#include "postgres.h"
#include "access/xact.h"
#include "catalog/dependency.h"
#include "catalog/namespace.h"
#include "commands/defrem.h"
@ -23,6 +24,7 @@
#include "distributed/metadata/distobject.h"
#include "distributed/metadata_cache.h"
#include "distributed/metadata_sync.h"
#include "distributed/worker_create_or_replace.h"
#include "nodes/parsenodes.h"
#include "utils/builtins.h"
#include "utils/lsyscache.h"
@ -239,7 +241,7 @@ PreprocessDropSequenceStmt(Node *node, const char *queryString,
return NIL;
}
if (!EnableDependencyCreation)
if (!EnableMetadataSync)
{
/*
* we are configured to disable object propagation, should not propagate anything
@ -656,3 +658,77 @@ PostprocessAlterSequenceOwnerStmt(Node *node, const char *queryString)
return NIL;
}
/*
* GenerateBackupNameForSequenceCollision generates a new sequence name for an existing
* sequence. The name is generated in such a way that the new name doesn't overlap with
* an existing relation by adding a suffix with incrementing number after the new name.
*/
char *
GenerateBackupNameForSequenceCollision(const ObjectAddress *address)
{
char *newName = palloc0(NAMEDATALEN);
char suffix[NAMEDATALEN] = { 0 };
int count = 0;
char *namespaceName = get_namespace_name(get_rel_namespace(address->objectId));
Oid schemaId = get_namespace_oid(namespaceName, false);
char *baseName = get_rel_name(address->objectId);
int baseLength = strlen(baseName);
while (true)
{
int suffixLength = SafeSnprintf(suffix, NAMEDATALEN - 1, "(citus_backup_%d)",
count);
/* trim the base name at the end to leave space for the suffix and trailing \0 */
baseLength = Min(baseLength, NAMEDATALEN - suffixLength - 1);
/* clear newName before copying the potentially trimmed baseName and suffix */
memset(newName, 0, NAMEDATALEN);
strncpy_s(newName, NAMEDATALEN, baseName, baseLength);
strncpy_s(newName + baseLength, NAMEDATALEN - baseLength, suffix,
suffixLength);
Oid newRelationId = get_relname_relid(newName, schemaId);
if (newRelationId == InvalidOid)
{
return newName;
}
count++;
}
}
/*
* RenameExistingSequenceWithDifferentTypeIfExists renames the sequence's type if
* that sequence exists and the desired sequence type is different than it's type.
*/
void
RenameExistingSequenceWithDifferentTypeIfExists(RangeVar *sequence, Oid desiredSeqTypeId)
{
Oid sequenceOid;
RangeVarGetAndCheckCreationNamespace(sequence, NoLock, &sequenceOid);
if (OidIsValid(sequenceOid))
{
Form_pg_sequence pgSequenceForm = pg_get_sequencedef(sequenceOid);
if (pgSequenceForm->seqtypid != desiredSeqTypeId)
{
ObjectAddress sequenceAddress = { 0 };
ObjectAddressSet(sequenceAddress, RelationRelationId, sequenceOid);
char *newName = GenerateBackupNameForCollision(&sequenceAddress);
RenameStmt *renameStmt = CreateRenameStatement(&sequenceAddress, newName);
const char *sqlRenameStmt = DeparseTreeNode((Node *) renameStmt);
ProcessUtilityParseTree((Node *) renameStmt, sqlRenameStmt,
PROCESS_UTILITY_QUERY,
NULL, None_Receiver, NULL);
CommandCounterIncrement();
}
}
}

View File

@ -1955,6 +1955,12 @@ PostprocessAlterTableStmt(AlterTableStmt *alterTableStatement)
return;
}
/*
* Before ensuring each dependency exist, update dependent sequences
* types if necessary.
*/
EnsureRelationHasCompatibleSequenceTypes(relationId);
/* changing a relation could introduce new dependencies */
ObjectAddress tableAddress = { 0 };
ObjectAddressSet(tableAddress, RelationRelationId, relationId);
@ -2045,18 +2051,9 @@ PostprocessAlterTableStmt(AlterTableStmt *alterTableStatement)
Oid seqOid = GetSequenceOid(relationId, attnum);
if (seqOid != InvalidOid)
{
EnsureDistributedSequencesHaveOneType(relationId,
list_make1_oid(
seqOid),
list_make1_int(
attnum));
if (ShouldSyncTableMetadata(relationId) &&
ClusterHasKnownMetadataWorkers())
if (ShouldSyncTableMetadata(relationId))
{
needMetadataSyncForNewSequences = true;
MarkSequenceDistributedAndPropagateWithDependencies(
relationId, seqOid);
alterTableDefaultNextvalCmd =
GetAddColumnWithNextvalDefaultCmd(seqOid,
relationId,
@ -2088,16 +2085,9 @@ PostprocessAlterTableStmt(AlterTableStmt *alterTableStatement)
Oid seqOid = GetSequenceOid(relationId, attnum);
if (seqOid != InvalidOid)
{
EnsureDistributedSequencesHaveOneType(relationId,
list_make1_oid(seqOid),
list_make1_int(attnum));
if (ShouldSyncTableMetadata(relationId) &&
ClusterHasKnownMetadataWorkers())
if (ShouldSyncTableMetadata(relationId))
{
needMetadataSyncForNewSequences = true;
MarkSequenceDistributedAndPropagateWithDependencies(relationId,
seqOid);
alterTableDefaultNextvalCmd = GetAlterColumnWithNextvalDefaultCmd(
seqOid, relationId, command->name);
}
@ -2627,8 +2617,7 @@ ErrorIfUnsupportedAlterTableStmt(AlterTableStmt *alterTableStatement)
* We currently don't support adding a serial column for an MX table
* TODO: record the dependency in the workers
*/
if (ShouldSyncTableMetadata(relationId) &&
ClusterHasKnownMetadataWorkers())
if (ShouldSyncTableMetadata(relationId))
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg(

View File

@ -437,8 +437,12 @@ AcquireDistributedLockOnRelations(List *relationIdList, LOCKMODE lockMode)
/*
* We only acquire distributed lock on relation if
* the relation is sync'ed between mx nodes.
*
* Even if users disable metadata sync, we cannot
* allow them not to acquire the remote locks.
* Hence, we have !IsCoordinator() check.
*/
if (ShouldSyncTableMetadata(relationId))
if (ShouldSyncTableMetadata(relationId) || !IsCoordinator())
{
char *qualifiedRelationName = generate_qualified_relation_name(relationId);
StringInfo lockRelationCommand = makeStringInfo();

View File

@ -897,7 +897,7 @@ ShouldCheckUndistributeCitusLocalTables(void)
return false;
}
if (IsCitusInitiatedRemoteBackend())
if (IsCitusInternalBackend() || IsRebalancerInternalBackend())
{
/* connection from the coordinator operating on a shard */
return false;

View File

@ -10,6 +10,7 @@
#include "postgres.h"
#include "distributed/backend_data.h"
#include "distributed/citus_safe_lib.h"
#include "distributed/connection_management.h"
#include "distributed/metadata_cache.h"
@ -232,6 +233,10 @@ GetConnParams(ConnectionHashKey *key, char ***keywords, char ***values,
*/
char nodePortString[12] = "";
StringInfo applicationName = makeStringInfo();
appendStringInfo(applicationName, "%s%ld", CITUS_APPLICATION_NAME_PREFIX,
GetGlobalPID());
/*
* This function has three sections:
* - Initialize the keywords and values (to be copied later) of global parameters
@ -260,7 +265,7 @@ GetConnParams(ConnectionHashKey *key, char ***keywords, char ***values,
key->database,
key->user,
GetDatabaseEncodingName(),
CITUS_APPLICATION_NAME
applicationName->data
};
/*

View File

@ -19,6 +19,7 @@
#include "access/hash.h"
#include "commands/dbcommands.h"
#include "distributed/backend_data.h"
#include "distributed/connection_management.h"
#include "distributed/errormessage.h"
#include "distributed/error_codes.h"
@ -1429,7 +1430,7 @@ ShouldShutdownConnection(MultiConnection *connection, const int cachedConnection
* escalating the number of cached connections. We can recognize such backends
* from their application name.
*/
return IsCitusInitiatedRemoteBackend() ||
return (IsCitusInternalBackend() || IsRebalancerInternalBackend()) ||
connection->initilizationState != POOL_STATE_INITIALIZED ||
cachedConnectionCount >= MaxCachedConnectionsPerWorker ||
connection->forceCloseAtTransactionEnd ||
@ -1441,14 +1442,25 @@ ShouldShutdownConnection(MultiConnection *connection, const int cachedConnection
}
/*
* IsRebalancerInitiatedBackend returns true if we are in a backend that citus
* rebalancer initiated.
*/
bool
IsRebalancerInternalBackend(void)
{
return application_name && strcmp(application_name, CITUS_REBALANCER_NAME) == 0;
}
/*
* IsCitusInitiatedRemoteBackend returns true if we are in a backend that citus
* initiated via remote connection.
*/
bool
IsCitusInitiatedRemoteBackend(void)
IsCitusInternalBackend(void)
{
return application_name && strcmp(application_name, CITUS_APPLICATION_NAME) == 0;
return ExtractGlobalPID(application_name) != INVALID_CITUS_INTERNAL_BACKEND_GPID;
}

View File

@ -14,16 +14,47 @@
#include "distributed/citus_ruleutils.h"
#include "distributed/deparser.h"
#include "distributed/listutils.h"
#include "lib/stringinfo.h"
#include "nodes/nodes.h"
#include "utils/builtins.h"
static void AppendCreateSchemaStmt(StringInfo buf, CreateSchemaStmt *stmt);
static void AppendDropSchemaStmt(StringInfo buf, DropStmt *stmt);
static void AppendGrantOnSchemaStmt(StringInfo buf, GrantStmt *stmt);
static void AppendGrantOnSchemaPrivileges(StringInfo buf, GrantStmt *stmt);
static void AppendGrantOnSchemaSchemas(StringInfo buf, GrantStmt *stmt);
static void AppendGrantOnSchemaGrantees(StringInfo buf, GrantStmt *stmt);
static void AppendAlterSchemaRenameStmt(StringInfo buf, RenameStmt *stmt);
char *
DeparseCreateSchemaStmt(Node *node)
{
CreateSchemaStmt *stmt = castNode(CreateSchemaStmt, node);
StringInfoData str = { 0 };
initStringInfo(&str);
AppendCreateSchemaStmt(&str, stmt);
return str.data;
}
char *
DeparseDropSchemaStmt(Node *node)
{
DropStmt *stmt = castNode(DropStmt, node);
StringInfoData str = { 0 };
initStringInfo(&str);
AppendDropSchemaStmt(&str, stmt);
return str.data;
}
char *
DeparseGrantOnSchemaStmt(Node *node)
{
@ -53,6 +84,70 @@ DeparseAlterSchemaRenameStmt(Node *node)
}
static void
AppendCreateSchemaStmt(StringInfo buf, CreateSchemaStmt *stmt)
{
if (stmt->schemaElts != NIL)
{
elog(ERROR, "schema creating is not supported with other create commands");
}
if (stmt->schemaname == NULL)
{
elog(ERROR, "schema name should be specified");
}
appendStringInfoString(buf, "CREATE SCHEMA ");
if (stmt->if_not_exists)
{
appendStringInfoString(buf, "IF NOT EXISTS ");
}
appendStringInfo(buf, "%s ", quote_identifier(stmt->schemaname));
if (stmt->authrole != NULL)
{
appendStringInfo(buf, "AUTHORIZATION %s", RoleSpecString(stmt->authrole, true));
}
}
static void
AppendDropSchemaStmt(StringInfo buf, DropStmt *stmt)
{
Assert(stmt->removeType == OBJECT_SCHEMA);
appendStringInfoString(buf, "DROP SCHEMA ");
if (stmt->missing_ok)
{
appendStringInfoString(buf, "IF EXISTS ");
}
Value *schemaValue = NULL;
foreach_ptr(schemaValue, stmt->objects)
{
const char *schemaString = quote_identifier(strVal(schemaValue));
appendStringInfo(buf, "%s", schemaString);
if (schemaValue != llast(stmt->objects))
{
appendStringInfoString(buf, ", ");
}
}
if (stmt->behavior == DROP_CASCADE)
{
appendStringInfoString(buf, " CASCADE");
}
else if (stmt->behavior == DROP_RESTRICT)
{
appendStringInfoString(buf, " RESTRICT");
}
}
static void
AppendGrantOnSchemaStmt(StringInfo buf, GrantStmt *stmt)
{

View File

@ -779,7 +779,7 @@ InTaskExecution(void)
* is considered a task execution, but an exception is when we
* are in a delegated function/procedure call.
*/
return IsCitusInitiatedRemoteBackend() &&
return IsCitusInternalBackend() &&
!InTopLevelDelegatedFunctionCall &&
!InDelegatedProcedureCall;
}

View File

@ -37,6 +37,7 @@
#include "distributed/metadata/dependency.h"
#include "distributed/metadata/distobject.h"
#include "distributed/metadata_cache.h"
#include "distributed/metadata_sync.h"
#include "distributed/version_compat.h"
#include "miscadmin.h"
#include "utils/fmgroids.h"
@ -120,6 +121,7 @@ typedef struct ViewDependencyNode
}ViewDependencyNode;
static List * GetRelationSequenceDependencyList(Oid relationId);
static List * GetRelationTriggerFunctionDependencyList(Oid relationId);
static List * GetRelationStatsSchemaDependencyList(Oid relationId);
static DependencyDefinition * CreateObjectAddressDependencyDef(Oid classId, Oid objectId);
@ -556,7 +558,7 @@ IsObjectAddressCollected(ObjectAddress findAddress,
bool
SupportedDependencyByCitus(const ObjectAddress *address)
{
if (!EnableDependencyCreation)
if (!EnableMetadataSync)
{
/*
* If the user has disabled object propagation we need to fall back to the legacy
@ -671,11 +673,20 @@ SupportedDependencyByCitus(const ObjectAddress *address)
case OCLASS_CLASS:
{
char relKind = get_rel_relkind(address->objectId);
/*
* composite types have a reference to a relation of composite type, we need
* to follow those to get the dependencies of type fields.
*
* As we also handle tables as objects as well, follow dependencies
* for tables.
*/
if (get_rel_relkind(address->objectId) == RELKIND_COMPOSITE_TYPE)
if (relKind == RELKIND_COMPOSITE_TYPE ||
relKind == RELKIND_RELATION ||
relKind == RELKIND_PARTITIONED_TABLE ||
relKind == RELKIND_FOREIGN_TABLE ||
relKind == RELKIND_SEQUENCE)
{
return true;
}
@ -985,6 +996,15 @@ ExpandCitusSupportedTypes(ObjectAddressCollector *collector, ObjectAddress targe
List *statisticsSchemaDependencyList =
GetRelationStatsSchemaDependencyList(relationId);
result = list_concat(result, statisticsSchemaDependencyList);
/*
* Get the dependent sequences for tables (both as serial columns and
* columns have nextval with existing sequences) and expand dependency list
* with them.
*/
List *sequenceDependencyList = GetRelationSequenceDependencyList(relationId);
result = list_concat(result, sequenceDependencyList);
}
default:
@ -997,6 +1017,24 @@ ExpandCitusSupportedTypes(ObjectAddressCollector *collector, ObjectAddress targe
}
/*
* GetRelationSequenceDependencyList returns the sequence dependency definition
* list for the given relation.
*/
static List *
GetRelationSequenceDependencyList(Oid relationId)
{
List *attnumList = NIL;
List *dependentSequenceList = NIL;
GetDependentSequencesWithRelation(relationId, &attnumList, &dependentSequenceList, 0);
List *sequenceDependencyDefList =
CreateObjectAddressDependencyDefList(RelationRelationId, dependentSequenceList);
return sequenceDependencyDefList;
}
/*
* GetRelationStatsSchemaDependencyList returns a list of DependencyDefinition
* objects for the schemas that statistics' of the relation with relationId depends.

View File

@ -46,6 +46,8 @@
#include "utils/rel.h"
static void MarkObjectDistributedLocally(const ObjectAddress *distAddress);
static char * CreatePgDistObjectEntryCommand(const ObjectAddress *objectAddress);
static int ExecuteCommandAsSuperuser(char *query, int paramCount, Oid *paramTypes,
Datum *paramValues);
@ -141,14 +143,60 @@ ObjectExists(const ObjectAddress *address)
/*
* MarkObjectDistributed marks an object as a distributed object by citus. Marking is done
* by adding appropriate entries to citus.pg_dist_object.
* MarkObjectDistributed marks an object as a distributed object. Marking is done
* by adding appropriate entries to citus.pg_dist_object and also marking the object
* as distributed by opening a connection using current user to all of the workers
* with metadata if object propagation is on.
*
* This also marks the object as distributed on all of the workers with metadata
* if object propagation is on.
* This function should be used if the user creating the given object. If you want
* to mark dependent objects as distributed check MarkObjectDistributedViaSuperUser.
*/
void
MarkObjectDistributed(const ObjectAddress *distAddress)
{
MarkObjectDistributedLocally(distAddress);
if (EnableMetadataSync)
{
char *workerPgDistObjectUpdateCommand =
CreatePgDistObjectEntryCommand(distAddress);
SendCommandToWorkersWithMetadata(workerPgDistObjectUpdateCommand);
}
}
/*
* MarkObjectDistributedViaSuperUser marks an object as a distributed object. Marking
* is done by adding appropriate entries to citus.pg_dist_object and also marking the
* object as distributed by opening a connection using super user to all of the workers
* with metadata if object propagation is on.
*
* This function should be used to mark dependent object as distributed. If you want
* to mark the object you are creating please check MarkObjectDistributed.
*/
void
MarkObjectDistributedViaSuperUser(const ObjectAddress *distAddress)
{
MarkObjectDistributedLocally(distAddress);
if (EnableMetadataSync)
{
char *workerPgDistObjectUpdateCommand =
CreatePgDistObjectEntryCommand(distAddress);
SendCommandToWorkersWithMetadataViaSuperUser(workerPgDistObjectUpdateCommand);
}
}
/*
* MarkObjectDistributedLocally marks an object as a distributed object by citus.
* Marking is done by adding appropriate entries to citus.pg_dist_object.
*
* This function should never be called alone, MarkObjectDistributed() or
* MarkObjectDistributedViaSuperUser() should be called.
*/
static void
MarkObjectDistributedLocally(const ObjectAddress *distAddress)
{
int paramCount = 3;
Oid paramTypes[3] = {
@ -161,32 +209,38 @@ MarkObjectDistributed(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";
int spiStatus = ExecuteCommandAsSuperuser(insertQuery, paramCount, paramTypes,
paramValues);
if (spiStatus < 0)
{
ereport(ERROR, (errmsg("failed to insert object into citus.pg_dist_object")));
}
}
if (EnableDependencyCreation)
{
/* create a list by adding the address of value to not to have warning */
List *objectAddressList = list_make1((ObjectAddress *) distAddress);
List *distArgumetIndexList = list_make1_int(INVALID_DISTRIBUTION_ARGUMENT_INDEX);
List *colocationIdList = list_make1_int(INVALID_COLOCATION_ID);
List *forceDelegationList = list_make1_int(NO_FORCE_PUSHDOWN);
char *workerPgDistObjectUpdateCommand =
MarkObjectsDistributedCreateCommand(objectAddressList,
distArgumetIndexList,
colocationIdList,
forceDelegationList);
SendCommandToWorkersWithMetadata(workerPgDistObjectUpdateCommand);
}
/*
* CreatePgDistObjectEntryCommand creates command to insert pg_dist_object tuple
* for the given object address.
*/
static char *
CreatePgDistObjectEntryCommand(const ObjectAddress *objectAddress)
{
/* create a list by adding the address of value to not to have warning */
List *objectAddressList =
list_make1((ObjectAddress *) objectAddress);
List *distArgumetIndexList = list_make1_int(INVALID_DISTRIBUTION_ARGUMENT_INDEX);
List *colocationIdList = list_make1_int(INVALID_COLOCATION_ID);
List *forceDelegationList = list_make1_int(NO_FORCE_PUSHDOWN);
char *workerPgDistObjectUpdateCommand =
MarkObjectsDistributedCreateCommand(objectAddressList,
distArgumetIndexList,
colocationIdList,
forceDelegationList);
return workerPgDistObjectUpdateCommand;
}

View File

@ -201,6 +201,9 @@ static bool workerNodeHashValid = false;
/* default value is -1, for coordinator it's 0 and for worker nodes > 0 */
static int32 LocalGroupId = -1;
/* default value is -1, increases with every node starting from 1 */
static int32 LocalNodeId = -1;
/* built first time through in InitializeDistCache */
static ScanKeyData DistPartitionScanKey[1];
static ScanKeyData DistShardScanKey[1];
@ -3618,6 +3621,62 @@ GetLocalGroupId(void)
}
/*
* GetNodeId returns the node identifier of the local node.
*/
int32
GetLocalNodeId(void)
{
InitializeCaches();
/*
* Already set the node id, no need to read the heap again.
*/
if (LocalNodeId != -1)
{
return LocalNodeId;
}
uint32 nodeId = -1;
int32 localGroupId = GetLocalGroupId();
bool includeNodesFromOtherClusters = false;
List *workerNodeList = ReadDistNode(includeNodesFromOtherClusters);
WorkerNode *workerNode = NULL;
foreach_ptr(workerNode, workerNodeList)
{
if (workerNode->groupId == localGroupId &&
workerNode->isActive)
{
nodeId = workerNode->nodeId;
break;
}
}
/*
* nodeId is -1 if we cannot find an active node whose group id is
* localGroupId in pg_dist_node.
*/
if (nodeId == -1)
{
elog(DEBUG4, "there is no active node with group id '%d' on pg_dist_node",
localGroupId);
/*
* This is expected if the coordinator is not added to the metadata.
* We'll return 0 for this case and for all cases so views can function almost normally
*/
nodeId = 0;
}
LocalNodeId = nodeId;
return nodeId;
}
/*
* RegisterLocalGroupIdCacheCallbacks registers the callbacks required to
* maintain LocalGroupId at a consistent value. It's separate from
@ -4019,6 +4078,7 @@ InvalidateMetadataSystemCache(void)
memset(&MetadataCache, 0, sizeof(MetadataCache));
workerNodeHashValid = false;
LocalGroupId = -1;
LocalNodeId = -1;
}
@ -4110,6 +4170,7 @@ InvalidateNodeRelationCacheCallback(Datum argument, Oid relationId)
if (relationId == InvalidOid || relationId == MetadataCache.distNodeRelationId)
{
workerNodeHashValid = false;
LocalNodeId = -1;
}
}

View File

@ -83,20 +83,15 @@
char *EnableManualMetadataChangesForUser = "";
static void EnsureSequentialModeMetadataOperations(void);
static List * DistributedObjectMetadataSyncCommandList(void);
static List * GetDistributedTableDDLEvents(Oid relationId);
static void EnsureObjectMetadataIsSane(int distributionArgumentIndex,
int colocationId);
static char * LocalGroupIdUpdateCommand(int32 groupId);
static List * SequenceDependencyCommandList(Oid relationId);
static char * TruncateTriggerCreateCommand(Oid relationId);
static char * SchemaOwnerName(Oid objectId);
static bool HasMetadataWorkers(void);
static List * DetachPartitionCommandList(void);
static void CreateShellTableOnWorkers(Oid relationId);
static void CreateTableMetadataOnWorkers(Oid relationId);
static bool ShouldSyncTableMetadataInternal(bool hashDistributed,
bool citusTableWithNoDistKey);
static bool SyncMetadataSnapshotToNode(WorkerNode *workerNode, bool raiseOnError);
static bool SyncNodeMetadataSnapshotToNode(WorkerNode *workerNode, bool raiseOnError);
static void DropMetadataSnapshotOnNode(WorkerNode *workerNode);
static char * CreateSequenceDependencyCommand(Oid relationId, Oid sequenceId,
char *columnName);
@ -110,7 +105,8 @@ static AccessPriv * GetAccessPrivObjectForGrantStmt(char *permission);
static RoleSpec * GetRoleSpecObjectForGrantStmt(Oid roleOid);
static List * GenerateGrantOnSchemaQueriesFromAclItem(Oid schemaOid,
AclItem *aclItem);
static void SetLocalEnableDependencyCreation(bool state);
static void SetLocalEnableMetadataSync(bool state);
static void SetLocalReplicateReferenceTablesOnActivate(bool state);
static char * GenerateSetRoleQuery(Oid roleOid);
static void MetadataSyncSigTermHandler(SIGNAL_ARGS);
static void MetadataSyncSigAlrmHandler(SIGNAL_ARGS);
@ -155,7 +151,7 @@ static bool got_SIGALRM = false;
/*
* start_metadata_sync_to_node function sets hasmetadata column of the given
* node to true, and then synchronizes the metadata on the node.
* node to true, and then activate node without replicating reference tables.
*/
Datum
start_metadata_sync_to_node(PG_FUNCTION_ARGS)
@ -165,20 +161,29 @@ start_metadata_sync_to_node(PG_FUNCTION_ARGS)
text *nodeName = PG_GETARG_TEXT_P(0);
int32 nodePort = PG_GETARG_INT32(1);
EnsureSuperUser();
EnsureCoordinator();
char *nodeNameString = text_to_cstring(nodeName);
StartMetadataSyncToNode(nodeNameString, nodePort);
bool prevReplicateRefTablesOnActivate = ReplicateReferenceTablesOnActivate;
SetLocalReplicateReferenceTablesOnActivate(false);
ActivateNode(nodeNameString, nodePort);
TransactionModifiedNodeMetadata = true;
SetLocalReplicateReferenceTablesOnActivate(prevReplicateRefTablesOnActivate);
PG_RETURN_VOID();
}
/*
* StartMetadataSyncToNode is the internal API for
* SyncNodeMetadataToNode is the internal API for
* start_metadata_sync_to_node().
*/
void
StartMetadataSyncToNode(const char *nodeNameString, int32 nodePort)
SyncNodeMetadataToNode(const char *nodeNameString, int32 nodePort)
{
char *escapedNodeName = quote_literal_cstr(nodeNameString);
@ -244,7 +249,29 @@ StartMetadataSyncToNode(const char *nodeNameString, int32 nodePort)
/* fail if metadata synchronization doesn't succeed */
bool raiseInterrupts = true;
SyncMetadataSnapshotToNode(workerNode, raiseInterrupts);
SyncNodeMetadataSnapshotToNode(workerNode, raiseInterrupts);
}
/*
* SyncCitusTableMetadata syncs citus table metadata to worker nodes with metadata.
* Our definition of metadata includes the shell table and its inter relations with
* other shell tables, corresponding pg_dist_object, pg_dist_partiton, pg_dist_shard
* and pg_dist_shard placement entries.
*/
void
SyncCitusTableMetadata(Oid relationId)
{
CreateShellTableOnWorkers(relationId);
CreateTableMetadataOnWorkers(relationId);
CreateInterTableRelationshipOfRelationOnWorkers(relationId);
if (!IsTableOwnedByExtension(relationId))
{
ObjectAddress relationAddress = { 0 };
ObjectAddressSet(relationAddress, RelationRelationId, relationId);
MarkObjectDistributed(&relationAddress);
}
}
@ -260,7 +287,7 @@ StartMetadataSyncToNode(const char *nodeNameString, int32 nodePort)
* visible on all connections used by the transaction, meaning we can only use 1
* connection per node.
*/
static void
void
EnsureSequentialModeMetadataOperations(void)
{
if (!IsTransactionBlock())
@ -302,6 +329,7 @@ stop_metadata_sync_to_node(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
EnsureCoordinator();
EnsureSuperUser();
text *nodeName = PG_GETARG_TEXT_P(0);
int32 nodePort = PG_GETARG_INT32(1);
@ -350,6 +378,8 @@ stop_metadata_sync_to_node(PG_FUNCTION_ARGS)
workerNode = SetWorkerColumn(workerNode, Anum_pg_dist_node_metadatasynced,
BoolGetDatum(false));
TransactionModifiedNodeMetadata = true;
PG_RETURN_VOID();
}
@ -387,7 +417,8 @@ ClusterHasKnownMetadataWorkers()
bool
ShouldSyncTableMetadata(Oid relationId)
{
if (!OidIsValid(relationId) || !IsCitusTable(relationId))
if (!EnableMetadataSync ||
!OidIsValid(relationId) || !IsCitusTable(relationId))
{
return false;
}
@ -443,25 +474,25 @@ ShouldSyncTableMetadataInternal(bool hashDistributed, bool citusTableWithNoDistK
/*
* SyncMetadataSnapshotToNode does the following:
* SyncNodeMetadataSnapshotToNode does the following:
* 1. Sets the localGroupId on the worker so the worker knows which tuple in
* pg_dist_node represents itself.
* 2. Recreates the distributed metadata on the given worker.
* 2. Recreates the node metadata on the given worker.
* If raiseOnError is true, it errors out if synchronization fails.
*/
static bool
SyncMetadataSnapshotToNode(WorkerNode *workerNode, bool raiseOnError)
SyncNodeMetadataSnapshotToNode(WorkerNode *workerNode, bool raiseOnError)
{
char *currentUser = CurrentUserName();
/* generate and add the local group id's update query */
char *localGroupIdUpdateCommand = LocalGroupIdUpdateCommand(workerNode->groupId);
/* generate the queries which drop the metadata */
List *dropMetadataCommandList = MetadataDropCommands();
/* generate the queries which drop the node metadata */
List *dropMetadataCommandList = NodeMetadataDropCommands();
/* generate the queries which create the metadata from scratch */
List *createMetadataCommandList = MetadataCreateCommands();
/* generate the queries which create the node metadata from scratch */
List *createMetadataCommandList = NodeMetadataCreateCommands();
List *recreateMetadataSnapshotCommandList = list_make1(localGroupIdUpdateCommand);
recreateMetadataSnapshotCommandList = list_concat(recreateMetadataSnapshotCommandList,
@ -506,12 +537,28 @@ DropMetadataSnapshotOnNode(WorkerNode *workerNode)
char *userName = CurrentUserName();
/* generate the queries which drop the metadata */
List *dropMetadataCommandList = MetadataDropCommands();
/*
* Detach partitions, break dependencies between sequences and table then
* remove shell tables first.
*/
List *dropMetadataCommandList = DetachPartitionCommandList();
dropMetadataCommandList = lappend(dropMetadataCommandList,
BREAK_CITUS_TABLE_SEQUENCE_DEPENDENCY_COMMAND);
dropMetadataCommandList = lappend(dropMetadataCommandList,
REMOVE_ALL_SHELL_TABLES_COMMAND);
dropMetadataCommandList = list_concat(dropMetadataCommandList,
NodeMetadataDropCommands());
dropMetadataCommandList = lappend(dropMetadataCommandList,
LocalGroupIdUpdateCommand(0));
/* remove all dist table and object/table related metadata afterwards */
dropMetadataCommandList = lappend(dropMetadataCommandList, DELETE_ALL_PARTITIONS);
dropMetadataCommandList = lappend(dropMetadataCommandList, DELETE_ALL_SHARDS);
dropMetadataCommandList = lappend(dropMetadataCommandList, DELETE_ALL_PLACEMENTS);
dropMetadataCommandList = lappend(dropMetadataCommandList,
DELETE_ALL_DISTRIBUTED_OBJECTS);
Assert(superuser());
SendOptionalMetadataCommandListToWorkerInCoordinatedTransaction(
workerNode->workerName,
workerNode->workerPort,
@ -521,28 +568,19 @@ DropMetadataSnapshotOnNode(WorkerNode *workerNode)
/*
* MetadataCreateCommands returns list of queries that are
* NodeMetadataCreateCommands returns list of queries that are
* required to create the current metadata snapshot of the node that the
* function is called. The metadata snapshot commands includes the
* following queries:
*
* (i) Query that populates pg_dist_node table
* (ii) Queries that create the clustered tables (including foreign keys,
* partitioning hierarchy etc.)
* (iii) Queries that populate pg_dist_partition table referenced by (ii)
* (iv) Queries that populate pg_dist_shard table referenced by (iii)
* (v) Queries that populate pg_dist_placement table referenced by (iv)
* (vi) Queries that populate pg_dist_object table
*/
List *
MetadataCreateCommands(void)
NodeMetadataCreateCommands(void)
{
List *metadataSnapshotCommandList = NIL;
List *distributedTableList = CitusTableList();
List *propagatedTableList = NIL;
bool includeNodesFromOtherClusters = true;
List *workerNodeList = ReadDistNode(includeNodesFromOtherClusters);
IncludeSequenceDefaults includeSequenceDefaults = WORKER_NEXTVAL_SEQUENCE_DEFAULTS;
/* make sure we have deterministic output for our tests */
workerNodeList = SortList(workerNodeList, CompareWorkerNodes);
@ -552,170 +590,6 @@ MetadataCreateCommands(void)
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
nodeListInsertCommand);
/* create the list of tables whose metadata will be created */
CitusTableCacheEntry *cacheEntry = NULL;
foreach_ptr(cacheEntry, distributedTableList)
{
if (ShouldSyncTableMetadata(cacheEntry->relationId))
{
propagatedTableList = lappend(propagatedTableList, cacheEntry);
}
}
/* create the tables, but not the metadata */
foreach_ptr(cacheEntry, propagatedTableList)
{
Oid relationId = cacheEntry->relationId;
ObjectAddress tableAddress = { 0 };
if (IsTableOwnedByExtension(relationId))
{
/* skip table creation when the Citus table is owned by an extension */
continue;
}
List *ddlCommandList = GetFullTableCreationCommands(relationId,
includeSequenceDefaults);
char *tableOwnerResetCommand = TableOwnerResetCommand(relationId);
/*
* Tables might have dependencies on different objects, since we create shards for
* table via multiple sessions these objects will be created via their own connection
* and committed immediately so they become visible to all sessions creating shards.
*/
ObjectAddressSet(tableAddress, RelationRelationId, relationId);
/*
* Set object propagation to off as we will mark objects distributed
* at the end of this function.
*/
bool prevDependencyCreationValue = EnableDependencyCreation;
SetLocalEnableDependencyCreation(false);
EnsureDependenciesExistOnAllNodes(&tableAddress);
/*
* Ensure sequence dependencies and mark them as distributed
*/
List *attnumList = NIL;
List *dependentSequenceList = NIL;
GetDependentSequencesWithRelation(relationId, &attnumList,
&dependentSequenceList, 0);
Oid sequenceOid = InvalidOid;
foreach_oid(sequenceOid, dependentSequenceList)
{
ObjectAddress sequenceAddress = { 0 };
ObjectAddressSet(sequenceAddress, RelationRelationId, sequenceOid);
EnsureDependenciesExistOnAllNodes(&sequenceAddress);
/*
* Sequences are not marked as distributed while creating table
* if no metadata worker node exists. We are marking all sequences
* distributed while syncing metadata in such case.
*/
MarkObjectDistributed(&sequenceAddress);
}
SetLocalEnableDependencyCreation(prevDependencyCreationValue);
List *workerSequenceDDLCommands = SequenceDDLCommandsForTable(relationId);
metadataSnapshotCommandList = list_concat(metadataSnapshotCommandList,
workerSequenceDDLCommands);
/* ddlCommandList contains TableDDLCommand information, need to materialize */
TableDDLCommand *tableDDLCommand = NULL;
foreach_ptr(tableDDLCommand, ddlCommandList)
{
Assert(CitusIsA(tableDDLCommand, TableDDLCommand));
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
GetTableDDLCommand(tableDDLCommand));
}
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
tableOwnerResetCommand);
List *sequenceDependencyCommandList = SequenceDependencyCommandList(
relationId);
metadataSnapshotCommandList = list_concat(metadataSnapshotCommandList,
sequenceDependencyCommandList);
}
/* construct the foreign key constraints after all tables are created */
foreach_ptr(cacheEntry, propagatedTableList)
{
Oid relationId = cacheEntry->relationId;
if (IsTableOwnedByExtension(relationId))
{
/* skip foreign key creation when the Citus table is owned by an extension */
continue;
}
List *foreignConstraintCommands =
GetReferencingForeignConstaintCommands(relationId);
metadataSnapshotCommandList = list_concat(metadataSnapshotCommandList,
foreignConstraintCommands);
}
/* construct partitioning hierarchy after all tables are created */
foreach_ptr(cacheEntry, propagatedTableList)
{
Oid relationId = cacheEntry->relationId;
if (IsTableOwnedByExtension(relationId))
{
/* skip partition creation when the Citus table is owned by an extension */
continue;
}
if (PartitionTable(relationId))
{
char *alterTableAttachPartitionCommands =
GenerateAlterTableAttachPartitionCommand(relationId);
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
alterTableAttachPartitionCommands);
}
}
/* after all tables are created, create the metadata */
foreach_ptr(cacheEntry, propagatedTableList)
{
Oid relationId = cacheEntry->relationId;
/* add the table metadata command first*/
char *metadataCommand = DistributionCreateCommand(cacheEntry);
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
metadataCommand);
if (!IsForeignTable(relationId))
{
/* add the truncate trigger command after the table became distributed */
char *truncateTriggerCreateCommand =
TruncateTriggerCreateCommand(cacheEntry->relationId);
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
truncateTriggerCreateCommand);
}
/* add the pg_dist_shard{,placement} entries */
List *shardIntervalList = LoadShardIntervalList(relationId);
List *shardCreateCommandList = ShardListInsertCommand(shardIntervalList);
metadataSnapshotCommandList = list_concat(metadataSnapshotCommandList,
shardCreateCommandList);
}
/* As the last step, propagate the pg_dist_object entities */
if (ShouldPropagate())
{
List *distributedObjectSyncCommandList =
DistributedObjectMetadataSyncCommandList();
metadataSnapshotCommandList = list_concat(metadataSnapshotCommandList,
distributedObjectSyncCommandList);
}
return metadataSnapshotCommandList;
}
@ -724,7 +598,7 @@ MetadataCreateCommands(void)
* DistributedObjectMetadataSyncCommandList returns the necessary commands to create
* pg_dist_object entries on the new node.
*/
static List *
List *
DistributedObjectMetadataSyncCommandList(void)
{
HeapTuple pgDistObjectTup = NULL;
@ -829,118 +703,42 @@ DistributedObjectMetadataSyncCommandList(void)
/*
* GetDistributedTableDDLEvents returns the full set of DDL commands necessary to
* create the given distributed table on a worker. The list includes setting up any
* sequences, setting the owner of the table, inserting table and shard metadata,
* setting the truncate trigger and foreign key constraints.
* CitusTableMetadataCreateCommandList returns the set of commands necessary to
* create the given distributed table metadata on a worker.
*/
static List *
GetDistributedTableDDLEvents(Oid relationId)
List *
CitusTableMetadataCreateCommandList(Oid relationId)
{
CitusTableCacheEntry *cacheEntry = GetCitusTableCacheEntry(relationId);
List *commandList = NIL;
IncludeSequenceDefaults includeSequenceDefaults = WORKER_NEXTVAL_SEQUENCE_DEFAULTS;
/* if the table is owned by an extension we only propagate pg_dist_* records */
bool tableOwnedByExtension = IsTableOwnedByExtension(relationId);
if (!tableOwnedByExtension)
{
/*
* Commands to create the table, these commands are TableDDLCommands so lets
* materialize to the non-sharded version
*/
List *tableDDLCommands = GetFullTableCreationCommands(relationId,
includeSequenceDefaults);
TableDDLCommand *tableDDLCommand = NULL;
foreach_ptr(tableDDLCommand, tableDDLCommands)
{
Assert(CitusIsA(tableDDLCommand, TableDDLCommand));
commandList = lappend(commandList, GetTableDDLCommand(tableDDLCommand));
}
/* command to associate sequences with table */
List *sequenceDependencyCommandList = SequenceDependencyCommandList(
relationId);
commandList = list_concat(commandList, sequenceDependencyCommandList);
}
/* command to insert pg_dist_partition entry */
char *metadataCommand = DistributionCreateCommand(cacheEntry);
commandList = lappend(commandList, metadataCommand);
/* commands to create the truncate trigger of the table */
if (!IsForeignTable(relationId))
{
char *truncateTriggerCreateCommand = TruncateTriggerCreateCommand(relationId);
commandList = lappend(commandList, truncateTriggerCreateCommand);
}
/* commands to insert pg_dist_shard & pg_dist_placement entries */
List *shardIntervalList = LoadShardIntervalList(relationId);
List *shardMetadataInsertCommandList = ShardListInsertCommand(shardIntervalList);
commandList = list_concat(commandList, shardMetadataInsertCommandList);
if (!tableOwnedByExtension)
{
/* commands to create foreign key constraints */
List *foreignConstraintCommands =
GetReferencingForeignConstaintCommands(relationId);
commandList = list_concat(commandList, foreignConstraintCommands);
/* commands to create partitioning hierarchy */
if (PartitionTable(relationId))
{
char *alterTableAttachPartitionCommands =
GenerateAlterTableAttachPartitionCommand(relationId);
commandList = lappend(commandList, alterTableAttachPartitionCommands);
}
}
return commandList;
}
/*
* MetadataDropCommands returns list of queries that are required to
* drop all the metadata of the node that are related to clustered tables.
* NodeMetadataDropCommands returns list of queries that are required to
* drop all the metadata of the node that are not related to clustered tables.
* The drop metadata snapshot commands includes the following queries:
*
* (i) Query to disable DDL propagation (necessary for (ii)
* (ii) Queries that DETACH all partitions of distributed tables
* (iii) Queries that delete all the rows from pg_dist_node table
* (iv) Queries that drop the clustered tables and remove its references from
* the pg_dist_partition. Note that distributed relation ids are gathered
* from the worker itself to prevent dropping any non-distributed tables
* with the same name.
* (v) Queries that delete all the rows from pg_dist_shard table referenced by (iv)
* (vi) Queries that delete all the rows from pg_dist_placement table
* referenced by (v)
* (vii) Queries that delete all the rows from pg_dist_object table
* (i) Queries that delete all the rows from pg_dist_node table
*/
List *
MetadataDropCommands(void)
NodeMetadataDropCommands(void)
{
List *dropSnapshotCommandList = NIL;
List *detachPartitionCommandList = DetachPartitionCommandList();
dropSnapshotCommandList = list_concat(dropSnapshotCommandList,
detachPartitionCommandList);
/*
* We are re-creating the metadata, so not lose track of the
* sequences by preventing them dropped via DROP TABLE.
*/
dropSnapshotCommandList =
lappend(dropSnapshotCommandList,
BREAK_CITUS_TABLE_SEQUENCE_DEPENDENCY_COMMAND);
dropSnapshotCommandList = lappend(dropSnapshotCommandList,
REMOVE_ALL_CITUS_TABLES_COMMAND);
dropSnapshotCommandList = lappend(dropSnapshotCommandList, DELETE_ALL_NODES);
dropSnapshotCommandList = lappend(dropSnapshotCommandList,
DELETE_ALL_DISTRIBUTED_OBJECTS);
return dropSnapshotCommandList;
}
@ -1153,8 +951,8 @@ citus_internal_add_object_metadata(PG_FUNCTION_ARGS)
argsArray);
/* First, disable propagation off to not to cause infinite propagation */
bool prevDependencyCreationValue = EnableDependencyCreation;
SetLocalEnableDependencyCreation(false);
bool prevDependencyCreationValue = EnableMetadataSync;
SetLocalEnableMetadataSync(false);
MarkObjectDistributed(&objectAddress);
@ -1181,7 +979,7 @@ citus_internal_add_object_metadata(PG_FUNCTION_ARGS)
forceDelegationAddress);
}
SetLocalEnableDependencyCreation(prevDependencyCreationValue);
SetLocalEnableMetadataSync(prevDependencyCreationValue);
PG_RETURN_VOID();
}
@ -1530,7 +1328,7 @@ PlacementUpsertCommand(uint64 shardId, uint64 placementId, int shardState,
* LocalGroupIdUpdateCommand creates the SQL command required to set the local group id
* of a worker and returns the command in a string.
*/
static char *
char *
LocalGroupIdUpdateCommand(int32 groupId)
{
StringInfo updateCommand = makeStringInfo();
@ -1542,36 +1340,6 @@ LocalGroupIdUpdateCommand(int32 groupId)
}
/*
* SequenceDDLCommandsForTable returns a list of commands which create sequences (and
* their schemas) to run on workers before creating the relation. The sequence creation
* commands are wrapped with a `worker_apply_sequence_command` call, which sets the
* sequence space uniquely for each worker. Notice that this function is relevant only
* during metadata propagation to workers and adds nothing to the list of sequence
* commands if none of the workers is marked as receiving metadata changes.
*/
List *
SequenceDDLCommandsForTable(Oid relationId)
{
List *allSequencesDDLList = NIL;
List *attnumList = NIL;
List *dependentSequenceList = NIL;
GetDependentSequencesWithRelation(relationId, &attnumList, &dependentSequenceList, 0);
char *ownerName = TableOwner(relationId);
Oid sequenceOid = InvalidOid;
foreach_oid(sequenceOid, dependentSequenceList)
{
List *sequenceDDLCommands = DDLCommandsForSequence(sequenceOid, ownerName);
allSequencesDDLList = list_concat(allSequencesDDLList, sequenceDDLCommands);
}
return allSequencesDDLList;
}
/*
* DDLCommandsForSequence returns the DDL commands needs to be run to create the
* sequence and alter the owner to the given owner name.
@ -1785,7 +1553,7 @@ GetSequencesFromAttrDef(Oid attrdefOid)
* necessary to ensure that the sequence is dropped when the table is
* dropped.
*/
static List *
List *
SequenceDependencyCommandList(Oid relationId)
{
List *sequenceCommandList = NIL;
@ -1815,7 +1583,8 @@ SequenceDependencyCommandList(Oid relationId)
CreateSequenceDependencyCommand(relationId, sequenceId, columnName);
sequenceCommandList = lappend(sequenceCommandList,
sequenceDependencyCommand);
makeTableDDLCommandString(
sequenceDependencyCommand));
}
return sequenceCommandList;
@ -2079,12 +1848,26 @@ GetRoleSpecObjectForGrantStmt(Oid roleOid)
/*
* SetLocalEnableDependencyCreation sets the enable_object_propagation locally
* SetLocalEnableMetadataSync sets the enable_metadata_sync locally
*/
static void
SetLocalEnableDependencyCreation(bool state)
SetLocalEnableMetadataSync(bool state)
{
set_config_option("citus.enable_object_propagation", state == true ? "on" : "off",
set_config_option("citus.enable_metadata_sync", state == true ? "on" : "off",
(superuser() ? PGC_SUSET : PGC_USERSET), PGC_S_SESSION,
GUC_ACTION_LOCAL, true, 0, false);
}
/*
* SetLocalReplicateReferenceTablesOnActivate sets the
* replicate_reference_tables_on_activate locally
*/
void
SetLocalReplicateReferenceTablesOnActivate(bool state)
{
set_config_option("citus.replicate_reference_tables_on_activate",
state == true ? "on" : "off",
(superuser() ? PGC_SUSET : PGC_USERSET), PGC_S_SESSION,
GUC_ACTION_LOCAL, true, 0, false);
}
@ -2104,7 +1887,7 @@ GenerateSetRoleQuery(Oid roleOid)
* TruncateTriggerCreateCommand creates a SQL query calling worker_create_truncate_trigger
* function, which creates the truncate trigger on the worker.
*/
static char *
TableDDLCommand *
TruncateTriggerCreateCommand(Oid relationId)
{
StringInfo triggerCreateCommand = makeStringInfo();
@ -2114,7 +1897,10 @@ TruncateTriggerCreateCommand(Oid relationId)
"SELECT worker_create_truncate_trigger(%s)",
quote_literal_cstr(tableName));
return triggerCreateCommand->data;
TableDDLCommand *triggerDDLCommand = makeTableDDLCommandString(
triggerCreateCommand->data);
return triggerDDLCommand;
}
@ -2168,16 +1954,101 @@ HasMetadataWorkers(void)
/*
* CreateTableMetadataOnWorkers creates the list of commands needed to create the
* given distributed table and sends these commands to all metadata workers i.e. workers
* with hasmetadata=true. Before sending the commands, in order to prevent recursive
* propagation, DDL propagation on workers are disabled with a
* `SET citus.enable_ddl_propagation TO off;` command.
* CreateInterTableRelationshipOfRelationOnWorkers create inter table relationship
* for the the given relation id on each worker node with metadata.
*/
void
CreateInterTableRelationshipOfRelationOnWorkers(Oid relationId)
{
/* if the table is owned by an extension we don't create */
bool tableOwnedByExtension = IsTableOwnedByExtension(relationId);
if (tableOwnedByExtension)
{
return;
}
List *commandList =
InterTableRelationshipOfRelationCommandList(relationId);
/* prevent recursive propagation */
SendCommandToWorkersWithMetadata(DISABLE_DDL_PROPAGATION);
const char *command = NULL;
foreach_ptr(command, commandList)
{
SendCommandToWorkersWithMetadata(command);
}
}
/*
* InterTableRelationshipOfRelationCommandList returns the command list to create
* inter table relationship for the given relation.
*/
List *
InterTableRelationshipOfRelationCommandList(Oid relationId)
{
/* commands to create foreign key constraints */
List *commandList = GetReferencingForeignConstaintCommands(relationId);
/* commands to create partitioning hierarchy */
if (PartitionTable(relationId))
{
char *alterTableAttachPartitionCommands =
GenerateAlterTableAttachPartitionCommand(relationId);
commandList = lappend(commandList, alterTableAttachPartitionCommands);
}
return commandList;
}
/*
* CreateShellTableOnWorkers creates the shell table on each worker node with metadata
* including sequence dependency and truncate triggers.
*/
static void
CreateShellTableOnWorkers(Oid relationId)
{
if (IsTableOwnedByExtension(relationId))
{
return;
}
List *commandList = list_make1(DISABLE_DDL_PROPAGATION);
IncludeSequenceDefaults includeSequenceDefaults = WORKER_NEXTVAL_SEQUENCE_DEFAULTS;
bool creatingShellTableOnRemoteNode = true;
List *tableDDLCommands = GetFullTableCreationCommands(relationId,
includeSequenceDefaults,
creatingShellTableOnRemoteNode);
TableDDLCommand *tableDDLCommand = NULL;
foreach_ptr(tableDDLCommand, tableDDLCommands)
{
Assert(CitusIsA(tableDDLCommand, TableDDLCommand));
commandList = lappend(commandList, GetTableDDLCommand(tableDDLCommand));
}
const char *command = NULL;
foreach_ptr(command, commandList)
{
SendCommandToWorkersWithMetadata(command);
}
}
/*
* CreateTableMetadataOnWorkers creates the list of commands needed to create the
* metadata of the given distributed table and sends these commands to all metadata
* workers i.e. workers with hasmetadata=true. Before sending the commands, in order
* to prevent recursive propagation, DDL propagation on workers are disabled with a
* `SET citus.enable_ddl_propagation TO off;` command.
*/
static void
CreateTableMetadataOnWorkers(Oid relationId)
{
List *commandList = GetDistributedTableDDLEvents(relationId);
List *commandList = CitusTableMetadataCreateCommandList(relationId);
/* prevent recursive propagation */
SendCommandToWorkersWithMetadata(DISABLE_DDL_PROPAGATION);
@ -2200,7 +2071,7 @@ CreateTableMetadataOnWorkers(Oid relationId)
* an extra step, if there are no partitions to DETACH, this function simply returns
* empty list to not disable/enable DDL propagation for nothing.
*/
static List *
List *
DetachPartitionCommandList(void)
{
List *detachPartitionCommandList = NIL;
@ -2242,7 +2113,7 @@ DetachPartitionCommandList(void)
/*
* SyncMetadataToNodes tries recreating the metadata snapshot in the
* SyncNodeMetadataToNodes tries recreating the metadata snapshot in the
* metadata workers that are out of sync. Returns the result of
* synchronization.
*
@ -2250,13 +2121,13 @@ DetachPartitionCommandList(void)
* since updates on the pg_dist_node metadata must be rollbacked if anything
* goes wrong.
*/
static MetadataSyncResult
SyncMetadataToNodes(void)
static NodeMetadataSyncResult
SyncNodeMetadataToNodes(void)
{
MetadataSyncResult result = METADATA_SYNC_SUCCESS;
NodeMetadataSyncResult result = NODE_METADATA_SYNC_SUCCESS;
if (!IsCoordinator())
{
return METADATA_SYNC_SUCCESS;
return NODE_METADATA_SYNC_SUCCESS;
}
/*
@ -2266,7 +2137,7 @@ SyncMetadataToNodes(void)
*/
if (!ConditionalLockRelationOid(DistNodeRelationId(), RowExclusiveLock))
{
return METADATA_SYNC_FAILED_LOCK;
return NODE_METADATA_SYNC_FAILED_LOCK;
}
List *syncedWorkerList = NIL;
@ -2277,12 +2148,12 @@ SyncMetadataToNodes(void)
if (workerNode->hasMetadata && !workerNode->metadataSynced)
{
bool raiseInterrupts = false;
if (!SyncMetadataSnapshotToNode(workerNode, raiseInterrupts))
if (!SyncNodeMetadataSnapshotToNode(workerNode, raiseInterrupts))
{
ereport(WARNING, (errmsg("failed to sync metadata to %s:%d",
workerNode->workerName,
workerNode->workerPort)));
result = METADATA_SYNC_FAILED_SYNC;
result = NODE_METADATA_SYNC_FAILED_SYNC;
}
else
{
@ -2303,7 +2174,7 @@ SyncMetadataToNodes(void)
if (!nodeUpdated->metadataSynced)
{
/* set the result to FAILED to trigger the sync again */
result = METADATA_SYNC_FAILED_SYNC;
result = NODE_METADATA_SYNC_FAILED_SYNC;
}
}
@ -2312,11 +2183,11 @@ SyncMetadataToNodes(void)
/*
* SyncMetadataToNodesMain is the main function for syncing metadata to
* SyncNodeMetadataToNodesMain is the main function for syncing node metadata to
* MX nodes. It retries until success and then exits.
*/
void
SyncMetadataToNodesMain(Datum main_arg)
SyncNodeMetadataToNodesMain(Datum main_arg)
{
Oid databaseOid = DatumGetObjectId(main_arg);
@ -2357,11 +2228,11 @@ SyncMetadataToNodesMain(Datum main_arg)
{
UseCoordinatedTransaction();
MetadataSyncResult result = SyncMetadataToNodes();
syncedAllNodes = (result == METADATA_SYNC_SUCCESS);
NodeMetadataSyncResult result = SyncNodeMetadataToNodes();
syncedAllNodes = (result == NODE_METADATA_SYNC_SUCCESS);
/* we use LISTEN/NOTIFY to wait for metadata syncing in tests */
if (result != METADATA_SYNC_FAILED_LOCK)
if (result != NODE_METADATA_SYNC_FAILED_LOCK)
{
Async_Notify(METADATA_SYNC_CHANNEL, NULL);
}
@ -2445,11 +2316,11 @@ MetadataSyncSigAlrmHandler(SIGNAL_ARGS)
/*
* SpawnSyncMetadataToNodes starts a background worker which runs metadata
* SpawnSyncNodeMetadataToNodes starts a background worker which runs node metadata
* sync. On success it returns workers' handle. Otherwise it returns NULL.
*/
BackgroundWorkerHandle *
SpawnSyncMetadataToNodes(Oid database, Oid extensionOwner)
SpawnSyncNodeMetadataToNodes(Oid database, Oid extensionOwner)
{
BackgroundWorker worker;
BackgroundWorkerHandle *handle = NULL;
@ -2467,7 +2338,7 @@ SpawnSyncMetadataToNodes(Oid database, Oid extensionOwner)
worker.bgw_restart_time = BGW_NEVER_RESTART;
strcpy_s(worker.bgw_library_name, sizeof(worker.bgw_library_name), "citus");
strcpy_s(worker.bgw_function_name, sizeof(worker.bgw_library_name),
"SyncMetadataToNodesMain");
"SyncNodeMetadataToNodesMain");
worker.bgw_main_arg = ObjectIdGetDatum(MyDatabaseId);
memcpy_s(worker.bgw_extra, sizeof(worker.bgw_extra), &extensionOwner,
sizeof(Oid));
@ -2766,7 +2637,8 @@ EnsureCoordinatorInitiatedOperation(void)
* check. The other two checks are to ensure that the operation is initiated
* by the coordinator.
*/
if (!IsCitusInitiatedRemoteBackend() || !MyBackendIsInDisributedTransaction() ||
if (!(IsCitusInternalBackend() || IsRebalancerInternalBackend()) ||
!MyBackendIsInDisributedTransaction() ||
GetLocalGroupId() == COORDINATOR_GROUP_ID)
{
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),

View File

@ -40,8 +40,10 @@
#include "distributed/remote_commands.h"
#include "distributed/resource_lock.h"
#include "distributed/shardinterval_utils.h"
#include "distributed/multi_partitioning_utils.h"
#include "distributed/shared_connection_stats.h"
#include "distributed/string_utils.h"
#include "distributed/metadata/pg_dist_object.h"
#include "distributed/transaction_recovery.h"
#include "distributed/version_compat.h"
#include "distributed/worker_manager.h"
@ -75,7 +77,7 @@ bool ReplicateReferenceTablesOnActivate = true;
/* did current transaction modify pg_dist_node? */
bool TransactionModifiedNodeMetadata = false;
bool EnableMetadataSyncByDefault = true;
bool EnableMetadataSync = true;
typedef struct NodeMetadata
{
@ -90,7 +92,6 @@ typedef struct NodeMetadata
} NodeMetadata;
/* local function forward declarations */
static int ActivateNode(char *nodeName, int nodePort);
static void RemoveNodeFromCluster(char *nodeName, int32 nodePort);
static void ErrorIfNodeContainsNonRemovablePlacements(WorkerNode *workerNode);
static bool PlacementHasActivePlacementOnAnotherGroup(GroupShardPlacement
@ -105,9 +106,12 @@ static void InsertPlaceholderCoordinatorRecord(void);
static void InsertNodeRow(int nodeid, char *nodename, int32 nodeport, NodeMetadata
*nodeMetadata);
static void DeleteNodeRow(char *nodename, int32 nodeport);
static void SetUpDistributedTableDependencies(WorkerNode *workerNode);
static void SyncDistributedObjectsToNode(WorkerNode *workerNode);
static void UpdateLocalGroupIdOnNode(WorkerNode *workerNode);
static void SyncPgDistTableMetadataToNode(WorkerNode *workerNode);
static List * InterTableRelationshipCommandList();
static WorkerNode * TupleToWorkerNode(TupleDesc tupleDescriptor, HeapTuple heapTuple);
static void PropagateNodeWideObjects(WorkerNode *newWorkerNode);
static List * PropagateNodeWideObjectsCommandList();
static WorkerNode * ModifiableWorkerNode(const char *nodeName, int32 nodePort);
static bool NodeIsLocal(WorkerNode *worker);
static void SetLockTimeoutLocally(int32 lock_cooldown);
@ -573,54 +577,118 @@ master_set_node_property(PG_FUNCTION_ARGS)
/*
* SetUpDistributedTableDependencies sets up up the following on a node if it's
* a primary node that currently stores data:
* - All dependencies (e.g., types, schemas)
* - Reference tables, because they are needed to handle queries efficiently.
* - Distributed functions
* InterTableRelationshipCommandList returns the command list to
* set up the multiple integrations including
*
* Note that we do not create the distributed dependencies on the coordinator
* since all the dependencies should be present in the coordinator already.
* (i) Foreign keys
* (ii) Partionining hierarchy
*
* for each citus table.
*/
static void
SetUpDistributedTableDependencies(WorkerNode *newWorkerNode)
static List *
InterTableRelationshipCommandList()
{
if (NodeIsPrimary(newWorkerNode))
List *distributedTableList = CitusTableList();
List *propagatedTableList = NIL;
List *multipleTableIntegrationCommandList = NIL;
CitusTableCacheEntry *cacheEntry = NULL;
foreach_ptr(cacheEntry, distributedTableList)
{
EnsureNoModificationsHaveBeenDone();
if (ShouldPropagate() && !NodeIsCoordinator(newWorkerNode))
/*
* Skip foreign key and partition creation when we shouldn't need to sync
* tablem metadata or the Citus table is owned by an extension.
*/
if (ShouldSyncTableMetadata(cacheEntry->relationId) &&
!IsTableOwnedByExtension(cacheEntry->relationId))
{
PropagateNodeWideObjects(newWorkerNode);
ReplicateAllDependenciesToNode(newWorkerNode->workerName,
newWorkerNode->workerPort);
}
else if (!NodeIsCoordinator(newWorkerNode))
{
ereport(WARNING, (errmsg("citus.enable_object_propagation is off, not "
"creating distributed objects on worker"),
errdetail("distributed objects are only kept in sync when "
"citus.enable_object_propagation is set to on. "
"Newly activated nodes will not get these "
"objects created")));
}
if (ReplicateReferenceTablesOnActivate)
{
ReplicateAllReferenceTablesToNode(newWorkerNode->workerName,
newWorkerNode->workerPort);
propagatedTableList = lappend(propagatedTableList, cacheEntry);
}
}
foreach_ptr(cacheEntry, propagatedTableList)
{
Oid relationId = cacheEntry->relationId;
List *commandListForRelation =
InterTableRelationshipOfRelationCommandList(relationId);
multipleTableIntegrationCommandList = list_concat(
multipleTableIntegrationCommandList,
commandListForRelation);
}
multipleTableIntegrationCommandList = lcons(DISABLE_DDL_PROPAGATION,
multipleTableIntegrationCommandList);
multipleTableIntegrationCommandList = lappend(multipleTableIntegrationCommandList,
ENABLE_DDL_PROPAGATION);
return multipleTableIntegrationCommandList;
}
/*
* PropagateNodeWideObjects is called during node activation to propagate any object that
* should be propagated for every node. These are generally not linked to any distributed
* object but change system wide behaviour.
* PgDistTableMetadataSyncCommandList returns the command list to sync the pg_dist_*
* (except pg_dist_node) metadata. We call them as table metadata.
*/
static void
PropagateNodeWideObjects(WorkerNode *newWorkerNode)
List *
PgDistTableMetadataSyncCommandList(void)
{
List *distributedTableList = CitusTableList();
List *propagatedTableList = NIL;
List *metadataSnapshotCommandList = NIL;
/* create the list of tables whose metadata will be created */
CitusTableCacheEntry *cacheEntry = NULL;
foreach_ptr(cacheEntry, distributedTableList)
{
if (ShouldSyncTableMetadata(cacheEntry->relationId))
{
propagatedTableList = lappend(propagatedTableList, cacheEntry);
}
}
/* remove all dist table and object related metadata first */
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
DELETE_ALL_PARTITIONS);
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList, DELETE_ALL_SHARDS);
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
DELETE_ALL_PLACEMENTS);
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
DELETE_ALL_DISTRIBUTED_OBJECTS);
/* create pg_dist_partition, pg_dist_shard and pg_dist_placement entries */
foreach_ptr(cacheEntry, propagatedTableList)
{
List *tableMetadataCreateCommandList =
CitusTableMetadataCreateCommandList(cacheEntry->relationId);
metadataSnapshotCommandList = list_concat(metadataSnapshotCommandList,
tableMetadataCreateCommandList);
}
/* As the last step, propagate the pg_dist_object entities */
Assert(ShouldPropagate());
List *distributedObjectSyncCommandList = DistributedObjectMetadataSyncCommandList();
metadataSnapshotCommandList = list_concat(metadataSnapshotCommandList,
distributedObjectSyncCommandList);
metadataSnapshotCommandList = lcons(DISABLE_DDL_PROPAGATION,
metadataSnapshotCommandList);
metadataSnapshotCommandList = lappend(metadataSnapshotCommandList,
ENABLE_DDL_PROPAGATION);
return metadataSnapshotCommandList;
}
/*
* PropagateNodeWideObjectsCommandList is called during node activation to
* propagate any object that should be propagated for every node. These are
* generally not linked to any distributed object but change system wide behaviour.
*/
static List *
PropagateNodeWideObjectsCommandList()
{
/* collect all commands */
List *ddlCommands = NIL;
@ -640,13 +708,141 @@ PropagateNodeWideObjects(WorkerNode *newWorkerNode)
/* if there are command wrap them in enable_ddl_propagation off */
ddlCommands = lcons(DISABLE_DDL_PROPAGATION, ddlCommands);
ddlCommands = lappend(ddlCommands, ENABLE_DDL_PROPAGATION);
}
return ddlCommands;
}
/*
* SyncDistributedObjectsCommandList returns commands to sync object dependencies
* to the given worker node. To be idempotent, it first drops the ones required to be
* dropped.
*
* Object dependencies include:
*
* - All dependencies (e.g., types, schemas, sequences)
* - All shell distributed tables
* - Inter relation between those shell tables
* - Node wide objects
*
* We also update the local group id here, as handling sequence dependencies
* requires it.
*/
List *
SyncDistributedObjectsCommandList(WorkerNode *workerNode)
{
List *commandList = NIL;
/*
* Propagate node wide objects. It includes only roles for now.
*/
commandList = list_concat(commandList, PropagateNodeWideObjectsCommandList());
/*
* Detach partitions, break dependencies between sequences and table then
* remove shell tables first.
*/
commandList = list_concat(commandList, DetachPartitionCommandList());
commandList = lappend(commandList, BREAK_CITUS_TABLE_SEQUENCE_DEPENDENCY_COMMAND);
commandList = lappend(commandList, REMOVE_ALL_SHELL_TABLES_COMMAND);
/*
* Replicate all objects of the pg_dist_object to the remote node.
*/
commandList = list_concat(commandList, ReplicateAllObjectsToNodeCommandList(
workerNode->workerName, workerNode->workerPort));
/*
* After creating each table, handle the inter table relationship between
* those tables.
*/
commandList = list_concat(commandList, InterTableRelationshipCommandList());
return commandList;
}
/*
* SyncDistributedObjectsToNode sync the distributed objects to the node. It includes
* - All dependencies (e.g., types, schemas, sequences)
* - All shell distributed table
* - Inter relation between those shell tables
*
* Note that we do not create the distributed dependencies on the coordinator
* since all the dependencies should be present in the coordinator already.
*/
static void
SyncDistributedObjectsToNode(WorkerNode *workerNode)
{
if (NodeIsCoordinator(workerNode))
{
/* coordinator has all the objects */
return;
}
if (!NodeIsPrimary(workerNode))
{
/* secondary nodes gets the objects from their primaries via replication */
return;
}
EnsureSequentialModeMetadataOperations();
Assert(ShouldPropagate());
List *commandList = SyncDistributedObjectsCommandList(workerNode);
/* send commands to new workers, the current user should be a superuser */
Assert(superuser());
SendMetadataCommandListToWorkerInCoordinatedTransaction(
workerNode->workerName,
workerNode->workerPort,
CurrentUserName(),
commandList);
}
/*
* UpdateLocalGroupIdOnNode updates local group id on node.
*/
static void
UpdateLocalGroupIdOnNode(WorkerNode *workerNode)
{
if (NodeIsPrimary(workerNode) && !NodeIsCoordinator(workerNode))
{
List *commandList = list_make1(LocalGroupIdUpdateCommand(workerNode->groupId));
/* send commands to new workers, the current user should be a superuser */
Assert(superuser());
SendMetadataCommandListToWorkerInCoordinatedTransaction(newWorkerNode->workerName,
newWorkerNode->workerPort,
CurrentUserName(),
ddlCommands);
SendMetadataCommandListToWorkerInCoordinatedTransaction(
workerNode->workerName,
workerNode->workerPort,
CurrentUserName(),
commandList);
}
}
/*
* SyncPgDistTableMetadataToNode syncs the pg_dist_partition, pg_dist_shard
* pg_dist_placement and pg_dist_object metadata entries.
*
*/
static void
SyncPgDistTableMetadataToNode(WorkerNode *workerNode)
{
if (NodeIsPrimary(workerNode) && !NodeIsCoordinator(workerNode))
{
List *syncPgDistMetadataCommandList = PgDistTableMetadataSyncCommandList();
/* send commands to new workers, the current user should be a superuser */
Assert(superuser());
SendMetadataCommandListToWorkerInCoordinatedTransaction(
workerNode->workerName,
workerNode->workerPort,
CurrentUserName(),
syncPgDistMetadataCommandList);
}
}
@ -847,7 +1043,7 @@ PrimaryNodeForGroup(int32 groupId, bool *groupContainsNodes)
* includes only replicating the reference tables and setting isactive column of the
* given node.
*/
static int
int
ActivateNode(char *nodeName, int nodePort)
{
bool isActive = true;
@ -897,8 +1093,9 @@ ActivateNode(char *nodeName, int nodePort)
workerNode =
SetWorkerColumnLocalOnly(workerNode, Anum_pg_dist_node_isactive,
BoolGetDatum(isActive));
bool syncMetadata =
EnableMetadataSyncByDefault && NodeIsPrimary(workerNode);
/* TODO: Once all tests will be enabled for MX, we can remove sync by default check */
bool syncMetadata = EnableMetadataSync && NodeIsPrimary(workerNode);
if (syncMetadata)
{
@ -908,13 +1105,43 @@ ActivateNode(char *nodeName, int nodePort)
*/
SetWorkerColumn(workerNode, Anum_pg_dist_node_metadatasynced,
BoolGetDatum(true));
}
SetUpDistributedTableDependencies(workerNode);
/*
* Update local group id first, as object dependency logic requires to have
* updated local group id.
*/
UpdateLocalGroupIdOnNode(workerNode);
if (syncMetadata)
{
StartMetadataSyncToNode(nodeName, nodePort);
/*
* Sync distributed objects first. We must sync distributed objects before
* replicating reference tables to the remote node, as reference tables may
* need such objects.
*/
SyncDistributedObjectsToNode(workerNode);
/*
* We need to replicate reference tables before syncing node metadata, otherwise
* reference table replication logic would try to get lock on the new node before
* having the shard placement on it
*/
if (ReplicateReferenceTablesOnActivate)
{
ReplicateAllReferenceTablesToNode(workerNode);
}
/*
* Sync node metadata. We must sync node metadata before syncing table
* related pg_dist_xxx metadata. Since table related metadata requires
* to have right pg_dist_node entries.
*/
SyncNodeMetadataToNode(nodeName, nodePort);
/*
* As the last step, sync the table related metadata to the remote node.
* We must handle it as the last step because of limitations shared with
* above comments.
*/
SyncPgDistTableMetadataToNode(workerNode);
}
/* finally, let all other active metadata nodes to learn about this change */

View File

@ -38,7 +38,6 @@ static void ErrorIfCurrentUserCanNotDistributeObject(ObjectType type,
ObjectAddress *addr,
Node *node,
Relation *relation);
static void ErrorIfUserNotAllowedToPropagateExtension(char *extensionName);
static List * textarray_to_strvaluelist(ArrayType *arr);
/* It is defined on PG >= 13 versions by default */
@ -398,12 +397,8 @@ ErrorIfCurrentUserCanNotDistributeObject(ObjectType type, ObjectAddress *addr,
Node *node, Relation *relation)
{
Oid userId = GetUserId();
AclMode aclMaskResult = 0;
bool skipAclCheck = false;
Oid idToCheck = InvalidOid;
/* Since we don't handle sequences like object, add it separately */
if (!(SupportedDependencyByCitus(addr) || type == OBJECT_SEQUENCE))
if (!SupportedDependencyByCitus(addr))
{
ereport(ERROR, (errmsg("Object type %d can not be distributed by Citus", type)));
}
@ -411,27 +406,19 @@ ErrorIfCurrentUserCanNotDistributeObject(ObjectType type, ObjectAddress *addr,
switch (type)
{
case OBJECT_SCHEMA:
{
idToCheck = addr->objectId;
aclMaskResult = pg_namespace_aclmask(idToCheck, userId, ACL_USAGE,
ACLMASK_ANY);
break;
}
case OBJECT_DATABASE:
case OBJECT_FUNCTION:
case OBJECT_PROCEDURE:
case OBJECT_AGGREGATE:
case OBJECT_TYPE:
case OBJECT_FOREIGN_SERVER:
case OBJECT_SEQUENCE:
case OBJECT_FOREIGN_TABLE:
case OBJECT_TABLE:
case OBJECT_EXTENSION:
case OBJECT_COLLATION:
{
check_object_ownership(userId, type, *addr, node, *relation);
skipAclCheck = true;
break;
}
case OBJECT_DATABASE:
{
idToCheck = addr->objectId;
aclMaskResult = pg_database_aclmask(idToCheck, userId, ACL_CONNECT,
ACLMASK_ANY);
break;
}
@ -444,53 +431,6 @@ ErrorIfCurrentUserCanNotDistributeObject(ObjectType type, ObjectAddress *addr,
"access privileges on role %d with type %d",
addr->objectId, type)));
}
skipAclCheck = true;
break;
}
case OBJECT_TYPE:
{
idToCheck = addr->objectId;
aclMaskResult = pg_type_aclmask(idToCheck, userId, ACL_USAGE,
ACLMASK_ANY);
break;
}
case OBJECT_FOREIGN_SERVER:
{
idToCheck = addr->objectId;
aclMaskResult = pg_foreign_server_aclmask(idToCheck, userId, ACL_USAGE,
ACLMASK_ANY);
break;
}
case OBJECT_SEQUENCE:
{
idToCheck = addr->objectId;
aclMaskResult = pg_class_aclmask(idToCheck, userId, ACL_USAGE, ACLMASK_ANY);
break;
}
case OBJECT_TABLE:
{
/* table distribution already does the ownership check, so we can stick to that over acl_check */
check_object_ownership(userId, type, *addr, node, *relation);
skipAclCheck = true;
break;
}
case OBJECT_EXTENSION:
{
Value *valueNode = (Value *) node;
char *extensionName = strVal(valueNode);
ErrorIfUserNotAllowedToPropagateExtension(extensionName);
skipAclCheck = true;
break;
}
case OBJECT_COLLATION:
{
skipAclCheck = true;
break;
}
@ -501,119 +441,6 @@ ErrorIfCurrentUserCanNotDistributeObject(ObjectType type, ObjectAddress *addr,
break;
}
}
if (!skipAclCheck && aclMaskResult == ACL_NO_RIGHTS)
{
ereport(ERROR, (errmsg("Current user does not have required privileges "
"on %d with type id %d to distribute it",
idToCheck, type)));
}
}
/*
* ErrorIfUserNotAllowedToPropagateExtension errors out if the current user does
* not have required privileges to propagate extension
*/
static void
ErrorIfUserNotAllowedToPropagateExtension(char *extensionName)
{
const int nameAttributeIndex = 1;
const int superuserAttributeIndex = 4;
#if PG_VERSION_NUM >= PG_VERSION_13
const int trustedAttributeIndex = 5;
#endif
LOCAL_FCINFO(fcinfo, 0);
FmgrInfo flinfo;
bool goForward = true;
bool doCopy = false;
EState *estate = CreateExecutorState();
ReturnSetInfo *extensionsResultSet = makeNode(ReturnSetInfo);
extensionsResultSet->econtext = GetPerTupleExprContext(estate);
extensionsResultSet->allowedModes = SFRM_Materialize;
fmgr_info(F_PG_AVAILABLE_EXTENSION_VERSIONS, &flinfo);
InitFunctionCallInfoData(*fcinfo, &flinfo, 0, InvalidOid, NULL,
(Node *) extensionsResultSet);
/*
* pg_available_extensions_versions returns result set containing all
* available extension versions with whether the extension requires
* superuser and it is trusted information.
*/
(*pg_available_extension_versions)(fcinfo);
TupleTableSlot *tupleTableSlot = MakeSingleTupleTableSlotCompat(
extensionsResultSet->setDesc,
&TTSOpsMinimalTuple);
bool hasTuple = tuplestore_gettupleslot(extensionsResultSet->setResult,
goForward,
doCopy,
tupleTableSlot);
while (hasTuple)
{
bool isNull = false;
Datum curExtensionNameDatum = slot_getattr(tupleTableSlot,
nameAttributeIndex,
&isNull);
char *curExtensionName = NameStr(*DatumGetName(curExtensionNameDatum));
if (strcmp(extensionName, curExtensionName) == 0)
{
Datum superuserExpectedDatum = slot_getattr(tupleTableSlot,
superuserAttributeIndex,
&isNull);
bool superuserExpected = DatumGetBool(superuserExpectedDatum);
#if PG_VERSION_NUM < PG_VERSION_13
if (superuserExpected)
{
EnsureSuperUser();
}
#else
if (superuserExpected)
{
/*
* After PG 13, if the extension is trusted it can be created
* by the user having CREATE privilege on the database even if
* the extension requires superuser.
*/
Datum trustedExtensionDatum = slot_getattr(tupleTableSlot,
trustedAttributeIndex,
&isNull);
bool trustedExtension = DatumGetBool(trustedExtensionDatum);
if (trustedExtension)
{
/* Allow if user has CREATE privilege on current database */
AclResult aclresult = pg_database_aclcheck(MyDatabaseId,
GetUserId(),
ACL_CREATE);
if (aclresult != ACLCHECK_OK)
{
ereport(ERROR, (errmsg("operation is not allowed"),
errhint("Must have CREATE privilege "
"on database to propagate "
"extension %s", curExtensionName)));
}
}
else
{
EnsureSuperUser();
}
}
#endif
break;
}
ExecClearTuple(tupleTableSlot);
hasTuple = tuplestore_gettupleslot(extensionsResultSet->setResult, goForward,
doCopy, tupleTableSlot);
}
ExecDropSingleTupleTableSlot(tupleTableSlot);
}

View File

@ -67,7 +67,6 @@
/* Shard related configuration */
int ShardCount = 32;
int ShardReplicationFactor = 1; /* desired replication factor for shards */
int ShardPlacementPolicy = SHARD_PLACEMENT_ROUND_ROBIN;
int NextShardId = 0;
int NextPlacementId = 0;
@ -141,8 +140,10 @@ master_get_table_ddl_events(PG_FUNCTION_ARGS)
functionContext->multi_call_memory_ctx);
/* allocate DDL statements, and then save position in DDL statements */
bool creatingShellTableOnRemoteNode = false;
List *tableDDLEventList = GetFullTableCreationCommands(relationId,
includeSequenceDefaults);
includeSequenceDefaults,
creatingShellTableOnRemoteNode);
tableDDLEventCell = list_head(tableDDLEventList);
ListCellAndListWrapper *wrapper = palloc0(sizeof(ListCellAndListWrapper));
wrapper->list = tableDDLEventList;
@ -458,8 +459,9 @@ ResolveRelationId(text *relationName, bool missingOk)
* constraint and trigger definitions.
*/
List *
GetFullTableCreationCommands(Oid relationId, IncludeSequenceDefaults
includeSequenceDefaults)
GetFullTableCreationCommands(Oid relationId,
IncludeSequenceDefaults includeSequenceDefaults,
bool creatingShellTableOnRemoteNode)
{
List *tableDDLEventList = NIL;
@ -471,6 +473,25 @@ GetFullTableCreationCommands(Oid relationId, IncludeSequenceDefaults
List *postLoadCreationCommandList =
GetPostLoadTableCreationCommands(relationId, true, true);
if (creatingShellTableOnRemoteNode)
{
/*
* While creating shell tables, we need to associate dependencies between
* sequences and the relation. We also need to add truncate trigger for it
* if it is not the foreign table.
*/
List *sequenceDependencyCommandList = SequenceDependencyCommandList(relationId);
tableDDLEventList = list_concat(tableDDLEventList, sequenceDependencyCommandList);
if (!IsForeignTable(relationId))
{
TableDDLCommand *truncateTriggerCommand = TruncateTriggerCreateCommand(
relationId);
tableDDLEventList = lappend(tableDDLEventList,
truncateTriggerCommand);
}
}
tableDDLEventList = list_concat(tableDDLEventList, postLoadCreationCommandList);
return tableDDLEventList;

View File

@ -556,8 +556,12 @@ BlockWritesToShardList(List *shardList)
Oid firstDistributedTableId = firstShardInterval->relationId;
bool shouldSyncMetadata = ShouldSyncTableMetadata(firstDistributedTableId);
if (shouldSyncMetadata)
if (shouldSyncMetadata || !IsCoordinator())
{
/*
* Even if users disable metadata sync, we cannot allow them not to
* acquire the remote locks. Hence, we have !IsCoordinator() check.
*/
LockShardListMetadataOnWorkers(ExclusiveLock, shardList);
}
}

View File

@ -96,7 +96,7 @@ isolation_cleanup_orphaned_shards(PG_FUNCTION_ARGS)
void
DropOrphanedShardsInSeparateTransaction(void)
{
ExecuteCriticalCommandInSeparateTransaction("CALL citus_cleanup_orphaned_shards()");
ExecuteRebalancerCommandInSeparateTransaction("CALL citus_cleanup_orphaned_shards()");
}

View File

@ -919,10 +919,10 @@ citus_drain_node(PG_FUNCTION_ARGS)
* This is done in a separate session. This way it's not undone if the
* draining fails midway through.
*/
ExecuteCriticalCommandInSeparateTransaction(psprintf(
"SELECT master_set_node_property(%s, %i, 'shouldhaveshards', false)",
quote_literal_cstr(nodeName),
nodePort));
ExecuteRebalancerCommandInSeparateTransaction(psprintf(
"SELECT master_set_node_property(%s, %i, 'shouldhaveshards', false)",
quote_literal_cstr(nodeName),
nodePort));
RebalanceTableShards(&options, shardTransferModeOid);
@ -1696,7 +1696,7 @@ UpdateShardPlacement(PlacementUpdateEvent *placementUpdateEvent,
* In case of failure, we throw an error such that rebalance_table_shards
* fails early.
*/
ExecuteCriticalCommandInSeparateTransaction(placementUpdateCommand->data);
ExecuteRebalancerCommandInSeparateTransaction(placementUpdateCommand->data);
UpdateColocatedShardPlacementProgress(shardId,
sourceNode->workerName,
@ -1711,12 +1711,18 @@ UpdateShardPlacement(PlacementUpdateEvent *placementUpdateEvent,
* don't want to rollback when the current transaction is rolled back.
*/
void
ExecuteCriticalCommandInSeparateTransaction(char *command)
ExecuteRebalancerCommandInSeparateTransaction(char *command)
{
int connectionFlag = FORCE_NEW_CONNECTION;
MultiConnection *connection = GetNodeConnection(connectionFlag, LocalHostName,
PostPortNumber);
StringInfo setApplicationName = makeStringInfo();
appendStringInfo(setApplicationName, "SET application_name TO %s",
CITUS_REBALANCER_NAME);
ExecuteCriticalRemoteCommand(connection, setApplicationName->data);
ExecuteCriticalRemoteCommand(connection, command);
CloseConnection(connection);
}

View File

@ -171,26 +171,9 @@ master_create_empty_shard(PG_FUNCTION_ARGS)
/* first retrieve a list of random nodes for shard placements */
while (candidateNodeIndex < attemptableNodeCount)
{
WorkerNode *candidateNode = NULL;
if (ShardPlacementPolicy == SHARD_PLACEMENT_LOCAL_NODE_FIRST)
{
candidateNode = WorkerGetLocalFirstCandidateNode(candidateNodeList);
}
else if (ShardPlacementPolicy == SHARD_PLACEMENT_ROUND_ROBIN)
{
candidateNode = WorkerGetRoundRobinCandidateNode(workerNodeList, shardId,
candidateNodeIndex);
}
else if (ShardPlacementPolicy == SHARD_PLACEMENT_RANDOM)
{
candidateNode = WorkerGetRandomCandidateNode(candidateNodeList);
}
else
{
ereport(ERROR, (errmsg("unrecognized shard placement policy")));
}
WorkerNode *candidateNode = WorkerGetRoundRobinCandidateNode(workerNodeList,
shardId,
candidateNodeIndex);
if (candidateNode == NULL)
{
ereport(ERROR, (errmsg("could only find %u of %u possible nodes",
@ -325,8 +308,10 @@ CreateAppendDistributedShardPlacements(Oid relationId, int64 shardId,
List *foreignConstraintCommandList =
GetReferencingForeignConstaintCommands(relationId);
IncludeSequenceDefaults includeSequenceDefaults = NO_SEQUENCE_DEFAULTS;
bool creatingShellTableOnRemoteNode = false;
List *ddlCommandList = GetFullTableCreationCommands(relationId,
includeSequenceDefaults);
includeSequenceDefaults,
creatingShellTableOnRemoteNode);
uint32 connectionFlag = FOR_DDL;
char *relationOwner = TableOwner(relationId);
@ -438,8 +423,10 @@ CreateShardsOnWorkers(Oid distributedRelationId, List *shardPlacements,
bool useExclusiveConnection, bool colocatedShard)
{
IncludeSequenceDefaults includeSequenceDefaults = NO_SEQUENCE_DEFAULTS;
bool creatingShellTableOnRemoteNode = false;
List *ddlCommandList = GetFullTableCreationCommands(distributedRelationId,
includeSequenceDefaults);
includeSequenceDefaults,
creatingShellTableOnRemoteNode);
List *foreignConstraintCommandList =
GetReferencingForeignConstaintCommands(distributedRelationId);

View File

@ -40,12 +40,6 @@ int MaxWorkerNodesTracked = 2048; /* determines worker node hash table size *
/* Local functions forward declarations */
static WorkerNode * WorkerGetNodeWithName(const char *hostname);
static char * ClientHostAddress(StringInfo remoteHostStringInfo);
static List * PrimaryNodesNotInList(List *currentList);
static WorkerNode * FindRandomNodeFromList(List *candidateWorkerNodeList);
static bool OddNumber(uint32 number);
static bool ListMember(List *currentList, WorkerNode *workerNode);
static bool NodeIsPrimaryWorker(WorkerNode *node);
static bool NodeIsReadableWorker(WorkerNode *node);
@ -55,73 +49,6 @@ static bool NodeIsReadableWorker(WorkerNode *node);
* ------------------------------------------------------------
*/
/*
* WorkerGetRandomCandidateNode accepts a list of WorkerNode's and returns a random
* primary node which is not in that list.
*
* Note that the function returns null if the worker membership list does not
* contain enough nodes to allocate a new worker node.
*/
WorkerNode *
WorkerGetRandomCandidateNode(List *currentNodeList)
{
WorkerNode *workerNode = NULL;
bool wantSameRack = false;
uint32 tryCount = WORKER_RACK_TRIES;
uint32 currentNodeCount = list_length(currentNodeList);
List *candidateWorkerNodeList = PrimaryNodesNotInList(currentNodeList);
/* we check if the shard has already been placed on all nodes known to us */
if (list_length(candidateWorkerNodeList) == 0)
{
return NULL;
}
/* if current node list is empty, randomly pick one node and return */
if (currentNodeCount == 0)
{
workerNode = FindRandomNodeFromList(candidateWorkerNodeList);
return workerNode;
}
/*
* If the current list has an odd number of nodes (1, 3, 5, etc), we want to
* place the shard on a different rack than the first node's rack.
* Otherwise, we want to place the shard on the same rack as the first node.
*/
if (OddNumber(currentNodeCount))
{
wantSameRack = false;
}
else
{
wantSameRack = true;
}
/*
* We try to find a worker node that fits our rack-aware placement strategy.
* If after a predefined number of tries, we still cannot find such a node,
* we simply give up and return the last worker node we found.
*/
for (uint32 tryIndex = 0; tryIndex < tryCount; tryIndex++)
{
WorkerNode *firstNode = (WorkerNode *) linitial(currentNodeList);
char *firstRack = firstNode->workerRack;
workerNode = FindRandomNodeFromList(candidateWorkerNodeList);
char *workerRack = workerNode->workerRack;
bool sameRack = (strncmp(workerRack, firstRack, WORKER_LENGTH) == 0);
if ((sameRack && wantSameRack) || (!sameRack && !wantSameRack))
{
break;
}
}
return workerNode;
}
/*
* WorkerGetRoundRobinCandidateNode takes in a list of worker nodes and returns
@ -152,147 +79,6 @@ WorkerGetRoundRobinCandidateNode(List *workerNodeList, uint64 shardId,
}
/*
* WorkerGetLocalFirstCandidateNode takes in a list of worker nodes, and then
* allocates a new worker node. The allocation is performed according to the
* following policy: if the list is empty, the node where the caller is connecting
* from is allocated; if the list is not empty, a node is allocated according
* to random policy.
*/
WorkerNode *
WorkerGetLocalFirstCandidateNode(List *currentNodeList)
{
WorkerNode *candidateNode = NULL;
uint32 currentNodeCount = list_length(currentNodeList);
/* choose first candidate node to be the client's host */
if (currentNodeCount == 0)
{
StringInfo clientHostStringInfo = makeStringInfo();
char *errorMessage = ClientHostAddress(clientHostStringInfo);
if (errorMessage != NULL)
{
ereport(ERROR, (errmsg("%s", errorMessage),
errdetail("Could not find the first worker "
"node for local-node-first policy."),
errhint("Make sure that you are not on the "
"master node.")));
}
/* if hostname is localhost.localdomain, change it to localhost */
char *clientHost = clientHostStringInfo->data;
if (strncmp(clientHost, "localhost.localdomain", WORKER_LENGTH) == 0)
{
clientHost = pstrdup("localhost");
}
candidateNode = WorkerGetNodeWithName(clientHost);
if (candidateNode == NULL)
{
ereport(ERROR, (errmsg("could not find worker node for "
"host: %s", clientHost)));
}
}
else
{
/* find a candidate node different from those already selected */
candidateNode = WorkerGetRandomCandidateNode(currentNodeList);
}
return candidateNode;
}
/*
* ClientHostAddress appends the connecting client's fully qualified hostname
* to the given StringInfo. If there is no such connection or the connection is
* over Unix domain socket, the function fills the error message and returns it.
* On success, it just returns NULL.
*/
static char *
ClientHostAddress(StringInfo clientHostStringInfo)
{
Port *port = MyProcPort;
char *clientHost = NULL;
char *errorMessage = NULL;
int clientHostLength = NI_MAXHOST;
int flags = NI_NAMEREQD; /* require fully qualified hostname */
int nameFound = 0;
if (port == NULL)
{
errorMessage = "cannot find tcp/ip connection to client";
return errorMessage;
}
switch (port->raddr.addr.ss_family)
{
case AF_INET:
#ifdef HAVE_IPV6
case AF_INET6:
#endif
{
break;
}
default:
{
errorMessage = "invalid address family in connection";
return errorMessage;
}
}
clientHost = palloc0(clientHostLength);
nameFound = pg_getnameinfo_all(&port->raddr.addr, port->raddr.salen,
clientHost, clientHostLength, NULL, 0, flags);
if (nameFound == 0)
{
appendStringInfo(clientHostStringInfo, "%s", clientHost);
}
else
{
StringInfo errorMessageStringInfo = makeStringInfo();
appendStringInfo(errorMessageStringInfo, "could not resolve client host: %s",
gai_strerror(nameFound));
errorMessage = errorMessageStringInfo->data;
return errorMessage;
}
return errorMessage;
}
/*
* WorkerGetNodeWithName finds and returns a node from the membership list that
* has the given hostname. The function returns null if no such node exists.
*/
static WorkerNode *
WorkerGetNodeWithName(const char *hostname)
{
WorkerNode *workerNode = NULL;
HASH_SEQ_STATUS status;
HTAB *workerNodeHash = GetWorkerNodeHash();
hash_seq_init(&status, workerNodeHash);
while ((workerNode = hash_seq_search(&status)) != NULL)
{
int nameCompare = strncmp(workerNode->workerName, hostname, WORKER_LENGTH);
if (nameCompare == 0)
{
/* we need to terminate the scan since we break */
hash_seq_term(&status);
break;
}
}
return workerNode;
}
/*
* ActivePrimaryNonCoordinatorNodeCount returns the number of groups with a primary in the cluster.
* This method excludes coordinator even if it is added as a worker to cluster.
@ -542,84 +328,6 @@ NodeIsReadableWorker(WorkerNode *node)
}
/*
* PrimaryNodesNotInList scans through the worker node hash and returns a list of all
* primary nodes which are not in currentList. It runs in O(n*m) but currentList is
* quite small.
*/
static List *
PrimaryNodesNotInList(List *currentList)
{
List *workerNodeList = NIL;
HTAB *workerNodeHash = GetWorkerNodeHash();
WorkerNode *workerNode = NULL;
HASH_SEQ_STATUS status;
hash_seq_init(&status, workerNodeHash);
while ((workerNode = hash_seq_search(&status)) != NULL)
{
if (ListMember(currentList, workerNode))
{
continue;
}
if (NodeIsPrimary(workerNode))
{
workerNodeList = lappend(workerNodeList, workerNode);
}
}
return workerNodeList;
}
/* FindRandomNodeFromList picks a random node from the list provided to it. */
static WorkerNode *
FindRandomNodeFromList(List *candidateWorkerNodeList)
{
uint32 candidateNodeCount = list_length(candidateWorkerNodeList);
/* nb, the random seed has already been set by the postmaster when starting up */
uint32 workerPosition = (random() % candidateNodeCount);
WorkerNode *workerNode =
(WorkerNode *) list_nth(candidateWorkerNodeList, workerPosition);
return workerNode;
}
/*
* OddNumber function returns true if given number is odd; returns false otherwise.
*/
static bool
OddNumber(uint32 number)
{
bool oddNumber = ((number % 2) == 1);
return oddNumber;
}
/* Checks if given worker node is a member of the current list. */
static bool
ListMember(List *currentList, WorkerNode *workerNode)
{
Size keySize = WORKER_LENGTH + sizeof(uint32);
WorkerNode *currentNode = NULL;
foreach_ptr(currentNode, currentList)
{
if (WorkerNodeCompare(workerNode, currentNode, keySize) == 0)
{
return true;
}
}
return false;
}
/*
* CompareWorkerNodes compares two pointers to worker nodes using the exact
* same logic employed by WorkerNodeCompare.

View File

@ -260,7 +260,7 @@ TryToDelegateFunctionCall(DistributedPlanningContext *planContext)
ereport(DEBUG4, (errmsg("function is distributed")));
}
if (IsCitusInitiatedRemoteBackend())
if (IsCitusInternalBackend())
{
bool isFunctionForceDelegated = procedure->forceDelegation;
@ -303,17 +303,6 @@ TryToDelegateFunctionCall(DistributedPlanningContext *planContext)
return NULL;
}
if (localGroupId != COORDINATOR_GROUP_ID)
{
/*
* We are calling a distributed function on a worker node. We currently
* only delegate from the coordinator.
*
* TODO: remove this restriction.
*/
return NULL;
}
/*
* Cannot delegate functions for INSERT ... SELECT func(), since they require
* coordinated transactions.
@ -324,16 +313,6 @@ TryToDelegateFunctionCall(DistributedPlanningContext *planContext)
return NULL;
}
if (fromFuncExpr && !IsMultiStatementTransaction())
{
/*
* For now, let's not push the function from the FROM clause unless it's in a
* multistatement transaction with the forceDelegation flag ON.
*/
ereport(DEBUG2, (errmsg("function from the FROM clause is not pushed")));
return NULL;
}
/* dissuade the planner from trying a generic plan with parameters */
(void) expression_tree_walker((Node *) funcExpr->args, contain_param_walker,
&walkerParamContext);
@ -374,14 +353,16 @@ TryToDelegateFunctionCall(DistributedPlanningContext *planContext)
}
/*
* If the expression is simple, such as, SELECT fn() in
* PL/PgSQL code, PL engine is doing simple expression
* evaluation, which can't interpret the CustomScan Node.
* Function from FROM clause is not simple, so it's ok.
* If the expression is simple, such as, SELECT function() or PEFORM function()
* in PL/PgSQL code, PL engine does a simple expression evaluation which can't
* interpret the Citus CustomScan Node.
* Note: Function from FROM clause is not simple, so it's ok to pushdown.
*/
if (MaybeExecutingUDF() && IsQuerySimple(planContext->query) && !fromFuncExpr)
if ((MaybeExecutingUDF() || DoBlockLevel > 0) &&
IsQuerySimple(planContext->query) &&
!fromFuncExpr)
{
ereport(DEBUG1, (errmsg("Skipping delegation of function "
ereport(DEBUG1, (errmsg("Skipping pushdown of function "
"from a PL/PgSQL simple expression")));
return NULL;
}

View File

@ -1616,7 +1616,19 @@ MasterAggregateExpression(Aggref *originalAggregate,
Expr *directarg;
foreach_ptr(directarg, originalAggregate->aggdirectargs)
{
if (!IsA(directarg, Const) && !IsA(directarg, Param))
/*
* Need to replace nodes that contain any Vars with Vars referring
* to the related column of the result set returned for the worker
* aggregation.
*
* When there are no Vars, then the expression can be fully evaluated
* on the coordinator, so we skip it here. This is not just an
* optimization, but the result of the expression might require
* calling the final function of the aggregate, and doing so when
* there are no input rows (i.e.: with an empty tuple slot) is not
* desirable for the node-executor methods.
*/
if (pull_var_clause_default((Node *) directarg) != NIL)
{
Var *var = makeVar(masterTableId, walkerContext->columnId,
exprType((Node *) directarg),
@ -3070,7 +3082,13 @@ WorkerAggregateExpressionList(Aggref *originalAggregate,
Expr *directarg;
foreach_ptr(directarg, originalAggregate->aggdirectargs)
{
if (!IsA(directarg, Const) && !IsA(directarg, Param))
/*
* The worker aggregation should execute any node that contains any
* Var nodes and return the result in the targetlist, so that the
* combine query can then fetch the result via remote scan; see
* MasterAggregateExpression.
*/
if (pull_var_clause_default((Node *) directarg) != NIL)
{
workerAggregateList = lappend(workerAggregateList, directarg);
}

View File

@ -108,7 +108,6 @@ static GucStringAssignHook OldApplicationNameAssignHook = NULL;
void _PG_init(void);
void _PG_fini(void);
static void DoInitialCleanup(void);
static void ResizeStackToMaximumDepth(void);
@ -169,13 +168,6 @@ static const struct config_enum_entry task_executor_type_options[] = {
{ NULL, 0, false }
};
static const struct config_enum_entry shard_placement_policy_options[] = {
{ "local-node-first", SHARD_PLACEMENT_LOCAL_NODE_FIRST, false },
{ "round-robin", SHARD_PLACEMENT_ROUND_ROBIN, false },
{ "random", SHARD_PLACEMENT_RANDOM, false },
{ NULL, 0, false }
};
static const struct config_enum_entry use_secondary_nodes_options[] = {
{ "never", USE_SECONDARY_NODES_NEVER, false },
{ "always", USE_SECONDARY_NODES_ALWAYS, false },
@ -358,14 +350,6 @@ _PG_init(void)
}
/* shared library deconstruction function */
void
_PG_fini(void)
{
columnar_fini();
}
/*
* DoInitialCleanup does cleanup at start time.
* Currently it:
@ -474,6 +458,7 @@ StartupCitusBackend(void)
InitializeMaintenanceDaemonBackend();
InitializeBackendData();
RegisterConnectionCleanup();
AssignGlobalPID();
}
@ -911,21 +896,10 @@ RegisterCitusConfigVariables(void)
NULL, NULL, NULL);
DefineCustomBoolVariable(
"citus.enable_metadata_sync_by_default",
gettext_noop("Enables MX in the new nodes by default"),
"citus.enable_metadata_sync",
gettext_noop("Enables object and metadata syncing."),
NULL,
&EnableMetadataSyncByDefault,
true,
PGC_USERSET,
GUC_SUPERUSER_ONLY | GUC_NO_SHOW_ALL,
NULL, NULL, NULL);
DefineCustomBoolVariable(
"citus.enable_object_propagation",
gettext_noop("Enables propagating object creation for more complex objects, "
"schema's will always be created"),
NULL,
&EnableDependencyCreation,
&EnableMetadataSync,
true,
PGC_USERSET,
GUC_NO_SHOW_ALL,
@ -1649,22 +1623,6 @@ RegisterCitusConfigVariables(void)
GUC_STANDARD,
NULL, NULL, NULL);
DefineCustomEnumVariable(
"citus.shard_placement_policy",
gettext_noop("Sets the policy to use when choosing nodes for shard placement."),
gettext_noop("The master node chooses which worker nodes to place new shards "
"on. This configuration value specifies the policy to use when "
"selecting these nodes. The local-node-first policy places the "
"first replica on the client node and chooses others randomly. "
"The round-robin policy aims to distribute shards evenly across "
"the cluster by selecting nodes in a round-robin fashion."
"The random policy picks all workers randomly."),
&ShardPlacementPolicy,
SHARD_PLACEMENT_ROUND_ROBIN, shard_placement_policy_options,
PGC_USERSET,
GUC_STANDARD,
NULL, NULL, NULL);
DefineCustomIntVariable(
"citus.shard_replication_factor",
gettext_noop("Sets the replication factor for shards."),
@ -2026,7 +1984,6 @@ NodeConninfoGucCheckHook(char **newval, void **extra, GucSource source)
{
/* this array _must_ be kept in an order usable by bsearch */
const char *allowedConninfoKeywords[] = {
"application_name",
"connect_timeout",
#if defined(ENABLE_GSS) && defined(ENABLE_SSPI)
"gsslib",

View File

@ -12,7 +12,27 @@
#include "udfs/citus_internal_add_object_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"
#include "udfs/get_all_active_transactions/11.0-1.sql"
#include "udfs/get_global_active_transactions/11.0-1.sql"
#include "udfs/citus_worker_stat_activity/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;
#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;
-- 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"
DROP FUNCTION IF EXISTS pg_catalog.master_apply_delete_command(text);
DROP FUNCTION pg_catalog.master_get_table_metadata(text);

View File

@ -47,6 +47,7 @@ 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_run_local_command(text);
DROP FUNCTION pg_catalog.worker_drop_sequence_dependency(text);
DROP FUNCTION pg_catalog.worker_drop_shell_table(table_name text);
CREATE OR REPLACE VIEW pg_catalog.citus_shards_on_worker AS
SELECT n.nspname as "Schema",
@ -83,3 +84,128 @@ DROP FUNCTION pg_catalog.citus_shards_on_worker();
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';
DROP FUNCTION IF EXISTS get_all_active_transactions();
CREATE OR REPLACE FUNCTION get_all_active_transactions(OUT datid oid, OUT process_id int, OUT initiator_node_identifier int4, OUT worker_query BOOL,
OUT transaction_number int8, OUT transaction_stamp timestamptz)
RETURNS SETOF RECORD
LANGUAGE C STRICT AS 'MODULE_PATHNAME',
$$get_all_active_transactions$$;
COMMENT ON FUNCTION get_all_active_transactions(OUT datid oid, OUT datname text, OUT process_id int, OUT initiator_node_identifier int4, OUT worker_query BOOL,
OUT transaction_number int8, OUT transaction_stamp timestamptz)
IS 'returns distributed transaction ids of active distributed transactions';
DROP FUNCTION IF EXISTS get_global_active_transactions();
CREATE FUNCTION get_global_active_transactions(OUT datid oid, OUT process_id int, OUT initiator_node_identifier int4, OUT worker_query BOOL, OUT transaction_number int8, OUT transaction_stamp timestamptz)
RETURNS SETOF RECORD
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$get_global_active_transactions$$;
COMMENT ON FUNCTION get_global_active_transactions(OUT database_id oid, OUT process_id int, OUT initiator_node_identifier int4, OUT transaction_number int8, OUT transaction_stamp timestamptz)
IS 'returns distributed transaction ids of active distributed transactions from each node of the cluster';
RESET search_path;
DROP FUNCTION pg_catalog.citus_dist_stat_activity CASCADE;
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)
RETURNS SETOF RECORD
LANGUAGE C STRICT AS 'MODULE_PATHNAME',
$$citus_dist_stat_activity$$;
COMMENT ON FUNCTION pg_catalog.citus_dist_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int,
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)
IS 'returns distributed transaction activity on distributed tables';
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;
SET search_path = 'pg_catalog';
-- we have to recreate this view because we drop citus_dist_stat_activity that this view depends
CREATE VIEW citus.citus_lock_waits AS
WITH
citus_dist_stat_activity AS
(
SELECT * FROM citus_dist_stat_activity
),
unique_global_wait_edges AS
(
SELECT DISTINCT ON(waiting_node_id, waiting_transaction_num, blocking_node_id, blocking_transaction_num) * FROM dump_global_wait_edges()
),
citus_dist_stat_activity_with_node_id 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
)
SELECT
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
FROM
unique_global_wait_edges
JOIN
citus_dist_stat_activity_with_node_id waiting ON (unique_global_wait_edges.waiting_transaction_num = waiting.transaction_number AND unique_global_wait_edges.waiting_node_id = waiting.initiator_node_id)
JOIN
citus_dist_stat_activity_with_node_id blocking ON (unique_global_wait_edges.blocking_transaction_num = blocking.transaction_number AND unique_global_wait_edges.blocking_node_id = blocking.initiator_node_id);
ALTER VIEW citus.citus_lock_waits SET SCHEMA pg_catalog;
GRANT SELECT ON pg_catalog.citus_lock_waits TO PUBLIC;
DROP FUNCTION citus_worker_stat_activity CASCADE;
CREATE OR REPLACE FUNCTION citus_worker_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int,
OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name,
OUT 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)
RETURNS SETOF RECORD
LANGUAGE C STRICT AS 'MODULE_PATHNAME',
$$citus_worker_stat_activity$$;
COMMENT ON FUNCTION citus_worker_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int,
OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name,
OUT 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)
IS 'returns distributed transaction activity on shards of distributed tables';
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;
RESET search_path;

View File

@ -0,0 +1,19 @@
DROP FUNCTION IF EXISTS pg_catalog.citus_dist_stat_activity CASCADE;
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$$;
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';

View File

@ -0,0 +1,19 @@
DROP FUNCTION IF EXISTS pg_catalog.citus_dist_stat_activity CASCADE;
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$$;
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';

View File

@ -0,0 +1,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 AS
(
SELECT DISTINCT ON(waiting_node_id, waiting_transaction_num, blocking_node_id, blocking_transaction_num) * FROM dump_global_wait_edges()
),
citus_dist_stat_activity_with_node_id 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
)
SELECT
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
FROM
unique_global_wait_edges
JOIN
citus_dist_stat_activity_with_node_id waiting ON (unique_global_wait_edges.waiting_transaction_num = waiting.transaction_number AND unique_global_wait_edges.waiting_node_id = waiting.initiator_node_id)
JOIN
citus_dist_stat_activity_with_node_id blocking ON (unique_global_wait_edges.blocking_transaction_num = blocking.transaction_number AND unique_global_wait_edges.blocking_node_id = blocking.initiator_node_id);
ALTER VIEW citus.citus_lock_waits SET SCHEMA pg_catalog;
GRANT SELECT ON pg_catalog.citus_lock_waits TO PUBLIC;
RESET search_path;

View File

@ -0,0 +1,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 AS
(
SELECT DISTINCT ON(waiting_node_id, waiting_transaction_num, blocking_node_id, blocking_transaction_num) * FROM dump_global_wait_edges()
),
citus_dist_stat_activity_with_node_id 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
)
SELECT
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
FROM
unique_global_wait_edges
JOIN
citus_dist_stat_activity_with_node_id waiting ON (unique_global_wait_edges.waiting_transaction_num = waiting.transaction_number AND unique_global_wait_edges.waiting_node_id = waiting.initiator_node_id)
JOIN
citus_dist_stat_activity_with_node_id blocking ON (unique_global_wait_edges.blocking_transaction_num = blocking.transaction_number AND unique_global_wait_edges.blocking_node_id = blocking.initiator_node_id);
ALTER VIEW citus.citus_lock_waits SET SCHEMA pg_catalog;
GRANT SELECT ON pg_catalog.citus_lock_waits TO PUBLIC;
RESET search_path;

View File

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

View File

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

View File

@ -0,0 +1,12 @@
DROP FUNCTION IF EXISTS pg_catalog.get_all_active_transactions();
CREATE OR REPLACE FUNCTION pg_catalog.get_all_active_transactions(OUT datid oid, OUT process_id int, OUT initiator_node_identifier int4,
OUT worker_query BOOL, OUT transaction_number int8, OUT transaction_stamp timestamptz,
OUT global_pid int8)
RETURNS SETOF RECORD
LANGUAGE C STRICT AS 'MODULE_PATHNAME',
$$get_all_active_transactions$$;
COMMENT ON FUNCTION pg_catalog.get_all_active_transactions(OUT datid oid, OUT datname text, OUT process_id int, OUT initiator_node_identifier int4,
OUT worker_query BOOL, OUT transaction_number int8, OUT transaction_stamp timestamptz,
OUT global_pid int8)
IS 'returns transaction information for all Citus initiated transactions';

View File

@ -0,0 +1,12 @@
DROP FUNCTION IF EXISTS pg_catalog.get_all_active_transactions();
CREATE OR REPLACE FUNCTION pg_catalog.get_all_active_transactions(OUT datid oid, OUT process_id int, OUT initiator_node_identifier int4,
OUT worker_query BOOL, OUT transaction_number int8, OUT transaction_stamp timestamptz,
OUT global_pid int8)
RETURNS SETOF RECORD
LANGUAGE C STRICT AS 'MODULE_PATHNAME',
$$get_all_active_transactions$$;
COMMENT ON FUNCTION pg_catalog.get_all_active_transactions(OUT datid oid, OUT datname text, OUT process_id int, OUT initiator_node_identifier int4,
OUT worker_query BOOL, OUT transaction_number int8, OUT transaction_stamp timestamptz,
OUT global_pid int8)
IS 'returns transaction information for all Citus initiated transactions';

View File

@ -0,0 +1,9 @@
DROP FUNCTION IF EXISTS pg_catalog.get_global_active_transactions();
CREATE OR REPLACE FUNCTION pg_catalog.get_global_active_transactions(OUT datid oid, OUT process_id int, OUT initiator_node_identifier int4, OUT worker_query BOOL,
OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT global_pid int8)
RETURNS SETOF RECORD
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$get_global_active_transactions$$;
COMMENT ON FUNCTION pg_catalog.get_global_active_transactions(OUT datid oid, OUT process_id int, OUT initiator_node_identifier int4, OUT worker_query BOOL,
OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT global_pid int8)
IS 'returns transaction information for all Citus initiated transactions from each node of the cluster';

View File

@ -0,0 +1,9 @@
DROP FUNCTION IF EXISTS pg_catalog.get_global_active_transactions();
CREATE OR REPLACE FUNCTION pg_catalog.get_global_active_transactions(OUT datid oid, OUT process_id int, OUT initiator_node_identifier int4, OUT worker_query BOOL,
OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT global_pid int8)
RETURNS SETOF RECORD
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$get_global_active_transactions$$;
COMMENT ON FUNCTION pg_catalog.get_global_active_transactions(OUT datid oid, OUT process_id int, OUT initiator_node_identifier int4, OUT worker_query BOOL,
OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT global_pid int8)
IS 'returns transaction information for all Citus initiated transactions from each node of the cluster';

View File

@ -0,0 +1,7 @@
CREATE FUNCTION pg_catalog.worker_drop_shell_table(table_name text)
RETURNS VOID
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$worker_drop_shell_table$$;
COMMENT ON FUNCTION worker_drop_shell_table(table_name text)
IS 'drop the distributed table only without the metadata';

View File

@ -0,0 +1,7 @@
CREATE FUNCTION pg_catalog.worker_drop_shell_table(table_name text)
RETURNS VOID
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$worker_drop_shell_table$$;
COMMENT ON FUNCTION worker_drop_shell_table(table_name text)
IS 'drop the distributed table only without the metadata';

View File

@ -20,6 +20,7 @@
#include "distributed/maintenanced.h"
#include "distributed/metadata_sync.h"
#include "distributed/remote_commands.h"
#include "distributed/worker_manager.h"
#include "postmaster/postmaster.h"
#include "miscadmin.h"
#include "storage/latch.h"
@ -28,45 +29,66 @@
/* declarations for dynamic loading */
PG_FUNCTION_INFO_V1(master_metadata_snapshot);
PG_FUNCTION_INFO_V1(activate_node_snapshot);
PG_FUNCTION_INFO_V1(wait_until_metadata_sync);
PG_FUNCTION_INFO_V1(trigger_metadata_sync);
PG_FUNCTION_INFO_V1(raise_error_in_metadata_sync);
/*
* master_metadata_snapshot prints all the queries that are required
* to generate a metadata snapshot.
* activate_node_snapshot prints all the queries that are required
* to activate a node.
*/
Datum
master_metadata_snapshot(PG_FUNCTION_ARGS)
activate_node_snapshot(PG_FUNCTION_ARGS)
{
List *dropSnapshotCommands = MetadataDropCommands();
List *createSnapshotCommands = MetadataCreateCommands();
List *snapshotCommandList = NIL;
int snapshotCommandIndex = 0;
/*
* Activate node commands are created using the given worker node,
* so we are using first primary worker node just for test purposes.
*/
WorkerNode *dummyWorkerNode = GetFirstPrimaryWorkerNode();
List *updateLocalGroupCommand =
list_make1(LocalGroupIdUpdateCommand(dummyWorkerNode->groupId));
List *syncDistObjCommands = SyncDistributedObjectsCommandList(dummyWorkerNode);
List *dropSnapshotCommands = NodeMetadataDropCommands();
List *createSnapshotCommands = NodeMetadataCreateCommands();
List *pgDistTableMetadataSyncCommands = PgDistTableMetadataSyncCommandList();
List *activateNodeCommandList = NIL;
int activateNodeCommandIndex = 0;
Oid ddlCommandTypeId = TEXTOID;
snapshotCommandList = list_concat(snapshotCommandList, dropSnapshotCommands);
snapshotCommandList = list_concat(snapshotCommandList, createSnapshotCommands);
activateNodeCommandList = list_concat(activateNodeCommandList,
updateLocalGroupCommand);
activateNodeCommandList = list_concat(activateNodeCommandList, syncDistObjCommands);
activateNodeCommandList = list_concat(activateNodeCommandList, dropSnapshotCommands);
activateNodeCommandList = list_concat(activateNodeCommandList,
createSnapshotCommands);
activateNodeCommandList = list_concat(activateNodeCommandList,
pgDistTableMetadataSyncCommands);
int snapshotCommandCount = list_length(snapshotCommandList);
Datum *snapshotCommandDatumArray = palloc0(snapshotCommandCount * sizeof(Datum));
int activateNodeCommandCount = list_length(activateNodeCommandList);
Datum *activateNodeCommandDatumArray = palloc0(activateNodeCommandCount *
sizeof(Datum));
const char *metadataSnapshotCommand = NULL;
foreach_ptr(metadataSnapshotCommand, snapshotCommandList)
const char *activateNodeSnapshotCommand = NULL;
foreach_ptr(activateNodeSnapshotCommand, activateNodeCommandList)
{
Datum metadataSnapshotCommandDatum = CStringGetTextDatum(metadataSnapshotCommand);
Datum activateNodeSnapshotCommandDatum = CStringGetTextDatum(
activateNodeSnapshotCommand);
snapshotCommandDatumArray[snapshotCommandIndex] = metadataSnapshotCommandDatum;
snapshotCommandIndex++;
activateNodeCommandDatumArray[activateNodeCommandIndex] =
activateNodeSnapshotCommandDatum;
activateNodeCommandIndex++;
}
ArrayType *snapshotCommandArrayType = DatumArrayToArrayType(snapshotCommandDatumArray,
snapshotCommandCount,
ddlCommandTypeId);
ArrayType *activateNodeCommandArrayType = DatumArrayToArrayType(
activateNodeCommandDatumArray,
activateNodeCommandCount,
ddlCommandTypeId);
PG_RETURN_ARRAYTYPE_P(snapshotCommandArrayType);
PG_RETURN_ARRAYTYPE_P(activateNodeCommandArrayType);
}

View File

@ -15,6 +15,9 @@
#include "distributed/pg_version_constants.h"
#include "miscadmin.h"
#include "unistd.h"
#include "safe_lib.h"
#include "funcapi.h"
#include "access/htup_details.h"
@ -43,7 +46,7 @@
#define GET_ACTIVE_TRANSACTION_QUERY "SELECT * FROM get_all_active_transactions();"
#define ACTIVE_TRANSACTION_COLUMN_COUNT 6
#define ACTIVE_TRANSACTION_COLUMN_COUNT 7
/*
* Each backend's data reside in the shared memory
@ -78,6 +81,7 @@ typedef struct BackendManagementShmemData
static void StoreAllActiveTransactions(Tuplestorestate *tupleStore, TupleDesc
tupleDescriptor);
static uint64 GenerateGlobalPID(void);
static shmem_startup_hook_type prev_shmem_startup_hook = NULL;
static BackendManagementShmemData *backendManagementShmemData = NULL;
@ -86,6 +90,7 @@ static BackendData *MyBackendData = NULL;
static void BackendManagementShmemInit(void);
static size_t BackendManagementShmemSize(void);
static void UnSetGlobalPID(void);
PG_FUNCTION_INFO_V1(assign_distributed_transaction_id);
@ -315,6 +320,7 @@ get_global_active_transactions(PG_FUNCTION_ARGS)
values[3] = ParseBoolField(result, rowIndex, 3);
values[4] = ParseIntField(result, rowIndex, 4);
values[5] = ParseTimestampTzField(result, rowIndex, 5);
values[6] = ParseIntField(result, rowIndex, 6);
tuplestore_putvalues(tupleStore, tupleDescriptor, values, isNulls);
}
@ -384,8 +390,7 @@ StoreAllActiveTransactions(Tuplestorestate *tupleStore, TupleDesc tupleDescripto
SpinLockAcquire(&currentBackend->mutex);
/* we're only interested in backends initiated by Citus */
if (currentBackend->citusBackend.initiatorNodeIdentifier < 0)
if (currentBackend->globalPID == INVALID_CITUS_INTERNAL_BACKEND_GPID)
{
SpinLockRelease(&currentBackend->mutex);
continue;
@ -427,6 +432,7 @@ StoreAllActiveTransactions(Tuplestorestate *tupleStore, TupleDesc tupleDescripto
values[3] = !coordinatorOriginatedQuery;
values[4] = UInt64GetDatum(transactionNumber);
values[5] = TimestampTzGetDatum(transactionIdTimestamp);
values[6] = UInt64GetDatum(currentBackend->globalPID);
tuplestore_putvalues(tupleStore, tupleDescriptor, values, isNulls);
@ -631,6 +637,7 @@ InitializeBackendData(void)
/* zero out the backend data */
UnSetDistributedTransactionId();
UnSetGlobalPID();
UnlockBackendSharedMemory();
}
@ -664,6 +671,24 @@ UnSetDistributedTransactionId(void)
}
/*
* UnSetGlobalPID resets the global pid for the current backend.
*/
static void
UnSetGlobalPID(void)
{
/* backend does not exist if the extension is not created */
if (MyBackendData)
{
SpinLockAcquire(&MyBackendData->mutex);
MyBackendData->globalPID = 0;
SpinLockRelease(&MyBackendData->mutex);
}
}
/*
* LockBackendSharedMemory is a simple wrapper around LWLockAcquire on the
* shared memory lock.
@ -780,6 +805,109 @@ MarkCitusInitiatedCoordinatorBackend(void)
}
/*
* AssignGlobalPID assigns a global process id for the current backend.
* If this is a Citus initiated backend, which means it is distributed part of a distributed
* query, then this function assigns the global pid extracted from the application name.
* If not, this function assigns a new generated global pid.
*/
void
AssignGlobalPID(void)
{
uint64 globalPID = INVALID_CITUS_INTERNAL_BACKEND_GPID;
if (!IsCitusInternalBackend())
{
globalPID = GenerateGlobalPID();
}
else
{
globalPID = ExtractGlobalPID(application_name);
}
SpinLockAcquire(&MyBackendData->mutex);
MyBackendData->globalPID = globalPID;
SpinLockRelease(&MyBackendData->mutex);
}
/*
* GetGlobalPID returns the global process id of the current backend.
*/
uint64
GetGlobalPID(void)
{
uint64 globalPID = INVALID_CITUS_INTERNAL_BACKEND_GPID;
if (MyBackendData)
{
SpinLockAcquire(&MyBackendData->mutex);
globalPID = MyBackendData->globalPID;
SpinLockRelease(&MyBackendData->mutex);
}
return globalPID;
}
/*
* GenerateGlobalPID generates the global process id for the current backend.
*/
static uint64
GenerateGlobalPID(void)
{
/*
* We try to create a human readable global pid that consists of node id and process id.
* By multiplying node id with 10^10 and adding pid we generate a number where the smallest
* 10 digit represent the pid and the remaining digits are the node id.
*
* Both node id and pid are 32 bit. We use 10^10 to fit all possible pids. Some very large
* 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()) * 10000000000) + getpid();
}
/*
* ExtractGlobalPID extracts the global process id from the application name and returns it
* if the application name is not compatible with Citus' application names returns 0.
*/
uint64
ExtractGlobalPID(char *applicationName)
{
/* does application name exist */
if (!applicationName)
{
return INVALID_CITUS_INTERNAL_BACKEND_GPID;
}
/* we create our own copy of application name incase the original changes */
char *applicationNameCopy = pstrdup(applicationName);
uint64 prefixLength = strlen(CITUS_APPLICATION_NAME_PREFIX);
/* does application name start with Citus's application name prefix */
if (strncmp(applicationNameCopy, CITUS_APPLICATION_NAME_PREFIX, prefixLength) != 0)
{
return INVALID_CITUS_INTERNAL_BACKEND_GPID;
}
/* are the remaining characters of the application name numbers */
uint64 numberOfRemainingChars = strlen(applicationNameCopy) - prefixLength;
if (numberOfRemainingChars <= 0 ||
!strisdigit_s(applicationNameCopy + prefixLength, numberOfRemainingChars))
{
return INVALID_CITUS_INTERNAL_BACKEND_GPID;
}
char *globalPIDString = &applicationNameCopy[prefixLength];
uint64 globalPID = strtoul(globalPIDString, NULL, 10);
return globalPID;
}
/*
* CurrentDistributedTransactionNumber returns the transaction number of the
* current distributed transaction. The caller must make sure a distributed

View File

@ -108,7 +108,7 @@
* showing the initiator_node_id we expand it to initiator_node_host and
* initiator_node_port.
*/
#define CITUS_DIST_STAT_ACTIVITY_QUERY_COLS 23
#define CITUS_DIST_STAT_ACTIVITY_QUERY_COLS 24
#define CITUS_DIST_STAT_ADDITIONAL_COLS 3
#define CITUS_DIST_STAT_ACTIVITY_COLS \
CITUS_DIST_STAT_ACTIVITY_QUERY_COLS + CITUS_DIST_STAT_ADDITIONAL_COLS
@ -147,14 +147,20 @@ SELECT \
pg_stat_activity.backend_xid, \
pg_stat_activity.backend_xmin, \
pg_stat_activity.query, \
pg_stat_activity.backend_type \
pg_stat_activity.backend_type, \
dist_txs.global_pid \
FROM \
pg_stat_activity \
INNER JOIN \
get_all_active_transactions() AS dist_txs(database_id, process_id, initiator_node_identifier, worker_query, transaction_number, transaction_stamp) \
get_all_active_transactions() AS dist_txs(database_id, process_id, initiator_node_identifier, worker_query, transaction_number, transaction_stamp, global_pid) \
ON pg_stat_activity.pid = dist_txs.process_id \
WHERE \
dist_txs.worker_query = false;"
backend_type = 'client backend' \
AND \
pg_stat_activity.query NOT ILIKE '%stat_activity%' \
AND \
pg_stat_activity.application_name NOT SIMILAR TO 'citus_internal gpid=\\d+'; \
"
#define CITUS_WORKER_STAT_ACTIVITY_QUERY \
"\
@ -181,14 +187,15 @@ SELECT \
pg_stat_activity.backend_xid, \
pg_stat_activity.backend_xmin, \
pg_stat_activity.query, \
pg_stat_activity.backend_type \
pg_stat_activity.backend_type, \
dist_txs.global_id \
FROM \
pg_stat_activity \
LEFT JOIN \
get_all_active_transactions() AS dist_txs(database_id, process_id, initiator_node_identifier, worker_query, transaction_number, transaction_stamp) \
JOIN \
get_all_active_transactions() AS dist_txs(database_id, process_id, initiator_node_identifier, worker_query, transaction_number, transaction_stamp, global_id) \
ON pg_stat_activity.pid = dist_txs.process_id \
WHERE \
pg_stat_activity.application_name = 'citus' \
pg_stat_activity.application_name SIMILAR TO 'citus_internal gpid=\\d+' \
AND \
pg_stat_activity.query NOT ILIKE '%stat_activity%';"
@ -223,6 +230,7 @@ typedef struct CitusDistStat
TransactionId backend_xmin;
text *query;
text *backend_type;
uint64 global_pid;
} CitusDistStat;
@ -501,6 +509,7 @@ ParseCitusDistStat(PGresult *result, int64 rowIndex)
citusDistStat->backend_xmin = ParseXIDField(result, rowIndex, 20);
citusDistStat->query = ParseTextField(result, rowIndex, 21);
citusDistStat->backend_type = ParseTextField(result, rowIndex, 22);
citusDistStat->global_pid = ParseIntField(result, rowIndex, 23);
return citusDistStat;
}
@ -688,6 +697,7 @@ HeapTupleToCitusDistStat(HeapTuple result, TupleDesc rowDescriptor)
citusDistStat->backend_xmin = ParseXIDFieldFromHeapTuple(result, rowDescriptor, 21);
citusDistStat->query = ParseTextFieldFromHeapTuple(result, rowDescriptor, 22);
citusDistStat->backend_type = ParseTextFieldFromHeapTuple(result, rowDescriptor, 23);
citusDistStat->global_pid = ParseIntFieldFromHeapTuple(result, rowDescriptor, 24);
return citusDistStat;
}
@ -1098,6 +1108,8 @@ ReturnCitusDistStats(List *citusStatsList, FunctionCallInfo fcinfo)
nulls[25] = true;
}
values[26] = Int32GetDatum(citusDistStat->global_pid);
tuplestore_putvalues(tupleStore, tupleDesc, values, nulls);
}
}

View File

@ -824,7 +824,7 @@ EnsurePrepareTransactionIsAllowed(void)
return;
}
if (IsCitusInitiatedRemoteBackend())
if (IsCitusInternalBackend())
{
/*
* If this is a Citus-initiated backend.

View File

@ -112,8 +112,7 @@ SendCommandToWorkerAsUser(const char *nodeName, int32 nodePort, const char *node
/*
* SendCommandToWorkers sends a command to all workers in
* parallel. Commands are committed on the workers when the local
* transaction commits. The connection are made as the extension
* owner to ensure write access to the Citus metadata tables.
* transaction commits.
*/
void
SendCommandToWorkersWithMetadata(const char *command)
@ -123,6 +122,24 @@ SendCommandToWorkersWithMetadata(const char *command)
}
/*
* SendCommandToWorkersWithMetadataViaSuperUser sends a command to all workers in
* parallel by opening a super user connection. Commands are committed on the workers
* when the local transaction commits. The connection are made as the extension
* owner to ensure write access to the Citus metadata tables.
*
* Since we prevent to open superuser connections for metadata tables, it is
* discourated to use it. Consider using it only for propagating pg_dist_object
* tuples for dependent objects.
*/
void
SendCommandToWorkersWithMetadataViaSuperUser(const char *command)
{
SendCommandToMetadataWorkersParams(command, CitusExtensionOwnerName(),
0, NULL, NULL);
}
/*
* TargetWorkerSetNodeList returns a list of WorkerNode's that satisfies the
* TargetWorkerSet.

View File

@ -524,7 +524,7 @@ CitusMaintenanceDaemonMain(Datum main_arg)
if (syncMetadata)
{
metadataSyncBgwHandle =
SpawnSyncMetadataToNodes(MyDatabaseId, myDbData->userOid);
SpawnSyncNodeMetadataToNodes(MyDatabaseId, myDbData->userOid);
}
nextMetadataSyncTime =

View File

@ -537,7 +537,7 @@ ReferenceTableReplicationFactor(void)
* reference table to prevent unnecessary data transfer.
*/
void
ReplicateAllReferenceTablesToNode(char *nodeName, int nodePort)
ReplicateAllReferenceTablesToNode(WorkerNode *workerNode)
{
List *referenceTableList = CitusTableTypeIdList(REFERENCE_TABLE);
@ -560,7 +560,9 @@ ReplicateAllReferenceTablesToNode(char *nodeName, int nodePort)
List *shardPlacementList =
ShardPlacementListIncludingOrphanedPlacements(shardInterval->shardId);
ShardPlacement *targetPlacement =
SearchShardPlacementInList(shardPlacementList, nodeName, nodePort);
SearchShardPlacementInList(shardPlacementList,
workerNode->workerName,
workerNode->workerPort);
if (targetPlacement != NULL &&
targetPlacement->shardState == SHARD_STATE_ACTIVE)
{
@ -585,7 +587,9 @@ ReplicateAllReferenceTablesToNode(char *nodeName, int nodePort)
LockShardDistributionMetadata(shardId, ExclusiveLock);
ReplicateReferenceTableShardToNode(shardInterval, nodeName, nodePort);
ReplicateReferenceTableShardToNode(shardInterval,
workerNode->workerName,
workerNode->workerPort);
}
/* create foreign constraints between reference tables */
@ -595,9 +599,11 @@ ReplicateAllReferenceTablesToNode(char *nodeName, int nodePort)
/* send commands to new workers, the current user should be a superuser */
Assert(superuser());
SendMetadataCommandListToWorkerInCoordinatedTransaction(nodeName, nodePort,
CurrentUserName(),
commandList);
SendMetadataCommandListToWorkerInCoordinatedTransaction(
workerNode->workerName,
workerNode->workerPort,
CurrentUserName(),
commandList);
}
}
}

View File

@ -33,8 +33,6 @@
#include "distributed/worker_protocol.h"
static const char * CreateStmtByObjectAddress(const ObjectAddress *address);
static RenameStmt * CreateRenameStatement(const ObjectAddress *address, char *newName);
static char * GenerateBackupNameForCollision(const ObjectAddress *address);
PG_FUNCTION_INFO_V1(worker_create_or_replace_object);
@ -166,7 +164,7 @@ CreateStmtByObjectAddress(const ObjectAddress *address)
* address. This name should be used when renaming an existing object before creating the
* new object locally on the worker.
*/
static char *
char *
GenerateBackupNameForCollision(const ObjectAddress *address)
{
switch (getObjectClass(address))
@ -186,13 +184,23 @@ GenerateBackupNameForCollision(const ObjectAddress *address)
return GenerateBackupNameForTypeCollision(address);
}
case OCLASS_CLASS:
{
char relKind = get_rel_relkind(address->objectId);
if (relKind == RELKIND_SEQUENCE)
{
return GenerateBackupNameForSequenceCollision(address);
}
}
default:
{
ereport(ERROR, (errmsg("unsupported object to construct a rename statement"),
errdetail(
"unable to generate a backup name for the old type")));
break;
}
}
ereport(ERROR, (errmsg("unsupported object to construct a rename statement"),
errdetail("unable to generate a backup name for the old type")));
}
@ -243,6 +251,7 @@ CreateRenameTypeStmt(const ObjectAddress *address, char *newName)
address->objectId));
stmt->newname = newName;
return stmt;
}
@ -265,11 +274,43 @@ CreateRenameProcStmt(const ObjectAddress *address, char *newName)
}
/*
* CreateRenameSequenceStmt creates a rename statement for a sequence based on its
* ObjectAddress. The rename statement will rename the existing object on its address
* to the value provided in newName.
*/
static RenameStmt *
CreateRenameSequenceStmt(const ObjectAddress *address, char *newName)
{
RenameStmt *stmt = makeNode(RenameStmt);
Oid seqOid = address->objectId;
HeapTuple seqClassTuple = SearchSysCache1(RELOID, seqOid);
if (!HeapTupleIsValid(seqClassTuple))
{
ereport(ERROR, (errmsg("citus cache lookup error")));
}
Form_pg_class seqClassForm = (Form_pg_class) GETSTRUCT(seqClassTuple);
char *schemaName = get_namespace_name(seqClassForm->relnamespace);
char *seqName = NameStr(seqClassForm->relname);
List *name = list_make2(makeString(schemaName), makeString(seqName));
ReleaseSysCache(seqClassTuple);
stmt->renameType = OBJECT_SEQUENCE;
stmt->object = (Node *) name;
stmt->relation = makeRangeVar(schemaName, seqName, -1);
stmt->newname = newName;
return stmt;
}
/*
* CreateRenameStatement creates a rename statement for an existing object to rename the
* object to newName.
*/
static RenameStmt *
RenameStmt *
CreateRenameStatement(const ObjectAddress *address, char *newName)
{
switch (getObjectClass(address))
@ -289,10 +330,21 @@ CreateRenameStatement(const ObjectAddress *address, char *newName)
return CreateRenameTypeStmt(address, newName);
}
case OCLASS_CLASS:
{
char relKind = get_rel_relkind(address->objectId);
if (relKind == RELKIND_SEQUENCE)
{
return CreateRenameSequenceStmt(address, newName);
}
}
default:
{
ereport(ERROR, (errmsg("unsupported object to construct a rename statement"),
errdetail("unable to generate a parsetree for the rename")));
break;
}
}
ereport(ERROR, (errmsg("unsupported object to construct a rename statement"),
errdetail("unable to generate a parsetree for the rename")));
}

View File

@ -28,13 +28,16 @@
#include "commands/extension.h"
#include "commands/sequence.h"
#include "distributed/citus_ruleutils.h"
#include "distributed/commands.h"
#include "distributed/commands/multi_copy.h"
#include "distributed/commands/utility_hook.h"
#include "distributed/connection_management.h"
#include "distributed/coordinator_protocol.h"
#include "distributed/deparser.h"
#include "distributed/intermediate_results.h"
#include "distributed/listutils.h"
#include "distributed/metadata_cache.h"
#include "distributed/metadata_sync.h"
#include "distributed/multi_client_executor.h"
#include "distributed/multi_logical_optimizer.h"
#include "distributed/multi_partitioning_utils.h"
@ -43,8 +46,10 @@
#include "distributed/remote_commands.h"
#include "distributed/resource_lock.h"
#include "distributed/worker_create_or_replace.h"
#include "distributed/worker_protocol.h"
#include "distributed/version_compat.h"
#include "executor/spi.h"
#include "nodes/makefuncs.h"
#include "parser/parse_relation.h"
#include "storage/lmgr.h"
@ -461,19 +466,25 @@ worker_apply_sequence_command(PG_FUNCTION_ARGS)
" SEQUENCE command string")));
}
/*
* If sequence with the same name exist for different type, it must have been
* stayed on that node after a rollbacked create_distributed_table operation.
* We must change it's name first to create the sequence with the correct type.
*/
CreateSeqStmt *createSequenceStatement = (CreateSeqStmt *) commandNode;
RenameExistingSequenceWithDifferentTypeIfExists(createSequenceStatement->sequence,
sequenceTypeId);
/* run the CREATE SEQUENCE command */
ProcessUtilityParseTree(commandNode, commandString, PROCESS_UTILITY_QUERY, NULL,
None_Receiver, NULL);
CommandCounterIncrement();
CreateSeqStmt *createSequenceStatement = (CreateSeqStmt *) commandNode;
char *sequenceName = createSequenceStatement->sequence->relname;
char *sequenceSchema = createSequenceStatement->sequence->schemaname;
createSequenceStatement = (CreateSeqStmt *) commandNode;
Oid sequenceRelationId = RangeVarGetRelid(createSequenceStatement->sequence,
AccessShareLock, false);
char *sequenceName = createSequenceStatement->sequence->relname;
char *sequenceSchema = createSequenceStatement->sequence->schemaname;
Assert(sequenceRelationId != InvalidOid);
AlterSequenceMinMax(sequenceRelationId, sequenceSchema, sequenceName, sequenceTypeId);

View File

@ -34,6 +34,7 @@
#include "utils/fmgroids.h"
PG_FUNCTION_INFO_V1(worker_drop_distributed_table);
PG_FUNCTION_INFO_V1(worker_drop_shell_table);
PG_FUNCTION_INFO_V1(worker_drop_sequence_dependency);
@ -104,6 +105,8 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS)
UnmarkObjectDistributed(&ownedSequenceAddress);
}
UnmarkObjectDistributed(&distributedTableObject);
if (!IsObjectAddressOwnedByExtension(&distributedTableObject, NULL))
{
/*
@ -142,6 +145,82 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS)
}
/*
* worker_drop_shell_table drops the shell table of with the given distributed
* table without deleting related entries on pg_dist_placement, pg_dist_shard
* and pg_dist_placement. We've separated that logic since we handle object
* dependencies and table metadata separately while activating nodes.
*/
Datum
worker_drop_shell_table(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
text *relationName = PG_GETARG_TEXT_P(0);
Oid relationId = ResolveRelationId(relationName, true);
if (!OidIsValid(relationId))
{
ereport(NOTICE, (errmsg("relation %s does not exist, skipping",
text_to_cstring(relationName))));
PG_RETURN_VOID();
}
EnsureTableOwner(relationId);
if (GetLocalGroupId() == COORDINATOR_GROUP_ID)
{
ereport(ERROR, (errmsg("worker_drop_shell_table is only allowed to run"
" on worker nodes")));
}
/* first check the relation type */
Relation distributedRelation = relation_open(relationId, AccessShareLock);
EnsureRelationKindSupported(relationId);
/* close the relation since we do not need anymore */
relation_close(distributedRelation, AccessShareLock);
/* prepare distributedTableObject for dropping the table */
ObjectAddress distributedTableObject = { InvalidOid, InvalidOid, 0 };
distributedTableObject.classId = RelationRelationId;
distributedTableObject.objectId = relationId;
distributedTableObject.objectSubId = 0;
if (IsObjectAddressOwnedByExtension(&distributedTableObject, NULL))
{
PG_RETURN_VOID();
}
/* Drop dependent sequences from pg_dist_object */
#if PG_VERSION_NUM >= PG_VERSION_13
List *ownedSequences = getOwnedSequences(relationId);
#else
List *ownedSequences = getOwnedSequences(relationId, InvalidAttrNumber);
#endif
Oid ownedSequenceOid = InvalidOid;
foreach_oid(ownedSequenceOid, ownedSequences)
{
ObjectAddress ownedSequenceAddress = { 0 };
ObjectAddressSet(ownedSequenceAddress, RelationRelationId, ownedSequenceOid);
UnmarkObjectDistributed(&ownedSequenceAddress);
}
/*
* If the table is owned by an extension, we cannot drop it, nor should we
* until the user runs DROP EXTENSION. Therefore, we skip dropping the
* table and only delete the metadata.
*
* We drop the table with cascade since other tables may be referring to it.
*/
performDeletion(&distributedTableObject, DROP_CASCADE,
PERFORM_DELETION_INTERNAL);
PG_RETURN_VOID();
}
/*
* worker_drop_sequence_dependency is a UDF that removes the dependency
* of all the sequences for the given table.

View File

@ -150,7 +150,8 @@ ErrorIfRelationIsAKnownShard(Oid relationId)
void
ErrorIfIllegallyChangingKnownShard(Oid relationId)
{
if (LocalExecutorLevel > 0 || IsCitusInitiatedRemoteBackend() ||
if (LocalExecutorLevel > 0 ||
(IsCitusInternalBackend() || IsRebalancerInternalBackend()) ||
EnableManualChangesToShards)
{
return;
@ -330,7 +331,7 @@ ResetHideShardsDecision(void)
static bool
ShouldHideShardsInternal(void)
{
if (IsCitusInitiatedRemoteBackend())
if (IsCitusInternalBackend() || IsRebalancerInternalBackend())
{
/* we never hide shards from Citus */
return false;

View File

@ -7,9 +7,9 @@
#include "access/tableam.h"
#include "access/skey.h"
#include "nodes/bitmapset.h"
#include "distributed/coordinator_protocol.h"
#include "access/heapam.h"
#include "catalog/indexing.h"
#include "utils/acl.h"
/*
* Number of valid ItemPointer Offset's for "row number" <> "ItemPointer"
@ -50,8 +50,7 @@ typedef struct ColumnarScanDescData *ColumnarScanDesc;
const TableAmRoutine * GetColumnarTableAmRoutine(void);
extern void columnar_tableam_init(void);
extern void columnar_tableam_finish(void);
extern bool CheckCitusVersion(int elevel);
extern TableScanDesc columnar_beginscan_extended(Relation relation, Snapshot snapshot,
int nkeys, ScanKey key,
ParallelTableScanDesc parallel_scan,

View File

@ -50,6 +50,7 @@ typedef struct BackendData
Oid userId;
slock_t mutex;
bool cancelledDueToDeadlock;
uint64 globalPID;
CitusInitiatedBackend citusBackend;
DistributedTransactionId transactionId;
} BackendData;
@ -63,6 +64,9 @@ extern void UnlockBackendSharedMemory(void);
extern void UnSetDistributedTransactionId(void);
extern void AssignDistributedTransactionId(void);
extern void MarkCitusInitiatedCoordinatorBackend(void);
extern void AssignGlobalPID(void);
extern uint64 GetGlobalPID(void);
extern uint64 ExtractGlobalPID(char *applicationName);
extern void GetBackendDataForProc(PGPROC *proc, BackendData *result);
extern void CancelTransactionDueToDeadlock(PGPROC *proc);
extern bool MyBackendGotCancelledDueToDeadlock(bool clearState);
@ -73,4 +77,6 @@ extern int GetAllActiveClientBackendCount(void);
extern void IncrementClientBackendCounter(void);
extern void DecrementClientBackendCounter(void);
#define INVALID_CITUS_INTERNAL_BACKEND_GPID 0
#endif /* BACKEND_DATA_H */

View File

@ -366,6 +366,8 @@ extern ObjectAddress AlterRoleSetStmtObjectAddress(Node *node,
extern List * GenerateCreateOrAlterRoleCommand(Oid roleOid);
/* schema.c - forward declarations */
extern List * PreprocessCreateSchemaStmt(Node *node, const char *queryString,
ProcessUtilityContext processUtilityContext);
extern List * PreprocessDropSchemaStmt(Node *dropSchemaStatement,
const char *queryString,
ProcessUtilityContext processUtilityContext);
@ -375,6 +377,7 @@ extern List * PreprocessGrantOnSchemaStmt(Node *node, const char *queryString,
ProcessUtilityContext processUtilityContext);
extern List * PreprocessAlterSchemaRenameStmt(Node *node, const char *queryString,
ProcessUtilityContext processUtilityContext);
extern ObjectAddress CreateSchemaStmtObjectAddress(Node *node, bool missing_ok);
extern ObjectAddress AlterSchemaRenameStmtObjectAddress(Node *node, bool missing_ok);
/* sequence.c - forward declarations */
@ -397,6 +400,9 @@ extern ObjectAddress AlterSequenceOwnerStmtObjectAddress(Node *node, bool missin
extern ObjectAddress RenameSequenceStmtObjectAddress(Node *node, bool missing_ok);
extern void ErrorIfUnsupportedSeqStmt(CreateSeqStmt *createSeqStmt);
extern void ErrorIfDistributedAlterSeqOwnedBy(AlterSeqStmt *alterSeqStmt);
extern char * GenerateBackupNameForSequenceCollision(const ObjectAddress *address);
extern void RenameExistingSequenceWithDifferentTypeIfExists(RangeVar *sequence,
Oid desiredSeqTypeId);
/* statistics.c - forward declarations */
extern List * PreprocessCreateStatisticsStmt(Node *node, const char *queryString,

View File

@ -32,7 +32,6 @@ typedef enum
} PropSetCmdBehavior;
extern PropSetCmdBehavior PropagateSetCommands;
extern bool EnableDDLPropagation;
extern bool EnableDependencyCreation;
extern bool EnableCreateTypePropagation;
extern bool EnableAlterRolePropagation;
extern bool EnableAlterRoleSetPropagation;

View File

@ -29,7 +29,10 @@
#define ERROR_BUFFER_SIZE 256
/* application name used for internal connections in Citus */
#define CITUS_APPLICATION_NAME "citus"
#define CITUS_APPLICATION_NAME_PREFIX "citus_internal gpid="
/* application name used for internal connections in rebalancer */
#define CITUS_REBALANCER_NAME "citus_rebalancer"
/* forward declare, to avoid forcing large headers on everyone */
struct pg_conn; /* target of the PGconn typedef */
@ -277,7 +280,8 @@ extern void FinishConnectionListEstablishment(List *multiConnectionList);
extern void FinishConnectionEstablishment(MultiConnection *connection);
extern void ClaimConnectionExclusively(MultiConnection *connection);
extern void UnclaimConnection(MultiConnection *connection);
extern bool IsCitusInitiatedRemoteBackend(void);
extern bool IsCitusInternalBackend(void);
extern bool IsRebalancerInternalBackend(void);
extern void MarkConnectionConnected(MultiConnection *connection);
/* time utilities */

View File

@ -72,15 +72,6 @@
#define DROP_FOREIGN_TABLE_COMMAND "DROP FOREIGN TABLE IF EXISTS %s CASCADE"
#define CREATE_SCHEMA_COMMAND "CREATE SCHEMA IF NOT EXISTS %s AUTHORIZATION %s"
/* Enumeration that defines the shard placement policy to use while staging */
typedef enum
{
SHARD_PLACEMENT_INVALID_FIRST = 0,
SHARD_PLACEMENT_LOCAL_NODE_FIRST = 1,
SHARD_PLACEMENT_ROUND_ROBIN = 2,
SHARD_PLACEMENT_RANDOM = 3
} ShardPlacementPolicyType;
/*
* TableDDLCommandType encodes the implementation used by TableDDLCommand. See comments in
* TableDDLCpmmand for details.
@ -212,7 +203,6 @@ extern TableDDLCommand * ColumnarGetCustomTableOptionsDDL(char *schemaName,
/* Config variables managed via guc.c */
extern int ShardCount;
extern int ShardReplicationFactor;
extern int ShardPlacementPolicy;
extern int NextShardId;
extern int NextPlacementId;
@ -224,7 +214,8 @@ extern uint64 GetNextShardId(void);
extern uint64 GetNextPlacementId(void);
extern Oid ResolveRelationId(text *relationName, bool missingOk);
extern List * GetFullTableCreationCommands(Oid relationId,
IncludeSequenceDefaults includeSequenceDefaults);
IncludeSequenceDefaults includeSequenceDefaults,
bool creatingShellTableOnRemoteNode);
extern List * GetPostLoadTableCreationCommands(Oid relationId, bool includeIndexes,
bool includeReplicaIdentity);
extern List * GetPreLoadTableCreationCommands(Oid relationId, IncludeSequenceDefaults

View File

@ -60,6 +60,8 @@ extern char * DeparseAlterTableStmt(Node *node);
extern void QualifyAlterTableSchemaStmt(Node *stmt);
/* forward declarations for deparse_schema_stmts.c */
extern char * DeparseCreateSchemaStmt(Node *node);
extern char * DeparseDropSchemaStmt(Node *node);
extern char * DeparseGrantOnSchemaStmt(Node *stmt);
extern char * DeparseAlterSchemaRenameStmt(Node *stmt);

View File

@ -23,6 +23,7 @@ extern bool CitusExtensionObject(const ObjectAddress *objectAddress);
extern bool IsObjectDistributed(const ObjectAddress *address);
extern bool ClusterHasDistributedFunctionWithDistArgument(void);
extern void MarkObjectDistributed(const ObjectAddress *distAddress);
extern void MarkObjectDistributedViaSuperUser(const ObjectAddress *distAddress);
extern void UnmarkObjectDistributed(const ObjectAddress *address);
extern bool IsTableOwnedByExtension(Oid relationId);
extern bool IsObjectAddressOwnedByExtension(const ObjectAddress *target,

View File

@ -165,6 +165,7 @@ extern CitusTableCacheEntry * LookupCitusTableCacheEntry(Oid relationId);
extern DistObjectCacheEntry * LookupDistObjectCacheEntry(Oid classid, Oid objid, int32
objsubid);
extern int32 GetLocalGroupId(void);
extern int32 GetLocalNodeId(void);
extern void CitusTableCacheFlushInvalidatedEntries(void);
extern Oid LookupShardRelationFromCatalog(int64 shardId, bool missing_ok);
extern List * ShardPlacementListIncludingOrphanedPlacements(uint64 shardId);

View File

@ -13,6 +13,7 @@
#define METADATA_SYNC_H
#include "distributed/coordinator_protocol.h"
#include "distributed/metadata_cache.h"
#include "nodes/pg_list.h"
@ -22,18 +23,23 @@ extern int MetadataSyncRetryInterval;
typedef enum
{
METADATA_SYNC_SUCCESS = 0,
METADATA_SYNC_FAILED_LOCK = 1,
METADATA_SYNC_FAILED_SYNC = 2
} MetadataSyncResult;
NODE_METADATA_SYNC_SUCCESS = 0,
NODE_METADATA_SYNC_FAILED_LOCK = 1,
NODE_METADATA_SYNC_FAILED_SYNC = 2
} NodeMetadataSyncResult;
/* Functions declarations for metadata syncing */
extern void StartMetadataSyncToNode(const char *nodeNameString, int32 nodePort);
extern void SyncNodeMetadataToNode(const char *nodeNameString, int32 nodePort);
extern void SyncCitusTableMetadata(Oid relationId);
extern void EnsureSequentialModeMetadataOperations(void);
extern bool ClusterHasKnownMetadataWorkers(void);
extern char * LocalGroupIdUpdateCommand(int32 groupId);
extern bool ShouldSyncTableMetadata(Oid relationId);
extern bool ShouldSyncTableMetadataViaCatalog(Oid relationId);
extern List * MetadataCreateCommands(void);
extern List * MetadataDropCommands(void);
extern List * NodeMetadataCreateCommands(void);
extern List * DistributedObjectMetadataSyncCommandList(void);
extern List * CitusTableMetadataCreateCommandList(Oid relationId);
extern List * NodeMetadataDropCommands(void);
extern char * MarkObjectsDistributedCreateCommand(List *addresses,
List *distributionArgumentIndexes,
List *colocationIds,
@ -52,22 +58,30 @@ extern char * CreateSchemaDDLCommand(Oid schemaId);
extern List * GrantOnSchemaDDLCommands(Oid schemaId);
extern char * PlacementUpsertCommand(uint64 shardId, uint64 placementId, int shardState,
uint64 shardLength, int32 groupId);
extern void CreateTableMetadataOnWorkers(Oid relationId);
extern BackgroundWorkerHandle * SpawnSyncMetadataToNodes(Oid database, Oid owner);
extern void SyncMetadataToNodesMain(Datum main_arg);
extern TableDDLCommand * TruncateTriggerCreateCommand(Oid relationId);
extern void CreateInterTableRelationshipOfRelationOnWorkers(Oid relationId);
extern List * InterTableRelationshipOfRelationCommandList(Oid relationId);
extern List * DetachPartitionCommandList(void);
extern BackgroundWorkerHandle * SpawnSyncNodeMetadataToNodes(Oid database, Oid owner);
extern void SyncNodeMetadataToNodesMain(Datum main_arg);
extern void SignalMetadataSyncDaemon(Oid database, int sig);
extern bool ShouldInitiateMetadataSync(bool *lockFailure);
extern List * SequenceDependencyCommandList(Oid relationId);
extern List * DDLCommandsForSequence(Oid sequenceOid, char *ownerName);
extern List * SequenceDDLCommandsForTable(Oid relationId);
extern List * GetSequencesFromAttrDef(Oid attrdefOid);
extern void GetDependentSequencesWithRelation(Oid relationId, List **attnumList,
List **dependentSequenceList, AttrNumber
attnum);
extern Oid GetAttributeTypeOid(Oid relationId, AttrNumber attnum);
#define DELETE_ALL_NODES "TRUNCATE pg_dist_node CASCADE"
#define DELETE_ALL_DISTRIBUTED_OBJECTS "TRUNCATE citus.pg_dist_object"
#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_PARTITIONS "DELETE FROM pg_dist_partition"
#define REMOVE_ALL_SHELL_TABLES_COMMAND \
"SELECT worker_drop_shell_table(logicalrelid::regclass::text) FROM pg_dist_partition"
#define REMOVE_ALL_CITUS_TABLES_COMMAND \
"SELECT worker_drop_distributed_table(logicalrelid::regclass::text) FROM pg_dist_partition"
#define BREAK_CITUS_TABLE_SEQUENCE_DEPENDENCY_COMMAND \
@ -75,8 +89,8 @@ extern Oid GetAttributeTypeOid(Oid relationId, AttrNumber attnum);
#define DISABLE_DDL_PROPAGATION "SET citus.enable_ddl_propagation TO 'off'"
#define ENABLE_DDL_PROPAGATION "SET citus.enable_ddl_propagation TO 'on'"
#define DISABLE_OBJECT_PROPAGATION "SET citus.enable_object_propagation TO 'off'"
#define ENABLE_OBJECT_PROPAGATION "SET citus.enable_object_propagation TO 'on'"
#define DISABLE_METADATA_SYNC "SET citus.enable_metadata_sync TO 'off'"
#define ENABLE_METADATA_SYNC "SET citus.enable_metadata_sync TO 'on'"
#define WORKER_APPLY_SEQUENCE_COMMAND "SELECT worker_apply_sequence_command (%s,%s)"
#define UPSERT_PLACEMENT \
"INSERT INTO pg_dist_placement " \
@ -94,6 +108,6 @@ extern Oid GetAttributeTypeOid(Oid relationId, AttrNumber attnum);
/* controlled via GUC */
extern char *EnableManualMetadataChangesForUser;
extern bool EnableMetadataSyncByDefault;
extern bool EnableMetadataSync;
#endif /* METADATA_SYNC_H */

View File

@ -249,7 +249,7 @@ extern void EnsureDependenciesExistOnAllNodes(const ObjectAddress *target);
extern List * GetDistributableDependenciesForObject(const ObjectAddress *target);
extern bool ShouldPropagate(void);
extern bool ShouldPropagateObject(const ObjectAddress *address);
extern void ReplicateAllDependenciesToNode(const char *nodeName, int nodePort);
extern List * ReplicateAllObjectsToNodeCommandList(const char *nodeName, int nodePort);
/* Remaining metadata utility functions */
extern char * TableOwner(Oid relationId);
@ -286,13 +286,7 @@ extern bool GetNodeDiskSpaceStatsForConnection(MultiConnection *connection,
uint64 *availableBytes,
uint64 *totalBytes);
extern void ExecuteQueryViaSPI(char *query, int SPIOK);
extern void EnsureSequenceTypeSupported(Oid seqOid, Oid seqTypId);
extern void EnsureSequenceTypeSupported(Oid seqOid, Oid seqTypId, Oid ownerRelationId);
extern void AlterSequenceType(Oid seqOid, Oid typeOid);
extern void MarkSequenceListDistributedAndPropagateWithDependencies(Oid relationId,
List *sequenceList);
extern void MarkSequenceDistributedAndPropagateWithDependencies(Oid relationId, Oid
sequenceOid);
extern void EnsureDistributedSequencesHaveOneType(Oid relationId,
List *dependentSequenceList,
List *attnumList);
extern void EnsureRelationHasCompatibleSequenceTypes(Oid relationId);
#endif /* METADATA_UTILITY_H */

View File

@ -25,6 +25,6 @@ extern void DeleteAllReplicatedTablePlacementsFromNodeGroup(int32 groupId,
bool localOnly);
extern int CompareOids(const void *leftElement, const void *rightElement);
extern int ReferenceTableReplicationFactor(void);
extern void ReplicateAllReferenceTablesToNode(char *nodeName, int nodePort);
extern void ReplicateAllReferenceTablesToNode(WorkerNode *workerNode);
#endif /* REFERENCE_TABLE_UTILS_H_ */

View File

@ -190,7 +190,7 @@ extern List * RebalancePlacementUpdates(List *workerNodeList, List *shardPlaceme
RebalancePlanFunctions *rebalancePlanFunctions);
extern List * ReplicationPlacementUpdates(List *workerNodeList, List *shardPlacementList,
int shardReplicationFactor);
extern void ExecuteCriticalCommandInSeparateTransaction(char *command);
extern void ExecuteRebalancerCommandInSeparateTransaction(char *command);
#endif /* SHARD_REBALANCER_H */

View File

@ -14,8 +14,12 @@
#ifndef WORKER_CREATE_OR_REPLACE_H
#define WORKER_CREATE_OR_REPLACE_H
#include "catalog/objectaddress.h"
#define CREATE_OR_REPLACE_COMMAND "SELECT worker_create_or_replace_object(%s);"
extern char * WrapCreateOrReplace(const char *sql);
extern char * GenerateBackupNameForCollision(const ObjectAddress *address);
extern RenameStmt * CreateRenameStatement(const ObjectAddress *address, char *newName);
#endif /* WORKER_CREATE_OR_REPLACE_H */

View File

@ -63,13 +63,13 @@ extern char *WorkerListFileName;
extern char *CurrentCluster;
extern bool ReplicateReferenceTablesOnActivate;
extern int ActivateNode(char *nodeName, int nodePort);
/* Function declarations for finding worker nodes to place shards on */
extern WorkerNode * WorkerGetRandomCandidateNode(List *currentNodeList);
extern WorkerNode * WorkerGetRoundRobinCandidateNode(List *workerNodeList,
uint64 shardId,
uint32 placementIndex);
extern WorkerNode * WorkerGetLocalFirstCandidateNode(List *currentNodeList);
extern uint32 ActivePrimaryNonCoordinatorNodeCount(void);
extern uint32 ActivePrimaryNodeCount(void);
extern List * ActivePrimaryNonCoordinatorNodeList(LOCKMODE lockMode);
@ -103,6 +103,8 @@ extern WorkerNode * SetWorkerColumnLocalOnly(WorkerNode *workerNode, int columnI
Datum value);
extern uint32 CountPrimariesWithMetadata(void);
extern WorkerNode * GetFirstPrimaryWorkerNode(void);
extern List * SyncDistributedObjectsCommandList(WorkerNode *workerNode);
extern List * PgDistTableMetadataSyncCommandList(void);
/* Function declarations for worker node utilities */
extern int CompareWorkerNodes(const void *leftElement, const void *rightElement);

View File

@ -49,6 +49,7 @@ extern bool SendOptionalMetadataCommandListToWorkerInCoordinatedTransaction(cons
List *
commandList);
extern void SendCommandToWorkersWithMetadata(const char *command);
extern void SendCommandToWorkersWithMetadataViaSuperUser(const char *command);
extern void SendBareCommandListToMetadataWorkers(List *commandList);
extern void EnsureNoModificationsHaveBeenDone(void);
extern void SendCommandListToWorkerOutsideTransaction(const char *nodeName,

View File

@ -712,6 +712,19 @@ select array_agg(val order by valf) from aggdata;
{0,NULL,2,3,5,2,4,NULL,NULL,8,NULL}
(1 row)
-- test by using some other node types as arguments to agg
select key, percentile_cont((key - (key > 4)::int) / 10.0) within group(order by val) from aggdata group by key;
key | percentile_cont
---------------------------------------------------------------------
1 | 2
2 | 2.4
3 | 4
5 |
6 |
7 | 8
9 | 0
(7 rows)
-- Test TransformSubqueryNode
select * FROM (
SELECT key, mode() within group (order by floor(agg1.val/2)) m from aggdata agg1
@ -932,5 +945,100 @@ SELECT square_func(5), a, count(a) FROM t1 GROUP BY a;
ERROR: function aggregate_support.square_func(integer) does not exist
HINT: No function matches the given name and argument types. You might need to add explicit type casts.
CONTEXT: while executing command on localhost:xxxxx
-- Test the cases where the worker agg exec. returns no tuples.
CREATE TABLE dist_table (dist_col int, agg_col numeric);
SELECT create_distributed_table('dist_table', 'dist_col');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE TABLE ref_table (int_col int);
SELECT create_reference_table('ref_table');
create_reference_table
---------------------------------------------------------------------
(1 row)
SELECT PERCENTILE_DISC(.25) WITHIN GROUP (ORDER BY agg_col)
FROM dist_table
LEFT JOIN ref_table ON TRUE;
percentile_disc
---------------------------------------------------------------------
(1 row)
SELECT PERCENTILE_DISC(.25) WITHIN GROUP (ORDER BY agg_col)
FROM (SELECT *, random() FROM dist_table) a;
percentile_disc
---------------------------------------------------------------------
(1 row)
SELECT PERCENTILE_DISC((2 > random())::int::numeric / 10) WITHIN GROUP (ORDER BY agg_col)
FROM dist_table
LEFT JOIN ref_table ON TRUE;
percentile_disc
---------------------------------------------------------------------
(1 row)
SELECT SUM(COALESCE(agg_col, 3))
FROM dist_table
LEFT JOIN ref_table ON TRUE;
sum
---------------------------------------------------------------------
(1 row)
SELECT AVG(COALESCE(agg_col, 10))
FROM dist_table
LEFT JOIN ref_table ON TRUE;
avg
---------------------------------------------------------------------
(1 row)
insert into dist_table values (2, 11.2), (3, NULL), (6, 3.22), (3, 4.23), (5, 5.25), (4, 63.4), (75, NULL), (80, NULL), (96, NULL), (8, 1078), (0, 1.19);
-- run the same queries after loading some data
SELECT PERCENTILE_DISC(.25) WITHIN GROUP (ORDER BY agg_col)
FROM dist_table
LEFT JOIN ref_table ON TRUE;
percentile_disc
---------------------------------------------------------------------
3.22
(1 row)
SELECT PERCENTILE_DISC(.25) WITHIN GROUP (ORDER BY agg_col)
FROM (SELECT *, random() FROM dist_table) a;
percentile_disc
---------------------------------------------------------------------
3.22
(1 row)
SELECT PERCENTILE_DISC((2 > random())::int::numeric / 10) WITHIN GROUP (ORDER BY agg_col)
FROM dist_table
LEFT JOIN ref_table ON TRUE;
percentile_disc
---------------------------------------------------------------------
1.19
(1 row)
SELECT floor(SUM(COALESCE(agg_col, 3)))
FROM dist_table
LEFT JOIN ref_table ON TRUE;
floor
---------------------------------------------------------------------
1178
(1 row)
SELECT floor(AVG(COALESCE(agg_col, 10)))
FROM dist_table
LEFT JOIN ref_table ON TRUE;
floor
---------------------------------------------------------------------
109
(1 row)
set client_min_messages to error;
drop schema aggregate_support cascade;

View File

@ -1,5 +1,5 @@
SHOW citus.enable_metadata_sync_by_default;
citus.enable_metadata_sync_by_default
SHOW citus.enable_metadata_sync;
citus.enable_metadata_sync
---------------------------------------------------------------------
on
(1 row)

View File

@ -193,8 +193,8 @@ FOR EACH STATEMENT EXECUTE FUNCTION dummy_function();
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1507008, 'interesting!schema', E'CREATE TRIGGER "trigger\\''name"
BEFORE INSERT ON "interesting!schema"."citus_local!_table"
FOR EACH STATEMENT EXECUTE FUNCTION dummy_function();')
CREATE EXTENSION seg;
BEGIN;
CREATE EXTENSION seg;
-- ALTER TRIGGER DEPENDS ON
ALTER TRIGGER "trigger\'name" ON "interesting!schema"."citus_local!_table" DEPENDS ON EXTENSION seg;
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1507008, 'interesting!schema', E'ALTER TRIGGER "trigger\\''name" ON "interesting!schema"."citus_local!_table" DEPENDS ON EXTENSION seg;')
@ -578,5 +578,5 @@ NOTICE: executing the command locally: SELECT val FROM citus_local_table_trigge
ROLLBACK;
-- cleanup at exit
SET client_min_messages TO ERROR;
DROP SCHEMA citus_local_table_triggers, "interesting!schema" CASCADE;
NOTICE: drop cascades to 22 other objects

View File

@ -43,12 +43,12 @@ FOR EACH ROW EXECUTE FUNCTION dummy_function();
NOTICE: executing the command locally: SELECT worker_apply_shard_ddl_command (1508000, 'citus_local_tables_mx', 'CREATE TRIGGER dummy_function_trigger
BEFORE UPDATE OF value ON citus_local_table
FOR EACH ROW EXECUTE FUNCTION dummy_function();')
-- Show that we can sync metadata successfully. That means, we create
-- Show that we can activate node successfully. That means, we create
-- the function that trigger needs in mx workers too.
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
start_metadata_sync_to_node
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
?column?
---------------------------------------------------------------------
1
(1 row)
CREATE EXTENSION seg;

View File

@ -265,6 +265,13 @@ set columnar.compression = 'pglz';
INSERT INTO truncate_schema.truncate_tbl SELECT generate_series(1, 100);
set columnar.compression to default;
-- create a user that can not truncate
SELECT run_command_on_workers($$CREATE USER truncate_user;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"CREATE ROLE")
(localhost,57638,t,"CREATE ROLE")
(2 rows)
CREATE USER truncate_user;
NOTICE: not propagating CREATE ROLE/USER commands to worker nodes
HINT: Connect to worker nodes directly to manually create all necessary users and roles.

View File

@ -1,5 +1,5 @@
SET citus.next_shard_id TO 20030000;
SET citus.enable_object_propagation TO false; -- all tests here verify old behaviour without distributing types,functions,etc automatically
SET citus.enable_metadata_sync TO false; -- all tests here verify old behaviour without distributing types,functions,etc automatically
CREATE USER typeowner_for_disabled_object_propagation_guc;
NOTICE: not propagating CREATE ROLE/USER commands to worker nodes
HINT: Connect to worker nodes directly to manually create all necessary users and roles.
@ -65,7 +65,7 @@ SELECT create_distributed_table('t3', 'a');
-- verify ALTER TYPE statements are not propagated for types, even though they are marked distributed
BEGIN;
-- object propagation is turned off after xact finished, type is already marked distributed by then
SET LOCAL citus.enable_object_propagation TO on;
SET LOCAL citus.enable_metadata_sync TO on;
CREATE TYPE tt3 AS (a int, b int);
CREATE TABLE t4 (a int PRIMARY KEY, b tt3);
SELECT create_distributed_table('t4','a');
@ -120,7 +120,7 @@ $$);
-- suppress any warnings during cleanup
SET client_min_messages TO error;
RESET citus.enable_object_propagation;
RESET citus.enable_metadata_sync;
DROP SCHEMA disabled_object_propagation CASCADE;
DROP SCHEMA disabled_object_propagation2 CASCADE;
DROP USER typeowner_for_disabled_object_propagation_guc;

View File

@ -155,16 +155,6 @@ ORDER BY 1,2,3;
SET client_min_messages TO error; -- suppress cascading objects dropping
DROP SCHEMA collation_tests CASCADE;
DROP SCHEMA collation_tests2 CASCADE;
-- This is hacky, but we should clean-up the resources as below
\c - - - :worker_1_port
SET client_min_messages TO error; -- suppress cascading objects dropping
DROP SCHEMA collation_tests CASCADE;
DROP SCHEMA collation_tests2 CASCADE;
\c - - - :worker_2_port
SET client_min_messages TO error; -- suppress cascading objects dropping
DROP SCHEMA collation_tests CASCADE;
DROP SCHEMA collation_tests2 CASCADE;
\c - - - :master_port
DROP USER collationuser;
SELECT run_command_on_workers($$DROP USER collationuser;$$);
run_command_on_workers

View File

@ -1,13 +1,7 @@
CREATE SCHEMA collation_conflict;
SELECT run_command_on_workers($$CREATE SCHEMA collation_conflict;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"CREATE SCHEMA")
(localhost,57638,t,"CREATE SCHEMA")
(2 rows)
\c - - - :worker_1_port
SET search_path TO collation_conflict;
SET citus.enable_metadata_sync TO off;
CREATE COLLATION caseinsensitive (
provider = icu,
locale = 'und-u-ks-level2'
@ -45,6 +39,7 @@ SET search_path TO collation_conflict;
DROP TABLE tblcoll;
DROP COLLATION caseinsensitive;
\c - - - :worker_1_port
SET citus.enable_metadata_sync TO off;
SET search_path TO collation_conflict;
CREATE COLLATION caseinsensitive (
provider = icu,

View File

@ -25,13 +25,6 @@ SELECT create_distributed_table('notices', 'id');
(1 row)
INSERT INTO notices VALUES (1, 'hello world');
-- Create the necessary test utility function
CREATE OR REPLACE FUNCTION master_metadata_snapshot()
RETURNS text[]
LANGUAGE C STRICT
AS 'citus';
COMMENT ON FUNCTION master_metadata_snapshot()
IS 'commands to create the metadata snapshot';
CREATE FUNCTION notice(text)
RETURNS void
LANGUAGE plpgsql AS $$
@ -476,8 +469,7 @@ ALTER FUNCTION eq(macaddr,macaddr) DEPENDS ON EXTENSION citus;
ERROR: distrtibuted functions are not allowed to depend on an extension
DETAIL: Function "function_tests.eq(pg_catalog.macaddr,pg_catalog.macaddr)" is already distributed. Functions from extensions are expected to be created on the workers by the extension they depend on.
SELECT create_distributed_function('pg_catalog.citus_drop_trigger()');
ERROR: unable to create a distributed function from functions owned by an extension
DETAIL: Function "pg_catalog.citus_drop_trigger()" has a dependency on extension "citus". Functions depending on an extension cannot be distributed. Create the function by creating the extension on the workers.
ERROR: Citus extension functions(citus_drop_trigger) cannot be distributed.
DROP FUNCTION eq(macaddr,macaddr);
-- call should fail as function should have been dropped
SELECT * FROM run_command_on_workers($$SELECT function_tests.eq('0123456789ab','ba9876543210');$$) ORDER BY 1,2;
@ -595,13 +587,6 @@ SELECT create_distributed_function('eq_with_param_names(macaddr, macaddr)', dist
(1 row)
-- show that we are able to propagate objects with multiple item on address arrays
SELECT * FROM (SELECT unnest(master_metadata_snapshot()) as metadata_command order by 1) as innerResult WHERE metadata_command like '%distributed_object_data%';
metadata_command
---------------------------------------------------------------------
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation) AS (VALUES ('type', ARRAY['public.usage_access_type']::text[], ARRAY[]::text[], -1, 0, false), ('type', ARRAY['function_tests.dup_result']::text[], ARRAY[]::text[], -1, 0, false), ('function', ARRAY['public', 'usage_access_func']::text[], ARRAY['public.usage_access_type', 'integer[]']::text[], -1, 0, false), ('function', ARRAY['public', 'usage_access_func_third']::text[], ARRAY['integer', 'integer[]']::text[], 0, 50, false), ('function', ARRAY['function_tests', 'notice']::text[], ARRAY['pg_catalog.text']::text[], -1, 0, false), ('function', ARRAY['function_tests', 'dup']::text[], ARRAY['pg_catalog.macaddr']::text[], 0, 52, false), ('function', ARRAY['function_tests', 'eq_with_param_names']::text[], ARRAY['pg_catalog.macaddr', 'pg_catalog.macaddr']::text[], 0, 52, false), ('function', ARRAY['function_tests', 'eq_mi''xed_param_names']::text[], ARRAY['pg_catalog.macaddr', 'pg_catalog.macaddr']::text[], -1, 0, false), ('function', ARRAY['function_tests', 'agg_sfunc']::text[], ARRAY['integer', 'integer']::text[], -1, 0, false), ('function', ARRAY['function_tests', 'agg_invfunc']::text[], ARRAY['integer', 'integer']::text[], -1, 0, false), ('function', ARRAY['function_tests', 'agg_finalfunc']::text[], ARRAY['integer', 'integer']::text[], -1, 0, false), ('aggregate', ARRAY['function_tests', 'my_rank']::text[], ARRAY['pg_catalog."any"']::text[], -1, 0, false), ('function', ARRAY['function_tests', 'agg_names_sfunc']::text[], ARRAY['function_tests.dup_result', 'function_tests.dup_result', 'function_tests.dup_result']::text[], -1, 0, false), ('function', ARRAY['function_tests', 'agg_names_finalfunc']::text[], ARRAY['function_tests.dup_result']::text[], -1, 0, false), ('aggregate', ARRAY['function_tests', 'agg_names']::text[], ARRAY['function_tests.dup_result', 'function_tests.dup_result']::text[], -1, 0, false), ('sequence', ARRAY['public', 'user_defined_seq']::text[], ARRAY[]::text[], -1, 0, false), ('role', ARRAY['postgres']::text[], ARRAY[]::text[], -1, 0, false), ('database', ARRAY['regression']::text[], ARRAY[]::text[], -1, 0, false), ('server', ARRAY['fake_fdw_server']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['public']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_testing_schema']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_testing_schema_2']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_test_schema_1']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['mx_test_schema_2']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['schema_colocation']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['function_tests']::text[], ARRAY[]::text[], -1, 0, false), ('schema', ARRAY['function_tests2']::text[], ARRAY[]::text[], -1, 0, false), ('extension', ARRAY['plpgsql']::text[], ARRAY[]::text[], -1, 0, false)) SELECT citus_internal_add_object_metadata(typetext, objnames, objargs, distargumentindex::int, colocationid::int, force_delegation::bool) FROM distributed_object_data;
(1 row)
-- valid distribution with distribution_arg_index
SELECT create_distributed_function('eq_with_param_names(macaddr, macaddr)','$1');
create_distributed_function
@ -1078,16 +1063,9 @@ SELECT stop_metadata_sync_to_node(nodename,nodeport) FROM pg_dist_node WHERE isa
\c - - - :worker_1_port
UPDATE pg_dist_local_group SET groupid = 0;
TRUNCATE pg_dist_node;
SET client_min_messages TO error; -- suppress cascading objects dropping
DROP SCHEMA function_tests CASCADE;
DROP SCHEMA function_tests2 CASCADE;
SET search_path TO function_tests, function_tests2;
\c - - - :worker_2_port
UPDATE pg_dist_local_group SET groupid = 0;
TRUNCATE pg_dist_node;
SET client_min_messages TO error; -- suppress cascading objects dropping
DROP SCHEMA function_tests CASCADE;
DROP SCHEMA function_tests2 CASCADE;
\c - - - :master_port
SET client_min_messages TO ERROR;
DROP USER functionuser;

View File

@ -1,13 +1,6 @@
-- This is designed to test worker_create_or_replace_object in PG11 with aggregates
-- Note in PG12 we use CREATE OR REPLACE AGGREGATE, thus the renaming does not occur
CREATE SCHEMA proc_conflict;
SELECT run_command_on_workers($$CREATE SCHEMA proc_conflict;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"CREATE SCHEMA")
(localhost,57638,t,"CREATE SCHEMA")
(2 rows)
\c - - - :worker_1_port
SET search_path TO proc_conflict;
CREATE FUNCTION existing_func(state int, i int) RETURNS int AS $$

View File

@ -173,21 +173,7 @@ SELECT * FROM run_command_on_workers($$CALL procedure_tests.raise_info('hello');
SET client_min_messages TO error; -- suppress cascading objects dropping
DROP SCHEMA procedure_tests CASCADE;
SELECT run_command_on_workers($$DROP SCHEMA procedure_tests CASCADE;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
DROP SCHEMA procedure_tests2 CASCADE;
SELECT run_command_on_workers($$DROP SCHEMA procedure_tests2 CASCADE;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
DROP USER procedureuser;
SELECT 1 FROM run_command_on_workers($$DROP USER procedureuser;$$);
?column?

View File

@ -456,21 +456,7 @@ SELECT * FROM field_indirection_test_2 ORDER BY 1,2,3;
-- clear objects
SET client_min_messages TO error; -- suppress cascading objects dropping
DROP SCHEMA type_tests CASCADE;
SELECT run_command_on_workers($$DROP SCHEMA type_tests CASCADE;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
DROP SCHEMA type_tests2 CASCADE;
SELECT run_command_on_workers($$DROP SCHEMA type_tests2 CASCADE;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
DROP USER typeuser;
SELECT run_command_on_workers($$DROP USER typeuser;$$);
run_command_on_workers

View File

@ -1,12 +1,5 @@
SET citus.next_shard_id TO 20020000;
CREATE SCHEMA type_conflict;
SELECT run_command_on_workers($$CREATE SCHEMA type_conflict;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"CREATE SCHEMA")
(localhost,57638,t,"CREATE SCHEMA")
(2 rows)
-- create a type on a worker that should not cause data loss once overwritten with a type
-- from the coordinator
\c - - :public_worker_1_host :worker_1_port

View File

@ -64,10 +64,3 @@ SELECT run_command_on_workers($$SELECT string_agg(enumlabel, ',' ORDER BY enumso
-- clear objects
SET client_min_messages TO error; -- suppress cascading objects dropping
DROP SCHEMA xact_enum_type CASCADE;
SELECT run_command_on_workers($$DROP SCHEMA xact_enum_type CASCADE;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)

View File

@ -47,7 +47,6 @@ WHERE n.nspname IN ('drop_partitioned_table', 'schema1')
AND c.relkind IN ('r','p')
ORDER BY 1, 2;
\c - - - :worker_1_port
CREATE SCHEMA drop_partitioned_table;
SET search_path = drop_partitioned_table;
CREATE VIEW tables_info AS
SELECT n.nspname as "Schema",
@ -395,11 +394,4 @@ NOTICE: issuing ROLLBACK
NOTICE: issuing ROLLBACK
DROP SCHEMA drop_partitioned_table CASCADE;
NOTICE: drop cascades to 3 other objects
SELECT run_command_on_workers('DROP SCHEMA IF EXISTS drop_partitioned_table CASCADE');
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
SET search_path TO public;

View File

@ -253,11 +253,3 @@ ORDER BY placementid;
RESET SEARCH_PATH;
DROP SCHEMA add_remove_node CASCADE;
NOTICE: drop cascades to table add_remove_node.user_table
SELECT * FROM run_command_on_workers('DROP SCHEMA IF EXISTS add_remove_node CASCADE')
ORDER BY nodeport;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 9060 | t | DROP SCHEMA
localhost | 57637 | t | DROP SCHEMA
(2 rows)

View File

@ -225,8 +225,8 @@ SELECT count(*) FROM single_replicatated WHERE key = 100;
RESET client_min_messages;
-- verify get_global_active_transactions works when a timeout happens on a connection
SELECT get_global_active_transactions();
get_global_active_transactions
SELECT * FROM get_global_active_transactions() WHERE transaction_number != 0;
datid | process_id | initiator_node_identifier | worker_query | transaction_number | transaction_stamp | global_pid
---------------------------------------------------------------------
(0 rows)

View File

@ -4,7 +4,9 @@
-- We have to keep two copies of this failure test
-- because if the shards are created via the executor
-- cancellations are processed, otherwise they are not
SET citus.enable_ddl_propagation TO OFF;
CREATE SCHEMA create_distributed_table_non_empty_failure;
SET citus.enable_ddl_propagation TO ON;
SET search_path TO 'create_distributed_table_non_empty_failure';
SET citus.next_shard_id TO 11000000;
SELECT citus.mitmproxy('conn.allow()');
@ -100,13 +102,6 @@ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata
(localhost,57637,t,1)
(2 rows)
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS create_distributed_table_non_empty_failure$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,9060,t,"DROP SCHEMA")
(localhost,57637,t,"DROP SCHEMA")
(2 rows)
-- this triggers a schema creation which prevents further transactions around dependency propagation
SELECT citus.mitmproxy('conn.allow()');
mitmproxy

View File

@ -193,7 +193,9 @@ SELECT citus.mitmproxy('conn.allow()');
DROP TABLE ref_table;
DROP SCHEMA failure_reference_table;
SET citus.enable_ddl_propagation TO OFF;
CREATE SCHEMA failure_reference_table;
SET citus.enable_ddl_propagation TO ON;
CREATE TABLE ref_table(id int);
INSERT INTO ref_table VALUES(1),(2),(3);
-- Test in transaction

View File

@ -1,7 +1,9 @@
--
-- failure_create_table adds failure tests for creating table without data.
--
SET citus.enable_ddl_propagation TO OFF;
CREATE SCHEMA failure_create_table;
SET citus.enable_ddl_propagation TO ON;
SET search_path TO 'failure_create_table';
SELECT citus.mitmproxy('conn.allow()');
mitmproxy

View File

@ -42,7 +42,7 @@ SELECT citus.mitmproxy('conn.onQuery(query="^UPDATE pg_dist_local_group SET grou
(1 row)
SELECT start_metadata_sync_to_node('localhost', :worker_2_proxy_port);
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.onQuery(query="^UPDATE pg_dist_local_group SET groupid").kill()');
mitmproxy
@ -50,33 +50,33 @@ SELECT citus.mitmproxy('conn.onQuery(query="^UPDATE pg_dist_local_group SET grou
(1 row)
SELECT start_metadata_sync_to_node('localhost', :worker_2_proxy_port);
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:xxxxx
-- Failure to drop all tables in pg_dist_partition
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT worker_drop_distributed_table").cancel(' || :pid || ')');
SELECT citus.mitmproxy('conn.onQuery(query="^DELETE FROM pg_dist_partition").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT start_metadata_sync_to_node('localhost', :worker_2_proxy_port);
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT worker_drop_distributed_table").kill()');
SELECT citus.mitmproxy('conn.onQuery(query="^DELETE FROM pg_dist_partition").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT start_metadata_sync_to_node('localhost', :worker_2_proxy_port);
SELECT citus_activate_node('localhost', :worker_2_proxy_port);
ERROR: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:xxxxx
-- Failure to truncate pg_dist_node in the worker
SELECT citus.mitmproxy('conn.onQuery(query="^TRUNCATE pg_dist_node CASCADE").cancel(' || :pid || ')');
-- Failure to delete pg_dist_node entries from the worker
SELECT citus.mitmproxy('conn.onQuery(query="^DELETE FROM pg_dist_node").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
@ -84,7 +84,7 @@ SELECT citus.mitmproxy('conn.onQuery(query="^TRUNCATE pg_dist_node CASCADE").can
SELECT start_metadata_sync_to_node('localhost', :worker_2_proxy_port);
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.onQuery(query="^TRUNCATE pg_dist_node CASCADE").kill()');
SELECT citus.mitmproxy('conn.onQuery(query="^DELETE FROM pg_dist_node").kill()');
mitmproxy
---------------------------------------------------------------------
@ -122,17 +122,17 @@ SELECT hasmetadata FROM pg_dist_node WHERE nodeport=:worker_2_proxy_port;
f
(1 row)
-- Verify we can sync metadata after unsuccessful attempts
-- Verify we can activate node after unsuccessful attempts
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT start_metadata_sync_to_node('localhost', :worker_2_proxy_port);
start_metadata_sync_to_node
SELECT 1 FROM citus_activate_node('localhost', :worker_2_proxy_port);
?column?
---------------------------------------------------------------------
1
(1 row)
SELECT hasmetadata FROM pg_dist_node WHERE nodeport=:worker_2_proxy_port;
@ -224,8 +224,8 @@ CONTEXT: while executing command on localhost:xxxxx
(1 row)
-- Failure to drop all tables in pg_dist_partition
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT worker_drop_distributed_table").cancel(' || :pid || ')');
-- Failure to delete pg_dist_node entries from the worker
SELECT citus.mitmproxy('conn.onQuery(query="^DELETE FROM pg_dist_node").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
@ -234,60 +234,7 @@ SELECT citus.mitmproxy('conn.onQuery(query="^SELECT worker_drop_distributed_tabl
SELECT stop_metadata_sync_to_node('localhost', :worker_2_proxy_port);
NOTICE: dropping metadata on the node (localhost,9060)
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT worker_drop_distributed_table").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT stop_metadata_sync_to_node('localhost', :worker_2_proxy_port);
NOTICE: dropping metadata on the node (localhost,9060)
WARNING: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:xxxxx
WARNING: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
connection not open
CONTEXT: while executing command on localhost:xxxxx
WARNING: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
connection not open
connection not open
CONTEXT: while executing command on localhost:xxxxx
WARNING: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
connection not open
connection not open
connection not open
CONTEXT: while executing command on localhost:xxxxx
WARNING: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
connection not open
connection not open
connection not open
connection not open
CONTEXT: while executing command on localhost:xxxxx
stop_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
-- Failure to truncate pg_dist_node in the worker
SELECT citus.mitmproxy('conn.onQuery(query="^TRUNCATE pg_dist_node CASCADE").cancel(' || :pid || ')');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT stop_metadata_sync_to_node('localhost', :worker_2_proxy_port);
NOTICE: dropping metadata on the node (localhost,9060)
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.onQuery(query="^TRUNCATE pg_dist_node CASCADE").kill()');
SELECT citus.mitmproxy('conn.onQuery(query="^DELETE FROM pg_dist_node").kill()');
mitmproxy
---------------------------------------------------------------------
@ -344,7 +291,7 @@ SELECT hasmetadata FROM pg_dist_node WHERE nodeport=:worker_2_proxy_port;
f
(1 row)
-- Verify we can drop metadata after unsuccessful attempts
-- Verify we can stop metadata sync after unsuccessful attempts
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
---------------------------------------------------------------------

View File

@ -332,6 +332,7 @@ BEGIN;
ERROR: insert or update on table "local_table_5_1518073" violates foreign key constraint "local_table_5_col_1_fkey1_1518073"
ROLLBACK;
BEGIN;
SET LOCAL citus.multi_shard_modify_mode TO 'sequential';
CREATE SCHEMA another_schema_fkeys_between_local_ref;
CREATE TABLE another_schema_fkeys_between_local_ref.local_table_6 (col_1 INT PRIMARY KEY);
-- first convert local tables to citus local tables in graph
@ -376,6 +377,7 @@ BEGIN;
ROLLBACK;
BEGIN;
SET LOCAL citus.multi_shard_modify_mode TO 'sequential';
CREATE TABLE local_table_6 (col_1 INT PRIMARY KEY);
-- first convert local tables to citus local tables in graph
ALTER TABLE local_table_2 ADD CONSTRAINT fkey_11 FOREIGN KEY (col_1) REFERENCES reference_table_1(col_1) ON DELETE CASCADE;

View File

@ -627,7 +627,7 @@ DETAIL: A distributed function is created. To make sure subsequent commands see
(1 row)
SELECT outer_emp();
DEBUG: Skipping delegation of function from a PL/PgSQL simple expression
DEBUG: Skipping pushdown of function from a PL/PgSQL simple expression
CONTEXT: SQL statement "SELECT inner_emp('hello')"
PL/pgSQL function outer_emp() line XX at PERFORM
outer_emp
@ -1162,18 +1162,11 @@ SELECT * FROM forcepushdown_schema.test_subquery ORDER BY 1;
(5 rows)
-- Query with targetList greater than 1
-- Function from FROM clause is not delegated outside of a BEGIN (for now)
-- Function from FROM clause is delegated outside of a BEGIN
SELECT 1,2,3 FROM select_data(100);
DEBUG: generating subplan XXX_1 for subquery SELECT data FROM forcepushdown_schema.test_subquery WHERE (data OPERATOR(pg_catalog.=) 100)
CONTEXT: SQL statement "SELECT result FROM forcepushdown_schema.test_subquery WHERE data =
(SELECT data FROM forcepushdown_schema.test_subquery WHERE data = a)"
PL/pgSQL function select_data(integer) line XX at SQL statement
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT result FROM forcepushdown_schema.test_subquery WHERE (data OPERATOR(pg_catalog.=) (SELECT intermediate_result.data FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(data integer)))
CONTEXT: SQL statement "SELECT result FROM forcepushdown_schema.test_subquery WHERE data =
(SELECT data FROM forcepushdown_schema.test_subquery WHERE data = a)"
PL/pgSQL function select_data(integer) line XX at SQL statement
DEBUG: pushing down the function call
NOTICE: Result: -1
CONTEXT: PL/pgSQL function select_data(integer) line XX at RAISE
DETAIL: from localhost:xxxxx
?column? | ?column? | ?column?
---------------------------------------------------------------------
1 | 2 | 3
@ -1388,10 +1381,47 @@ SELECT COUNT(*) FROM table_test_prepare;
28
(1 row)
CREATE TABLE test_perform(i int);
SELECT create_distributed_table('test_perform', 'i', colocate_with := 'none');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE OR REPLACE FUNCTION test(x int)
RETURNS int
AS $$
DECLARE
BEGIN
RAISE NOTICE 'INPUT %', x;
RETURN x;
END;
$$ LANGUAGE plpgsql;
SELECT create_distributed_function('test(int)', 'x',
colocate_with := 'test_perform', force_delegation := true);
DEBUG: switching to sequential query execution mode
DETAIL: A distributed function is created. To make sure subsequent commands see the type correctly we need to make sure to use only one connection for all future commands
create_distributed_function
---------------------------------------------------------------------
(1 row)
DO $$
BEGIN
PERFORM test(3);
END;
$$ LANGUAGE plpgsql;
DEBUG: Skipping pushdown of function from a PL/PgSQL simple expression
CONTEXT: SQL statement "SELECT test(3)"
PL/pgSQL function inline_code_block line XX at PERFORM
NOTICE: INPUT 3
CONTEXT: PL/pgSQL function test(integer) line XX at RAISE
SQL statement "SELECT test(3)"
PL/pgSQL function inline_code_block line XX at PERFORM
RESET client_min_messages;
SET citus.log_remote_commands TO off;
DROP SCHEMA forcepushdown_schema CASCADE;
NOTICE: drop cascades to 36 other objects
NOTICE: drop cascades to 38 other objects
DETAIL: drop cascades to table test_forcepushdown
drop cascades to table test_forcepushdown_noncolocate
drop cascades to function insert_data(integer)
@ -1428,3 +1458,5 @@ drop cascades to function insert_data_cte_nondist(integer)
drop cascades to table table_test_prepare
drop cascades to function test_prepare(integer,integer)
drop cascades to function outer_test_prepare(integer,integer)
drop cascades to table test_perform
drop cascades to function test(integer)

View File

@ -6,7 +6,9 @@ CREATE SCHEMA dist_schema;
CREATE TABLE dist_schema.dist_table (id int);
CREATE SCHEMA another_dist_schema;
CREATE TABLE another_dist_schema.dist_table (id int);
SET citus.enable_ddl_propagation TO off;
CREATE SCHEMA non_dist_schema;
SET citus.enable_ddl_propagation TO on;
-- create roles on all nodes
SELECT run_command_on_coordinator_and_workers('CREATE USER role_1');
NOTICE: not propagating CREATE ROLE/USER commands to worker nodes
@ -193,24 +195,9 @@ SET citus.enable_alter_role_propagation TO ON;
ALTER ROLE role_1 NOSUPERUSER;
SET citus.enable_alter_role_propagation TO OFF;
DROP TABLE dist_schema.dist_table, another_dist_schema.dist_table;
SELECT run_command_on_coordinator_and_workers('DROP SCHEMA dist_schema');
run_command_on_coordinator_and_workers
---------------------------------------------------------------------
(1 row)
SELECT run_command_on_coordinator_and_workers('DROP SCHEMA another_dist_schema');
run_command_on_coordinator_and_workers
---------------------------------------------------------------------
(1 row)
SELECT run_command_on_coordinator_and_workers('DROP SCHEMA non_dist_schema');
run_command_on_coordinator_and_workers
---------------------------------------------------------------------
(1 row)
DROP SCHEMA dist_schema;
DROP SCHEMA another_dist_schema;
DROP SCHEMA non_dist_schema;
-- test if the grantors are propagated correctly
-- first remove one of the worker nodes
SET citus.shard_replication_factor TO 1;
@ -319,12 +306,7 @@ SELECT nspname, nspacl FROM pg_namespace WHERE nspname = 'grantor_schema' ORDER
\c - - - :master_port
DROP TABLE grantor_schema.grantor_table;
SELECT run_command_on_coordinator_and_workers('DROP SCHEMA grantor_schema CASCADE');
run_command_on_coordinator_and_workers
---------------------------------------------------------------------
(1 row)
DROP SCHEMA grantor_schema CASCADE;
-- test distributing the schema with another user
CREATE SCHEMA dist_schema;
GRANT ALL ON SCHEMA dist_schema TO role_1 WITH GRANT OPTION;
@ -352,12 +334,7 @@ SELECT nspname, nspacl FROM pg_namespace WHERE nspname = 'dist_schema' ORDER BY
\c - - - :master_port
DROP TABLE dist_schema.dist_table;
SELECT run_command_on_coordinator_and_workers('DROP SCHEMA dist_schema CASCADE');
run_command_on_coordinator_and_workers
---------------------------------------------------------------------
(1 row)
DROP SCHEMA dist_schema CASCADE;
-- test grants on public schema
-- first remove one of the worker nodes
SET citus.shard_replication_factor TO 1;

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