Merge branch 'main' into refactor_extended_task_params

pull/7372/head
Gürkan İndibay 2024-01-23 14:22:24 +03:00 committed by GitHub
commit c3f75b69d3
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
139 changed files with 4961 additions and 777 deletions

View File

@ -471,14 +471,30 @@ jobs:
run: |-
detected_changes=$(git diff origin/main... --name-only --diff-filter=AM | (grep 'src/test/regress/sql/.*\.sql\|src/test/regress/spec/.*\.spec\|src/test/regress/citus_tests/test/test_.*\.py' || true))
tests=${detected_changes}
if [ -z "$tests" ]; then
echo "No test found."
# split the tests to be skipped --today we only skip upgrade tests
skipped_tests=""
not_skipped_tests=""
for test in $tests; do
if [[ $test =~ ^src/test/regress/sql/upgrade_ ]]; then
skipped_tests="$skipped_tests $test"
else
not_skipped_tests="$not_skipped_tests $test"
fi
done
if [ ! -z "$skipped_tests" ]; then
echo "Skipped tests " $skipped_tests
fi
if [ -z "$not_skipped_tests" ]; then
echo "Not detected any tests that flaky test detection should run"
else
echo "Detected tests " $tests
echo "Detected tests " $not_skipped_tests
fi
echo 'tests<<EOF' >> $GITHUB_OUTPUT
echo "$tests" >> "$GITHUB_OUTPUT"
echo "$not_skipped_tests" >> "$GITHUB_OUTPUT"
echo 'EOF' >> $GITHUB_OUTPUT
test-flakyness:
if: ${{ needs.test-flakyness-pre.outputs.tests != ''}}

View File

@ -61,6 +61,7 @@ check-style:
# depend on install-all so that downgrade scripts are installed as well
check: all install-all
$(MAKE) -C src/test/regress check-full
# explicetely does not use $(MAKE) to avoid parallelism
make -C src/test/regress check
.PHONY: all check clean install install-downgrades install-all

View File

@ -209,12 +209,9 @@ static void ReplaceTable(Oid sourceId, Oid targetId, List *justBeforeDropCommand
static bool HasAnyGeneratedStoredColumns(Oid relationId);
static List * GetNonGeneratedStoredColumnNameList(Oid relationId);
static void CheckAlterDistributedTableConversionParameters(TableConversionState *con);
static char * CreateWorkerChangeSequenceDependencyCommand(char *sequenceSchemaName,
char *sequenceName,
char *sourceSchemaName,
char *sourceName,
char *targetSchemaName,
char *targetName);
static char * CreateWorkerChangeSequenceDependencyCommand(char *qualifiedSequeceName,
char *qualifiedSourceName,
char *qualifiedTargetName);
static void ErrorIfMatViewSizeExceedsTheLimit(Oid matViewOid);
static char * CreateMaterializedViewDDLCommand(Oid matViewOid);
static char * GetAccessMethodForMatViewIfExists(Oid viewOid);
@ -791,13 +788,15 @@ ConvertTableInternal(TableConversionState *con)
justBeforeDropCommands = lappend(justBeforeDropCommands, detachFromParentCommand);
}
char *qualifiedRelationName = quote_qualified_identifier(con->schemaName,
con->relationName);
if (PartitionedTable(con->relationId))
{
if (!con->suppressNoticeMessages)
{
ereport(NOTICE, (errmsg("converting the partitions of %s",
quote_qualified_identifier(con->schemaName,
con->relationName))));
qualifiedRelationName)));
}
List *partitionList = PartitionList(con->relationId);
@ -870,9 +869,7 @@ ConvertTableInternal(TableConversionState *con)
if (!con->suppressNoticeMessages)
{
ereport(NOTICE, (errmsg("creating a new table for %s",
quote_qualified_identifier(con->schemaName,
con->relationName))));
ereport(NOTICE, (errmsg("creating a new table for %s", qualifiedRelationName)));
}
TableDDLCommand *tableCreationCommand = NULL;
@ -999,8 +996,6 @@ ConvertTableInternal(TableConversionState *con)
{
continue;
}
char *qualifiedRelationName = quote_qualified_identifier(con->schemaName,
con->relationName);
TableConversionParameters cascadeParam = {
.relationId = colocatedTableId,
@ -1750,9 +1745,7 @@ CreateMaterializedViewDDLCommand(Oid matViewOid)
{
StringInfo query = makeStringInfo();
char *viewName = get_rel_name(matViewOid);
char *schemaName = get_namespace_name(get_rel_namespace(matViewOid));
char *qualifiedViewName = quote_qualified_identifier(schemaName, viewName);
char *qualifiedViewName = generate_qualified_relation_name(matViewOid);
/* here we need to get the access method of the view to recreate it */
char *accessMethodName = GetAccessMethodForMatViewIfExists(matViewOid);
@ -1801,9 +1794,8 @@ ReplaceTable(Oid sourceId, Oid targetId, List *justBeforeDropCommands,
bool suppressNoticeMessages)
{
char *sourceName = get_rel_name(sourceId);
char *targetName = get_rel_name(targetId);
Oid schemaId = get_rel_namespace(sourceId);
char *schemaName = get_namespace_name(schemaId);
char *qualifiedSourceName = generate_qualified_relation_name(sourceId);
char *qualifiedTargetName = generate_qualified_relation_name(targetId);
StringInfo query = makeStringInfo();
@ -1811,8 +1803,7 @@ ReplaceTable(Oid sourceId, Oid targetId, List *justBeforeDropCommands,
{
if (!suppressNoticeMessages)
{
ereport(NOTICE, (errmsg("moving the data of %s",
quote_qualified_identifier(schemaName, sourceName))));
ereport(NOTICE, (errmsg("moving the data of %s", qualifiedSourceName)));
}
if (!HasAnyGeneratedStoredColumns(sourceId))
@ -1822,8 +1813,7 @@ ReplaceTable(Oid sourceId, Oid targetId, List *justBeforeDropCommands,
* "INSERT INTO .. SELECT *"".
*/
appendStringInfo(query, "INSERT INTO %s SELECT * FROM %s",
quote_qualified_identifier(schemaName, targetName),
quote_qualified_identifier(schemaName, sourceName));
qualifiedTargetName, qualifiedSourceName);
}
else
{
@ -1838,9 +1828,8 @@ ReplaceTable(Oid sourceId, Oid targetId, List *justBeforeDropCommands,
char *insertColumnString = StringJoin(nonStoredColumnNameList, ',');
appendStringInfo(query,
"INSERT INTO %s (%s) OVERRIDING SYSTEM VALUE SELECT %s FROM %s",
quote_qualified_identifier(schemaName, targetName),
insertColumnString, insertColumnString,
quote_qualified_identifier(schemaName, sourceName));
qualifiedTargetName, insertColumnString,
insertColumnString, qualifiedSourceName);
}
ExecuteQueryViaSPI(query->data, SPI_OK_INSERT);
@ -1864,14 +1853,11 @@ ReplaceTable(Oid sourceId, Oid targetId, List *justBeforeDropCommands,
*/
if (ShouldSyncTableMetadata(targetId))
{
Oid sequenceSchemaOid = get_rel_namespace(sequenceOid);
char *sequenceSchemaName = get_namespace_name(sequenceSchemaOid);
char *sequenceName = get_rel_name(sequenceOid);
char *qualifiedSequenceName = generate_qualified_relation_name(sequenceOid);
char *workerChangeSequenceDependencyCommand =
CreateWorkerChangeSequenceDependencyCommand(sequenceSchemaName,
sequenceName,
schemaName, sourceName,
schemaName, targetName);
CreateWorkerChangeSequenceDependencyCommand(qualifiedSequenceName,
qualifiedSourceName,
qualifiedTargetName);
SendCommandToWorkersWithMetadata(workerChangeSequenceDependencyCommand);
}
else if (ShouldSyncTableMetadata(sourceId))
@ -1894,25 +1880,23 @@ ReplaceTable(Oid sourceId, Oid targetId, List *justBeforeDropCommands,
if (!suppressNoticeMessages)
{
ereport(NOTICE, (errmsg("dropping the old %s",
quote_qualified_identifier(schemaName, sourceName))));
ereport(NOTICE, (errmsg("dropping the old %s", qualifiedSourceName)));
}
resetStringInfo(query);
appendStringInfo(query, "DROP %sTABLE %s CASCADE",
IsForeignTable(sourceId) ? "FOREIGN " : "",
quote_qualified_identifier(schemaName, sourceName));
qualifiedSourceName);
ExecuteQueryViaSPI(query->data, SPI_OK_UTILITY);
if (!suppressNoticeMessages)
{
ereport(NOTICE, (errmsg("renaming the new table to %s",
quote_qualified_identifier(schemaName, sourceName))));
ereport(NOTICE, (errmsg("renaming the new table to %s", qualifiedSourceName)));
}
resetStringInfo(query);
appendStringInfo(query, "ALTER TABLE %s RENAME TO %s",
quote_qualified_identifier(schemaName, targetName),
qualifiedTargetName,
quote_identifier(sourceName));
ExecuteQueryViaSPI(query->data, SPI_OK_UTILITY);
}
@ -2172,18 +2156,13 @@ CheckAlterDistributedTableConversionParameters(TableConversionState *con)
* worker_change_sequence_dependency query with the parameters.
*/
static char *
CreateWorkerChangeSequenceDependencyCommand(char *sequenceSchemaName, char *sequenceName,
char *sourceSchemaName, char *sourceName,
char *targetSchemaName, char *targetName)
CreateWorkerChangeSequenceDependencyCommand(char *qualifiedSequeceName,
char *qualifiedSourceName,
char *qualifiedTargetName)
{
char *qualifiedSchemaName = quote_qualified_identifier(sequenceSchemaName,
sequenceName);
char *qualifiedSourceName = quote_qualified_identifier(sourceSchemaName, sourceName);
char *qualifiedTargetName = quote_qualified_identifier(targetSchemaName, targetName);
StringInfo query = makeStringInfo();
appendStringInfo(query, "SELECT worker_change_sequence_dependency(%s, %s, %s)",
quote_literal_cstr(qualifiedSchemaName),
quote_literal_cstr(qualifiedSequeceName),
quote_literal_cstr(qualifiedSourceName),
quote_literal_cstr(qualifiedTargetName));

View File

@ -1160,9 +1160,7 @@ DropIdentitiesOnTable(Oid relationId)
if (attributeForm->attidentity)
{
char *tableName = get_rel_name(relationId);
char *schemaName = get_namespace_name(get_rel_namespace(relationId));
char *qualifiedTableName = quote_qualified_identifier(schemaName, tableName);
char *qualifiedTableName = generate_qualified_relation_name(relationId);
StringInfo dropCommand = makeStringInfo();
@ -1222,9 +1220,7 @@ DropViewsOnTable(Oid relationId)
Oid viewId = InvalidOid;
foreach_oid(viewId, reverseOrderedViews)
{
char *viewName = get_rel_name(viewId);
char *schemaName = get_namespace_name(get_rel_namespace(viewId));
char *qualifiedViewName = quote_qualified_identifier(schemaName, viewName);
char *qualifiedViewName = generate_qualified_relation_name(viewId);
StringInfo dropCommand = makeStringInfo();
appendStringInfo(dropCommand, "DROP %sVIEW IF EXISTS %s",

View File

@ -0,0 +1,131 @@
/*-------------------------------------------------------------------------
*
* comment.c
* Commands to interact with the comments for all database
* object types.
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "access/genam.h"
#include "access/htup_details.h"
#include "access/table.h"
#include "catalog/pg_shdescription.h"
#include "nodes/parsenodes.h"
#include "utils/builtins.h"
#include "utils/fmgroids.h"
#include "utils/rel.h"
#include "distributed/comment.h"
static char * GetCommentForObject(Oid classOid, Oid objectOid);
List *
GetCommentPropagationCommands(Oid classOid, Oid objOoid, char *objectName, ObjectType
objectType)
{
List *commands = NIL;
StringInfo commentStmt = makeStringInfo();
/* Get the comment for the database */
char *comment = GetCommentForObject(classOid, objOoid);
char const *commentObjectType = ObjectTypeNames[objectType];
/* Create the SQL command to propagate the comment to other nodes */
if (comment != NULL)
{
appendStringInfo(commentStmt, "COMMENT ON %s %s IS %s;", commentObjectType,
quote_identifier(objectName),
quote_literal_cstr(comment));
}
/* Add the command to the list */
if (commentStmt->len > 0)
{
commands = list_make1(commentStmt->data);
}
return commands;
}
static char *
GetCommentForObject(Oid classOid, Oid objectOid)
{
HeapTuple tuple;
char *comment = NULL;
/* Open pg_shdescription catalog */
Relation shdescRelation = table_open(SharedDescriptionRelationId, AccessShareLock);
/* Scan the table */
ScanKeyData scanKey[2];
ScanKeyInit(&scanKey[0],
Anum_pg_shdescription_objoid,
BTEqualStrategyNumber, F_OIDEQ,
ObjectIdGetDatum(objectOid));
ScanKeyInit(&scanKey[1],
Anum_pg_shdescription_classoid,
BTEqualStrategyNumber, F_OIDEQ,
ObjectIdGetDatum(classOid));
bool indexOk = true;
int scanKeyCount = 2;
SysScanDesc scan = systable_beginscan(shdescRelation, SharedDescriptionObjIndexId,
indexOk, NULL, scanKeyCount,
scanKey);
if ((tuple = systable_getnext(scan)) != NULL)
{
bool isNull = false;
TupleDesc tupdesc = RelationGetDescr(shdescRelation);
Datum descDatum = heap_getattr(tuple, Anum_pg_shdescription_description, tupdesc,
&isNull);
/* Add the command to the list */
if (!isNull)
{
comment = TextDatumGetCString(descDatum);
}
else
{
comment = NULL;
}
}
/* End the scan and close the catalog */
systable_endscan(scan);
table_close(shdescRelation, AccessShareLock);
return comment;
}
/*
* CommentObjectAddress resolves the ObjectAddress for the object
* on which the comment is placed. Optionally errors if the object does not
* exist based on the missing_ok flag passed in by the caller.
*/
List *
CommentObjectAddress(Node *node, bool missing_ok, bool isPostprocess)
{
CommentStmt *stmt = castNode(CommentStmt, node);
Relation relation;
ObjectAddress objectAddress = get_object_address(stmt->objtype, stmt->object,
&relation, AccessExclusiveLock,
missing_ok);
ObjectAddress *objectAddressCopy = palloc0(sizeof(ObjectAddress));
*objectAddressCopy = objectAddress;
return list_make1(objectAddressCopy);
}

View File

@ -1323,10 +1323,7 @@ CreateCitusTable(Oid relationId, CitusTableType tableType,
{
List *partitionList = PartitionList(relationId);
Oid partitionRelationId = InvalidOid;
Oid namespaceId = get_rel_namespace(relationId);
char *schemaName = get_namespace_name(namespaceId);
char *relationName = get_rel_name(relationId);
char *parentRelationName = quote_qualified_identifier(schemaName, relationName);
char *parentRelationName = generate_qualified_relation_name(relationId);
/*
* when there are many partitions, each call to CreateDistributedTable

View File

@ -13,8 +13,10 @@
#include "miscadmin.h"
#include "access/genam.h"
#include "access/heapam.h"
#include "access/htup_details.h"
#include "access/table.h"
#include "access/xact.h"
#include "catalog/objectaddress.h"
#include "catalog/pg_collation.h"
@ -25,6 +27,7 @@
#include "commands/defrem.h"
#include "nodes/parsenodes.h"
#include "utils/builtins.h"
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
#include "utils/rel.h"
#include "utils/relcache.h"
@ -33,6 +36,7 @@
#include "distributed/adaptive_executor.h"
#include "distributed/commands.h"
#include "distributed/commands/utility_hook.h"
#include "distributed/comment.h"
#include "distributed/deparse_shard_query.h"
#include "distributed/deparser.h"
#include "distributed/listutils.h"
@ -41,10 +45,10 @@
#include "distributed/metadata_utility.h"
#include "distributed/multi_executor.h"
#include "distributed/relation_access_tracking.h"
#include "distributed/serialize_distributed_ddls.h"
#include "distributed/worker_protocol.h"
#include "distributed/worker_transaction.h"
/*
* DatabaseCollationInfo is used to store collation related information of a database.
*/
@ -74,6 +78,7 @@ static char * GetTablespaceName(Oid tablespaceOid);
static ObjectAddress * GetDatabaseAddressFromDatabaseName(char *databaseName,
bool missingOk);
static List * FilterDistributedDatabases(List *databases);
static Oid get_database_owner(Oid dbId);
@ -173,17 +178,23 @@ PreprocessGrantOnDatabaseStmt(Node *node, const char *queryString,
GrantStmt *stmt = castNode(GrantStmt, node);
Assert(stmt->objtype == OBJECT_DATABASE);
List *databaseList = stmt->objects;
List *distributedDatabases = FilterDistributedDatabases(stmt->objects);
if (list_length(databaseList) == 0)
if (list_length(distributedDatabases) == 0)
{
return NIL;
}
EnsureCoordinator();
List *originalObjects = stmt->objects;
stmt->objects = distributedDatabases;
char *sql = DeparseTreeNode((Node *) stmt);
stmt->objects = originalObjects;
List *commands = list_make3(DISABLE_DDL_PROPAGATION,
(void *) sql,
ENABLE_DDL_PROPAGATION);
@ -192,57 +203,127 @@ PreprocessGrantOnDatabaseStmt(Node *node, const char *queryString,
}
/*
* FilterDistributedDatabases filters the database list and returns the distributed ones,
* as a list.
*/
static List *
FilterDistributedDatabases(List *databases)
{
List *distributedDatabases = NIL;
String *databaseName = NULL;
foreach_ptr(databaseName, databases)
{
bool missingOk = true;
ObjectAddress *dbAddress =
GetDatabaseAddressFromDatabaseName(strVal(databaseName), missingOk);
if (IsAnyObjectDistributed(list_make1(dbAddress)))
{
distributedDatabases = lappend(distributedDatabases, databaseName);
}
}
return distributedDatabases;
}
/*
* IsSetTablespaceStatement returns true if the statement is a SET TABLESPACE statement,
* false otherwise.
*/
static bool
IsSetTablespaceStatement(AlterDatabaseStmt *stmt)
{
DefElem *def = NULL;
foreach_ptr(def, stmt->options)
{
if (strcmp(def->defname, "tablespace") == 0)
{
return true;
}
}
return false;
}
/*
* PreprocessAlterDatabaseStmt is executed before the statement is applied to the local
* postgres instance.
*
* In this stage we can prepare the commands that need to be run on all workers to grant
* on databases.
*
* We also serialize database commands globally by acquiring a Citus specific advisory
* lock based on OCLASS_DATABASE on the first primary worker node.
*/
List *
PreprocessAlterDatabaseStmt(Node *node, const char *queryString,
ProcessUtilityContext processUtilityContext)
{
if (!ShouldPropagate())
bool missingOk = false;
AlterDatabaseStmt *stmt = castNode(AlterDatabaseStmt, node);
ObjectAddress *dbAddress = GetDatabaseAddressFromDatabaseName(stmt->dbname,
missingOk);
if (!ShouldPropagate() || !IsAnyObjectDistributed(list_make1(dbAddress)))
{
return NIL;
}
AlterDatabaseStmt *stmt = castNode(AlterDatabaseStmt, node);
EnsureCoordinator();
SerializeDistributedDDLsOnObjectClassObject(OCLASS_DATABASE, stmt->dbname);
char *sql = DeparseTreeNode((Node *) stmt);
List *commands = list_make3(DISABLE_DDL_PROPAGATION,
(void *) sql,
sql,
ENABLE_DDL_PROPAGATION);
return NodeDDLTaskList(NON_COORDINATOR_NODES, commands);
if (IsSetTablespaceStatement(stmt))
{
/*
* Set tablespace does not work inside a transaction.Therefore, we need to use
* NontransactionalNodeDDLTask to run the command on the workers outside
* the transaction block.
*/
return NontransactionalNodeDDLTaskList(NON_COORDINATOR_NODES, commands);
}
else
{
return NodeDDLTaskList(NON_COORDINATOR_NODES, commands);
}
}
#if PG_VERSION_NUM >= PG_VERSION_15
/*
* PreprocessAlterDatabaseSetStmt is executed before the statement is applied to the local
* postgres instance.
* PreprocessAlterDatabaseRefreshCollStmt is executed before the statement is applied to
* the local postgres instance.
*
* In this stage we can prepare the commands that need to be run on all workers to grant
* on databases.
*
* We also serialize database commands globally by acquiring a Citus specific advisory
* lock based on OCLASS_DATABASE on the first primary worker node.
*/
List *
PreprocessAlterDatabaseRefreshCollStmt(Node *node, const char *queryString,
ProcessUtilityContext processUtilityContext)
{
if (!ShouldPropagate())
bool missingOk = true;
AlterDatabaseRefreshCollStmt *stmt = castNode(AlterDatabaseRefreshCollStmt, node);
ObjectAddress *dbAddress = GetDatabaseAddressFromDatabaseName(stmt->dbname,
missingOk);
if (!ShouldPropagate() || !IsAnyObjectDistributed(list_make1(dbAddress)))
{
return NIL;
}
AlterDatabaseRefreshCollStmt *stmt = castNode(AlterDatabaseRefreshCollStmt, node);
EnsureCoordinator();
SerializeDistributedDDLsOnObjectClassObject(OCLASS_DATABASE, stmt->dbname);
char *sql = DeparseTreeNode((Node *) stmt);
@ -256,23 +337,66 @@ PreprocessAlterDatabaseRefreshCollStmt(Node *node, const char *queryString,
#endif
/*
* PreprocessAlterDatabaseSetStmt is executed before the statement is applied to the local
* postgres instance.
* PreprocessAlterDatabaseRenameStmt is executed before the statement is applied to
* the local postgres instance.
*
* In this stage we can prepare the commands that need to be run on all workers to grant
* on databases.
* We also serialize database commands globally by acquiring a Citus specific advisory
* lock based on OCLASS_DATABASE on the first primary worker node.
*
* We acquire this lock here instead of PostprocessAlterDatabaseRenameStmt because the
* command renames the database and SerializeDistributedDDLsOnObjectClass resolves the
* object on workers based on database name. For this reason, we need to acquire the lock
* before the command is applied to the local postgres instance.
*/
List *
PreprocessAlterDatabaseSetStmt(Node *node, const char *queryString,
ProcessUtilityContext processUtilityContext)
PreprocessAlterDatabaseRenameStmt(Node *node, const char *queryString,
ProcessUtilityContext processUtilityContext)
{
if (!ShouldPropagate())
bool missingOk = true;
RenameStmt *stmt = castNode(RenameStmt, node);
ObjectAddress *dbAddress = GetDatabaseAddressFromDatabaseName(stmt->subname,
missingOk);
if (!ShouldPropagate() || !IsAnyObjectDistributed(list_make1(dbAddress)))
{
return NIL;
}
AlterDatabaseSetStmt *stmt = castNode(AlterDatabaseSetStmt, node);
EnsureCoordinator();
/*
* Different than other ALTER DATABASE commands, we first acquire a lock
* by providing InvalidOid because we want ALTER TABLE .. RENAME TO ..
* commands to block not only with ALTER DATABASE operations but also
* with CREATE DATABASE operations because they might cause name conflicts
* and that could also cause deadlocks too.
*/
SerializeDistributedDDLsOnObjectClass(OCLASS_DATABASE);
SerializeDistributedDDLsOnObjectClassObject(OCLASS_DATABASE, stmt->subname);
return NIL;
}
/*
* PostprocessAlterDatabaseRenameStmt is executed after the statement is applied to the local
* postgres instance. In this stage we prepare ALTER DATABASE RENAME statement to be run on
* all workers.
*/
List *
PostprocessAlterDatabaseRenameStmt(Node *node, const char *queryString)
{
bool missingOk = false;
RenameStmt *stmt = castNode(RenameStmt, node);
ObjectAddress *dbAddress = GetDatabaseAddressFromDatabaseName(stmt->newname,
missingOk);
if (!ShouldPropagate() || !IsAnyObjectDistributed(list_make1(dbAddress)))
{
return NIL;
}
EnsureCoordinator();
@ -287,12 +411,52 @@ PreprocessAlterDatabaseSetStmt(Node *node, const char *queryString,
/*
* PostprocessAlterDatabaseStmt is executed before the statement is applied to the local
* PreprocessAlterDatabaseSetStmt is executed before the statement is applied to the local
* postgres instance.
*
* In this stage we can prepare the commands that need to be run on all workers to grant
* on databases.
*
* We also serialize database commands globally by acquiring a Citus specific advisory
* lock based on OCLASS_DATABASE on the first primary worker node.
*/
List *
PreprocessAlterDatabaseSetStmt(Node *node, const char *queryString,
ProcessUtilityContext processUtilityContext)
{
AlterDatabaseSetStmt *stmt = castNode(AlterDatabaseSetStmt, node);
bool missingOk = true;
ObjectAddress *dbAddress = GetDatabaseAddressFromDatabaseName(stmt->dbname,
missingOk);
if (!ShouldPropagate() || !IsAnyObjectDistributed(list_make1(dbAddress)))
{
return NIL;
}
EnsureCoordinator();
SerializeDistributedDDLsOnObjectClassObject(OCLASS_DATABASE, stmt->dbname);
char *sql = DeparseTreeNode((Node *) stmt);
List *commands = list_make3(DISABLE_DDL_PROPAGATION,
(void *) sql,
ENABLE_DDL_PROPAGATION);
return NodeDDLTaskList(NON_COORDINATOR_NODES, commands);
}
/*
* PreprocessCreateDatabaseStmt is executed before the statement is applied to the local
* Postgres instance.
*
* In this stage, we perform validations that we want to ensure before delegating to
* previous utility hooks because it might not be convenient to throw an error in an
* implicit transaction that creates a database.
*
* We also serialize database commands globally by acquiring a Citus specific advisory
* lock based on OCLASS_DATABASE on the first primary worker node.
*/
List *
PreprocessCreateDatabaseStmt(Node *node, const char *queryString,
@ -303,11 +467,13 @@ PreprocessCreateDatabaseStmt(Node *node, const char *queryString,
return NIL;
}
EnsureCoordinator();
EnsurePropagationToCoordinator();
CreatedbStmt *stmt = castNode(CreatedbStmt, node);
EnsureSupportedCreateDatabaseCommand(stmt);
SerializeDistributedDDLsOnObjectClass(OCLASS_DATABASE);
return NIL;
}
@ -328,7 +494,7 @@ PostprocessCreateDatabaseStmt(Node *node, const char *queryString)
return NIL;
}
EnsureCoordinator();
EnsurePropagationToCoordinator();
/*
* Given that CREATE DATABASE doesn't support "IF NOT EXISTS" and we're
@ -346,16 +512,19 @@ PostprocessCreateDatabaseStmt(Node *node, const char *queryString)
(void *) createDatabaseCommand,
ENABLE_DDL_PROPAGATION);
return NontransactionalNodeDDLTaskList(NON_COORDINATOR_NODES, commands);
return NontransactionalNodeDDLTaskList(REMOTE_NODES, commands);
}
/*
* PreprocessDropDatabaseStmt is executed after the statement is applied to the local
* PreprocessDropDatabaseStmt is executed before the statement is applied to the local
* postgres instance. In this stage we can prepare the commands that need to be run on
* all workers to drop the database. Since the DROP DATABASE statement gives error in
* transaction context, we need to use NontransactionalNodeDDLTaskList to send the
* DROP DATABASE statement to the workers.
*
* We also serialize database commands globally by acquiring a Citus specific advisory
* lock based on OCLASS_DATABASE on the first primary worker node.
*/
List *
PreprocessDropDatabaseStmt(Node *node, const char *queryString,
@ -366,7 +535,7 @@ PreprocessDropDatabaseStmt(Node *node, const char *queryString,
return NIL;
}
EnsureCoordinator();
EnsurePropagationToCoordinator();
DropdbStmt *stmt = (DropdbStmt *) node;
@ -386,13 +555,15 @@ PreprocessDropDatabaseStmt(Node *node, const char *queryString,
return NIL;
}
SerializeDistributedDDLsOnObjectClassObject(OCLASS_DATABASE, stmt->dbname);
char *dropDatabaseCommand = DeparseTreeNode(node);
List *commands = list_make3(DISABLE_DDL_PROPAGATION,
(void *) dropDatabaseCommand,
ENABLE_DDL_PROPAGATION);
return NontransactionalNodeDDLTaskList(NON_COORDINATOR_NODES, commands);
return NontransactionalNodeDDLTaskList(REMOTE_NODES, commands);
}
@ -504,6 +675,31 @@ GetTablespaceName(Oid tablespaceOid)
}
/*
* GetDatabaseMetadataSyncCommands returns a list of sql statements
* for the given database id. The list contains the database ddl command,
* grant commands and comment propagation commands.
*/
List *
GetDatabaseMetadataSyncCommands(Oid dbOid)
{
char *databaseName = get_database_name(dbOid);
char *databaseDDLCommand = CreateDatabaseDDLCommand(dbOid);
List *ddlCommands = list_make1(databaseDDLCommand);
List *grantDDLCommands = GrantOnDatabaseDDLCommands(dbOid);
List *commentDDLCommands = GetCommentPropagationCommands(DatabaseRelationId, dbOid,
databaseName,
OBJECT_DATABASE);
ddlCommands = list_concat(ddlCommands, grantDDLCommands);
ddlCommands = list_concat(ddlCommands, commentDDLCommands);
return ddlCommands;
}
/*
* GetDatabaseCollation gets oid of a database and returns all the collation related information
* We need this method since collation related info in Form_pg_database is not accessible.

View File

@ -31,20 +31,90 @@
#include "distributed/worker_manager.h"
#include "distributed/worker_transaction.h"
typedef enum RequiredObjectSet
{
REQUIRE_ONLY_DEPENDENCIES = 1,
REQUIRE_OBJECT_AND_DEPENDENCIES = 2,
} RequiredObjectSet;
static void EnsureDependenciesCanBeDistributed(const ObjectAddress *relationAddress);
static void ErrorIfCircularDependencyExists(const ObjectAddress *objectAddress);
static int ObjectAddressComparator(const void *a, const void *b);
static void EnsureDependenciesExistOnAllNodes(const ObjectAddress *target);
static void EnsureRequiredObjectSetExistOnAllNodes(const ObjectAddress *target,
RequiredObjectSet requiredObjectSet);
static List * GetDependencyCreateDDLCommands(const ObjectAddress *dependency);
static bool ShouldPropagateObject(const ObjectAddress *address);
static char * DropTableIfExistsCommand(Oid relationId);
/*
* EnsureDependenciesExistOnAllNodes finds all the dependencies that we support and makes
* sure these are available on all nodes. If not available they will be created on the
* nodes via a separate session that will be committed directly so that the objects are
* visible to potentially multiple sessions creating the shards.
* EnsureObjectAndDependenciesExistOnAllNodes is a wrapper around
* EnsureRequiredObjectSetExistOnAllNodes to ensure the "object itself" (together
* with its dependencies) is available on all nodes.
*
* Different than EnsureDependenciesExistOnAllNodes, we return early if the
* target object is distributed already.
*
* The reason why we don't do the same in EnsureDependenciesExistOnAllNodes
* is that it's is used when altering an object too and hence the target object
* may instantly have a dependency that needs to be propagated now. For example,
* when "GRANT non_dist_role TO dist_role" is executed, we need to propagate
* "non_dist_role" to all nodes before propagating the "GRANT" command itself.
* For this reason, we call EnsureDependenciesExistOnAllNodes for "dist_role"
* and it would automatically discover that "non_dist_role" is a dependency of
* "dist_role" and propagate it beforehand.
*
* However, when we're requested to create the target object itself (and
* implicitly its dependencies), we're sure that we're not altering the target
* object itself, hence we can return early if the target object is already
* distributed. This is the case, for example, when
* "REASSIGN OWNED BY dist_role TO non_dist_role" is executed. In that case,
* "non_dist_role" is not a dependency of "dist_role" but we want to distribute
* "non_dist_role" beforehand and we call this function for "non_dist_role",
* not for "dist_role".
*
* See EnsureRequiredObjectExistOnAllNodes to learn more about how this
* function deals with an object created within the same transaction.
*/
void
EnsureObjectAndDependenciesExistOnAllNodes(const ObjectAddress *target)
{
if (IsAnyObjectDistributed(list_make1((ObjectAddress *) target)))
{
return;
}
EnsureRequiredObjectSetExistOnAllNodes(target, REQUIRE_OBJECT_AND_DEPENDENCIES);
}
/*
* EnsureDependenciesExistOnAllNodes is a wrapper around
* EnsureRequiredObjectSetExistOnAllNodes to ensure "all dependencies" of given
* object --but not the object itself-- are available on all nodes.
*
* See EnsureRequiredObjectSetExistOnAllNodes to learn more about how this
* function deals with an object created within the same transaction.
*/
static void
EnsureDependenciesExistOnAllNodes(const ObjectAddress *target)
{
EnsureRequiredObjectSetExistOnAllNodes(target, REQUIRE_ONLY_DEPENDENCIES);
}
/*
* EnsureRequiredObjectSetExistOnAllNodes finds all the dependencies that we support and makes
* sure these are available on all nodes if required object set is REQUIRE_ONLY_DEPENDENCIES.
* Otherwise, i.e., if required object set is REQUIRE_OBJECT_AND_DEPENDENCIES, then this
* function creates the object itself on all nodes too. This function ensures that each
* of the dependencies are supported by Citus but doesn't check the same for the target
* object itself (when REQUIRE_OBJECT_AND_DEPENDENCIES) is provided because we assume that
* callers don't call this function for an unsupported function at all.
*
* If not available, they will be created on the nodes via a separate session that will be
* committed directly so that the objects are visible to potentially multiple sessions creating
* the shards.
*
* Note; only the actual objects are created via a separate session, the records to
* pg_dist_object are created in this session. As a side effect the objects could be
@ -55,29 +125,52 @@ static char * DropTableIfExistsCommand(Oid relationId);
* postgres native CREATE IF NOT EXISTS, or citus helper functions.
*/
static void
EnsureDependenciesExistOnAllNodes(const ObjectAddress *target)
EnsureRequiredObjectSetExistOnAllNodes(const ObjectAddress *target,
RequiredObjectSet requiredObjectSet)
{
List *dependenciesWithCommands = NIL;
Assert(requiredObjectSet == REQUIRE_ONLY_DEPENDENCIES ||
requiredObjectSet == REQUIRE_OBJECT_AND_DEPENDENCIES);
List *objectsWithCommands = NIL;
List *ddlCommands = NULL;
/*
* If there is any unsupported dependency or circular dependency exists, Citus can
* not ensure dependencies will exist on all nodes.
*
* Note that we don't check whether "target" is distributable (in case
* REQUIRE_OBJECT_AND_DEPENDENCIES is provided) because we expect callers
* to not even call this function if Citus doesn't know how to propagate
* "target" object itself.
*/
EnsureDependenciesCanBeDistributed(target);
/* collect all dependencies in creation order and get their ddl commands */
List *dependencies = GetDependenciesForObject(target);
ObjectAddress *dependency = NULL;
foreach_ptr(dependency, dependencies)
List *objectsToBeCreated = GetDependenciesForObject(target);
/*
* Append the target object to make sure that it's created after its
* dependencies are created, if requested.
*/
if (requiredObjectSet == REQUIRE_OBJECT_AND_DEPENDENCIES)
{
List *dependencyCommands = GetDependencyCreateDDLCommands(dependency);
ObjectAddress *targetCopy = palloc(sizeof(ObjectAddress));
*targetCopy = *target;
objectsToBeCreated = lappend(objectsToBeCreated, targetCopy);
}
ObjectAddress *object = NULL;
foreach_ptr(object, objectsToBeCreated)
{
List *dependencyCommands = GetDependencyCreateDDLCommands(object);
ddlCommands = list_concat(ddlCommands, dependencyCommands);
/* create a new list with dependencies that actually created commands */
/* create a new list with objects that actually created commands */
if (list_length(dependencyCommands) > 0)
{
dependenciesWithCommands = lappend(dependenciesWithCommands, dependency);
objectsWithCommands = lappend(objectsWithCommands, object);
}
}
if (list_length(ddlCommands) <= 0)
@ -100,26 +193,28 @@ EnsureDependenciesExistOnAllNodes(const ObjectAddress *target)
List *remoteNodeList = ActivePrimaryRemoteNodeList(RowShareLock);
/*
* Lock dependent objects explicitly to make sure same DDL command won't be sent
* Lock objects to be created explicitly to make sure same DDL command won't be sent
* multiple times from parallel sessions.
*
* Sort dependencies that will be created on workers to not to have any deadlock
* Sort the objects that will be created on workers to not to have any deadlock
* issue if different sessions are creating different objects.
*/
List *addressSortedDependencies = SortList(dependenciesWithCommands,
List *addressSortedDependencies = SortList(objectsWithCommands,
ObjectAddressComparator);
foreach_ptr(dependency, addressSortedDependencies)
foreach_ptr(object, addressSortedDependencies)
{
LockDatabaseObject(dependency->classId, dependency->objectId,
dependency->objectSubId, ExclusiveLock);
LockDatabaseObject(object->classId, object->objectId,
object->objectSubId, ExclusiveLock);
}
/*
* We need to propagate dependencies via the current user's metadata connection if
* any dependency for the target is created in the current transaction. Our assumption
* is that if we rely on a dependency created in the current transaction, then the
* current user, most probably, has permissions to create the target object as well.
* We need to propagate objects via the current user's metadata connection if
* any of the objects that we're interested in are created in the current transaction.
* Our assumption is that if we rely on an object created in the current transaction,
* then the current user, most probably, has permissions to create the target object
* as well.
*
* Note that, user still may not be able to create the target due to no permissions
* for any of its dependencies. But this is ok since it should be rare.
*
@ -127,7 +222,18 @@ EnsureDependenciesExistOnAllNodes(const ObjectAddress *target)
* have visibility issues since propagated dependencies would be invisible to
* the separate connection until we locally commit.
*/
if (HasAnyDependencyInPropagatedObjects(target))
List *createdObjectList = GetAllSupportedDependenciesForObject(target);
/* consider target as well if we're requested to create it too */
if (requiredObjectSet == REQUIRE_OBJECT_AND_DEPENDENCIES)
{
ObjectAddress *targetCopy = palloc(sizeof(ObjectAddress));
*targetCopy = *target;
createdObjectList = lappend(createdObjectList, targetCopy);
}
if (HasAnyObjectInPropagatedObjects(createdObjectList))
{
SendCommandListToRemoteNodesWithMetadata(ddlCommands);
}
@ -150,7 +256,7 @@ EnsureDependenciesExistOnAllNodes(const ObjectAddress *target)
* that objects have been created on remote nodes before marking them
* distributed, so MarkObjectDistributed wouldn't fail.
*/
foreach_ptr(dependency, dependenciesWithCommands)
foreach_ptr(object, objectsWithCommands)
{
/*
* pg_dist_object entries must be propagated with the super user, since
@ -160,7 +266,7 @@ EnsureDependenciesExistOnAllNodes(const ObjectAddress *target)
* 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);
MarkObjectDistributedViaSuperUser(object);
}
}
@ -478,15 +584,7 @@ GetDependencyCreateDDLCommands(const ObjectAddress *dependency)
*/
if (dependency->objectId != MyDatabaseId && EnableCreateDatabasePropagation)
{
char *databaseDDLCommand = CreateDatabaseDDLCommand(dependency->objectId);
List *ddlCommands = list_make1(databaseDDLCommand);
List *grantDDLCommands = GrantOnDatabaseDDLCommands(dependency->objectId);
ddlCommands = list_concat(ddlCommands, grantDDLCommands);
return ddlCommands;
return GetDatabaseMetadataSyncCommands(dependency->objectId);
}
return NIL;

View File

@ -16,6 +16,7 @@
#include "distributed/commands.h"
#include "distributed/commands/utility_hook.h"
#include "distributed/comment.h"
#include "distributed/deparser.h"
#include "distributed/version_compat.h"
@ -275,6 +276,17 @@ static DistributeObjectOps Any_CreateRole = {
.address = CreateRoleStmtObjectAddress,
.markDistributed = true,
};
static DistributeObjectOps Any_ReassignOwned = {
.deparse = DeparseReassignOwnedStmt,
.qualify = NULL,
.preprocess = NULL,
.postprocess = PostprocessReassignOwnedStmt,
.operationType = DIST_OPS_ALTER,
.address = NULL,
.markDistributed = false,
};
static DistributeObjectOps Any_DropOwned = {
.deparse = DeparseDropOwnedStmt,
.qualify = NULL,
@ -293,6 +305,17 @@ static DistributeObjectOps Any_DropRole = {
.address = NULL,
.markDistributed = false,
};
static DistributeObjectOps Role_Comment = {
.deparse = DeparseCommentStmt,
.qualify = NULL,
.preprocess = PreprocessAlterDistributedObjectStmt,
.postprocess = NULL,
.objectType = OBJECT_DATABASE,
.operationType = DIST_OPS_ALTER,
.address = CommentObjectAddress,
.markDistributed = false,
};
static DistributeObjectOps Any_CreateForeignServer = {
.deparse = DeparseCreateForeignServerStmt,
.qualify = NULL,
@ -522,6 +545,27 @@ static DistributeObjectOps Database_Set = {
.markDistributed = false,
};
static DistributeObjectOps Database_Comment = {
.deparse = DeparseCommentStmt,
.qualify = NULL,
.preprocess = PreprocessAlterDistributedObjectStmt,
.postprocess = NULL,
.objectType = OBJECT_DATABASE,
.operationType = DIST_OPS_ALTER,
.address = CommentObjectAddress,
.markDistributed = false,
};
static DistributeObjectOps Database_Rename = {
.deparse = DeparseAlterDatabaseRenameStmt,
.qualify = NULL,
.preprocess = PreprocessAlterDatabaseRenameStmt,
.postprocess = PostprocessAlterDatabaseRenameStmt,
.objectType = OBJECT_DATABASE,
.operationType = DIST_OPS_ALTER,
.address = NULL,
.markDistributed = false,
};
static DistributeObjectOps Domain_Alter = {
.deparse = DeparseAlterDomainStmt,
@ -951,13 +995,18 @@ static DistributeObjectOps TextSearchConfig_AlterOwner = {
.markDistributed = false,
};
static DistributeObjectOps TextSearchConfig_Comment = {
.deparse = DeparseTextSearchConfigurationCommentStmt,
.deparse = DeparseCommentStmt,
/* TODO: When adding new comment types we should create an abstracted
* qualify function, just like we have an abstract deparse
* and adress function
*/
.qualify = QualifyTextSearchConfigurationCommentStmt,
.preprocess = PreprocessAlterDistributedObjectStmt,
.postprocess = NULL,
.objectType = OBJECT_TSCONFIGURATION,
.operationType = DIST_OPS_ALTER,
.address = TextSearchConfigurationCommentObjectAddress,
.address = CommentObjectAddress,
.markDistributed = false,
};
static DistributeObjectOps TextSearchConfig_Define = {
@ -1020,13 +1069,13 @@ static DistributeObjectOps TextSearchDict_AlterOwner = {
.markDistributed = false,
};
static DistributeObjectOps TextSearchDict_Comment = {
.deparse = DeparseTextSearchDictionaryCommentStmt,
.deparse = DeparseCommentStmt,
.qualify = QualifyTextSearchDictionaryCommentStmt,
.preprocess = PreprocessAlterDistributedObjectStmt,
.postprocess = NULL,
.objectType = OBJECT_TSDICTIONARY,
.operationType = DIST_OPS_ALTER,
.address = TextSearchDictCommentObjectAddress,
.address = CommentObjectAddress,
.markDistributed = false,
};
static DistributeObjectOps TextSearchDict_Define = {
@ -1759,6 +1808,16 @@ GetDistributeObjectOps(Node *node)
return &TextSearchDict_Comment;
}
case OBJECT_DATABASE:
{
return &Database_Comment;
}
case OBJECT_ROLE:
{
return &Role_Comment;
}
default:
{
return &NoDistributeOps;
@ -1868,6 +1927,11 @@ GetDistributeObjectOps(Node *node)
return &Any_DropOwned;
}
case T_ReassignOwnedStmt:
{
return &Any_ReassignOwned;
}
case T_DropStmt:
{
DropStmt *stmt = castNode(DropStmt, node);
@ -2087,6 +2151,11 @@ GetDistributeObjectOps(Node *node)
return &Collation_Rename;
}
case OBJECT_DATABASE:
{
return &Database_Rename;
}
case OBJECT_DOMAIN:
{
return &Domain_Rename;

View File

@ -885,6 +885,7 @@ UpdateFunctionDistributionInfo(const ObjectAddress *distAddress,
char *workerPgDistObjectUpdateCommand =
MarkObjectsDistributedCreateCommand(objectAddressList,
NIL,
distArgumentIndexList,
colocationIdList,
forceDelegationList);

View File

@ -2547,12 +2547,8 @@ ShardIdForTuple(CitusCopyDestReceiver *copyDest, Datum *columnValues, bool *colu
if (columnNulls[partitionColumnIndex])
{
Oid relationId = copyDest->distributedRelationId;
char *relationName = get_rel_name(relationId);
Oid schemaOid = get_rel_namespace(relationId);
char *schemaName = get_namespace_name(schemaOid);
char *qualifiedTableName = quote_qualified_identifier(schemaName,
relationName);
char *qualifiedTableName = generate_qualified_relation_name(
copyDest->distributedRelationId);
ereport(ERROR, (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
errmsg("the partition column of table %s cannot be NULL",

View File

@ -48,6 +48,9 @@
#include "distributed/version_compat.h"
#include "distributed/worker_transaction.h"
static ObjectAddress * GetNewRoleAddress(ReassignOwnedStmt *stmt);
/*
* PreprocessDropOwnedStmt finds the distributed role out of the ones
* being dropped and unmarks them distributed and creates the drop statements
@ -89,3 +92,81 @@ PreprocessDropOwnedStmt(Node *node, const char *queryString,
return NodeDDLTaskList(NON_COORDINATOR_NODES, commands);
}
/*
* PostprocessReassignOwnedStmt takes a Node pointer representing a REASSIGN
* OWNED statement and performs any necessary post-processing after the statement
* has been executed locally.
*
* We filter out local roles in OWNED BY clause before deparsing the command,
* meaning that we skip reassigning what is owned by local roles. However,
* if the role specified in TO clause is local, we automatically distribute
* it before deparsing the command.
*/
List *
PostprocessReassignOwnedStmt(Node *node, const char *queryString)
{
ReassignOwnedStmt *stmt = castNode(ReassignOwnedStmt, node);
List *allReassignRoles = stmt->roles;
List *distributedReassignRoles = FilterDistributedRoles(allReassignRoles);
if (list_length(distributedReassignRoles) <= 0)
{
return NIL;
}
if (!ShouldPropagate())
{
return NIL;
}
EnsureCoordinator();
stmt->roles = distributedReassignRoles;
char *sql = DeparseTreeNode((Node *) stmt);
stmt->roles = allReassignRoles;
ObjectAddress *newRoleAddress = GetNewRoleAddress(stmt);
/*
* We temporarily enable create / alter role propagation to properly
* propagate the role specified in TO clause.
*/
int saveNestLevel = NewGUCNestLevel();
set_config_option("citus.enable_create_role_propagation", "on",
(superuser() ? PGC_SUSET : PGC_USERSET), PGC_S_SESSION,
GUC_ACTION_LOCAL, true, 0, false);
set_config_option("citus.enable_alter_role_propagation", "on",
(superuser() ? PGC_SUSET : PGC_USERSET), PGC_S_SESSION,
GUC_ACTION_LOCAL, true, 0, false);
set_config_option("citus.enable_alter_role_set_propagation", "on",
(superuser() ? PGC_SUSET : PGC_USERSET), PGC_S_SESSION,
GUC_ACTION_LOCAL, true, 0, false);
EnsureObjectAndDependenciesExistOnAllNodes(newRoleAddress);
/* rollback GUCs to the state before this session */
AtEOXact_GUC(true, saveNestLevel);
List *commands = list_make3(DISABLE_DDL_PROPAGATION,
sql,
ENABLE_DDL_PROPAGATION);
return NodeDDLTaskList(NON_COORDINATOR_NODES, commands);
}
/*
* GetNewRoleAddress returns the ObjectAddress of the new role
*/
static ObjectAddress *
GetNewRoleAddress(ReassignOwnedStmt *stmt)
{
Oid roleOid = get_role_oid(stmt->newrole->rolename, false);
ObjectAddress *address = palloc0(sizeof(ObjectAddress));
ObjectAddressSet(*address, AuthIdRelationId, roleOid);
return address;
}

View File

@ -45,6 +45,7 @@
#include "distributed/citus_safe_lib.h"
#include "distributed/commands.h"
#include "distributed/commands/utility_hook.h"
#include "distributed/comment.h"
#include "distributed/coordinator_protocol.h"
#include "distributed/deparser.h"
#include "distributed/listutils.h"
@ -80,7 +81,6 @@ static const char * WrapQueryInAlterRoleIfExistsCall(const char *query, RoleSpec
static VariableSetStmt * MakeVariableSetStmt(const char *config);
static int ConfigGenericNameCompare(const void *lhs, const void *rhs);
static List * RoleSpecToObjectAddress(RoleSpec *role, bool missing_ok);
static bool IsGrantRoleWithInheritOrSetOption(GrantRoleStmt *stmt);
/* controlled via GUC */
bool EnableCreateRolePropagation = true;
@ -583,6 +583,17 @@ GenerateCreateOrAlterRoleCommand(Oid roleOid)
{
completeRoleList = lappend(completeRoleList, DeparseTreeNode(stmt));
}
/*
* append COMMENT ON ROLE commands for this specific user
* When we propagate user creation, we also want to make sure that we propagate
* all the comments it has been given. For this, we check pg_shdescription
* for the ROLE entry corresponding to roleOid, and generate the relevant
* Comment stmts to be run in the new node.
*/
List *commentStmts = GetCommentPropagationCommands(AuthIdRelationId, roleOid,
rolename, OBJECT_ROLE);
completeRoleList = list_concat(completeRoleList, commentStmts);
}
return completeRoleList;
@ -893,10 +904,31 @@ GenerateGrantRoleStmtsOfRole(Oid roleid)
grantRoleStmt->grantor = NULL;
#if PG_VERSION_NUM >= PG_VERSION_16
/* inherit option is always included */
DefElem *inherit_opt;
if (membership->inherit_option)
{
inherit_opt = makeDefElem("inherit", (Node *) makeBoolean(true), -1);
}
else
{
inherit_opt = makeDefElem("inherit", (Node *) makeBoolean(false), -1);
}
grantRoleStmt->opt = list_make1(inherit_opt);
/* admin option is false by default, only include true case */
if (membership->admin_option)
{
DefElem *opt = makeDefElem("admin", (Node *) makeBoolean(true), -1);
grantRoleStmt->opt = list_make1(opt);
DefElem *admin_opt = makeDefElem("admin", (Node *) makeBoolean(true), -1);
grantRoleStmt->opt = lappend(grantRoleStmt->opt, admin_opt);
}
/* set option is true by default, only include false case */
if (!membership->set_option)
{
DefElem *set_opt = makeDefElem("set", (Node *) makeBoolean(false), -1);
grantRoleStmt->opt = lappend(grantRoleStmt->opt, set_opt);
}
#else
grantRoleStmt->admin_opt = membership->admin_option;
@ -1209,19 +1241,6 @@ PreprocessGrantRoleStmt(Node *node, const char *queryString,
return NIL;
}
if (IsGrantRoleWithInheritOrSetOption(stmt))
{
if (EnableUnsupportedFeatureMessages)
{
ereport(NOTICE, (errmsg("not propagating GRANT/REVOKE commands with specified"
" INHERIT/SET options to worker nodes"),
errhint(
"Connect to worker nodes directly to manually run the same"
" GRANT/REVOKE command after disabling DDL propagation.")));
}
return NIL;
}
/*
* Postgres don't seem to use the grantor. Even dropping the grantor doesn't
* seem to affect the membership. If this changes, we might need to add grantors
@ -1273,27 +1292,6 @@ PostprocessGrantRoleStmt(Node *node, const char *queryString)
}
/*
* IsGrantRoleWithInheritOrSetOption returns true if the given
* GrantRoleStmt has inherit or set option specified in its options
*/
static bool
IsGrantRoleWithInheritOrSetOption(GrantRoleStmt *stmt)
{
#if PG_VERSION_NUM >= PG_VERSION_16
DefElem *opt = NULL;
foreach_ptr(opt, stmt->opt)
{
if (strcmp(opt->defname, "inherit") == 0 || strcmp(opt->defname, "set") == 0)
{
return true;
}
}
#endif
return false;
}
/*
* ConfigGenericNameCompare compares two config_generic structs based on their
* name fields. If the name fields contain the same strings two structs are

View File

@ -790,45 +790,6 @@ AlterTextSearchDictionarySchemaStmtObjectAddress(Node *node, bool missing_ok, bo
}
/*
* TextSearchConfigurationCommentObjectAddress resolves the ObjectAddress for the TEXT
* SEARCH CONFIGURATION on which the comment is placed. Optionally errors if the
* configuration does not exist based on the missing_ok flag passed in by the caller.
*/
List *
TextSearchConfigurationCommentObjectAddress(Node *node, bool missing_ok, bool
isPostprocess)
{
CommentStmt *stmt = castNode(CommentStmt, node);
Assert(stmt->objtype == OBJECT_TSCONFIGURATION);
Oid objid = get_ts_config_oid(castNode(List, stmt->object), missing_ok);
ObjectAddress *address = palloc0(sizeof(ObjectAddress));
ObjectAddressSet(*address, TSConfigRelationId, objid);
return list_make1(address);
}
/*
* TextSearchDictCommentObjectAddress resolves the ObjectAddress for the TEXT SEARCH
* DICTIONARY on which the comment is placed. Optionally errors if the dictionary does not
* exist based on the missing_ok flag passed in by the caller.
*/
List *
TextSearchDictCommentObjectAddress(Node *node, bool missing_ok, bool isPostprocess)
{
CommentStmt *stmt = castNode(CommentStmt, node);
Assert(stmt->objtype == OBJECT_TSDICTIONARY);
Oid objid = get_ts_dict_oid(castNode(List, stmt->object), missing_ok);
ObjectAddress *address = palloc0(sizeof(ObjectAddress));
ObjectAddressSet(*address, TSDictionaryRelationId, objid);
return list_make1(address);
}
/*
* AlterTextSearchConfigurationOwnerObjectAddress resolves the ObjectAddress for the TEXT
* SEARCH CONFIGURATION for which the owner is changed. Optionally errors if the

View File

@ -34,6 +34,7 @@
#include "access/htup_details.h"
#include "catalog/catalog.h"
#include "catalog/dependency.h"
#include "catalog/pg_authid.h"
#include "catalog/pg_database.h"
#include "commands/dbcommands.h"
#include "commands/defrem.h"
@ -44,6 +45,7 @@
#include "nodes/makefuncs.h"
#include "nodes/parsenodes.h"
#include "nodes/pg_list.h"
#include "postmaster/postmaster.h"
#include "tcop/utility.h"
#include "utils/builtins.h"
#include "utils/fmgroids.h"
@ -77,6 +79,7 @@
#include "distributed/multi_partitioning_utils.h"
#include "distributed/multi_physical_planner.h"
#include "distributed/reference_table_utils.h"
#include "distributed/remote_commands.h"
#include "distributed/resource_lock.h"
#include "distributed/string_utils.h"
#include "distributed/transaction_management.h"
@ -84,6 +87,13 @@
#include "distributed/worker_shard_visibility.h"
#include "distributed/worker_transaction.h"
#define EXECUTE_COMMAND_ON_REMOTE_NODES_AS_USER \
"SELECT citus_internal.execute_command_on_remote_nodes_as_user(%s, %s)"
#define START_MANAGEMENT_TRANSACTION \
"SELECT citus_internal.start_management_transaction('%lu')"
#define MARK_OBJECT_DISTRIBUTED \
"SELECT citus_internal.mark_object_distributed(%d, %s, %d)"
bool EnableDDLPropagation = true; /* ddl propagation is enabled */
int CreateObjectPropagationMode = CREATE_OBJECT_PROPAGATION_IMMEDIATE;
@ -112,6 +122,8 @@ static void PostStandardProcessUtility(Node *parsetree);
static void DecrementUtilityHookCountersIfNecessary(Node *parsetree);
static bool IsDropSchemaOrDB(Node *parsetree);
static bool ShouldCheckUndistributeCitusLocalTables(void);
static void RunPreprocessMainDBCommand(Node *parsetree, const char *queryString);
static void RunPostprocessMainDBCommand(Node *parsetree);
/*
* ProcessUtilityParseTree is a convenience method to create a PlannedStmt out of
@ -243,6 +255,11 @@ citus_ProcessUtility(PlannedStmt *pstmt,
if (!CitusHasBeenLoaded())
{
if (!IsMainDB)
{
RunPreprocessMainDBCommand(parsetree, queryString);
}
/*
* Ensure that utility commands do not behave any differently until CREATE
* EXTENSION is invoked.
@ -250,6 +267,11 @@ citus_ProcessUtility(PlannedStmt *pstmt,
PrevProcessUtility(pstmt, queryString, false, context,
params, queryEnv, dest, completionTag);
if (!IsMainDB)
{
RunPostprocessMainDBCommand(parsetree);
}
return;
}
else if (IsA(parsetree, CallStmt))
@ -704,9 +726,9 @@ citus_ProcessUtilityInternal(PlannedStmt *pstmt,
ereport(NOTICE, (errmsg("Citus partially supports CREATE DATABASE for "
"distributed databases"),
errdetail("Citus does not propagate CREATE DATABASE "
"command to workers"),
"command to other nodes"),
errhint("You can manually create a database and its "
"extensions on workers.")));
"extensions on other nodes.")));
}
}
else if (IsA(parsetree, CreateRoleStmt) && !EnableCreateRolePropagation)
@ -1572,3 +1594,49 @@ DropSchemaOrDBInProgress(void)
{
return activeDropSchemaOrDBs > 0;
}
/*
* RunPreprocessMainDBCommand runs the necessary commands for a query, in main
* database before query is run on the local node with PrevProcessUtility
*/
static void
RunPreprocessMainDBCommand(Node *parsetree, const char *queryString)
{
if (IsA(parsetree, CreateRoleStmt))
{
StringInfo mainDBQuery = makeStringInfo();
appendStringInfo(mainDBQuery,
START_MANAGEMENT_TRANSACTION,
GetCurrentFullTransactionId().value);
RunCitusMainDBQuery(mainDBQuery->data);
mainDBQuery = makeStringInfo();
appendStringInfo(mainDBQuery,
EXECUTE_COMMAND_ON_REMOTE_NODES_AS_USER,
quote_literal_cstr(queryString),
quote_literal_cstr(CurrentUserName()));
RunCitusMainDBQuery(mainDBQuery->data);
}
}
/*
* RunPostprocessMainDBCommand runs the necessary commands for a query, in main
* database after query is run on the local node with PrevProcessUtility
*/
static void
RunPostprocessMainDBCommand(Node *parsetree)
{
if (IsA(parsetree, CreateRoleStmt))
{
StringInfo mainDBQuery = makeStringInfo();
CreateRoleStmt *createRoleStmt = castNode(CreateRoleStmt, parsetree);
Oid roleOid = get_role_oid(createRoleStmt->role, false);
appendStringInfo(mainDBQuery,
MARK_OBJECT_DISTRIBUTED,
AuthIdRelationId,
quote_literal_cstr(createRoleStmt->role),
roleOid);
RunCitusMainDBQuery(mainDBQuery->data);
}
}

View File

@ -392,9 +392,7 @@ CreateViewDDLCommand(Oid viewOid)
static void
AppendQualifiedViewNameToCreateViewCommand(StringInfo buf, Oid viewOid)
{
char *viewName = get_rel_name(viewOid);
char *schemaName = get_namespace_name(get_rel_namespace(viewOid));
char *qualifiedViewName = quote_qualified_identifier(schemaName, viewName);
char *qualifiedViewName = generate_qualified_relation_name(viewOid);
appendStringInfo(buf, "%s ", qualifiedViewName);
}

View File

@ -425,11 +425,13 @@ GetConnParam(const char *keyword)
/*
* GetEffectiveConnKey checks whether there is any pooler configuration for the
* provided key (host/port combination). The one case where this logic is not
* applied is for loopback connections originating within the task tracker. If
* a corresponding row is found in the poolinfo table, a modified (effective)
* key is returned with the node, port, and dbname overridden, as applicable,
* otherwise, the original key is returned unmodified.
* provided key (host/port combination). If a corresponding row is found in the
* poolinfo table, a modified (effective) key is returned with the node, port,
* and dbname overridden, as applicable, otherwise, the original key is returned
* unmodified.
*
* In the case of Citus non-main databases we just return the key, since we
* would not have access to tables with worker information.
*/
ConnectionHashKey *
GetEffectiveConnKey(ConnectionHashKey *key)
@ -444,7 +446,17 @@ GetEffectiveConnKey(ConnectionHashKey *key)
return key;
}
if (!CitusHasBeenLoaded())
{
/*
* This happens when we connect to main database over localhost
* from some non Citus database.
*/
return key;
}
WorkerNode *worker = FindWorkerNode(key->hostname, key->port);
if (worker == NULL)
{
/* this can be hit when the key references an unknown node */

View File

@ -1046,8 +1046,15 @@ FinishConnectionListEstablishment(List *multiConnectionList)
continue;
}
bool beforePollSocket = PQsocket(connectionState->connection->pgConn);
bool connectionStateChanged = MultiConnectionStatePoll(connectionState);
if (beforePollSocket != PQsocket(connectionState->connection->pgConn))
{
/* rebuild the wait events if MultiConnectionStatePoll() changed the socket */
waitEventSetRebuild = true;
}
if (connectionStateChanged)
{
if (connectionState->phase != MULTI_CONNECTION_PHASE_CONNECTING)

View File

@ -0,0 +1,77 @@
/*-------------------------------------------------------------------------
*
* deparse_coment_stmts.c
*
* All routines to deparse comment statements.
*
* Copyright (c), Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "catalog/namespace.h"
#include "commands/defrem.h"
#include "lib/stringinfo.h"
#include "nodes/parsenodes.h"
#include "parser/parse_type.h"
#include "utils/builtins.h"
#include "utils/elog.h"
#include "pg_version_compat.h"
#include "distributed/citus_ruleutils.h"
#include "distributed/commands.h"
#include "distributed/comment.h"
#include "distributed/deparser.h"
#include "distributed/listutils.h"
#include "distributed/log_utils.h"
const char *ObjectTypeNames[] =
{
[OBJECT_DATABASE] = "DATABASE",
[OBJECT_ROLE] = "ROLE",
[OBJECT_TSCONFIGURATION] = "TEXT SEARCH CONFIGURATION",
[OBJECT_TSDICTIONARY] = "TEXT SEARCH DICTIONARY",
/* When support for propagating comments to new objects is introduced, an entry for each
* statement type should be added to this list. The first element in each entry is the 'object_type' keyword
* that will be included in the 'COMMENT ON <object_type> ..' statement (i.e. DATABASE,). The second element is the type of
* stmt->object, which represents the name of the propagated object.
*/
};
char *
DeparseCommentStmt(Node *node)
{
CommentStmt *stmt = castNode(CommentStmt, node);
StringInfoData str = { 0 };
initStringInfo(&str);
const char *objectName = NULL;
if (IsA(stmt->object, String))
{
objectName = quote_identifier(strVal(stmt->object));
}
else if (IsA(stmt->object, List))
{
objectName = NameListToQuotedString(castNode(List, stmt->object));
}
else
{
ereport(ERROR,
(errcode(ERRCODE_INTERNAL_ERROR),
errmsg("unknown object type")));
}
const char *objectType = ObjectTypeNames[stmt->objtype];
char *comment = stmt->comment != NULL ? quote_literal_cstr(stmt->comment) : "NULL";
appendStringInfo(&str, "COMMENT ON %s %s IS %s;", objectType, objectName, comment);
return str.data;
}

View File

@ -30,12 +30,14 @@
static void AppendAlterDatabaseOwnerStmt(StringInfo buf, AlterOwnerStmt *stmt);
static void AppendAlterDatabaseSetStmt(StringInfo buf, AlterDatabaseSetStmt *stmt);
static void AppendAlterDatabaseStmt(StringInfo buf, AlterDatabaseStmt *stmt);
static void AppendDefElemConnLimit(StringInfo buf, DefElem *def);
static void AppendCreateDatabaseStmt(StringInfo buf, CreatedbStmt *stmt);
static void AppendDropDatabaseStmt(StringInfo buf, DropdbStmt *stmt);
static void AppendGrantOnDatabaseStmt(StringInfo buf, GrantStmt *stmt);
static void AppendBasicAlterDatabaseOptions(StringInfo buf, AlterDatabaseStmt *stmt);
static void AppendGrantDatabases(StringInfo buf, GrantStmt *stmt);
static void AppendAlterDatabaseSetTablespace(StringInfo buf, DefElem *def, char *dbname);
const DefElemOptionFormat create_database_option_formats[] = {
const DefElemOptionFormat createDatabaseOptionFormats[] = {
{ "owner", " OWNER %s", OPTION_FORMAT_STRING },
{ "template", " TEMPLATE %s", OPTION_FORMAT_STRING },
{ "encoding", " ENCODING %s", OPTION_FORMAT_LITERAL_CSTR },
@ -53,6 +55,14 @@ const DefElemOptionFormat create_database_option_formats[] = {
{ "is_template", " IS_TEMPLATE %s", OPTION_FORMAT_BOOLEAN }
};
const DefElemOptionFormat alterDatabaseOptionFormats[] = {
{ "is_template", " IS_TEMPLATE %s", OPTION_FORMAT_BOOLEAN },
{ "allow_connections", " ALLOW_CONNECTIONS %s", OPTION_FORMAT_BOOLEAN },
{ "connection_limit", " CONNECTION LIMIT %d", OPTION_FORMAT_INTEGER },
};
char *
DeparseAlterDatabaseOwnerStmt(Node *node)
{
@ -112,48 +122,63 @@ AppendGrantOnDatabaseStmt(StringInfo buf, GrantStmt *stmt)
static void
AppendDefElemConnLimit(StringInfo buf, DefElem *def)
AppendAlterDatabaseStmt(StringInfo buf, AlterDatabaseStmt *stmt)
{
appendStringInfo(buf, " CONNECTION LIMIT %ld", (long int) defGetNumeric(def));
if (list_length(stmt->options) == 0)
{
elog(ERROR, "got unexpected number of options for ALTER DATABASE");
}
if (stmt->options)
{
DefElem *firstOption = linitial(stmt->options);
if (strcmp(firstOption->defname, "tablespace") == 0)
{
AppendAlterDatabaseSetTablespace(buf, firstOption, stmt->dbname);
/* SET tablespace cannot be combined with other options */
return;
}
appendStringInfo(buf, "ALTER DATABASE %s WITH",
quote_identifier(stmt->dbname));
AppendBasicAlterDatabaseOptions(buf, stmt);
}
appendStringInfo(buf, ";");
}
static void
AppendAlterDatabaseStmt(StringInfo buf, AlterDatabaseStmt *stmt)
AppendAlterDatabaseSetTablespace(StringInfo buf, DefElem *def, char *dbname)
{
appendStringInfo(buf, "ALTER DATABASE %s ", quote_identifier(stmt->dbname));
appendStringInfo(buf,
"ALTER DATABASE %s SET TABLESPACE %s",
quote_identifier(dbname), quote_identifier(defGetString(def)));
}
if (stmt->options)
/*
* AppendBasicAlterDatabaseOptions appends basic ALTER DATABASE options to a string buffer.
* Basic options are those that can be appended to the ALTER DATABASE statement
* after the "WITH" keyword.(i.e. ALLOW_CONNECTIONS, CONNECTION LIMIT, IS_TEMPLATE)
* For example, the tablespace option is not a basic option since it is defined via SET keyword.
*
* This function takes a string buffer and an AlterDatabaseStmt as input.
* It appends the basic options to the string buffer.
*
*/
static void
AppendBasicAlterDatabaseOptions(StringInfo buf, AlterDatabaseStmt *stmt)
{
DefElem *def = NULL;
foreach_ptr(def, stmt->options)
{
ListCell *cell = NULL;
appendStringInfo(buf, "WITH ");
foreach(cell, stmt->options)
{
DefElem *def = castNode(DefElem, lfirst(cell));
if (strcmp(def->defname, "is_template") == 0)
{
appendStringInfo(buf, "IS_TEMPLATE %s",
quote_literal_cstr(strVal(def->arg)));
}
else if (strcmp(def->defname, "connection_limit") == 0)
{
AppendDefElemConnLimit(buf, def);
}
else if (strcmp(def->defname, "allow_connections") == 0)
{
ereport(ERROR,
errmsg("ALLOW_CONNECTIONS is not supported"));
}
else
{
ereport(ERROR,
errmsg("unrecognized ALTER DATABASE option: %s",
def->defname));
}
}
DefElemOptionToStatement(buf, def, alterDatabaseOptionFormats, lengthof(
alterDatabaseOptionFormats));
}
appendStringInfo(buf, ";");
}
@ -216,6 +241,22 @@ AppendAlterDatabaseSetStmt(StringInfo buf, AlterDatabaseSetStmt *stmt)
}
char *
DeparseAlterDatabaseRenameStmt(Node *node)
{
RenameStmt *stmt = (RenameStmt *) node;
StringInfoData str;
initStringInfo(&str);
appendStringInfo(&str, "ALTER DATABASE %s RENAME TO %s",
quote_identifier(stmt->subname),
quote_identifier(stmt->newname));
return str.data;
}
char *
DeparseAlterDatabaseSetStmt(Node *node)
{
@ -246,8 +287,8 @@ AppendCreateDatabaseStmt(StringInfo buf, CreatedbStmt *stmt)
DefElem *option = NULL;
foreach_ptr(option, stmt->options)
{
DefElemOptionToStatement(buf, option, create_database_option_formats,
lengthof(create_database_option_formats));
DefElemOptionToStatement(buf, option, createDatabaseOptionFormats,
lengthof(createDatabaseOptionFormats));
}
}

View File

@ -71,7 +71,7 @@ AppendRoleList(StringInfo buf, List *roleList)
{
Node *roleNode = (Node *) lfirst(cell);
Assert(IsA(roleNode, RoleSpec) || IsA(roleNode, AccessPriv));
char const *rolename = NULL;
const char *rolename = NULL;
if (IsA(roleNode, RoleSpec))
{
rolename = RoleSpecString((RoleSpec *) roleNode, true);
@ -83,3 +83,27 @@ AppendRoleList(StringInfo buf, List *roleList)
}
}
}
static void
AppendReassignOwnedStmt(StringInfo buf, ReassignOwnedStmt *stmt)
{
appendStringInfo(buf, "REASSIGN OWNED BY ");
AppendRoleList(buf, stmt->roles);
const char *newRoleName = RoleSpecString(stmt->newrole, true);
appendStringInfo(buf, " TO %s", newRoleName);
}
char *
DeparseReassignOwnedStmt(Node *node)
{
ReassignOwnedStmt *stmt = castNode(ReassignOwnedStmt, node);
StringInfoData buf = { 0 };
initStringInfo(&buf);
AppendReassignOwnedStmt(&buf, stmt);
return buf.data;
}

View File

@ -411,6 +411,16 @@ AppendRevokeAdminOptionFor(StringInfo buf, GrantRoleStmt *stmt)
appendStringInfo(buf, "ADMIN OPTION FOR ");
break;
}
else if (strcmp(opt->defname, "inherit") == 0)
{
appendStringInfo(buf, "INHERIT OPTION FOR ");
break;
}
else if (strcmp(opt->defname, "set") == 0)
{
appendStringInfo(buf, "SET OPTION FOR ");
break;
}
}
}
#else
@ -428,16 +438,29 @@ AppendGrantWithAdminOption(StringInfo buf, GrantRoleStmt *stmt)
if (stmt->is_grant)
{
#if PG_VERSION_NUM >= PG_VERSION_16
int opt_count = 0;
DefElem *opt = NULL;
foreach_ptr(opt, stmt->opt)
{
bool admin_option = false;
char *optval = defGetString(opt);
if (strcmp(opt->defname, "admin") == 0 &&
parse_bool(optval, &admin_option) && admin_option)
bool option_value = false;
if (parse_bool(optval, &option_value))
{
appendStringInfo(buf, " WITH ADMIN OPTION");
break;
opt_count++;
char *prefix = opt_count > 1 ? "," : " WITH";
if (strcmp(opt->defname, "inherit") == 0)
{
appendStringInfo(buf, "%s INHERIT %s", prefix, option_value ? "TRUE" :
"FALSE");
}
else if (strcmp(opt->defname, "admin") == 0 && option_value)
{
appendStringInfo(buf, "%s ADMIN OPTION", prefix);
}
else if (strcmp(opt->defname, "set") == 0 && !option_value)
{
appendStringInfo(buf, "%s SET FALSE", prefix);
}
}
}
#else

View File

@ -395,68 +395,6 @@ DeparseAlterTextSearchDictionarySchemaStmt(Node *node)
}
/*
* DeparseTextSearchConfigurationCommentStmt returns the sql statement representing
* COMMENT ON TEXT SEARCH CONFIGURATION ... IS ...
*/
char *
DeparseTextSearchConfigurationCommentStmt(Node *node)
{
CommentStmt *stmt = castNode(CommentStmt, node);
Assert(stmt->objtype == OBJECT_TSCONFIGURATION);
StringInfoData buf = { 0 };
initStringInfo(&buf);
appendStringInfo(&buf, "COMMENT ON TEXT SEARCH CONFIGURATION %s IS ",
NameListToQuotedString(castNode(List, stmt->object)));
if (stmt->comment == NULL)
{
appendStringInfoString(&buf, "NULL");
}
else
{
appendStringInfoString(&buf, quote_literal_cstr(stmt->comment));
}
appendStringInfoString(&buf, ";");
return buf.data;
}
/*
* DeparseTextSearchDictionaryCommentStmt returns the sql statement representing
* COMMENT ON TEXT SEARCH DICTIONARY ... IS ...
*/
char *
DeparseTextSearchDictionaryCommentStmt(Node *node)
{
CommentStmt *stmt = castNode(CommentStmt, node);
Assert(stmt->objtype == OBJECT_TSDICTIONARY);
StringInfoData buf = { 0 };
initStringInfo(&buf);
appendStringInfo(&buf, "COMMENT ON TEXT SEARCH DICTIONARY %s IS ",
NameListToQuotedString(castNode(List, stmt->object)));
if (stmt->comment == NULL)
{
appendStringInfoString(&buf, "NULL");
}
else
{
appendStringInfoString(&buf, quote_literal_cstr(stmt->comment));
}
appendStringInfoString(&buf, ";");
return buf.data;
}
/*
* AppendStringInfoTokentypeList specializes in adding a comma separated list of
* token_tyoe's to TEXT SEARCH CONFIGURATION commands

View File

@ -1563,8 +1563,15 @@ set_join_column_names(deparse_namespace *dpns, RangeTblEntry *rte,
/* Assert we processed the right number of columns */
#ifdef USE_ASSERT_CHECKING
while (i < colinfo->num_cols && colinfo->colnames[i] == NULL)
i++;
for (int col_index = 0; col_index < colinfo->num_cols; col_index++)
{
/*
* In the above processing-loops, "i" advances only if
* the column is not new, check if this is a new column.
*/
if (colinfo->is_new_col[col_index])
i++;
}
Assert(i == colinfo->num_cols);
Assert(j == nnewcolumns);
#endif

View File

@ -401,7 +401,7 @@ typedef struct WorkerPool
/*
* Placement executions destined for worker node, but not assigned to any
* connection and not ready to start.
* connection and ready to start.
*/
dlist_head readyTaskQueue;
int readyTaskCount;
@ -492,8 +492,6 @@ typedef struct WorkerSession
} WorkerSession;
struct TaskPlacementExecution;
/* GUC, determining whether Citus opens 1 connection per task */
bool ForceMaxQueryParallelization = false;
int MaxAdaptiveExecutorPoolSize = 16;
@ -585,7 +583,7 @@ typedef enum TaskPlacementExecutionState
} TaskPlacementExecutionState;
/*
* TaskPlacementExecution represents the an execution of a command
* TaskPlacementExecution represents the execution of a command
* on a shard placement.
*/
typedef struct TaskPlacementExecution
@ -1925,7 +1923,7 @@ RunDistributedExecution(DistributedExecution *execution)
/*
* Iterate until all the tasks are finished. Once all the tasks
* are finished, ensure that that all the connection initializations
* are finished, ensure that all the connection initializations
* are also finished. Otherwise, those connections are terminated
* abruptly before they are established (or failed). Instead, we let
* the ConnectionStateMachine() to properly handle them.
@ -3135,7 +3133,7 @@ ConnectionStateMachine(WorkerSession *session)
*
* We can only retry connection when the remote transaction has
* not started over the connection. Otherwise, we'd have to deal
* with restoring the transaction state, which iis beyond our
* with restoring the transaction state, which is beyond our
* purpose at this time.
*/
RemoteTransaction *transaction = &connection->remoteTransaction;

View File

@ -143,15 +143,10 @@ NonPushableInsertSelectExecScan(CustomScanState *node)
targetRelation->partitionColumn);
if (distributionColumnIndex == -1)
{
char *relationName = get_rel_name(targetRelationId);
Oid schemaOid = get_rel_namespace(targetRelationId);
char *schemaName = get_namespace_name(schemaOid);
ereport(ERROR, (errcode(ERRCODE_NULL_VALUE_NOT_ALLOWED),
errmsg(
"the partition column of table %s should have a value",
quote_qualified_identifier(schemaName,
relationName))));
generate_qualified_relation_name(targetRelationId))));
}
TargetEntry *selectPartitionTE = list_nth(selectQuery->targetList,

View File

@ -168,7 +168,7 @@ CitusExecutorRun(QueryDesc *queryDesc,
executorBoundParams = queryDesc->params;
/*
* We do some potentially time consuming operations our self now before we hand of
* We do some potentially time consuming operations ourself now before we hand off
* control to postgres' executor. To make sure that time spent is accurately measured
* we remove the totaltime instrumentation from the queryDesc. Instead we will start
* and stop the instrumentation of the total time and put it back on the queryDesc

View File

@ -109,7 +109,7 @@ TupleStoreTupleDestPutTuple(TupleDestination *self, Task *task,
uint64 tupleSize = tupleLibpqSize;
if (tupleSize == 0)
{
tupleSize = HeapTupleHeaderGetDatumLength(heapTuple);
tupleSize = heapTuple->t_len;
}
/*

View File

@ -31,6 +31,7 @@
#include "nodes/makefuncs.h"
#include "nodes/pg_list.h"
#include "parser/parse_type.h"
#include "postmaster/postmaster.h"
#include "utils/builtins.h"
#include "utils/fmgroids.h"
#include "utils/lsyscache.h"
@ -49,18 +50,49 @@
#include "distributed/metadata/pg_dist_object.h"
#include "distributed/metadata_cache.h"
#include "distributed/metadata_sync.h"
#include "distributed/remote_commands.h"
#include "distributed/version_compat.h"
#include "distributed/worker_transaction.h"
static char * CreatePgDistObjectEntryCommand(const ObjectAddress *objectAddress);
static char * CreatePgDistObjectEntryCommand(const ObjectAddress *objectAddress,
char *objectName);
static int ExecuteCommandAsSuperuser(char *query, int paramCount, Oid *paramTypes,
Datum *paramValues);
static bool IsObjectDistributed(const ObjectAddress *address);
PG_FUNCTION_INFO_V1(mark_object_distributed);
PG_FUNCTION_INFO_V1(citus_unmark_object_distributed);
PG_FUNCTION_INFO_V1(master_unmark_object_distributed);
/*
* mark_object_distributed adds an object to pg_dist_object
* in all of the nodes.
*/
Datum
mark_object_distributed(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
EnsureSuperUser();
Oid classId = PG_GETARG_OID(0);
text *objectNameText = PG_GETARG_TEXT_P(1);
char *objectName = text_to_cstring(objectNameText);
Oid objectId = PG_GETARG_OID(2);
ObjectAddress *objectAddress = palloc0(sizeof(ObjectAddress));
ObjectAddressSet(*objectAddress, classId, objectId);
/*
* This function is called when a query is run from a Citus non-main database.
* We need to insert into local pg_dist_object over a connection to make sure
* 2PC still works.
*/
bool useConnectionForLocalQuery = true;
MarkObjectDistributedWithName(objectAddress, objectName, useConnectionForLocalQuery);
PG_RETURN_VOID();
}
/*
* citus_unmark_object_distributed(classid oid, objid oid, objsubid int)
*
@ -160,12 +192,48 @@ ObjectExists(const ObjectAddress *address)
void
MarkObjectDistributed(const ObjectAddress *distAddress)
{
MarkObjectDistributedLocally(distAddress);
bool useConnectionForLocalQuery = false;
MarkObjectDistributedWithName(distAddress, "", useConnectionForLocalQuery);
}
/*
* MarkObjectDistributedWithName marks an object as a distributed object.
* Same as MarkObjectDistributed but this function also allows passing an objectName
* that is used in case the object does not exists for the current transaction.
*/
void
MarkObjectDistributedWithName(const ObjectAddress *distAddress, char *objectName,
bool useConnectionForLocalQuery)
{
if (!CitusHasBeenLoaded())
{
elog(ERROR, "Cannot mark object distributed because Citus has not been loaded.");
}
/*
* When a query is run from a Citus non-main database we need to insert into pg_dist_object
* over a connection to make sure 2PC still works.
*/
if (useConnectionForLocalQuery)
{
StringInfo insertQuery = makeStringInfo();
appendStringInfo(insertQuery,
"INSERT INTO pg_catalog.pg_dist_object (classid, objid, objsubid)"
"VALUES (%d, %d, %d) ON CONFLICT DO NOTHING",
distAddress->classId, distAddress->objectId,
distAddress->objectSubId);
SendCommandToWorker(LocalHostName, PostPortNumber, insertQuery->data);
}
else
{
MarkObjectDistributedLocally(distAddress);
}
if (EnableMetadataSync)
{
char *workerPgDistObjectUpdateCommand =
CreatePgDistObjectEntryCommand(distAddress);
CreatePgDistObjectEntryCommand(distAddress, objectName);
SendCommandToRemoteNodesWithMetadata(workerPgDistObjectUpdateCommand);
}
}
@ -188,7 +256,7 @@ MarkObjectDistributedViaSuperUser(const ObjectAddress *distAddress)
if (EnableMetadataSync)
{
char *workerPgDistObjectUpdateCommand =
CreatePgDistObjectEntryCommand(distAddress);
CreatePgDistObjectEntryCommand(distAddress, "");
SendCommandToRemoteNodesWithMetadataViaSuperUser(workerPgDistObjectUpdateCommand);
}
}
@ -279,17 +347,21 @@ ShouldMarkRelationDistributed(Oid relationId)
* for the given object address.
*/
static char *
CreatePgDistObjectEntryCommand(const ObjectAddress *objectAddress)
CreatePgDistObjectEntryCommand(const ObjectAddress *objectAddress, char *objectName)
{
/* create a list by adding the address of value to not to have warning */
List *objectAddressList =
list_make1((ObjectAddress *) objectAddress);
/* names also require a list so we create a nested list here */
List *objectNameList = list_make1(list_make1((char *) objectName));
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,
objectNameList,
distArgumetIndexList,
colocationIdList,
forceDelegationList);

View File

@ -79,6 +79,7 @@
#include "distributed/pg_dist_partition.h"
#include "distributed/pg_dist_placement.h"
#include "distributed/pg_dist_shard.h"
#include "distributed/remote_commands.h"
#include "distributed/shardinterval_utils.h"
#include "distributed/shared_library_init.h"
#include "distributed/utils/array_type.h"
@ -380,7 +381,7 @@ EnsureModificationsCanRun(void)
/*
* EnsureModificationsCanRunOnRelation firsts calls into EnsureModificationsCanRun() and
* EnsureModificationsCanRunOnRelation first calls into EnsureModificationsCanRun() and
* then does one more additional check. The additional check is to give a proper error
* message if any relation that is modified is replicated, as replicated tables use
* 2PC and 2PC cannot happen when recovery is in progress.
@ -5722,6 +5723,14 @@ GetPoolinfoViaCatalog(int32 nodeId)
char *
GetAuthinfoViaCatalog(const char *roleName, int64 nodeId)
{
/*
* Citus will not be loaded when we run a global DDL command from a
* Citus non-main database.
*/
if (!CitusHasBeenLoaded())
{
return "";
}
char *authinfo = "";
Datum nodeIdDatumArray[2] = {
Int32GetDatum(nodeId),

View File

@ -83,6 +83,7 @@
#include "distributed/pg_dist_shard.h"
#include "distributed/relation_access_tracking.h"
#include "distributed/remote_commands.h"
#include "distributed/remote_transaction.h"
#include "distributed/resource_lock.h"
#include "distributed/tenant_schema_metadata.h"
#include "distributed/utils/array_type.h"
@ -900,6 +901,7 @@ NodeListIdempotentInsertCommand(List *workerNodeList)
*/
char *
MarkObjectsDistributedCreateCommand(List *addresses,
List *namesArg,
List *distributionArgumentIndexes,
List *colocationIds,
List *forceDelegations)
@ -924,9 +926,25 @@ MarkObjectsDistributedCreateCommand(List *addresses,
int forceDelegation = list_nth_int(forceDelegations, currentObjectCounter);
List *names = NIL;
List *args = NIL;
char *objectType = NULL;
char *objectType = getObjectTypeDescription(address, false);
getObjectIdentityParts(address, &names, &args, false);
if (IsMainDBCommand)
{
/*
* When we try to distribute an object that's being created in a non Citus
* main database, we cannot find the name, since the object is not visible
* in Citus main database.
* Because of that we need to pass the name to this function.
*/
names = list_nth(namesArg, currentObjectCounter);
bool missingOk = false;
objectType = getObjectTypeDescription(address, missingOk);
}
else
{
objectType = getObjectTypeDescription(address, false);
getObjectIdentityParts(address, &names, &args, IsMainDBCommand);
}
if (!isFirstObject)
{
@ -5148,6 +5166,7 @@ SendDistObjectCommands(MetadataSyncContext *context)
char *workerMetadataUpdateCommand =
MarkObjectsDistributedCreateCommand(list_make1(address),
NIL,
list_make1_int(distributionArgumentIndex),
list_make1_int(colocationId),
list_make1_int(forceDelegation));

View File

@ -1751,6 +1751,10 @@ citus_internal_mark_node_not_synced(PG_FUNCTION_ARGS)
/*
* FindWorkerNode searches over the worker nodes and returns the workerNode
* if it already exists. Else, the function returns NULL.
*
* NOTE: A special case that this handles is when nodeName and nodePort are set
* to LocalHostName and PostPortNumber. In that case we return the primary node
* for the local group.
*/
WorkerNode *
FindWorkerNode(const char *nodeName, int32 nodePort)
@ -1773,6 +1777,11 @@ FindWorkerNode(const char *nodeName, int32 nodePort)
return workerNode;
}
if (strcmp(LocalHostName, nodeName) == 0 && nodePort == PostPortNumber)
{
return PrimaryNodeForGroup(GetLocalGroupId(), NULL);
}
return NULL;
}
@ -2771,12 +2780,24 @@ EnsureCoordinatorIsInMetadata(void)
{
bool isCoordinatorInMetadata = false;
PrimaryNodeForGroup(COORDINATOR_GROUP_ID, &isCoordinatorInMetadata);
if (!isCoordinatorInMetadata)
if (isCoordinatorInMetadata)
{
return;
}
/* be more descriptive when we're not on coordinator */
if (IsCoordinator())
{
ereport(ERROR, (errmsg("coordinator is not added to the metadata"),
errhint("Use SELECT citus_set_coordinator_host('<hostname>') "
"to configure the coordinator hostname")));
}
else
{
ereport(ERROR, (errmsg("coordinator is not added to the metadata"),
errhint("Use SELECT citus_set_coordinator_host('<hostname>') "
"on coordinator to configure the coordinator hostname")));
}
}

View File

@ -1945,11 +1945,7 @@ ConstructQualifiedShardName(ShardInterval *shardInterval)
static List *
RecreateTableDDLCommandList(Oid relationId)
{
const char *relationName = get_rel_name(relationId);
Oid relationSchemaId = get_rel_namespace(relationId);
const char *relationSchemaName = get_namespace_name(relationSchemaId);
const char *qualifiedRelationName = quote_qualified_identifier(relationSchemaName,
relationName);
const char *qualifiedRelationName = generate_qualified_relation_name(relationId);
StringInfo dropCommand = makeStringInfo();

View File

@ -252,7 +252,7 @@ FastPathRouterQuery(Query *query, Node **distributionKeyValue)
/*
* Distribution column must be used in a simple equality match check and it must be
* place at top level conjustion operator. In simple words, we should have
* place at top level conjunction operator. In simple words, we should have
* WHERE dist_key = VALUE [AND ....];
*
* We're also not allowing any other appearances of the distribution key in the quals.

View File

@ -197,9 +197,7 @@ CitusExplainScan(CustomScanState *node, List *ancestors, struct ExplainState *es
if (!ExplainDistributedQueries)
{
appendStringInfoSpaces(es->str, es->indent * 2);
appendStringInfo(es->str, "explain statements for distributed queries ");
appendStringInfo(es->str, "are not enabled\n");
ExplainPropertyBool("citus.explain_distributed_queries", false, es);
return;
}

View File

@ -715,8 +715,8 @@ MultiNodeTree(Query *queryTree)
/*
* ContainsReadIntermediateResultFunction determines whether an expresion tree contains
* a call to the read_intermediate_result function.
* ContainsReadIntermediateResultFunction determines whether an expression tree
* contains a call to the read_intermediate_result function.
*/
bool
ContainsReadIntermediateResultFunction(Node *node)
@ -726,7 +726,7 @@ ContainsReadIntermediateResultFunction(Node *node)
/*
* ContainsReadIntermediateResultArrayFunction determines whether an expresion
* ContainsReadIntermediateResultArrayFunction determines whether an expression
* tree contains a call to the read_intermediate_results(result_ids, format)
* function.
*/

View File

@ -434,7 +434,7 @@ ExtractSelectRangeTableEntry(Query *query)
* for the given modification query.
*
* The function errors out if the input query is not a
* modify query (e.g., INSERT, UPDATE or DELETE). So, this
* modify query (e.g., INSERT, UPDATE, DELETE or MERGE). So, this
* function is not expected to be called on SELECT queries.
*/
Oid
@ -2271,13 +2271,13 @@ SelectsFromDistributedTable(List *rangeTableList, Query *query)
/*
* RouterQuery runs router pruning logic for SELECT, UPDATE, DELETE, and MERGE queries.
* If there are shards present and query is routable, all RTEs have been updated
* to point to the relevant shards in the originalQuery. Also, placementList is
* filled with the list of worker nodes that has all the required shard placements
* for the query execution. anchorShardId is set to the first pruned shardId of
* the given query. Finally, relationShardList is filled with the list of
* relation-to-shard mappings for the query.
* PlanRouterQuery runs router pruning logic for SELECT, UPDATE, DELETE, and
* MERGE queries. If there are shards present and query is routable, all RTEs
* have been updated to point to the relevant shards in the originalQuery. Also,
* placementList is filled with the list of worker nodes that has all the
* required shard placements for the query execution. anchorShardId is set to
* the first pruned shardId of the given query. Finally, relationShardList is
* filled with the list of relation-to-shard mappings for the query.
*
* If the given query is not routable, it fills planningError with the related
* DeferredErrorMessage. The caller can check this error message to see if query
@ -2510,7 +2510,7 @@ AllShardsColocated(List *relationShardList)
if (currentTableType == RANGE_DISTRIBUTED ||
currentTableType == APPEND_DISTRIBUTED)
{
/* we do not have further strict colocation chceks */
/* we do not have further strict colocation checks */
continue;
}
}
@ -2932,7 +2932,7 @@ TargetShardIntervalsForRestrictInfo(RelationRestrictionContext *restrictionConte
}
/*
* Different resrictions might have different partition columns.
* Different restrictions might have different partition columns.
* We report partition column value if there is only one.
*/
if (multiplePartitionValuesExist)

View File

@ -433,7 +433,7 @@ CreateTargetEntryForColumn(Form_pg_attribute attributeTuple, Index rteIndex,
attributeTuple->atttypmod, attributeTuple->attcollation, 0);
TargetEntry *targetEntry =
makeTargetEntry((Expr *) targetColumn, resno,
strdup(attributeTuple->attname.data), false);
pstrdup(attributeTuple->attname.data), false);
return targetEntry;
}

View File

@ -0,0 +1,275 @@
/*-------------------------------------------------------------------------
*
* serialize_distributed_ddls.c
*
* This file contains functions for serializing distributed DDLs.
*
* If you're adding support for serializing a new DDL, you should
* extend the following functions to support the new object class:
* AcquireCitusAdvisoryObjectClassLockGetOid()
* AcquireCitusAdvisoryObjectClassLockCheckPrivileges()
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "miscadmin.h"
#include "catalog/dependency.h"
#include "catalog/pg_database_d.h"
#include "commands/dbcommands.h"
#include "storage/lock.h"
#include "utils/builtins.h"
#include "pg_version_compat.h"
#include "distributed/adaptive_executor.h"
#include "distributed/argutils.h"
#include "distributed/deparse_shard_query.h"
#include "distributed/resource_lock.h"
#include "distributed/serialize_distributed_ddls.h"
PG_FUNCTION_INFO_V1(citus_internal_acquire_citus_advisory_object_class_lock);
static void SerializeDistributedDDLsOnObjectClassInternal(ObjectClass objectClass,
char *qualifiedObjectName);
static char * AcquireCitusAdvisoryObjectClassLockCommand(ObjectClass objectClass,
char *qualifiedObjectName);
static void AcquireCitusAdvisoryObjectClassLock(ObjectClass objectClass,
char *qualifiedObjectName);
static Oid AcquireCitusAdvisoryObjectClassLockGetOid(ObjectClass objectClass,
char *qualifiedObjectName);
static void AcquireCitusAdvisoryObjectClassLockCheckPrivileges(ObjectClass objectClass,
Oid oid);
/*
* citus_internal_acquire_citus_advisory_object_class_lock is an internal UDF
* to call AcquireCitusAdvisoryObjectClassLock().
*/
Datum
citus_internal_acquire_citus_advisory_object_class_lock(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
PG_ENSURE_ARGNOTNULL(0, "object_class");
ObjectClass objectClass = PG_GETARG_INT32(0);
char *qualifiedObjectName = PG_ARGISNULL(1) ? NULL : PG_GETARG_CSTRING(1);
AcquireCitusAdvisoryObjectClassLock(objectClass, qualifiedObjectName);
PG_RETURN_VOID();
}
/*
* SerializeDistributedDDLsOnObjectClass is a wrapper around
* SerializeDistributedDDLsOnObjectClassInternal to acquire the lock on given
* object class itself, see the comment in header file for more details about
* the difference between this function and
* SerializeDistributedDDLsOnObjectClassObject().
*/
void
SerializeDistributedDDLsOnObjectClass(ObjectClass objectClass)
{
SerializeDistributedDDLsOnObjectClassInternal(objectClass, NULL);
}
/*
* SerializeDistributedDDLsOnObjectClassObject is a wrapper around
* SerializeDistributedDDLsOnObjectClassInternal to acquire the lock on given
* object that belongs to given object class, see the comment in header file
* for more details about the difference between this function and
* SerializeDistributedDDLsOnObjectClass().
*/
void
SerializeDistributedDDLsOnObjectClassObject(ObjectClass objectClass,
char *qualifiedObjectName)
{
if (qualifiedObjectName == NULL)
{
elog(ERROR, "qualified object name cannot be NULL");
}
SerializeDistributedDDLsOnObjectClassInternal(objectClass, qualifiedObjectName);
}
/*
* SerializeDistributedDDLsOnObjectClassInternal serializes distributed DDLs
* that target given object class by acquiring a Citus specific advisory lock
* on the first primary worker node if there are any workers in the cluster.
*
* The lock is acquired via a coordinated transaction. For this reason,
* it automatically gets released when (maybe implicit) transaction on
* current server commits or rolls back.
*
* If qualifiedObjectName is provided to be non-null, then the oid of the
* object is first resolved on the first primary worker node and then the
* lock is acquired on that oid. If qualifiedObjectName is null, then the
* lock is acquired on the object class itself.
*
* Note that those two lock types don't conflict with each other and are
* acquired for different purposes. The lock on the object class
* (qualifiedObjectName = NULL) is used to serialize DDLs that target the
* object class itself, e.g., when creating a new object of that class, and
* the latter is used to serialize DDLs that target a specific object of
* that class, e.g., when altering an object.
*
* In some cases, we may want to acquire both locks at the same time. For
* example, when renaming a database, we want to acquire both lock types
* because while the object class lock is used to ensure that another session
* doesn't create a new database with the same name, the object lock is used
* to ensure that another session doesn't alter the same database.
*/
static void
SerializeDistributedDDLsOnObjectClassInternal(ObjectClass objectClass,
char *qualifiedObjectName)
{
WorkerNode *firstWorkerNode = GetFirstPrimaryWorkerNode();
if (firstWorkerNode == NULL)
{
/*
* If there are no worker nodes in the cluster, then we don't need
* to acquire the lock at all; and we cannot indeed.
*/
return;
}
/*
* Indeed we would already ensure permission checks in remote node
* --via AcquireCitusAdvisoryObjectClassLock()-- but we first do so on
* the local node to avoid from reporting confusing error messages.
*/
Oid oid = AcquireCitusAdvisoryObjectClassLockGetOid(objectClass, qualifiedObjectName);
AcquireCitusAdvisoryObjectClassLockCheckPrivileges(objectClass, oid);
Task *task = CitusMakeNode(Task);
task->taskType = DDL_TASK;
char *command = AcquireCitusAdvisoryObjectClassLockCommand(objectClass,
qualifiedObjectName);
SetTaskQueryString(task, command);
ShardPlacement *targetPlacement = CitusMakeNode(ShardPlacement);
SetPlacementNodeMetadata(targetPlacement, firstWorkerNode);
task->taskPlacementList = list_make1(targetPlacement);
/* need to be in a transaction to acquire a lock that's bound to transactions */
UseCoordinatedTransaction();
bool localExecutionSupported = true;
ExecuteUtilityTaskList(list_make1(task), localExecutionSupported);
}
/*
* AcquireCitusAdvisoryObjectClassLockCommand returns a command to call
* pg_catalog.citus_internal_acquire_citus_advisory_object_class_lock().
*/
static char *
AcquireCitusAdvisoryObjectClassLockCommand(ObjectClass objectClass,
char *qualifiedObjectName)
{
/* safe to cast to int as it's an enum */
int objectClassInt = (int) objectClass;
char *quotedObjectName =
!qualifiedObjectName ? "NULL" :
quote_literal_cstr(qualifiedObjectName);
StringInfo command = makeStringInfo();
appendStringInfo(command,
"SELECT pg_catalog.citus_internal_acquire_citus_advisory_object_class_lock(%d, %s)",
objectClassInt, quotedObjectName);
return command->data;
}
/*
* AcquireCitusAdvisoryObjectClassLock acquires a Citus specific advisory
* ExclusiveLock based on given object class.
*/
static void
AcquireCitusAdvisoryObjectClassLock(ObjectClass objectClass, char *qualifiedObjectName)
{
Oid oid = AcquireCitusAdvisoryObjectClassLockGetOid(objectClass, qualifiedObjectName);
AcquireCitusAdvisoryObjectClassLockCheckPrivileges(objectClass, oid);
LOCKTAG locktag;
SET_LOCKTAG_GLOBAL_DDL_SERIALIZATION(locktag, objectClass, oid);
LOCKMODE lockmode = ExclusiveLock;
bool sessionLock = false;
bool dontWait = false;
LockAcquire(&locktag, lockmode, sessionLock, dontWait);
}
/*
* AcquireCitusAdvisoryObjectClassLockGetOid returns the oid of given object
* that belongs to given object class. If qualifiedObjectName is NULL, then
* it returns InvalidOid.
*/
static Oid
AcquireCitusAdvisoryObjectClassLockGetOid(ObjectClass objectClass,
char *qualifiedObjectName)
{
if (qualifiedObjectName == NULL)
{
return InvalidOid;
}
bool missingOk = false;
switch (objectClass)
{
case OCLASS_DATABASE:
{
return get_database_oid(qualifiedObjectName, missingOk);
}
default:
elog(ERROR, "unsupported object class: %d", objectClass);
}
}
/*
* AcquireCitusAdvisoryObjectClassLockCheckPrivileges is used to perform privilege checks
* before acquiring the Citus specific advisory lock on given object class and oid.
*/
static void
AcquireCitusAdvisoryObjectClassLockCheckPrivileges(ObjectClass objectClass, Oid oid)
{
switch (objectClass)
{
case OCLASS_DATABASE:
{
if (OidIsValid(oid) && !object_ownercheck(DatabaseRelationId, oid,
GetUserId()))
{
aclcheck_error(ACLCHECK_NOT_OWNER, OBJECT_DATABASE,
get_database_name(oid));
}
else if (!OidIsValid(oid) && !have_createdb_privilege())
{
ereport(ERROR,
(errcode(ERRCODE_INSUFFICIENT_PRIVILEGE),
errmsg("permission denied to create / rename database")));
}
break;
}
default:
elog(ERROR, "unsupported object class: %d", objectClass);
}
}

View File

@ -94,6 +94,7 @@
#include "distributed/reference_table_utils.h"
#include "distributed/relation_access_tracking.h"
#include "distributed/remote_commands.h"
#include "distributed/remote_transaction.h"
#include "distributed/repartition_executor.h"
#include "distributed/replication_origin_session_utils.h"
#include "distributed/resource_lock.h"
@ -2570,6 +2571,17 @@ RegisterCitusConfigVariables(void)
GUC_NO_SHOW_ALL | GUC_NOT_IN_SAMPLE,
NoticeIfSubqueryPushdownEnabled, NULL, NULL);
DefineCustomStringVariable(
"citus.superuser",
gettext_noop("Name of a superuser role to be used in Citus main database "
"connections"),
NULL,
&SuperuserRole,
"",
PGC_SUSET,
GUC_STANDARD,
NULL, NULL, NULL);
DefineCustomEnumVariable(
"citus.task_assignment_policy",
gettext_noop("Sets the policy to use when assigning tasks to worker nodes."),
@ -3149,6 +3161,8 @@ CitusAuthHook(Port *port, int status)
*/
InitializeBackendData(port->application_name);
IsMainDB = (strncmp(MainDb, "", NAMEDATALEN) == 0 ||
strncmp(MainDb, port->database_name, NAMEDATALEN) == 0);
/* let other authentication hooks to kick in first */
if (original_client_auth_hook)

View File

@ -3,3 +3,12 @@
#include "udfs/citus_internal_database_command/12.2-1.sql"
#include "udfs/citus_add_rebalance_strategy/12.2-1.sql"
#include "udfs/start_management_transaction/12.2-1.sql"
#include "udfs/execute_command_on_remote_nodes_as_user/12.2-1.sql"
#include "udfs/mark_object_distributed/12.2-1.sql"
#include "udfs/commit_management_command_2pc/12.2-1.sql"
ALTER TABLE pg_catalog.pg_dist_transaction ADD COLUMN outer_xid xid8;
#include "udfs/citus_internal_acquire_citus_advisory_object_class_lock/12.2-1.sql"

View File

@ -1,5 +1,23 @@
-- citus--12.2-1--12.1-1
DROP FUNCTION pg_catalog.citus_internal_database_command(text);
DROP FUNCTION pg_catalog.citus_internal_acquire_citus_advisory_object_class_lock(int, cstring);
#include "../udfs/citus_add_rebalance_strategy/10.1-1.sql"
DROP FUNCTION citus_internal.start_management_transaction(
outer_xid xid8
);
DROP FUNCTION citus_internal.execute_command_on_remote_nodes_as_user(
query text,
username text
);
DROP FUNCTION citus_internal.mark_object_distributed(
classId Oid, objectName text, objectId Oid
);
DROP FUNCTION citus_internal.commit_management_command_2pc();
ALTER TABLE pg_catalog.pg_dist_transaction DROP COLUMN outer_xid;

View File

@ -0,0 +1,5 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_acquire_citus_advisory_object_class_lock(objectClass int, qualifiedObjectName cstring)
RETURNS void
LANGUAGE C
VOLATILE
AS 'MODULE_PATHNAME', $$citus_internal_acquire_citus_advisory_object_class_lock$$;

View File

@ -0,0 +1,5 @@
CREATE OR REPLACE FUNCTION pg_catalog.citus_internal_acquire_citus_advisory_object_class_lock(objectClass int, qualifiedObjectName cstring)
RETURNS void
LANGUAGE C
VOLATILE
AS 'MODULE_PATHNAME', $$citus_internal_acquire_citus_advisory_object_class_lock$$;

View File

@ -0,0 +1,7 @@
CREATE OR REPLACE FUNCTION citus_internal.commit_management_command_2pc()
RETURNS VOID
LANGUAGE C
AS 'MODULE_PATHNAME', $$commit_management_command_2pc$$;
COMMENT ON FUNCTION citus_internal.commit_management_command_2pc()
IS 'commits the coordinated remote transactions, is a wrapper function for CoordinatedRemoteTransactionsCommit';

View File

@ -0,0 +1,7 @@
CREATE OR REPLACE FUNCTION citus_internal.commit_management_command_2pc()
RETURNS VOID
LANGUAGE C
AS 'MODULE_PATHNAME', $$commit_management_command_2pc$$;
COMMENT ON FUNCTION citus_internal.commit_management_command_2pc()
IS 'commits the coordinated remote transactions, is a wrapper function for CoordinatedRemoteTransactionsCommit';

View File

@ -0,0 +1,7 @@
CREATE OR REPLACE FUNCTION citus_internal.execute_command_on_remote_nodes_as_user(query text, username text)
RETURNS VOID
LANGUAGE C
AS 'MODULE_PATHNAME', $$execute_command_on_remote_nodes_as_user$$;
COMMENT ON FUNCTION citus_internal.execute_command_on_remote_nodes_as_user(query text, username text)
IS 'executes a query on the nodes other than the current one';

View File

@ -0,0 +1,7 @@
CREATE OR REPLACE FUNCTION citus_internal.execute_command_on_remote_nodes_as_user(query text, username text)
RETURNS VOID
LANGUAGE C
AS 'MODULE_PATHNAME', $$execute_command_on_remote_nodes_as_user$$;
COMMENT ON FUNCTION citus_internal.execute_command_on_remote_nodes_as_user(query text, username text)
IS 'executes a query on the nodes other than the current one';

View File

@ -0,0 +1,7 @@
CREATE OR REPLACE FUNCTION citus_internal.mark_object_distributed(classId Oid, objectName text, objectId Oid)
RETURNS VOID
LANGUAGE C
AS 'MODULE_PATHNAME', $$mark_object_distributed$$;
COMMENT ON FUNCTION citus_internal.mark_object_distributed(classId Oid, objectName text, objectId Oid)
IS 'adds an object to pg_dist_object on all nodes';

View File

@ -0,0 +1,7 @@
CREATE OR REPLACE FUNCTION citus_internal.mark_object_distributed(classId Oid, objectName text, objectId Oid)
RETURNS VOID
LANGUAGE C
AS 'MODULE_PATHNAME', $$mark_object_distributed$$;
COMMENT ON FUNCTION citus_internal.mark_object_distributed(classId Oid, objectName text, objectId Oid)
IS 'adds an object to pg_dist_object on all nodes';

View File

@ -0,0 +1,7 @@
CREATE OR REPLACE FUNCTION citus_internal.start_management_transaction(outer_xid xid8)
RETURNS VOID
LANGUAGE C
AS 'MODULE_PATHNAME', $$start_management_transaction$$;
COMMENT ON FUNCTION citus_internal.start_management_transaction(outer_xid xid8)
IS 'internal Citus function that starts a management transaction in the main database';

View File

@ -0,0 +1,7 @@
CREATE OR REPLACE FUNCTION citus_internal.start_management_transaction(outer_xid xid8)
RETURNS VOID
LANGUAGE C
AS 'MODULE_PATHNAME', $$start_management_transaction$$;
COMMENT ON FUNCTION citus_internal.start_management_transaction(outer_xid xid8)
IS 'internal Citus function that starts a management transaction in the main database';

View File

@ -19,14 +19,19 @@
#include "miscadmin.h"
#include "access/xact.h"
#include "postmaster/postmaster.h"
#include "utils/builtins.h"
#include "utils/hsearch.h"
#include "utils/xid8.h"
#include "distributed/backend_data.h"
#include "distributed/citus_safe_lib.h"
#include "distributed/commands/utility_hook.h"
#include "distributed/connection_management.h"
#include "distributed/listutils.h"
#include "distributed/metadata/distobject.h"
#include "distributed/metadata_cache.h"
#include "distributed/metadata_sync.h"
#include "distributed/placement_connection.h"
#include "distributed/remote_commands.h"
#include "distributed/remote_transaction.h"
@ -56,6 +61,9 @@ static void FinishRemoteTransactionSavepointRollback(MultiConnection *connection
static void Assign2PCIdentifier(MultiConnection *connection);
PG_FUNCTION_INFO_V1(start_management_transaction);
PG_FUNCTION_INFO_V1(execute_command_on_remote_nodes_as_user);
PG_FUNCTION_INFO_V1(commit_management_command_2pc);
static char *IsolationLevelName[] = {
"READ UNCOMMITTED",
@ -64,6 +72,154 @@ static char *IsolationLevelName[] = {
"SERIALIZABLE"
};
/*
* These variables are necessary for running queries from a database that is not
* the Citus main database. Some of these queries need to be propagated to the
* workers and Citus main database will be used for these queries, such as
* CREATE ROLE. For that we create a connection to the Citus main database and
* run queries from there.
*/
/* The MultiConnection used for connecting Citus main database. */
MultiConnection *MainDBConnection = NULL;
/*
* IsMainDBCommand is true if this is a query in the Citus main database that is started
* by a query from a different database.
*/
bool IsMainDBCommand = false;
/*
* The transaction id of the query from the other database that started the
* main database query.
*/
FullTransactionId OuterXid;
/*
* Shows if this is the Citus main database or not. We needed a variable instead of
* checking if this database's name is the same as MainDb because we sometimes need
* this value outside a transaction where we cannot reach the current database name.
*/
bool IsMainDB = true;
/*
* Name of a superuser role to be used during main database connections.
*/
char *SuperuserRole = NULL;
/*
* start_management_transaction starts a management transaction
* in the main database by recording the outer transaction's transaction id and setting
* IsMainDBCommand to true.
*/
Datum
start_management_transaction(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
EnsureSuperUser();
OuterXid = PG_GETARG_FULLTRANSACTIONID(0);
IsMainDBCommand = true;
Use2PCForCoordinatedTransaction();
PG_RETURN_VOID();
}
/*
* execute_command_on_remote_nodes_as_user executes the query on the nodes
* other than the current node, using the user passed.
*/
Datum
execute_command_on_remote_nodes_as_user(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
EnsureSuperUser();
text *queryText = PG_GETARG_TEXT_P(0);
char *query = text_to_cstring(queryText);
text *usernameText = PG_GETARG_TEXT_P(1);
char *username = text_to_cstring(usernameText);
StringInfo queryToSend = makeStringInfo();
appendStringInfo(queryToSend, "%s;%s;%s", DISABLE_METADATA_SYNC, query,
ENABLE_METADATA_SYNC);
SendCommandToWorkersAsUser(REMOTE_NODES, username, queryToSend->data);
PG_RETURN_VOID();
}
/*
* commit_management_command_2pc is a wrapper UDF for
* CoordinatedRemoteTransactionsCommit
*/
Datum
commit_management_command_2pc(PG_FUNCTION_ARGS)
{
CheckCitusVersion(ERROR);
EnsureSuperUser();
RecoverTwoPhaseCommits();
PG_RETURN_VOID();
}
/*
* RunCitusMainDBQuery creates a connection to Citus main database if necessary
* and runs the query over the connection in the main database.
*/
void
RunCitusMainDBQuery(char *query)
{
if (MainDBConnection == NULL)
{
if (strlen(SuperuserRole) == 0)
{
ereport(ERROR, (errmsg("No superuser role is given for Citus main "
"database connection"),
errhint("Set citus.superuser to a superuser role name")));
}
int flags = 0;
MainDBConnection = GetNodeUserDatabaseConnection(flags, LocalHostName,
PostPortNumber,
SuperuserRole,
MainDb);
RemoteTransactionBegin(MainDBConnection);
}
SendRemoteCommand(MainDBConnection, query);
PGresult *result = GetRemoteCommandResult(MainDBConnection, true);
if (!IsResponseOK(result))
{
ReportResultError(MainDBConnection, result, ERROR);
}
ForgetResults(MainDBConnection);
}
/*
* CleanCitusMainDBConnection closes and removes the connection to Citus main database.
*/
void
CleanCitusMainDBConnection(void)
{
if (MainDBConnection == NULL)
{
return;
}
CloseConnection(MainDBConnection);
MainDBConnection = NULL;
}
/*
* StartRemoteTransactionBegin initiates beginning the remote transaction in
@ -616,7 +772,7 @@ StartRemoteTransactionPrepare(struct MultiConnection *connection)
WorkerNode *workerNode = FindWorkerNode(connection->hostname, connection->port);
if (workerNode != NULL)
{
LogTransactionRecord(workerNode->groupId, transaction->preparedName);
LogTransactionRecord(workerNode->groupId, transaction->preparedName, OuterXid);
}
/*

View File

@ -21,6 +21,7 @@
#include "catalog/dependency.h"
#include "common/hashfn.h"
#include "nodes/print.h"
#include "postmaster/postmaster.h"
#include "storage/fd.h"
#include "utils/datum.h"
#include "utils/guc.h"
@ -46,6 +47,7 @@
#include "distributed/multi_logical_replication.h"
#include "distributed/placement_connection.h"
#include "distributed/relation_access_tracking.h"
#include "distributed/remote_commands.h"
#include "distributed/repartition_join_execution.h"
#include "distributed/replication_origin_session_utils.h"
#include "distributed/shard_cleaner.h"
@ -55,6 +57,9 @@
#include "distributed/version_compat.h"
#include "distributed/worker_log_messages.h"
#define COMMIT_MANAGEMENT_COMMAND_2PC \
"SELECT citus_internal.commit_management_command_2pc()"
CoordinatedTransactionState CurrentCoordinatedTransactionState = COORD_TRANS_NONE;
@ -317,12 +322,23 @@ CoordinatedTransactionCallback(XactEvent event, void *arg)
MemoryContext previousContext =
MemoryContextSwitchTo(CitusXactCallbackContext);
if (CurrentCoordinatedTransactionState == COORD_TRANS_PREPARED)
if (CurrentCoordinatedTransactionState == COORD_TRANS_PREPARED &&
!IsMainDBCommand)
{
/* handles both already prepared and open transactions */
CoordinatedRemoteTransactionsCommit();
}
/*
* If this is a non-Citus main database we should try to commit the prepared
* transactions created by the Citus main database on the worker nodes.
*/
if (!IsMainDB && MainDBConnection != NULL)
{
RunCitusMainDBQuery(COMMIT_MANAGEMENT_COMMAND_2PC);
CleanCitusMainDBConnection();
}
/* close connections etc. */
if (CurrentCoordinatedTransactionState != COORD_TRANS_NONE)
{
@ -378,6 +394,8 @@ CoordinatedTransactionCallback(XactEvent event, void *arg)
RemoveIntermediateResultsDirectories();
CleanCitusMainDBConnection();
/* handles both already prepared and open transactions */
if (CurrentCoordinatedTransactionState > COORD_TRANS_IDLE)
{
@ -509,6 +527,17 @@ CoordinatedTransactionCallback(XactEvent event, void *arg)
break;
}
/*
* If this is a non-Citus main database we should commit the Citus
* main database query. So if some error happens on the distributed main
* database query we wouldn't have committed the current query.
*/
if (!IsMainDB && MainDBConnection != NULL)
{
RunCitusMainDBQuery("COMMIT");
}
/*
* TODO: It'd probably be a good idea to force constraints and
* such to 'immediate' here. Deferred triggers might try to send
@ -537,7 +566,10 @@ CoordinatedTransactionCallback(XactEvent event, void *arg)
* us to mark failed placements as invalid. Better don't use
* this for anything important (i.e. DDL/metadata).
*/
CoordinatedRemoteTransactionsCommit();
if (IsMainDB)
{
CoordinatedRemoteTransactionsCommit();
}
CurrentCoordinatedTransactionState = COORD_TRANS_COMMITTED;
}
@ -1139,18 +1171,17 @@ ResetPropagatedObjects(void)
/*
* HasAnyDependencyInPropagatedObjects decides if any dependency of given object is
* HasAnyObjectInPropagatedObjects decides if any of the objects in given list are
* propagated in the current transaction.
*/
bool
HasAnyDependencyInPropagatedObjects(const ObjectAddress *objectAddress)
HasAnyObjectInPropagatedObjects(List *objectList)
{
List *dependencyList = GetAllSupportedDependenciesForObject(objectAddress);
ObjectAddress *dependency = NULL;
foreach_ptr(dependency, dependencyList)
ObjectAddress *object = NULL;
foreach_ptr(object, objectList)
{
/* first search in root transaction */
if (DependencyInPropagatedObjectsHash(PropagatedObjectsInTx, dependency))
if (DependencyInPropagatedObjectsHash(PropagatedObjectsInTx, object))
{
return true;
}
@ -1163,7 +1194,7 @@ HasAnyDependencyInPropagatedObjects(const ObjectAddress *objectAddress)
SubXactContext *state = NULL;
foreach_ptr(state, activeSubXactContexts)
{
if (DependencyInPropagatedObjectsHash(state->propagatedObjects, dependency))
if (DependencyInPropagatedObjectsHash(state->propagatedObjects, object))
{
return true;
}

View File

@ -29,10 +29,12 @@
#include "lib/stringinfo.h"
#include "storage/lmgr.h"
#include "storage/lock.h"
#include "storage/procarray.h"
#include "utils/builtins.h"
#include "utils/fmgroids.h"
#include "utils/memutils.h"
#include "utils/rel.h"
#include "utils/xid8.h"
#include "pg_version_constants.h"
@ -82,7 +84,7 @@ recover_prepared_transactions(PG_FUNCTION_ARGS)
* prepared transaction should be committed.
*/
void
LogTransactionRecord(int32 groupId, char *transactionName)
LogTransactionRecord(int32 groupId, char *transactionName, FullTransactionId outerXid)
{
Datum values[Natts_pg_dist_transaction];
bool isNulls[Natts_pg_dist_transaction];
@ -93,6 +95,7 @@ LogTransactionRecord(int32 groupId, char *transactionName)
values[Anum_pg_dist_transaction_groupid - 1] = Int32GetDatum(groupId);
values[Anum_pg_dist_transaction_gid - 1] = CStringGetTextDatum(transactionName);
values[Anum_pg_dist_transaction_outerxid - 1] = FullTransactionIdGetDatum(outerXid);
/* open transaction relation and insert new tuple */
Relation pgDistTransaction = table_open(DistTransactionRelationId(),
@ -258,6 +261,54 @@ RecoverWorkerTransactions(WorkerNode *workerNode)
continue;
}
/* Check if the transaction is created by an outer transaction from a non-main database */
bool outerXidIsNull = false;
Datum outerXidDatum = heap_getattr(heapTuple,
Anum_pg_dist_transaction_outerxid,
tupleDescriptor, &outerXidIsNull);
TransactionId outerXid = 0;
if (!outerXidIsNull)
{
FullTransactionId outerFullXid = DatumGetFullTransactionId(outerXidDatum);
outerXid = XidFromFullTransactionId(outerFullXid);
}
if (outerXid != 0)
{
bool outerXactIsInProgress = TransactionIdIsInProgress(outerXid);
bool outerXactDidCommit = TransactionIdDidCommit(outerXid);
if (outerXactIsInProgress && !outerXactDidCommit)
{
/*
* The transaction is initiated from an outer transaction and the outer
* transaction is not yet committed, so we should not commit either.
* We remove this transaction from the pendingTransactionSet so it'll
* not be aborted by the loop below.
*/
hash_search(pendingTransactionSet, transactionName, HASH_REMOVE,
&foundPreparedTransactionBeforeCommit);
continue;
}
else if (!outerXactIsInProgress && !outerXactDidCommit)
{
/*
* Since outer transaction isn't in progress and did not commit we need to
* abort the prepared transaction too. We do this by simply doing the same
* thing we would do for transactions that are initiated from the main
* database.
*/
continue;
}
else
{
/*
* Outer transaction did commit, so we can try to commit the prepared
* transaction too.
*/
}
}
/*
* Remove the transaction from the pending list such that only transactions
* that need to be aborted remain at the end.

View File

@ -234,7 +234,8 @@ List *
TargetWorkerSetNodeList(TargetWorkerSet targetWorkerSet, LOCKMODE lockMode)
{
List *workerNodeList = NIL;
if (targetWorkerSet == ALL_SHARD_NODES || targetWorkerSet == METADATA_NODES)
if (targetWorkerSet == ALL_SHARD_NODES ||
targetWorkerSet == METADATA_NODES)
{
workerNodeList = ActivePrimaryNodeList(lockMode);
}

View File

@ -142,7 +142,17 @@ SetRangeTblExtraData(RangeTblEntry *rte, CitusRTEKind rteKind, char *fragmentSch
fauxFunction->funcexpr = (Node *) fauxFuncExpr;
/* set the column count to pass ruleutils checks, not used elsewhere */
fauxFunction->funccolcount = list_length(rte->eref->colnames);
if (rte->relid != 0)
{
Relation rel = RelationIdGetRelation(rte->relid);
fauxFunction->funccolcount = RelationGetNumberOfAttributes(rel);
RelationClose(rel);
}
else
{
fauxFunction->funccolcount = list_length(rte->eref->colnames);
}
fauxFunction->funccolnames = funcColumnNames;
fauxFunction->funccoltypes = funcColumnTypes;
fauxFunction->funccoltypmods = funcColumnTypeMods;

View File

@ -362,10 +362,8 @@ ErrorIfShardPlacementsNotColocated(Oid leftRelationId, Oid rightRelationId)
leftRelationName, rightRelationName)));
}
List *leftPlacementList = ShardPlacementListSortedByWorker(
leftShardId);
List *rightPlacementList = ShardPlacementListSortedByWorker(
rightShardId);
List *leftPlacementList = ShardPlacementList(leftShardId);
List *rightPlacementList = ShardPlacementList(rightShardId);
if (list_length(leftPlacementList) != list_length(rightPlacementList))
{

View File

@ -470,12 +470,11 @@ SingleReplicatedTable(Oid relationId)
return false;
}
List *shardIntervalList = LoadShardList(relationId);
uint64 *shardIdPointer = NULL;
foreach_ptr(shardIdPointer, shardIntervalList)
foreach_ptr(shardIdPointer, shardList)
{
uint64 shardId = *shardIdPointer;
shardPlacementList = ShardPlacementListSortedByWorker(shardId);
shardPlacementList = ShardPlacementList(shardId);
if (list_length(shardPlacementList) != 1)
{

View File

@ -170,14 +170,10 @@ WorkerDropDistributedTable(Oid relationId)
*/
if (!IsAnyObjectAddressOwnedByExtension(list_make1(distributedTableObject), NULL))
{
char *relName = get_rel_name(relationId);
Oid schemaId = get_rel_namespace(relationId);
char *schemaName = get_namespace_name(schemaId);
StringInfo dropCommand = makeStringInfo();
appendStringInfo(dropCommand, "DROP%sTABLE %s CASCADE",
IsForeignTable(relationId) ? " FOREIGN " : " ",
quote_qualified_identifier(schemaName, relName));
generate_qualified_relation_name(relationId));
Node *dropCommandNode = ParseTreeNode(dropCommand->data);

View File

@ -92,38 +92,21 @@ CitusNodeTagI(Node *node)
return ((CitusNode*)(node))->citus_tag;
}
/*
* Postgres's nodes/nodes.h has more information on why we do this.
*/
#ifdef __GNUC__
/* Citus variant of newNode(), don't use directly. */
#define CitusNewNode(size, tag) \
({ CitusNode *_result; \
AssertMacro((size) >= sizeof(CitusNode)); /* need the tag, at least */ \
_result = (CitusNode *) palloc0fast(size); \
_result->extensible.type = T_ExtensibleNode; \
_result->extensible.extnodename = CitusNodeTagNames[tag - CITUS_NODE_TAG_START]; \
_result->citus_tag =(int) (tag); \
_result; \
})
static inline CitusNode *
CitusNewNode(size_t size, CitusNodeTag tag)
{
CitusNode *result;
#else
extern CitusNode *newCitusNodeMacroHolder;
#define CitusNewNode(size, tag) \
( \
AssertMacro((size) >= sizeof(CitusNode)), /* need the tag, at least */ \
newCitusNodeMacroHolder = (CitusNode *) palloc0fast(size), \
newCitusNodeMacroHolder->extensible.type = T_ExtensibleNode, \
newCitusNodeMacroHolder->extensible.extnodename = CitusNodeTagNames[tag - CITUS_NODE_TAG_START], \
newCitusNodeMacroHolder->citus_tag =(int) (tag), \
newCitusNodeMacroHolder \
)
#endif
Assert(size >= sizeof(CitusNode)); /* need the ExtensibleNode and the tag, at least */
result = (CitusNode *) palloc0(size);
result->extensible.type = T_ExtensibleNode;
result->extensible.extnodename = CitusNodeTagNames[tag - CITUS_NODE_TAG_START];
result->citus_tag = (int) (tag);
return result;
}
/*
* IsA equivalent that compares node tags, including Citus-specific nodes.

View File

@ -230,6 +230,7 @@ extern List * PreprocessAlterDatabaseStmt(Node *node, const char *queryString,
extern List * PreprocessAlterDatabaseRefreshCollStmt(Node *node, const char *queryString,
ProcessUtilityContext
processUtilityContext);
extern List * GetDatabaseMetadataSyncCommands(Oid dbOid);
extern List * PreprocessAlterDatabaseSetStmt(Node *node, const char *queryString,
@ -244,6 +245,11 @@ extern List * DropDatabaseStmtObjectAddress(Node *node, bool missingOk,
bool isPostprocess);
extern List * CreateDatabaseStmtObjectAddress(Node *node, bool missingOk,
bool isPostprocess);
extern List * GenerateGrantDatabaseCommandList(void);
extern List * PreprocessAlterDatabaseRenameStmt(Node *node, const char *queryString,
ProcessUtilityContext
processUtilityContext);
extern List * PostprocessAlterDatabaseRenameStmt(Node *node, const char *queryString);
extern void EnsureSupportedCreateDatabaseCommand(CreatedbStmt *stmt);
extern char * CreateDatabaseDDLCommand(Oid dbId);
@ -440,6 +446,7 @@ extern List * CreateExtensionStmtObjectAddress(Node *stmt, bool missing_ok, bool
/* owned.c - forward declarations */
extern List * PreprocessDropOwnedStmt(Node *node, const char *queryString,
ProcessUtilityContext processUtilityContext);
extern List * PostprocessReassignOwnedStmt(Node *node, const char *queryString);
/* policy.c - forward declarations */
extern List * CreatePolicyCommands(Oid relationId);
@ -687,11 +694,6 @@ extern List * AlterTextSearchConfigurationSchemaStmtObjectAddress(Node *node,
extern List * AlterTextSearchDictionarySchemaStmtObjectAddress(Node *node,
bool missing_ok, bool
isPostprocess);
extern List * TextSearchConfigurationCommentObjectAddress(Node *node,
bool missing_ok, bool
isPostprocess);
extern List * TextSearchDictCommentObjectAddress(Node *node,
bool missing_ok, bool isPostprocess);
extern List * AlterTextSearchConfigurationOwnerObjectAddress(Node *node,
bool missing_ok, bool
isPostprocess);

View File

@ -0,0 +1,26 @@
/*-------------------------------------------------------------------------
*
* comment.h
* Declarations for comment related operations.
*
* Copyright (c) Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#ifndef COMMENT_H
#define COMMENT_H
#include "postgres.h"
#include "nodes/parsenodes.h"
extern const char *ObjectTypeNames[];
extern List * GetCommentPropagationCommands(Oid classOid, Oid oid, char *objectName,
ObjectType objectType);
extern List * CommentObjectAddress(Node *node, bool missing_ok, bool isPostprocess);
# endif /* COMMENT_H */

View File

@ -143,6 +143,9 @@ extern void DefElemOptionToStatement(StringInfo buf, DefElem *option,
const DefElemOptionFormat *opt_formats,
int opt_formats_len);
/* forward declarations for deparse_comment_stmts.c */
extern char * DeparseCommentStmt(Node *node);
/* forward declarations for deparse_statistics_stmts.c */
extern char * DeparseCreateStatisticsStmt(Node *node);
@ -231,6 +234,7 @@ extern void QualifyAlterRoleSetStmt(Node *stmt);
extern char * DeparseCreateRoleStmt(Node *stmt);
extern char * DeparseDropRoleStmt(Node *stmt);
extern char * DeparseGrantRoleStmt(Node *stmt);
extern char * DeparseReassignOwnedStmt(Node *node);
/* forward declarations for deparse_owned_stmts.c */
extern char * DeparseDropOwnedStmt(Node *node);
@ -251,6 +255,7 @@ extern char * DeparseAlterDatabaseRefreshCollStmt(Node *node);
extern char * DeparseAlterDatabaseSetStmt(Node *node);
extern char * DeparseCreateDatabaseStmt(Node *node);
extern char * DeparseDropDatabaseStmt(Node *node);
extern char * DeparseAlterDatabaseRenameStmt(Node *node);
/* forward declaration for deparse_publication_stmts.c */

View File

@ -104,7 +104,7 @@ typedef struct FastPathRestrictionContext
* Set to true when distKey = Param; in the queryTree
*/
bool distributionKeyHasParam;
}FastPathRestrictionContext;
} FastPathRestrictionContext;
typedef struct PlannerRestrictionContext
{

View File

@ -23,6 +23,8 @@ extern bool CitusExtensionObject(const ObjectAddress *objectAddress);
extern bool IsAnyObjectDistributed(const List *addresses);
extern bool ClusterHasDistributedFunctionWithDistArgument(void);
extern void MarkObjectDistributed(const ObjectAddress *distAddress);
extern void MarkObjectDistributedWithName(const ObjectAddress *distAddress, char *name,
bool useConnectionForLocalQuery);
extern void MarkObjectDistributedViaSuperUser(const ObjectAddress *distAddress);
extern void MarkObjectDistributedLocally(const ObjectAddress *distAddress);
extern void UnmarkObjectDistributed(const ObjectAddress *address);

View File

@ -89,6 +89,7 @@ extern List * NodeMetadataCreateCommands(void);
extern List * CitusTableMetadataCreateCommandList(Oid relationId);
extern List * NodeMetadataDropCommands(void);
extern char * MarkObjectsDistributedCreateCommand(List *addresses,
List *names,
List *distributionArgumentIndexes,
List *colocationIds,
List *forceDelegations);

View File

@ -386,6 +386,7 @@ extern void EnsureUndistributeTenantTableSafe(Oid relationId, const char *operat
extern TableConversionReturn * UndistributeTable(TableConversionParameters *params);
extern void UndistributeTables(List *relationIdList);
extern void EnsureObjectAndDependenciesExistOnAllNodes(const ObjectAddress *target);
extern void EnsureAllObjectDependenciesExistOnAllNodes(const List *targets);
extern DeferredErrorMessage * DeferErrorIfCircularDependencyExists(const
ObjectAddress *

View File

@ -238,8 +238,8 @@ typedef struct Task
TaskQuery taskQuery;
/*
* A task can have multiple queries, in which case queryCount will be > 1. If
* a task has more one query, then taskQuery->queryType == TASK_QUERY_TEXT_LIST.
* A task can have multiple queries, in which case queryCount will be > 1, and
* taskQuery->queryType == TASK_QUERY_TEXT_LIST.
*/
int queryCount;
@ -290,7 +290,7 @@ typedef struct Task
/*
* When we evaluate functions and parameters in the query string then
* we should no longer send the list of parameters long with the
* we should no longer send the list of parameters along with the
* query.
*/
bool parametersInQueryStringResolved;

View File

@ -35,9 +35,10 @@ typedef FormData_pg_dist_transaction *Form_pg_dist_transaction;
* compiler constants for pg_dist_transaction
* ----------------
*/
#define Natts_pg_dist_transaction 2
#define Natts_pg_dist_transaction 3
#define Anum_pg_dist_transaction_groupid 1
#define Anum_pg_dist_transaction_gid 2
#define Anum_pg_dist_transaction_outerxid 3
#endif /* PG_DIST_TRANSACTION_H */

View File

@ -144,4 +144,13 @@ extern void CoordinatedRemoteTransactionsSavepointBegin(SubTransactionId subId);
extern void CoordinatedRemoteTransactionsSavepointRelease(SubTransactionId subId);
extern void CoordinatedRemoteTransactionsSavepointRollback(SubTransactionId subId);
extern void RunCitusMainDBQuery(char *query);
extern void CleanCitusMainDBConnection(void);
extern bool IsMainDBCommand;
extern bool IsMainDB;
extern char *SuperuserRole;
extern char *MainDb;
extern struct MultiConnection *MainDBConnection;
#endif /* REMOTE_TRANSACTION_H */

View File

@ -14,6 +14,7 @@
#include "c.h"
#include "catalog/dependency.h"
#include "nodes/pg_list.h"
#include "storage/lock.h"
#include "tcop/utility.h"
@ -45,7 +46,8 @@ typedef enum AdvisoryLocktagClass
ADV_LOCKTAG_CLASS_CITUS_CLEANUP_OPERATION_ID = 10,
ADV_LOCKTAG_CLASS_CITUS_LOGICAL_REPLICATION = 12,
ADV_LOCKTAG_CLASS_CITUS_REBALANCE_PLACEMENT_COLOCATION = 13,
ADV_LOCKTAG_CLASS_CITUS_BACKGROUND_TASK = 14
ADV_LOCKTAG_CLASS_CITUS_BACKGROUND_TASK = 14,
ADV_LOCKTAG_CLASS_CITUS_GLOBAL_DDL_SERIALIZATION = 15
} AdvisoryLocktagClass;
/* CitusOperations has constants for citus operations */
@ -142,6 +144,72 @@ typedef enum CitusOperations
(uint32) (taskId), \
ADV_LOCKTAG_CLASS_CITUS_BACKGROUND_TASK)
/*
* IsNodeWideObjectClass returns true if the given object class is node-wide,
* i.e., that is not bound to a particular database but to whole server.
*
* Defined here as an inlined function so that SET_LOCKTAG_GLOBAL_DDL_SERIALIZATION
* macro can use it.
*/
static inline bool
IsNodeWideObjectClass(ObjectClass objectClass)
{
if ((int) objectClass < 0 || objectClass > LAST_OCLASS)
{
elog(ERROR, "invalid object class: %d", objectClass);
}
/*
* We don't expect Postgres to change an object class to a node-wide one in the
* future, but a newly added object class may be node-wide.
*
* So we put a static assert here to make sure that the developer who adds support
* for a new Postgres version is aware of this.
*
* If new object classes are added and none of them are node-wide, then update
* this assertion check based on latest supported major Postgres version.
*/
StaticAssertStmt(PG_MAJORVERSION_NUM <= 16,
"better to check if any of newly added ObjectClass'es are node-wide");
switch (objectClass)
{
case OCLASS_ROLE:
case OCLASS_DATABASE:
case OCLASS_TBLSPACE:
#if PG_VERSION_NUM >= PG_VERSION_15
case OCLASS_PARAMETER_ACL:
#endif
#if PG_VERSION_NUM >= PG_VERSION_16
case OCLASS_ROLE_MEMBERSHIP:
#endif
{
return true;
}
default:
return false;
}
}
/*
* Automatically sets databaseId to InvalidOid if the object class is
* node-wide, i.e., that is not bound to a particular database but to
* whole server. If the object class is not node-wide, sets databaseId
* to MyDatabaseId.
*
* That way, the lock is local to each database if the object class is
* not node-wide, and global if it is.
*/
#define SET_LOCKTAG_GLOBAL_DDL_SERIALIZATION(tag, objectClass, oid) \
SET_LOCKTAG_ADVISORY(tag, \
(uint32) (IsNodeWideObjectClass(objectClass) ? InvalidOid : \
MyDatabaseId), \
(uint32) objectClass, \
(uint32) oid, \
ADV_LOCKTAG_CLASS_CITUS_GLOBAL_DDL_SERIALIZATION)
/*
* DistLockConfigs are used to configure the locking behaviour of AcquireDistributedLockOnRelations
*/

View File

@ -0,0 +1,37 @@
/*-------------------------------------------------------------------------
*
* serialize_distributed_ddls.h
*
* Declarations for public functions related to serializing distributed
* DDLs.
*
*-------------------------------------------------------------------------
*/
#ifndef SERIALIZE_DDLS_OVER_CATALOG_H
#define SERIALIZE_DDLS_OVER_CATALOG_H
#include "postgres.h"
#include "catalog/dependency.h"
/*
* Note that those two lock types don't conflict with each other and are
* acquired for different purposes. The lock on the object class
* --SerializeDistributedDDLsOnObjectClass()-- is used to serialize DDLs
* that target the object class itself, e.g., when creating a new object
* of that class, and the latter one --SerializeDistributedDDLsOnObjectClassObject()--
* is used to serialize DDLs that target a specific object of that class,
* e.g., when altering an object.
*
* In some cases, we may want to acquire both locks at the same time. For
* example, when renaming a database, we want to acquire both lock types
* because while the object class lock is used to ensure that another session
* doesn't create a new database with the same name, the object lock is used
* to ensure that another session doesn't alter the same database.
*/
extern void SerializeDistributedDDLsOnObjectClass(ObjectClass objectClass);
extern void SerializeDistributedDDLsOnObjectClassObject(ObjectClass objectClass,
char *qualifiedObjectName);
#endif /* SERIALIZE_DDLS_OVER_CATALOG_H */

View File

@ -163,7 +163,7 @@ extern bool MaybeExecutingUDF(void);
extern void TrackPropagatedObject(const ObjectAddress *objectAddress);
extern void TrackPropagatedTableAndSequences(Oid relationId);
extern void ResetPropagatedObjects(void);
extern bool HasAnyDependencyInPropagatedObjects(const ObjectAddress *objectAddress);
extern bool HasAnyObjectInPropagatedObjects(List *objectList);
/* initialization function(s) */
extern void InitializeTransactionManagement(void);

View File

@ -17,7 +17,8 @@ extern int Recover2PCInterval;
/* Functions declarations for worker transactions */
extern void LogTransactionRecord(int32 groupId, char *transactionName);
extern void LogTransactionRecord(int32 groupId, char *transactionName,
FullTransactionId outerXid);
extern int RecoverTwoPhaseCommits(void);
extern void DeleteWorkerTransactions(WorkerNode *workerNode);

View File

@ -48,14 +48,21 @@ get_guc_variables_compat(int *gucCount)
#define pgstat_fetch_stat_local_beentry(a) pgstat_get_local_beentry_by_index(a)
#define have_createdb_privilege() have_createdb_privilege()
#else
#include "miscadmin.h"
#include "catalog/pg_authid.h"
#include "catalog/pg_class_d.h"
#include "catalog/pg_database_d.h"
#include "catalog/pg_namespace.h"
#include "catalog/pg_proc_d.h"
#include "storage/relfilenode.h"
#include "utils/guc.h"
#include "utils/guc_tables.h"
#include "utils/syscache.h"
#define pg_clean_ascii_compat(a, b) pg_clean_ascii(a)
@ -105,6 +112,11 @@ object_ownercheck(Oid classid, Oid objectid, Oid roleid)
return pg_proc_ownercheck(objectid, roleid);
}
case DatabaseRelationId:
{
return pg_database_ownercheck(objectid, roleid);
}
default:
{
ereport(ERROR,
@ -140,6 +152,28 @@ object_aclcheck(Oid classid, Oid objectid, Oid roleid, AclMode mode)
}
static inline bool
have_createdb_privilege(void)
{
bool result = false;
HeapTuple utup;
/* Superusers can always do everything */
if (superuser())
{
return true;
}
utup = SearchSysCache1(AUTHOID, ObjectIdGetDatum(GetUserId()));
if (HeapTupleIsValid(utup))
{
result = ((Form_pg_authid) GETSTRUCT(utup))->rolcreatedb;
ReleaseSysCache(utup);
}
return result;
}
typedef bool TU_UpdateIndexes;
/*

View File

@ -1,5 +1,5 @@
# this schedule is to be run on only coordinators
test: upgrade_basic_before
test: upgrade_basic_before upgrade_basic_before_non_mixed
test: upgrade_pg_dist_cleanup_before
test: upgrade_post_11_before

View File

@ -1,5 +1,5 @@
# The basic tests runs analyze which depends on shard numbers
test: multi_test_helpers multi_test_helpers_superuser
test: multi_test_helpers multi_test_helpers_superuser upgrade_basic_before_non_mixed
test: multi_test_catalog_views
test: upgrade_basic_before
test: upgrade_ref2ref_before

View File

@ -215,7 +215,7 @@ s/^(ERROR: The index name \(test_index_creation1_p2020_09_26)_([0-9])+_(tenant_
s/^(DEBUG: the index name on the shards of the partition is too long, switching to sequential and local execution mode to prevent self deadlocks: test_index_creation1_p2020_09_26)_([0-9])+_(tenant_id_timeperiod_idx)/\1_xxxxxx_\3/g
# normalize errors for not being able to connect to a non-existing host
s/could not translate host name "foobar" to address: .*$/could not translate host name "foobar" to address: <system specific error>/g
s/could not translate host name "([A-Za-z0-9\.\-]+)" to address: .*$/could not translate host name "\1" to address: <system specific error>/g
# ignore PL/pgSQL line numbers that differ on Mac builds
s/(CONTEXT: PL\/pgSQL function .* line )([0-9]+)/\1XX/g

View File

@ -92,7 +92,7 @@ PG_MAJOR_VERSION = get_pg_major_version()
OLDEST_SUPPORTED_CITUS_VERSION_MATRIX = {
14: "10.2.0",
15: "11.1.5",
16: "12.1devel",
16: "12.1.1",
}
OLDEST_SUPPORTED_CITUS_VERSION = OLDEST_SUPPORTED_CITUS_VERSION_MATRIX[PG_MAJOR_VERSION]
@ -431,6 +431,12 @@ next_port = PORT_LOWER_BOUND
def notice_handler(diag: psycopg.errors.Diagnostic):
print(f"{diag.severity}: {diag.message_primary}")
if diag.message_detail:
print(f"DETAIL: {diag.message_detail}")
if diag.message_hint:
print(f"HINT: {diag.message_hint}")
if diag.context:
print(f"CONTEXT: {diag.context}")
def cleanup_test_leftovers(nodes):

View File

@ -153,6 +153,9 @@ DEPS = {
"isolation_extension_commands": TestDeps(
None, ["isolation_setup", "isolation_add_remove_node"]
),
"isolation_update_node": TestDeps(
None, ["isolation_setup", "isolation_add_remove_node"]
),
"schema_based_sharding": TestDeps("minimal_schedule"),
"multi_sequence_default": TestDeps(
None, ["multi_test_helpers", "multi_cluster_management", "multi_table_ddl"]
@ -199,6 +202,16 @@ DEPS = {
repeatable=False,
),
"multi_prepare_plsql": TestDeps("base_schedule"),
"pg15": TestDeps("base_schedule"),
"foreign_key_to_reference_shard_rebalance": TestDeps(
"minimal_schedule", ["remove_coordinator_from_metadata"]
),
"limit_intermediate_size": TestDeps("base_schedule"),
"columnar_drop": TestDeps(
"minimal_schedule",
["columnar_create", "columnar_load"],
repeatable=False,
),
}

View File

@ -82,6 +82,7 @@ the name of the fixture. For example:
```python
def test_some_query(cluster):
cluster.coordinator.sql("SELECT 1")
assert cluster.workers[0].sql_value('SELECT 2') == 2
```
If you need a cluster of a specific size you can use the `cluster_factory`

View File

@ -0,0 +1,198 @@
def test_main_commited_outer_not_yet(cluster):
c = cluster.coordinator
w0 = cluster.workers[0]
# create a non-main database
c.sql("CREATE DATABASE db1")
# we will use cur1 to simulate non-main database user and
# cur2 to manually do the steps we would do in the main database
with c.cur(dbname="db1") as cur1, c.cur() as cur2:
# let's start a transaction and find its transaction id
cur1.execute("BEGIN")
cur1.execute("SELECT txid_current()")
txid = cur1.fetchall()
# using the transaction id of the cur1 simulate the main database commands manually
cur2.execute("BEGIN")
cur2.execute(
"SELECT citus_internal.start_management_transaction(%s)", (str(txid[0][0]),)
)
cur2.execute(
"SELECT citus_internal.execute_command_on_remote_nodes_as_user('CREATE USER u1;', 'postgres')"
)
cur2.execute(
"SELECT citus_internal.mark_object_distributed(1260, 'u1', 123123)"
)
cur2.execute("COMMIT")
# run the transaction recovery
c.sql("SELECT recover_prepared_transactions()")
# user should not be created on the worker because outer transaction is not committed yet
role_before_commit = w0.sql_value(
"SELECT count(*) FROM pg_roles WHERE rolname = 'u1'"
)
assert (
int(role_before_commit) == 0
), "role is in pg_dist_object despite not committing"
# user should not be in pg_dist_object on the coordinator because outer transaction is not committed yet
pdo_coordinator_before_commit = c.sql_value(
"SELECT count(*) FROM pg_dist_object WHERE objid = 123123"
)
assert (
int(pdo_coordinator_before_commit) == 0
), "role is in pg_dist_object on coordinator despite not committing"
# user should not be in pg_dist_object on the worker because outer transaction is not committed yet
pdo_worker_before_commit = w0.sql_value(
"SELECT count(*) FROM pg_dist_object WHERE objid::regrole::text = 'u1'"
)
assert (
int(pdo_worker_before_commit) == 0
), "role is in pg_dist_object on worker despite not committing"
# commit in cur1 so the transaction recovery thinks this is a successful transaction
cur1.execute("COMMIT")
# run the transaction recovery again after committing
c.sql("SELECT recover_prepared_transactions()")
# check that the user is created by the transaction recovery on the worker
role_after_commit = w0.sql_value(
"SELECT count(*) FROM pg_roles WHERE rolname = 'u1'"
)
assert (
int(role_after_commit) == 1
), "role is not created during recovery despite committing"
# check that the user is in pg_dist_object on the coordinator after transaction recovery
pdo_coordinator_after_commit = c.sql_value(
"SELECT count(*) FROM pg_dist_object WHERE objid = 123123"
)
assert (
int(pdo_coordinator_after_commit) == 1
), "role is not in pg_dist_object on coordinator after recovery despite committing"
# check that the user is in pg_dist_object on the worker after transaction recovery
pdo_worker_after_commit = w0.sql_value(
"SELECT count(*) FROM pg_dist_object WHERE objid::regrole::text = 'u1'"
)
assert (
int(pdo_worker_after_commit) == 1
), "role is not in pg_dist_object on worker after recovery despite committing"
c.sql("DROP DATABASE db1")
c.sql(
"SELECT citus_internal.execute_command_on_remote_nodes_as_user('DROP USER u1', 'postgres')"
)
c.sql(
"""
SELECT run_command_on_workers($$
DELETE FROM pg_dist_object
WHERE objid::regrole::text = 'u1'
$$)
"""
)
c.sql(
"""
DELETE FROM pg_dist_object
WHERE objid = 123123
"""
)
def test_main_commited_outer_aborted(cluster):
c = cluster.coordinator
w0 = cluster.workers[0]
# create a non-main database
c.sql("CREATE DATABASE db2")
# we will use cur1 to simulate non-main database user and
# cur2 to manually do the steps we would do in the main database
with c.cur(dbname="db2") as cur1, c.cur() as cur2:
# let's start a transaction and find its transaction id
cur1.execute("BEGIN")
cur1.execute("SELECT txid_current()")
txid = cur1.fetchall()
# using the transaction id of the cur1 simulate the main database commands manually
cur2.execute("BEGIN")
cur2.execute(
"SELECT citus_internal.start_management_transaction(%s)", (str(txid[0][0]),)
)
cur2.execute(
"SELECT citus_internal.execute_command_on_remote_nodes_as_user('CREATE USER u2;', 'postgres')"
)
cur2.execute(
"SELECT citus_internal.mark_object_distributed(1260, 'u2', 321321)"
)
cur2.execute("COMMIT")
# abort cur1 so the transaction recovery thinks this is an aborted transaction
cur1.execute("ABORT")
# check that the user is not yet created on the worker
role_before_recovery = w0.sql_value(
"SELECT count(*) FROM pg_roles WHERE rolname = 'u2'"
)
assert int(role_before_recovery) == 0, "role is already created before recovery"
# check that the user is not in pg_dist_object on the coordinator
pdo_coordinator_before_recovery = c.sql_value(
"SELECT count(*) FROM pg_dist_object WHERE objid = 321321"
)
assert (
int(pdo_coordinator_before_recovery) == 0
), "role is already in pg_dist_object on coordinator before recovery"
# check that the user is not in pg_dist_object on the worker
pdo_worker_before_recovery = w0.sql_value(
"SELECT count(*) FROM pg_dist_object WHERE objid::regrole::text = 'u2'"
)
assert (
int(pdo_worker_before_recovery) == 0
), "role is already in pg_dist_object on worker before recovery"
# run the transaction recovery
c.sql("SELECT recover_prepared_transactions()")
# check that the user is not created by the transaction recovery on the worker
role_after_recovery = w0.sql_value(
"SELECT count(*) FROM pg_roles WHERE rolname = 'u2'"
)
assert (
int(role_after_recovery) == 0
), "role is created during recovery despite aborting"
# check that the user is not in pg_dist_object on the coordinator after transaction recovery
pdo_coordinator_after_recovery = c.sql_value(
"SELECT count(*) FROM pg_dist_object WHERE objid = 321321"
)
assert (
int(pdo_coordinator_after_recovery) == 0
), "role is in pg_dist_object on coordinator after recovery despite aborting"
# check that the user is not in pg_dist_object on the worker after transaction recovery
pdo_worker_after_recovery = w0.sql_value(
"SELECT count(*) FROM pg_dist_object WHERE objid::regrole::text = 'u2'"
)
assert (
int(pdo_worker_after_recovery) == 0
), "role is in pg_dist_object on worker after recovery despite aborting"
c.sql("DROP DATABASE db2")

View File

@ -1,38 +1,30 @@
set citus.log_remote_commands = true;
set citus.grep_remote_commands = '%ALTER DATABASE%';
-- since ALLOW_CONNECTIONS alter option should be executed in a different database
-- and since we don't have a multiple database support for now,
-- this statement will get error
alter database regression ALLOW_CONNECTIONS false;
ERROR: ALLOW_CONNECTIONS is not supported
alter database regression with CONNECTION LIMIT 100;
NOTICE: issuing ALTER DATABASE regression WITH CONNECTION LIMIT 100;
NOTICE: issuing ALTER DATABASE regression WITH CONNECTION LIMIT 100;
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing ALTER DATABASE regression WITH CONNECTION LIMIT 100;
NOTICE: issuing ALTER DATABASE regression WITH CONNECTION LIMIT 100;
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
alter database regression with IS_TEMPLATE true CONNECTION LIMIT 50;
NOTICE: issuing ALTER DATABASE regression WITH IS_TEMPLATE 'true' CONNECTION LIMIT 50;
NOTICE: issuing ALTER DATABASE regression WITH IS_TEMPLATE true CONNECTION LIMIT 50;
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing ALTER DATABASE regression WITH IS_TEMPLATE 'true' CONNECTION LIMIT 50;
NOTICE: issuing ALTER DATABASE regression WITH IS_TEMPLATE true CONNECTION LIMIT 50;
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
alter database regression with CONNECTION LIMIT -1;
NOTICE: issuing ALTER DATABASE regression WITH CONNECTION LIMIT -1;
NOTICE: issuing ALTER DATABASE regression WITH CONNECTION LIMIT -1;
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing ALTER DATABASE regression WITH CONNECTION LIMIT -1;
NOTICE: issuing ALTER DATABASE regression WITH CONNECTION LIMIT -1;
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
alter database regression with IS_TEMPLATE true;
NOTICE: issuing ALTER DATABASE regression WITH IS_TEMPLATE 'true';
NOTICE: issuing ALTER DATABASE regression WITH IS_TEMPLATE true;
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing ALTER DATABASE regression WITH IS_TEMPLATE 'true';
NOTICE: issuing ALTER DATABASE regression WITH IS_TEMPLATE true;
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
alter database regression with IS_TEMPLATE false;
NOTICE: issuing ALTER DATABASE regression WITH IS_TEMPLATE 'false';
NOTICE: issuing ALTER DATABASE regression WITH IS_TEMPLATE false;
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing ALTER DATABASE regression WITH IS_TEMPLATE 'false';
NOTICE: issuing ALTER DATABASE regression WITH IS_TEMPLATE false;
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
-- this statement will get error since we don't have a multiple database support for now
alter database regression rename to regression2;
ERROR: current database cannot be renamed
alter database regression set default_transaction_read_only = true;
NOTICE: issuing ALTER DATABASE regression SET default_transaction_read_only = 'true'
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
@ -147,4 +139,86 @@ NOTICE: issuing ALTER DATABASE regression RESET lock_timeout
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing ALTER DATABASE regression RESET lock_timeout
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
set citus.enable_create_database_propagation=on;
create database "regression!'2";
alter database "regression!'2" with CONNECTION LIMIT 100;
NOTICE: issuing ALTER DATABASE "regression!'2" WITH CONNECTION LIMIT 100;
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing ALTER DATABASE "regression!'2" WITH CONNECTION LIMIT 100;
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
alter database "regression!'2" with IS_TEMPLATE true CONNECTION LIMIT 50;
NOTICE: issuing ALTER DATABASE "regression!'2" WITH IS_TEMPLATE true CONNECTION LIMIT 50;
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing ALTER DATABASE "regression!'2" WITH IS_TEMPLATE true CONNECTION LIMIT 50;
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
alter database "regression!'2" with IS_TEMPLATE false;
NOTICE: issuing ALTER DATABASE "regression!'2" WITH IS_TEMPLATE false;
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing ALTER DATABASE "regression!'2" WITH IS_TEMPLATE false;
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
\set alter_db_tablespace :abs_srcdir '/tmp_check/ts3'
CREATE TABLESPACE alter_db_tablespace LOCATION :'alter_db_tablespace';
\c - - - :worker_1_port
\set alter_db_tablespace :abs_srcdir '/tmp_check/ts4'
CREATE TABLESPACE alter_db_tablespace LOCATION :'alter_db_tablespace';
\c - - - :worker_2_port
\set alter_db_tablespace :abs_srcdir '/tmp_check/ts5'
CREATE TABLESPACE alter_db_tablespace LOCATION :'alter_db_tablespace';
\c - - - :master_port
set citus.log_remote_commands = true;
set citus.grep_remote_commands = '%ALTER DATABASE%';
alter database "regression!'2" set TABLESPACE alter_db_tablespace;
NOTICE: issuing ALTER DATABASE "regression!'2" SET TABLESPACE alter_db_tablespace
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing ALTER DATABASE "regression!'2" SET TABLESPACE alter_db_tablespace
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
set citus.enable_create_database_propagation=on;
alter database "regression!'2" rename to regression3;
NOTICE: issuing ALTER DATABASE "regression!'2" RENAME TO regression3
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing ALTER DATABASE "regression!'2" RENAME TO regression3
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
-- check that the local database rename and alter comnmand is not propagated
set citus.enable_create_database_propagation=off;
CREATE database local_regression;
NOTICE: Citus partially supports CREATE DATABASE for distributed databases
DETAIL: Citus does not propagate CREATE DATABASE command to other nodes
HINT: You can manually create a database and its extensions on other nodes.
alter DATABASE local_regression with CONNECTION LIMIT 100;
alter DATABASE local_regression rename to local_regression2;
drop database local_regression2;
set citus.enable_create_database_propagation=on;
drop database regression3;
create database "regression!'4";
SELECT result FROM run_command_on_all_nodes(
$$
ALTER TABLESPACE alter_db_tablespace RENAME TO "ts-needs\!escape"
$$
);
result
---------------------------------------------------------------------
ALTER TABLESPACE
ALTER TABLESPACE
ALTER TABLESPACE
(3 rows)
alter database "regression!'4" set TABLESPACE "ts-needs\!escape";
NOTICE: issuing ALTER DATABASE "regression!'4" SET TABLESPACE "ts-needs\!escape"
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing ALTER DATABASE "regression!'4" SET TABLESPACE "ts-needs\!escape"
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
drop database "regression!'4";
set citus.log_remote_commands = false;
set citus.enable_create_database_propagation=off;
SELECT result FROM run_command_on_all_nodes(
$$
drop tablespace "ts-needs\!escape"
$$
);
result
---------------------------------------------------------------------
DROP TABLESPACE
DROP TABLESPACE
DROP TABLESPACE
(3 rows)

View File

@ -254,8 +254,8 @@ SELECT run_command_on_workers('SHOW enable_hashagg');
-- also test case sensitivity
CREATE DATABASE "REGRESSION";
NOTICE: Citus partially supports CREATE DATABASE for distributed databases
DETAIL: Citus does not propagate CREATE DATABASE command to workers
HINT: You can manually create a database and its extensions on workers.
DETAIL: Citus does not propagate CREATE DATABASE command to other nodes
HINT: You can manually create a database and its extensions on other nodes.
ALTER ROLE CURRENT_USER IN DATABASE "REGRESSION" SET public.myguc TO "Hello from coordinator only";
SELECT d.datname, r.setconfig FROM pg_db_role_setting r LEFT JOIN pg_database d ON r.setdatabase=d.oid WHERE r.setconfig::text LIKE '%Hello from coordinator only%';
datname | setconfig

View File

@ -285,14 +285,7 @@ SELECT citus_schema_undistribute('tenant1');
ERROR: must be owner of schema tenant1
-- assign all tables to dummyregular except table5
SET role tenantuser;
SELECT result FROM run_command_on_all_nodes($$ REASSIGN OWNED BY tenantuser TO dummyregular; $$);
result
---------------------------------------------------------------------
REASSIGN OWNED
REASSIGN OWNED
REASSIGN OWNED
(3 rows)
REASSIGN OWNED BY tenantuser TO dummyregular;
CREATE TABLE tenant1.table5(id int);
-- table owner check fails the distribution
SET role dummyregular;
@ -366,14 +359,7 @@ SELECT result FROM run_command_on_all_nodes($$ SELECT array_agg(logicalrelid ORD
(3 rows)
RESET role;
SELECT result FROM run_command_on_all_nodes($$ REASSIGN OWNED BY dummyregular TO tenantuser; $$);
result
---------------------------------------------------------------------
REASSIGN OWNED
REASSIGN OWNED
REASSIGN OWNED
(3 rows)
REASSIGN OWNED BY dummyregular TO tenantuser;
DROP USER dummyregular;
CREATE USER dummysuper superuser;
SET role dummysuper;

View File

@ -189,14 +189,7 @@ SELECT citus_schema_move('s2', 'dummy_node', 1234);
ERROR: must be owner of schema s2
-- assign all tables to regularuser
RESET ROLE;
SELECT result FROM run_command_on_all_nodes($$ REASSIGN OWNED BY tenantuser TO regularuser; $$);
result
---------------------------------------------------------------------
REASSIGN OWNED
REASSIGN OWNED
REASSIGN OWNED
(3 rows)
REASSIGN OWNED BY tenantuser TO regularuser;
GRANT USAGE ON SCHEMA citus_schema_move TO regularuser;
SET ROLE regularuser;
SELECT nodeid AS s2_new_nodeid, quote_literal(nodename) AS s2_new_nodename, nodeport AS s2_new_nodeport

View File

@ -39,8 +39,8 @@ SELECT :columnar_stripes_before_drop - count(distinct storage_id) FROM columnar.
SELECT current_database() datname \gset
CREATE DATABASE db_to_drop;
NOTICE: Citus partially supports CREATE DATABASE for distributed databases
DETAIL: Citus does not propagate CREATE DATABASE command to workers
HINT: You can manually create a database and its extensions on workers.
DETAIL: Citus does not propagate CREATE DATABASE command to other nodes
HINT: You can manually create a database and its extensions on other nodes.
\c db_to_drop
CREATE EXTENSION citus_columnar;
SELECT oid::text databaseoid FROM pg_database WHERE datname = current_database() \gset

View File

@ -0,0 +1,101 @@
set citus.log_remote_commands to on;
set citus.enable_create_database_propagation to on;
set citus.grep_remote_commands to 'COMMENT ON DATABASE';
create database "test1-\!escape";
comment on DATABASE "test1-\!escape" is 'test-comment';
SELECT result FROM run_command_on_all_nodes(
$$
SELECT ds.description AS database_comment
FROM pg_database d
LEFT JOIN pg_shdescription ds ON d.oid = ds.objoid
WHERE d.datname = 'test1-\!escape';
$$
);
result
---------------------------------------------------------------------
test-comment
test-comment
test-comment
(3 rows)
comment on DATABASE "test1-\!escape" is 'comment-needs\!escape';
SELECT result FROM run_command_on_all_nodes(
$$
SELECT ds.description AS database_comment
FROM pg_database d
LEFT JOIN pg_shdescription ds ON d.oid = ds.objoid
WHERE d.datname = 'test1-\!escape';
$$
);
result
---------------------------------------------------------------------
comment-needs\!escape
comment-needs\!escape
comment-needs\!escape
(3 rows)
comment on DATABASE "test1-\!escape" is null;
SELECT result FROM run_command_on_all_nodes(
$$
SELECT ds.description AS database_comment
FROM pg_database d
LEFT JOIN pg_shdescription ds ON d.oid = ds.objoid
WHERE d.datname = 'test1-\!escape';
$$
);
result
---------------------------------------------------------------------
(3 rows)
drop DATABASE "test1-\!escape";
--test metadata sync
select 1 from citus_remove_node('localhost', :worker_2_port);
?column?
---------------------------------------------------------------------
1
(1 row)
create database "test1-\!escape";
comment on DATABASE "test1-\!escape" is 'test-comment';
SELECT result FROM run_command_on_all_nodes(
$$
SELECT ds.description AS database_comment
FROM pg_database d
LEFT JOIN pg_shdescription ds ON d.oid = ds.objoid
WHERE d.datname = 'test1-\!escape';
$$
);
result
---------------------------------------------------------------------
test-comment
test-comment
(2 rows)
select 1 from citus_add_node('localhost', :worker_2_port);
?column?
---------------------------------------------------------------------
1
(1 row)
SELECT result FROM run_command_on_all_nodes(
$$
SELECT ds.description AS database_comment
FROM pg_database d
LEFT JOIN pg_shdescription ds ON d.oid = ds.objoid
WHERE d.datname = 'test1-\!escape';
$$
);
result
---------------------------------------------------------------------
test-comment
test-comment
test-comment
(3 rows)
drop DATABASE "test1-\!escape";
reset citus.enable_create_database_propagation;
reset citus.grep_remote_commands;
reset citus.log_remote_commands;

View File

@ -0,0 +1,99 @@
set citus.log_remote_commands to on;
set citus.grep_remote_commands to 'COMMENT ON ROLE';
create role "role1-\!escape";
comment on ROLE "role1-\!escape" is 'test-comment';
SELECT result FROM run_command_on_all_nodes(
$$
SELECT ds.description AS role_comment
FROM pg_roles r
LEFT JOIN pg_shdescription ds ON r.oid = ds.objoid
WHERE r.rolname = 'role1-\!escape';
$$
);
result
---------------------------------------------------------------------
test-comment
test-comment
test-comment
(3 rows)
comment on role "role1-\!escape" is 'comment-needs\!escape';
SELECT result FROM run_command_on_all_nodes(
$$
SELECT ds.description AS role_comment
FROM pg_roles r
LEFT JOIN pg_shdescription ds ON r.oid = ds.objoid
WHERE r.rolname = 'role1-\!escape';
$$
);
result
---------------------------------------------------------------------
comment-needs\!escape
comment-needs\!escape
comment-needs\!escape
(3 rows)
comment on role "role1-\!escape" is NULL;
SELECT result FROM run_command_on_all_nodes(
$$
SELECT ds.description AS role_comment
FROM pg_roles r
LEFT JOIN pg_shdescription ds ON r.oid = ds.objoid
WHERE r.rolname = 'role1-\!escape';
$$
);
result
---------------------------------------------------------------------
(3 rows)
drop role "role1-\!escape";
--test metadata sync
select 1 from citus_remove_node('localhost', :worker_2_port);
?column?
---------------------------------------------------------------------
1
(1 row)
create role "role1-\!escape";
comment on ROLE "role1-\!escape" is 'test-comment';
SELECT result FROM run_command_on_all_nodes(
$$
SELECT ds.description AS role_comment
FROM pg_roles r
LEFT JOIN pg_shdescription ds ON r.oid = ds.objoid
WHERE r.rolname = 'role1-\!escape';
$$
);
result
---------------------------------------------------------------------
test-comment
test-comment
(2 rows)
select 1 from citus_add_node('localhost', :worker_2_port);
?column?
---------------------------------------------------------------------
1
(1 row)
SELECT result FROM run_command_on_all_nodes(
$$
SELECT ds.description AS role_comment
FROM pg_roles r
LEFT JOIN pg_shdescription ds ON r.oid = ds.objoid
WHERE r.rolname = 'role1-\!escape';
$$
);
result
---------------------------------------------------------------------
test-comment
test-comment
test-comment
(3 rows)
drop role "role1-\!escape";
reset citus.grep_remote_commands;
reset citus.log_remote_commands;

View File

@ -64,8 +64,8 @@ CREATE TABLESPACE create_drop_db_tablespace LOCATION :'create_drop_db_tablespace
\c - - - :master_port
CREATE DATABASE local_database;
NOTICE: Citus partially supports CREATE DATABASE for distributed databases
DETAIL: Citus does not propagate CREATE DATABASE command to workers
HINT: You can manually create a database and its extensions on workers.
DETAIL: Citus does not propagate CREATE DATABASE command to other nodes
HINT: You can manually create a database and its extensions on other nodes.
-- check that it's only created for coordinator
SELECT * FROM public.check_database_on_all_nodes('local_database') ORDER BY node_type;
node_type | result
@ -88,8 +88,8 @@ SELECT * FROM public.check_database_on_all_nodes('local_database') ORDER BY node
\c - - - :worker_1_port
CREATE DATABASE local_database;
NOTICE: Citus partially supports CREATE DATABASE for distributed databases
DETAIL: Citus does not propagate CREATE DATABASE command to workers
HINT: You can manually create a database and its extensions on workers.
DETAIL: Citus does not propagate CREATE DATABASE command to other nodes
HINT: You can manually create a database and its extensions on other nodes.
-- check that it's only created for coordinator
SELECT * FROM public.check_database_on_all_nodes('local_database') ORDER BY node_type;
node_type | result
@ -209,19 +209,7 @@ SELECT result FROM run_command_on_all_nodes(
CREATE USER "role-needs\!escape";
CREATE DATABASE "db-needs\!escape" owner "role-needs\!escape" tablespace "ts-needs\!escape";
-- Rename it to make check_database_on_all_nodes happy.
-- Today we don't support ALTER DATABASE .. RENAME TO .., so need to propagate it manually.
SELECT result FROM run_command_on_all_nodes(
$$
ALTER DATABASE "db-needs\!escape" RENAME TO db_needs_escape
$$
);
result
---------------------------------------------------------------------
ALTER DATABASE
ALTER DATABASE
ALTER DATABASE
(3 rows)
ALTER DATABASE "db-needs\!escape" RENAME TO db_needs_escape;
SELECT * FROM public.check_database_on_all_nodes('db_needs_escape') ORDER BY node_type;
node_type | result
---------------------------------------------------------------------
@ -464,18 +452,14 @@ drop database "mydatabase#1'2";
ERROR: database "mydatabase#1'2" does not exist
\c - - - :worker_1_port
SET citus.enable_create_database_propagation TO ON;
-- show that dropping the database from workers is not allowed when citus.enable_create_database_propagation is on
-- show that dropping the database from workers is allowed when citus.enable_create_database_propagation is on
DROP DATABASE db_needs_escape;
ERROR: operation is not allowed on this node
HINT: Connect to the coordinator and run it again.
-- and the same applies to create database too
create database error_test;
ERROR: operation is not allowed on this node
HINT: Connect to the coordinator and run it again.
drop database error_test;
\c - - - :master_port
SET citus.enable_create_database_propagation TO ON;
DROP DATABASE test_node_activation;
DROP DATABASE db_needs_escape;
DROP USER "role-needs\!escape";
-- drop database with force options test
create database db_force_test;
@ -506,8 +490,8 @@ select 1 from citus_remove_node('localhost', :worker_2_port);
SET citus.enable_create_database_propagation TO off;
CREATE DATABASE non_distributed_db;
NOTICE: Citus partially supports CREATE DATABASE for distributed databases
DETAIL: Citus does not propagate CREATE DATABASE command to workers
HINT: You can manually create a database and its extensions on workers.
DETAIL: Citus does not propagate CREATE DATABASE command to other nodes
HINT: You can manually create a database and its extensions on other nodes.
SET citus.enable_create_database_propagation TO on;
create database distributed_db;
select 1 from citus_add_node('localhost', :worker_2_port);
@ -556,10 +540,10 @@ SELECT result from run_command_on_all_nodes(
revoke connect,temp,temporary,create on database db_role_grants_test_non_distributed from public
$$
) ORDER BY result;
result
result
---------------------------------------------------------------------
ERROR: operation is not allowed on this node
ERROR: operation is not allowed on this node
REVOKE
REVOKE
REVOKE
(3 rows)
@ -589,11 +573,7 @@ grant CONNECT,TEMPORARY,CREATE on DATABASE db_role_grants_test to db_role_grants
NOTICE: issuing GRANT connect, temporary, create ON DATABASE db_role_grants_test TO db_role_grants_test_role_missing_on_node_2;
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
grant CONNECT,TEMPORARY,CREATE on DATABASE db_role_grants_test_non_distributed to db_role_grants_test_role_exists_on_node_2;
NOTICE: issuing GRANT connect, temporary, create ON DATABASE db_role_grants_test_non_distributed TO db_role_grants_test_role_exists_on_node_2;
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
grant CONNECT,TEMPORARY,CREATE on DATABASE db_role_grants_test_non_distributed to db_role_grants_test_role_missing_on_node_2;
NOTICE: issuing GRANT connect, temporary, create ON DATABASE db_role_grants_test_non_distributed TO db_role_grants_test_role_missing_on_node_2;
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
-- check the privileges before add_node for database db_role_grants_test,
-- role db_role_grants_test_role_exists_on_node_2
SELECT result from run_command_on_all_nodes(
@ -673,7 +653,7 @@ SELECT result from run_command_on_all_nodes(
) ORDER BY result;
result
---------------------------------------------------------------------
t
f
t
(2 rows)
@ -684,7 +664,7 @@ SELECT result from run_command_on_all_nodes(
) ORDER BY result;
result
---------------------------------------------------------------------
t
f
t
(2 rows)
@ -695,7 +675,7 @@ SELECT result from run_command_on_all_nodes(
) ORDER BY result;
result
---------------------------------------------------------------------
t
f
t
(2 rows)
@ -708,7 +688,7 @@ SELECT result from run_command_on_all_nodes(
) ORDER BY result;
result
---------------------------------------------------------------------
t
f
t
(2 rows)
@ -719,7 +699,7 @@ SELECT result from run_command_on_all_nodes(
) ORDER BY result;
result
---------------------------------------------------------------------
t
f
t
(2 rows)
@ -730,7 +710,7 @@ SELECT result from run_command_on_all_nodes(
) ORDER BY result;
result
---------------------------------------------------------------------
t
f
t
(2 rows)
@ -828,7 +808,7 @@ SELECT result from run_command_on_all_nodes(
result
---------------------------------------------------------------------
f
t
f
t
(3 rows)
@ -840,7 +820,7 @@ SELECT result from run_command_on_all_nodes(
result
---------------------------------------------------------------------
f
t
f
t
(3 rows)
@ -852,7 +832,7 @@ SELECT result from run_command_on_all_nodes(
result
---------------------------------------------------------------------
f
t
f
t
(3 rows)
@ -866,7 +846,7 @@ SELECT result from run_command_on_all_nodes(
result
---------------------------------------------------------------------
f
t
f
t
(3 rows)
@ -878,7 +858,7 @@ SELECT result from run_command_on_all_nodes(
result
---------------------------------------------------------------------
f
t
f
t
(3 rows)
@ -890,7 +870,7 @@ SELECT result from run_command_on_all_nodes(
result
---------------------------------------------------------------------
f
t
f
t
(3 rows)
@ -944,6 +924,347 @@ SELECT * FROM public.check_database_on_all_nodes('test_db') ORDER BY node_type;
REVOKE CONNECT ON DATABASE test_db FROM propagated_role;
DROP DATABASE test_db;
DROP ROLE propagated_role, non_propagated_role;
-- show that we don't try to propagate commands on non-distributed databases
SET citus.enable_create_database_propagation TO OFF;
CREATE DATABASE local_database_1;
NOTICE: Citus partially supports CREATE DATABASE for distributed databases
DETAIL: Citus does not propagate CREATE DATABASE command to other nodes
HINT: You can manually create a database and its extensions on other nodes.
SET citus.enable_create_database_propagation TO ON;
CREATE ROLE local_role_1;
GRANT CONNECT, TEMPORARY, CREATE ON DATABASE local_database_1 TO local_role_1;
ALTER DATABASE local_database_1 SET default_transaction_read_only = 'true';
REVOKE CONNECT, TEMPORARY, CREATE ON DATABASE local_database_1 FROM local_role_1;
DROP ROLE local_role_1;
DROP DATABASE local_database_1;
-- test create / drop database commands from workers
-- remove one of the workers to test node activation too
SELECT 1 from citus_remove_node('localhost', :worker_2_port);
?column?
---------------------------------------------------------------------
1
(1 row)
\c - - - :worker_1_port
CREATE DATABASE local_worker_db;
NOTICE: Citus partially supports CREATE DATABASE for distributed databases
DETAIL: Citus does not propagate CREATE DATABASE command to other nodes
HINT: You can manually create a database and its extensions on other nodes.
SET citus.enable_create_database_propagation TO ON;
CREATE DATABASE db_created_from_worker
WITH template=template1
OWNER = create_drop_db_test_user
ENCODING = 'UTF8'
CONNECTION LIMIT = 42
TABLESPACE = "ts-needs\!escape"
ALLOW_CONNECTIONS = false;
\c - - - :master_port
SET citus.enable_create_database_propagation TO ON;
SELECT 1 FROM citus_add_node('localhost', :worker_2_port);
?column?
---------------------------------------------------------------------
1
(1 row)
\c - - - :worker_1_port
SET citus.enable_create_database_propagation TO ON;
SELECT * FROM public.check_database_on_all_nodes('local_worker_db') ORDER BY node_type;
node_type | result
---------------------------------------------------------------------
coordinator (remote) | {"database_properties": null, "pg_dist_object_record_for_db_exists": false, "stale_pg_dist_object_record_for_a_db_exists": false}
worker node (local) | {"database_properties": {"datacl": null, "datname": "local_worker_db", "datctype": "C", "encoding": "UTF8", "datcollate": "C", "tablespace": "pg_default", "daticurules": null, "datallowconn": true, "datconnlimit": -1, "daticulocale": null, "datistemplate": false, "database_owner": "postgres", "datcollversion": null, "datlocprovider": "c"}, "pg_dist_object_record_for_db_exists": false, "stale_pg_dist_object_record_for_a_db_exists": false}
worker node (remote) | {"database_properties": null, "pg_dist_object_record_for_db_exists": false, "stale_pg_dist_object_record_for_a_db_exists": false}
(3 rows)
SELECT * FROM public.check_database_on_all_nodes('db_created_from_worker') ORDER BY node_type;
node_type | result
---------------------------------------------------------------------
coordinator (remote) | {"database_properties": {"datacl": null, "datname": "db_created_from_worker", "datctype": "C", "encoding": "UTF8", "datcollate": "C", "tablespace": "ts-needs\\!escape", "daticurules": null, "datallowconn": false, "datconnlimit": 42, "daticulocale": null, "datistemplate": false, "database_owner": "create_drop_db_test_user", "datcollversion": null, "datlocprovider": "c"}, "pg_dist_object_record_for_db_exists": true, "stale_pg_dist_object_record_for_a_db_exists": false}
worker node (local) | {"database_properties": {"datacl": null, "datname": "db_created_from_worker", "datctype": "C", "encoding": "UTF8", "datcollate": "C", "tablespace": "ts-needs\\!escape", "daticurules": null, "datallowconn": false, "datconnlimit": 42, "daticulocale": null, "datistemplate": false, "database_owner": "create_drop_db_test_user", "datcollversion": null, "datlocprovider": "c"}, "pg_dist_object_record_for_db_exists": true, "stale_pg_dist_object_record_for_a_db_exists": false}
worker node (remote) | {"database_properties": {"datacl": null, "datname": "db_created_from_worker", "datctype": "C", "encoding": "UTF8", "datcollate": "C", "tablespace": "ts-needs\\!escape", "daticurules": null, "datallowconn": false, "datconnlimit": 42, "daticulocale": null, "datistemplate": false, "database_owner": "create_drop_db_test_user", "datcollversion": null, "datlocprovider": "c"}, "pg_dist_object_record_for_db_exists": true, "stale_pg_dist_object_record_for_a_db_exists": false}
(3 rows)
DROP DATABASE db_created_from_worker;
SELECT * FROM public.check_database_on_all_nodes('db_created_from_worker') ORDER BY node_type;
node_type | result
---------------------------------------------------------------------
coordinator (remote) | {"database_properties": null, "pg_dist_object_record_for_db_exists": false, "stale_pg_dist_object_record_for_a_db_exists": false}
worker node (local) | {"database_properties": null, "pg_dist_object_record_for_db_exists": false, "stale_pg_dist_object_record_for_a_db_exists": false}
worker node (remote) | {"database_properties": null, "pg_dist_object_record_for_db_exists": false, "stale_pg_dist_object_record_for_a_db_exists": false}
(3 rows)
-- drop the local database while the GUC is on
DROP DATABASE local_worker_db;
SELECT * FROM public.check_database_on_all_nodes('local_worker_db') ORDER BY node_type;
node_type | result
---------------------------------------------------------------------
coordinator (remote) | {"database_properties": null, "pg_dist_object_record_for_db_exists": false, "stale_pg_dist_object_record_for_a_db_exists": false}
worker node (local) | {"database_properties": null, "pg_dist_object_record_for_db_exists": false, "stale_pg_dist_object_record_for_a_db_exists": false}
worker node (remote) | {"database_properties": null, "pg_dist_object_record_for_db_exists": false, "stale_pg_dist_object_record_for_a_db_exists": false}
(3 rows)
SET citus.enable_create_database_propagation TO OFF;
CREATE DATABASE local_worker_db;
NOTICE: Citus partially supports CREATE DATABASE for distributed databases
DETAIL: Citus does not propagate CREATE DATABASE command to other nodes
HINT: You can manually create a database and its extensions on other nodes.
-- drop the local database while the GUC is off
DROP DATABASE local_worker_db;
SELECT * FROM public.check_database_on_all_nodes('local_worker_db') ORDER BY node_type;
node_type | result
---------------------------------------------------------------------
coordinator (remote) | {"database_properties": null, "pg_dist_object_record_for_db_exists": false, "stale_pg_dist_object_record_for_a_db_exists": false}
worker node (local) | {"database_properties": null, "pg_dist_object_record_for_db_exists": false, "stale_pg_dist_object_record_for_a_db_exists": false}
worker node (remote) | {"database_properties": null, "pg_dist_object_record_for_db_exists": false, "stale_pg_dist_object_record_for_a_db_exists": false}
(3 rows)
SET citus.enable_create_database_propagation TO ON;
CREATE DATABASE another_db_created_from_worker;
\c - - - :master_port
SELECT 1 FROM citus_remove_node('localhost', :master_port);
?column?
---------------------------------------------------------------------
1
(1 row)
\c - - - :worker_1_port
SET citus.enable_create_database_propagation TO ON;
-- fails because coordinator is not added into metadata
DROP DATABASE another_db_created_from_worker;
ERROR: coordinator is not added to the metadata
HINT: Use SELECT citus_set_coordinator_host('<hostname>') on coordinator to configure the coordinator hostname
-- fails because coordinator is not added into metadata
CREATE DATABASE new_db;
ERROR: coordinator is not added to the metadata
HINT: Use SELECT citus_set_coordinator_host('<hostname>') on coordinator to configure the coordinator hostname
\c - - - :master_port
SET client_min_messages TO WARNING;
SELECT 1 FROM citus_add_node('localhost', :master_port, 0);
?column?
---------------------------------------------------------------------
1
(1 row)
RESET client_min_messages;
SET citus.enable_create_database_propagation TO ON;
-- dropping a database that was created from a worker via a different node works fine
DROP DATABASE another_db_created_from_worker;
SELECT * FROM public.check_database_on_all_nodes('another_db_created_from_worker') ORDER BY node_type;
node_type | result
---------------------------------------------------------------------
coordinator (local) | {"database_properties": null, "pg_dist_object_record_for_db_exists": false, "stale_pg_dist_object_record_for_a_db_exists": false}
worker node (remote) | {"database_properties": null, "pg_dist_object_record_for_db_exists": false, "stale_pg_dist_object_record_for_a_db_exists": false}
worker node (remote) | {"database_properties": null, "pg_dist_object_record_for_db_exists": false, "stale_pg_dist_object_record_for_a_db_exists": false}
(3 rows)
-- Show that we automatically propagate the dependencies (only roles atm) when
-- creating a database from workers too.
SELECT 1 from citus_remove_node('localhost', :worker_2_port);
?column?
---------------------------------------------------------------------
1
(1 row)
\c - - - :worker_1_port
set citus.enable_create_role_propagation TO off;
create role non_propagated_role;
NOTICE: not propagating CREATE ROLE/USER commands to other nodes
HINT: Connect to other nodes directly to manually create all necessary users and roles.
set citus.enable_create_role_propagation TO on;
set citus.enable_create_database_propagation TO on;
create database test_db OWNER non_propagated_role;
create role propagated_role;
\c - - - :master_port
-- not supported from workers, so need to execute this via coordinator
grant connect on database test_db to propagated_role;
SET citus.enable_create_database_propagation TO ON;
SELECT 1 FROM citus_add_node('localhost', :worker_2_port);
?column?
---------------------------------------------------------------------
1
(1 row)
SELECT * FROM public.check_database_on_all_nodes('test_db') ORDER BY node_type;
node_type | result
---------------------------------------------------------------------
coordinator (local) | {"database_properties": {"datacl": ["=Tc/non_propagated_role", "non_propagated_role=CTc/non_propagated_role", "propagated_role=c/non_propagated_role"], "datname": "test_db", "datctype": "C", "encoding": "UTF8", "datcollate": "C", "tablespace": "pg_default", "daticurules": null, "datallowconn": true, "datconnlimit": -1, "daticulocale": null, "datistemplate": false, "database_owner": "non_propagated_role", "datcollversion": null, "datlocprovider": "c"}, "pg_dist_object_record_for_db_exists": true, "stale_pg_dist_object_record_for_a_db_exists": false}
worker node (remote) | {"database_properties": {"datacl": ["=Tc/non_propagated_role", "non_propagated_role=CTc/non_propagated_role", "propagated_role=c/non_propagated_role"], "datname": "test_db", "datctype": "C", "encoding": "UTF8", "datcollate": "C", "tablespace": "pg_default", "daticurules": null, "datallowconn": true, "datconnlimit": -1, "daticulocale": null, "datistemplate": false, "database_owner": "non_propagated_role", "datcollversion": null, "datlocprovider": "c"}, "pg_dist_object_record_for_db_exists": true, "stale_pg_dist_object_record_for_a_db_exists": false}
worker node (remote) | {"database_properties": {"datacl": ["=Tc/non_propagated_role", "non_propagated_role=CTc/non_propagated_role", "propagated_role=c/non_propagated_role"], "datname": "test_db", "datctype": "C", "encoding": "UTF8", "datcollate": "C", "tablespace": "pg_default", "daticurules": null, "datallowconn": true, "datconnlimit": -1, "daticulocale": null, "datistemplate": false, "database_owner": "non_propagated_role", "datcollversion": null, "datlocprovider": "c"}, "pg_dist_object_record_for_db_exists": true, "stale_pg_dist_object_record_for_a_db_exists": false}
(3 rows)
REVOKE CONNECT ON DATABASE test_db FROM propagated_role;
DROP DATABASE test_db;
DROP ROLE propagated_role, non_propagated_role;
-- test pg_catalog.citus_internal_acquire_citus_advisory_object_class_lock with null input
SELECT pg_catalog.citus_internal_acquire_citus_advisory_object_class_lock(null, 'regression');
ERROR: object_class cannot be NULL
SELECT pg_catalog.citus_internal_acquire_citus_advisory_object_class_lock((SELECT CASE WHEN substring(version(), '\d+')::integer < 16 THEN 25 ELSE 26 END AS oclass_database), null);
citus_internal_acquire_citus_advisory_object_class_lock
---------------------------------------------------------------------
(1 row)
-- OCLASS_DATABASE
SELECT pg_catalog.citus_internal_acquire_citus_advisory_object_class_lock((SELECT CASE WHEN substring(version(), '\d+')::integer < 16 THEN 25 ELSE 26 END AS oclass_database), NULL);
citus_internal_acquire_citus_advisory_object_class_lock
---------------------------------------------------------------------
(1 row)
SELECT pg_catalog.citus_internal_acquire_citus_advisory_object_class_lock((SELECT CASE WHEN substring(version(), '\d+')::integer < 16 THEN 25 ELSE 26 END AS oclass_database), 'regression');
citus_internal_acquire_citus_advisory_object_class_lock
---------------------------------------------------------------------
(1 row)
SELECT pg_catalog.citus_internal_acquire_citus_advisory_object_class_lock((SELECT CASE WHEN substring(version(), '\d+')::integer < 16 THEN 25 ELSE 26 END AS oclass_database), '');
ERROR: database "" does not exist
SELECT pg_catalog.citus_internal_acquire_citus_advisory_object_class_lock((SELECT CASE WHEN substring(version(), '\d+')::integer < 16 THEN 25 ELSE 26 END AS oclass_database), 'no_such_db');
ERROR: database "no_such_db" does not exist
-- invalid OCLASS
SELECT pg_catalog.citus_internal_acquire_citus_advisory_object_class_lock(-1, NULL);
ERROR: unsupported object class: -1
SELECT pg_catalog.citus_internal_acquire_citus_advisory_object_class_lock(-1, 'regression');
ERROR: unsupported object class: -1
-- invalid OCLASS
SELECT pg_catalog.citus_internal_acquire_citus_advisory_object_class_lock(100, NULL);
ERROR: unsupported object class: 100
SELECT pg_catalog.citus_internal_acquire_citus_advisory_object_class_lock(100, 'regression');
ERROR: unsupported object class: 100
-- another valid OCLASS, but not implemented yet
SELECT pg_catalog.citus_internal_acquire_citus_advisory_object_class_lock(10, NULL);
ERROR: unsupported object class: 10
SELECT pg_catalog.citus_internal_acquire_citus_advisory_object_class_lock(10, 'regression');
ERROR: unsupported object class: 10
SELECT 1 FROM run_command_on_all_nodes('ALTER SYSTEM SET citus.enable_create_database_propagation TO ON');
?column?
---------------------------------------------------------------------
1
1
1
(3 rows)
SELECT 1 FROM run_command_on_all_nodes('SELECT pg_reload_conf()');
?column?
---------------------------------------------------------------------
1
1
1
(3 rows)
SELECT pg_sleep(0.1);
pg_sleep
---------------------------------------------------------------------
(1 row)
-- only one of them succeeds and we don't run into a distributed deadlock
SELECT COUNT(*) FROM run_command_on_all_nodes('CREATE DATABASE concurrent_create_db') WHERE success;
count
---------------------------------------------------------------------
1
(1 row)
SELECT * FROM public.check_database_on_all_nodes('concurrent_create_db') ORDER BY node_type;
node_type | result
---------------------------------------------------------------------
coordinator (local) | {"database_properties": {"datacl": null, "datname": "concurrent_create_db", "datctype": "C", "encoding": "UTF8", "datcollate": "C", "tablespace": "pg_default", "daticurules": null, "datallowconn": true, "datconnlimit": -1, "daticulocale": null, "datistemplate": false, "database_owner": "postgres", "datcollversion": null, "datlocprovider": "c"}, "pg_dist_object_record_for_db_exists": true, "stale_pg_dist_object_record_for_a_db_exists": false}
worker node (remote) | {"database_properties": {"datacl": null, "datname": "concurrent_create_db", "datctype": "C", "encoding": "UTF8", "datcollate": "C", "tablespace": "pg_default", "daticurules": null, "datallowconn": true, "datconnlimit": -1, "daticulocale": null, "datistemplate": false, "database_owner": "postgres", "datcollversion": null, "datlocprovider": "c"}, "pg_dist_object_record_for_db_exists": true, "stale_pg_dist_object_record_for_a_db_exists": false}
worker node (remote) | {"database_properties": {"datacl": null, "datname": "concurrent_create_db", "datctype": "C", "encoding": "UTF8", "datcollate": "C", "tablespace": "pg_default", "daticurules": null, "datallowconn": true, "datconnlimit": -1, "daticulocale": null, "datistemplate": false, "database_owner": "postgres", "datcollversion": null, "datlocprovider": "c"}, "pg_dist_object_record_for_db_exists": true, "stale_pg_dist_object_record_for_a_db_exists": false}
(3 rows)
SELECT COUNT(*) FROM run_command_on_all_nodes('DROP DATABASE concurrent_create_db') WHERE success;
count
---------------------------------------------------------------------
1
(1 row)
SELECT * FROM public.check_database_on_all_nodes('concurrent_create_db') ORDER BY node_type;
node_type | result
---------------------------------------------------------------------
coordinator (local) | {"database_properties": null, "pg_dist_object_record_for_db_exists": false, "stale_pg_dist_object_record_for_a_db_exists": false}
worker node (remote) | {"database_properties": null, "pg_dist_object_record_for_db_exists": false, "stale_pg_dist_object_record_for_a_db_exists": false}
worker node (remote) | {"database_properties": null, "pg_dist_object_record_for_db_exists": false, "stale_pg_dist_object_record_for_a_db_exists": false}
(3 rows)
-- revert the system wide change that enables citus.enable_create_database_propagation on all nodes
SELECT 1 FROM run_command_on_all_nodes('ALTER SYSTEM SET citus.enable_create_database_propagation TO OFF');
?column?
---------------------------------------------------------------------
1
1
1
(3 rows)
SELECT 1 FROM run_command_on_all_nodes('SELECT pg_reload_conf()');
?column?
---------------------------------------------------------------------
1
1
1
(3 rows)
SELECT pg_sleep(0.1);
pg_sleep
---------------------------------------------------------------------
(1 row)
-- but keep it enabled for coordinator for the rest of the tests
SET citus.enable_create_database_propagation TO ON;
CREATE DATABASE distributed_db;
CREATE USER no_createdb;
SET ROLE no_createdb;
SET citus.enable_create_database_propagation TO ON;
CREATE DATABASE no_createdb;
ERROR: permission denied to create / rename database
ALTER DATABASE distributed_db RENAME TO rename_test;
ERROR: permission denied to create / rename database
DROP DATABASE distributed_db;
ERROR: must be owner of database distributed_db
ALTER DATABASE distributed_db SET TABLESPACE pg_default;
ERROR: must be owner of database distributed_db
ALTER DATABASE distributed_db SET timezone TO 'UTC';
ERROR: must be owner of database distributed_db
ALTER DATABASE distributed_db RESET timezone;
ERROR: must be owner of database distributed_db
GRANT ALL ON DATABASE distributed_db TO postgres;
WARNING: no privileges were granted for "distributed_db"
RESET ROLE;
ALTER ROLE no_createdb createdb;
SET ROLE no_createdb;
CREATE DATABASE no_createdb;
ALTER DATABASE distributed_db RENAME TO rename_test;
ERROR: must be owner of database distributed_db
RESET ROLE;
SELECT 1 FROM run_command_on_all_nodes($$GRANT ALL ON TABLESPACE pg_default TO no_createdb$$);
?column?
---------------------------------------------------------------------
1
1
1
(3 rows)
ALTER DATABASE distributed_db OWNER TO no_createdb;
SET ROLE no_createdb;
ALTER DATABASE distributed_db SET TABLESPACE pg_default;
ALTER DATABASE distributed_db SET timezone TO 'UTC';
ALTER DATABASE distributed_db RESET timezone;
GRANT ALL ON DATABASE distributed_db TO postgres;
ALTER DATABASE distributed_db RENAME TO rename_test;
DROP DATABASE rename_test;
RESET ROLE;
SELECT 1 FROM run_command_on_all_nodes($$REVOKE ALL ON TABLESPACE pg_default FROM no_createdb$$);
?column?
---------------------------------------------------------------------
1
1
1
(3 rows)
DROP DATABASE no_createdb;
DROP USER no_createdb;
SET citus.enable_create_database_propagation TO ON;
--clean up resources created by this test
-- DROP TABLESPACE is not supported, so we need to drop it manually.
SELECT result FROM run_command_on_all_nodes(

View File

@ -6,14 +6,14 @@ SET citus.shard_replication_factor TO 1;
SET citus.next_shard_id TO 35137400;
CREATE DATABASE citus_created;
NOTICE: Citus partially supports CREATE DATABASE for distributed databases
DETAIL: Citus does not propagate CREATE DATABASE command to workers
HINT: You can manually create a database and its extensions on workers.
DETAIL: Citus does not propagate CREATE DATABASE command to other nodes
HINT: You can manually create a database and its extensions on other nodes.
\c citus_created
CREATE EXTENSION citus;
CREATE DATABASE citus_not_created;
NOTICE: Citus partially supports CREATE DATABASE for distributed databases
DETAIL: Citus does not propagate CREATE DATABASE command to workers
HINT: You can manually create a database and its extensions on workers.
DETAIL: Citus does not propagate CREATE DATABASE command to other nodes
HINT: You can manually create a database and its extensions on other nodes.
\c citus_not_created
DROP DATABASE citus_created;
\c regression
@ -26,14 +26,14 @@ SET citus.shard_replication_factor TO 1;
SET citus.next_shard_id TO 35137400;
CREATE DATABASE citus_created;
NOTICE: Citus partially supports CREATE DATABASE for distributed databases
DETAIL: Citus does not propagate CREATE DATABASE command to workers
HINT: You can manually create a database and its extensions on workers.
DETAIL: Citus does not propagate CREATE DATABASE command to other nodes
HINT: You can manually create a database and its extensions on other nodes.
\c citus_created
CREATE EXTENSION citus;
CREATE DATABASE citus_not_created;
NOTICE: Citus partially supports CREATE DATABASE for distributed databases
DETAIL: Citus does not propagate CREATE DATABASE command to workers
HINT: You can manually create a database and its extensions on workers.
DETAIL: Citus does not propagate CREATE DATABASE command to other nodes
HINT: You can manually create a database and its extensions on other nodes.
\c citus_not_created
DROP DATABASE citus_created;
\c regression

View File

@ -0,0 +1,154 @@
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
---------------------------------------------------------------------
(1 row)
CREATE SCHEMA failure_non_main_db_2pc;
SET SEARCH_PATH TO 'failure_non_main_db_2pc';
CREATE DATABASE other_db1;
NOTICE: Citus partially supports CREATE DATABASE for distributed databases
DETAIL: Citus does not propagate CREATE DATABASE command to other nodes
HINT: You can manually create a database and its extensions on other nodes.
SELECT citus.mitmproxy('conn.onQuery(query="COMMIT PREPARED").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
\c other_db1
CREATE USER user_1;
\c regression
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT nodeid, result FROM run_command_on_all_nodes($$SELECT rolname FROM pg_roles WHERE rolname::TEXT = 'user_1'$$) ORDER BY 1;
nodeid | result
---------------------------------------------------------------------
0 | user_1
1 | user_1
2 |
(3 rows)
SELECT recover_prepared_transactions();
recover_prepared_transactions
---------------------------------------------------------------------
1
(1 row)
SELECT nodeid, result FROM run_command_on_all_nodes($$SELECT rolname FROM pg_roles WHERE rolname::TEXT = 'user_1'$$) ORDER BY 1;
nodeid | result
---------------------------------------------------------------------
0 | user_1
1 | user_1
2 | user_1
(3 rows)
SELECT citus.mitmproxy('conn.onQuery(query="CREATE USER user_2").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
\c other_db1
CREATE USER user_2;
ERROR: connection not open
CONTEXT: while executing command on localhost:xxxxx
while executing command on localhost:xxxxx
\c regression
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT nodeid, result FROM run_command_on_all_nodes($$SELECT rolname FROM pg_roles WHERE rolname::TEXT = 'user_2'$$) ORDER BY 1;
nodeid | result
---------------------------------------------------------------------
0 |
1 |
2 |
(3 rows)
SELECT recover_prepared_transactions();
recover_prepared_transactions
---------------------------------------------------------------------
0
(1 row)
SELECT nodeid, result FROM run_command_on_all_nodes($$SELECT rolname FROM pg_roles WHERE rolname::TEXT = 'user_2'$$) ORDER BY 1;
nodeid | result
---------------------------------------------------------------------
0 |
1 |
2 |
(3 rows)
DROP DATABASE other_db1;
-- user_2 should not exist because the query to create it will fail
-- but let's make sure we try to drop it just in case
DROP USER IF EXISTS user_1, user_2;
NOTICE: role "user_2" does not exist, skipping
SELECT citus_set_coordinator_host('localhost');
citus_set_coordinator_host
---------------------------------------------------------------------
(1 row)
\c - - - :worker_1_port
CREATE DATABASE other_db2;
NOTICE: Citus partially supports CREATE DATABASE for distributed databases
DETAIL: Citus does not propagate CREATE DATABASE command to other nodes
HINT: You can manually create a database and its extensions on other nodes.
SELECT citus.mitmproxy('conn.onQuery(query="COMMIT PREPARED").kill()');
mitmproxy
---------------------------------------------------------------------
(1 row)
\c other_db2
CREATE USER user_3;
\c regression
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
---------------------------------------------------------------------
(1 row)
SELECT result FROM run_command_on_all_nodes($$SELECT rolname FROM pg_roles WHERE rolname::TEXT = 'user_3'$$) ORDER BY 1;
result
---------------------------------------------------------------------
user_3
user_3
(3 rows)
SELECT recover_prepared_transactions();
recover_prepared_transactions
---------------------------------------------------------------------
1
(1 row)
SELECT result FROM run_command_on_all_nodes($$SELECT rolname FROM pg_roles WHERE rolname::TEXT = 'user_3'$$) ORDER BY 1;
result
---------------------------------------------------------------------
user_3
user_3
user_3
(3 rows)
DROP DATABASE other_db2;
DROP USER user_3;
\c - - - :master_port
SELECT result FROM run_command_on_all_nodes($$DELETE FROM pg_dist_node WHERE groupid = 0$$);
result
---------------------------------------------------------------------
DELETE 1
DELETE 1
DELETE 1
(3 rows)
DROP SCHEMA failure_non_main_db_2pc;

View File

@ -210,6 +210,7 @@ select create_distributed_table('partitioned_tbl_with_fkey','x');
create table partition_1_with_fkey partition of partitioned_tbl_with_fkey for values from ('2022-01-01') to ('2022-12-31');
create table partition_2_with_fkey partition of partitioned_tbl_with_fkey for values from ('2023-01-01') to ('2023-12-31');
create table partition_3_with_fkey partition of partitioned_tbl_with_fkey DEFAULT;
insert into partitioned_tbl_with_fkey (x,y) select s,s%10 from generate_series(1,100) s;
ALTER TABLE partitioned_tbl_with_fkey ADD CONSTRAINT fkey_to_ref_tbl FOREIGN KEY (y) REFERENCES ref_table_with_fkey(id);
WITH shardid AS (SELECT shardid FROM pg_dist_shard where logicalrelid = 'partitioned_tbl_with_fkey'::regclass ORDER BY shardid LIMIT 1)

View File

@ -542,8 +542,8 @@ create user myuser;
create user myuser_1;
create database test_db;
NOTICE: Citus partially supports CREATE DATABASE for distributed databases
DETAIL: Citus does not propagate CREATE DATABASE command to workers
HINT: You can manually create a database and its extensions on workers.
DETAIL: Citus does not propagate CREATE DATABASE command to other nodes
HINT: You can manually create a database and its extensions on other nodes.
SELECT result FROM run_command_on_workers($$create database test_db$$);
result
---------------------------------------------------------------------
@ -677,7 +677,7 @@ select has_database_privilege('myuser','regression', 'TEMPORARY');
select has_database_privilege('myuser','test_db', 'CREATE');
has_database_privilege
---------------------------------------------------------------------
t
f
(1 row)
select has_database_privilege('myuser','test_db', 'CONNECT');
@ -725,7 +725,7 @@ select has_database_privilege('myuser_1','regression', 'TEMPORARY');
select has_database_privilege('myuser_1','test_db', 'CREATE');
has_database_privilege
---------------------------------------------------------------------
t
f
(1 row)
select has_database_privilege('myuser_1','test_db', 'CONNECT');
@ -884,19 +884,19 @@ select has_database_privilege('myuser','test_db', 'CREATE');
select has_database_privilege('myuser','test_db', 'CONNECT');
has_database_privilege
---------------------------------------------------------------------
f
t
(1 row)
select has_database_privilege('myuser','test_db', 'TEMP');
has_database_privilege
---------------------------------------------------------------------
f
t
(1 row)
select has_database_privilege('myuser','test_db', 'TEMPORARY');
has_database_privilege
---------------------------------------------------------------------
f
t
(1 row)
select has_database_privilege('myuser_1','regression', 'CREATE');
@ -932,19 +932,19 @@ select has_database_privilege('myuser_1','test_db', 'CREATE');
select has_database_privilege('myuser_1','test_db', 'CONNECT');
has_database_privilege
---------------------------------------------------------------------
f
t
(1 row)
select has_database_privilege('myuser_1','test_db', 'TEMP');
has_database_privilege
---------------------------------------------------------------------
f
t
(1 row)
select has_database_privilege('myuser_1','test_db', 'TEMPORARY');
has_database_privilege
---------------------------------------------------------------------
f
t
(1 row)
\c - - - :master_port

View File

@ -0,0 +1,211 @@
Parsed test spec with 2 sessions
starting permutation: s1-begin s2-begin s1-acquire-citus-adv-oclass-lock s2-acquire-citus-adv-oclass-lock s1-commit s2-commit
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-acquire-citus-adv-oclass-lock: SELECT citus_internal_acquire_citus_advisory_object_class_lock(value, NULL) FROM oclass_database;
citus_internal_acquire_citus_advisory_object_class_lock
---------------------------------------------------------------------
(1 row)
step s2-acquire-citus-adv-oclass-lock: SELECT citus_internal_acquire_citus_advisory_object_class_lock(value, NULL) FROM oclass_database; <waiting ...>
step s1-commit: COMMIT;
step s2-acquire-citus-adv-oclass-lock: <... completed>
citus_internal_acquire_citus_advisory_object_class_lock
---------------------------------------------------------------------
(1 row)
step s2-commit: COMMIT;
starting permutation: s1-create-testdb1 s1-begin s2-begin s1-acquire-citus-adv-oclass-lock-with-oid-testdb1 s2-acquire-citus-adv-oclass-lock-with-oid-testdb1 s1-commit s2-commit s1-drop-testdb1
step s1-create-testdb1: CREATE DATABASE testdb1;
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-acquire-citus-adv-oclass-lock-with-oid-testdb1: SELECT citus_internal_acquire_citus_advisory_object_class_lock(value, 'testdb1') FROM oclass_database;
citus_internal_acquire_citus_advisory_object_class_lock
---------------------------------------------------------------------
(1 row)
step s2-acquire-citus-adv-oclass-lock-with-oid-testdb1: SELECT citus_internal_acquire_citus_advisory_object_class_lock(value, 'testdb1') FROM oclass_database; <waiting ...>
step s1-commit: COMMIT;
step s2-acquire-citus-adv-oclass-lock-with-oid-testdb1: <... completed>
citus_internal_acquire_citus_advisory_object_class_lock
---------------------------------------------------------------------
(1 row)
step s2-commit: COMMIT;
step s1-drop-testdb1: DROP DATABASE testdb1;
starting permutation: s1-create-testdb1 s2-create-testdb2 s1-begin s2-begin s1-acquire-citus-adv-oclass-lock-with-oid-testdb1 s2-acquire-citus-adv-oclass-lock-with-oid-testdb2 s1-commit s2-commit s1-drop-testdb1 s2-drop-testdb2
step s1-create-testdb1: CREATE DATABASE testdb1;
step s2-create-testdb2: CREATE DATABASE testdb2;
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-acquire-citus-adv-oclass-lock-with-oid-testdb1: SELECT citus_internal_acquire_citus_advisory_object_class_lock(value, 'testdb1') FROM oclass_database;
citus_internal_acquire_citus_advisory_object_class_lock
---------------------------------------------------------------------
(1 row)
step s2-acquire-citus-adv-oclass-lock-with-oid-testdb2: SELECT citus_internal_acquire_citus_advisory_object_class_lock(value, 'testdb2') FROM oclass_database;
citus_internal_acquire_citus_advisory_object_class_lock
---------------------------------------------------------------------
(1 row)
step s1-commit: COMMIT;
step s2-commit: COMMIT;
step s1-drop-testdb1: DROP DATABASE testdb1;
step s2-drop-testdb2: DROP DATABASE testdb2;
starting permutation: s2-create-testdb2 s1-begin s2-begin s1-acquire-citus-adv-oclass-lock s2-acquire-citus-adv-oclass-lock-with-oid-testdb2 s1-commit s2-commit s2-drop-testdb2
step s2-create-testdb2: CREATE DATABASE testdb2;
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-acquire-citus-adv-oclass-lock: SELECT citus_internal_acquire_citus_advisory_object_class_lock(value, NULL) FROM oclass_database;
citus_internal_acquire_citus_advisory_object_class_lock
---------------------------------------------------------------------
(1 row)
step s2-acquire-citus-adv-oclass-lock-with-oid-testdb2: SELECT citus_internal_acquire_citus_advisory_object_class_lock(value, 'testdb2') FROM oclass_database;
citus_internal_acquire_citus_advisory_object_class_lock
---------------------------------------------------------------------
(1 row)
step s1-commit: COMMIT;
step s2-commit: COMMIT;
step s2-drop-testdb2: DROP DATABASE testdb2;
starting permutation: s2-create-testdb2 s2-begin s2-alter-testdb2-set-lc_monetary s1-create-db1 s2-rollback s2-drop-testdb2 s1-drop-db1
step s2-create-testdb2: CREATE DATABASE testdb2;
step s2-begin: BEGIN;
step s2-alter-testdb2-set-lc_monetary: ALTER DATABASE testdb2 SET lc_monetary TO 'C';
step s1-create-db1: CREATE DATABASE db1;
step s2-rollback: ROLLBACK;
step s2-drop-testdb2: DROP DATABASE testdb2;
step s1-drop-db1: DROP DATABASE db1;
starting permutation: s2-create-testdb2 s2-begin s2-alter-testdb2-set-lc_monetary s1-create-user-dbuser s1-grant-on-testdb2-to-dbuser s2-rollback s2-drop-testdb2 s1-drop-user-dbuser
step s2-create-testdb2: CREATE DATABASE testdb2;
step s2-begin: BEGIN;
step s2-alter-testdb2-set-lc_monetary: ALTER DATABASE testdb2 SET lc_monetary TO 'C';
step s1-create-user-dbuser: CREATE USER dbuser;
step s1-grant-on-testdb2-to-dbuser: GRANT ALL ON DATABASE testdb2 TO dbuser;
step s2-rollback: ROLLBACK;
step s2-drop-testdb2: DROP DATABASE testdb2;
step s1-drop-user-dbuser: DROP USER dbuser;
starting permutation: s2-create-testdb2 s2-begin s2-alter-testdb2-set-lc_monetary s1-create-testdb1 s1-create-user-dbuser s1-grant-on-testdb1-to-dbuser s2-rollback s2-drop-testdb2 s1-drop-testdb1 s1-drop-user-dbuser
step s2-create-testdb2: CREATE DATABASE testdb2;
step s2-begin: BEGIN;
step s2-alter-testdb2-set-lc_monetary: ALTER DATABASE testdb2 SET lc_monetary TO 'C';
step s1-create-testdb1: CREATE DATABASE testdb1;
step s1-create-user-dbuser: CREATE USER dbuser;
step s1-grant-on-testdb1-to-dbuser: GRANT ALL ON DATABASE testdb1 TO dbuser;
step s2-rollback: ROLLBACK;
step s2-drop-testdb2: DROP DATABASE testdb2;
step s1-drop-testdb1: DROP DATABASE testdb1;
step s1-drop-user-dbuser: DROP USER dbuser;
starting permutation: s1-create-testdb1 s2-create-testdb2 s1-begin s2-begin s1-alter-testdb1-rename-to-db1 s2-alter-testdb2-rename-to-db1 s1-commit s2-rollback s1-drop-db1 s2-drop-testdb2
step s1-create-testdb1: CREATE DATABASE testdb1;
step s2-create-testdb2: CREATE DATABASE testdb2;
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-alter-testdb1-rename-to-db1: ALTER DATABASE testdb1 RENAME TO db1;
step s2-alter-testdb2-rename-to-db1: ALTER DATABASE testdb2 RENAME TO db1; <waiting ...>
step s1-commit: COMMIT;
step s2-alter-testdb2-rename-to-db1: <... completed>
ERROR: database "db1" already exists
step s2-rollback: ROLLBACK;
step s1-drop-db1: DROP DATABASE db1;
step s2-drop-testdb2: DROP DATABASE testdb2;
starting permutation: s1-create-testdb1 s2-create-testdb2 s1-begin s2-begin s1-alter-testdb1-rename-to-db1 s2-alter-testdb2-rename-to-db1 s1-rollback s2-commit s1-drop-testdb1 s2-drop-db1
step s1-create-testdb1: CREATE DATABASE testdb1;
step s2-create-testdb2: CREATE DATABASE testdb2;
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-alter-testdb1-rename-to-db1: ALTER DATABASE testdb1 RENAME TO db1;
step s2-alter-testdb2-rename-to-db1: ALTER DATABASE testdb2 RENAME TO db1; <waiting ...>
step s1-rollback: ROLLBACK;
step s2-alter-testdb2-rename-to-db1: <... completed>
step s2-commit: COMMIT;
step s1-drop-testdb1: DROP DATABASE testdb1;
step s2-drop-db1: DROP DATABASE db1;
starting permutation: s1-create-testdb1 s1-begin s2-begin s1-alter-testdb1-rename-to-db1 s2-alter-testdb1-rename-to-db1 s1-commit s2-rollback s1-drop-db1
step s1-create-testdb1: CREATE DATABASE testdb1;
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-alter-testdb1-rename-to-db1: ALTER DATABASE testdb1 RENAME TO db1;
step s2-alter-testdb1-rename-to-db1: ALTER DATABASE testdb1 RENAME TO db1; <waiting ...>
step s1-commit: COMMIT;
step s2-alter-testdb1-rename-to-db1: <... completed>
ERROR: database "testdb1" does not exist
step s2-rollback: ROLLBACK;
step s1-drop-db1: DROP DATABASE db1;
starting permutation: s1-create-testdb1 s1-begin s2-begin s1-alter-testdb1-rename-to-db1 s2-alter-testdb1-rename-to-db1 s1-rollback s2-commit s2-drop-db1
step s1-create-testdb1: CREATE DATABASE testdb1;
step s1-begin: BEGIN;
step s2-begin: BEGIN;
step s1-alter-testdb1-rename-to-db1: ALTER DATABASE testdb1 RENAME TO db1;
step s2-alter-testdb1-rename-to-db1: ALTER DATABASE testdb1 RENAME TO db1; <waiting ...>
step s1-rollback: ROLLBACK;
step s2-alter-testdb1-rename-to-db1: <... completed>
step s2-commit: COMMIT;
step s2-drop-db1: DROP DATABASE db1;
starting permutation: s2-create-testdb2 s2-begin s2-alter-testdb2-rename-to-db1 s1-create-db1 s2-rollback s2-drop-testdb2 s1-drop-db1
step s2-create-testdb2: CREATE DATABASE testdb2;
step s2-begin: BEGIN;
step s2-alter-testdb2-rename-to-db1: ALTER DATABASE testdb2 RENAME TO db1;
step s1-create-db1: CREATE DATABASE db1; <waiting ...>
step s2-rollback: ROLLBACK;
step s1-create-db1: <... completed>
step s2-drop-testdb2: DROP DATABASE testdb2;
step s1-drop-db1: DROP DATABASE db1;
starting permutation: s2-create-testdb2 s2-begin s2-alter-testdb2-rename-to-db1 s1-create-db1 s2-commit s2-drop-db1
step s2-create-testdb2: CREATE DATABASE testdb2;
step s2-begin: BEGIN;
step s2-alter-testdb2-rename-to-db1: ALTER DATABASE testdb2 RENAME TO db1;
step s1-create-db1: CREATE DATABASE db1; <waiting ...>
step s2-commit: COMMIT;
step s1-create-db1: <... completed>
ERROR: database "db1" already exists
step s2-drop-db1: DROP DATABASE db1;
starting permutation: s2-create-testdb2 s2-begin s2-alter-testdb2-rename-to-db2 s1-create-db1 s2-commit s2-drop-db2 s1-drop-db1
step s2-create-testdb2: CREATE DATABASE testdb2;
step s2-begin: BEGIN;
step s2-alter-testdb2-rename-to-db2: ALTER DATABASE testdb2 RENAME TO db2;
step s1-create-db1: CREATE DATABASE db1; <waiting ...>
step s2-commit: COMMIT;
step s1-create-db1: <... completed>
step s2-drop-db2: DROP DATABASE db2;
step s1-drop-db1: DROP DATABASE db1;
starting permutation: s2-create-testdb2 s2-begin s2-alter-testdb2-rename-to-db1 s1-drop-testdb2 s2-rollback
step s2-create-testdb2: CREATE DATABASE testdb2;
step s2-begin: BEGIN;
step s2-alter-testdb2-rename-to-db1: ALTER DATABASE testdb2 RENAME TO db1;
step s1-drop-testdb2: DROP DATABASE testdb2; <waiting ...>
step s2-rollback: ROLLBACK;
step s1-drop-testdb2: <... completed>
starting permutation: s2-create-testdb2 s1-create-db1 s2-begin s2-alter-testdb2-rename-to-db2 s1-drop-db1 s2-commit s2-drop-db2
step s2-create-testdb2: CREATE DATABASE testdb2;
step s1-create-db1: CREATE DATABASE db1;
step s2-begin: BEGIN;
step s2-alter-testdb2-rename-to-db2: ALTER DATABASE testdb2 RENAME TO db2;
step s1-drop-db1: DROP DATABASE db1;
step s2-commit: COMMIT;
step s2-drop-db2: DROP DATABASE db2;

View File

@ -93,8 +93,8 @@ nodeid|nodename|nodeport
starting permutation: s1-begin s1-update-node-1 s2-begin s2-update-node-1 s1-commit s2-abort s1-show-nodes s3-update-node-1-back s3-manually-fix-metadata
nodeid|nodename |nodeport
---------------------------------------------------------------------
25|localhost| 57638
24|localhost| 57637
23|localhost| 57638
22|localhost| 57637
(2 rows)
step s1-begin:
@ -139,8 +139,8 @@ step s1-show-nodes:
nodeid|nodename |nodeport|isactive
---------------------------------------------------------------------
25|localhost| 57638|t
24|localhost| 58637|t
23|localhost| 57638|t
22|localhost| 58637|t
(2 rows)
step s3-update-node-1-back:
@ -178,8 +178,8 @@ nodeid|nodename|nodeport
starting permutation: s2-create-table s1-begin s1-update-node-nonexistent s1-prepare-transaction s2-cache-prepared-statement s1-commit-prepared s2-execute-prepared s1-update-node-existent s3-manually-fix-metadata
nodeid|nodename |nodeport
---------------------------------------------------------------------
27|localhost| 57638
26|localhost| 57637
23|localhost| 57638
22|localhost| 57637
(2 rows)
step s2-create-table:
@ -250,7 +250,7 @@ count
step s1-commit-prepared:
COMMIT prepared 'label';
s2: WARNING: connection to the remote node non-existent:57637 failed with the following error: could not translate host name "non-existent" to address: Name or service not known
s2: WARNING: connection to the remote node non-existent:57637 failed with the following error: could not translate host name "non-existent" to address: <system specific error>
step s2-execute-prepared:
EXECUTE foo;

View File

@ -16,7 +16,8 @@ SELECT cte.user_id, cte.value_2 FROM cte,cte2 ORDER BY 1,2 LIMIT 10;
ERROR: the intermediate result size exceeds citus.max_intermediate_result_size (currently 2 kB)
DETAIL: Citus restricts the size of intermediate results of complex subqueries and CTEs to avoid accidentally pulling large result sets into once place.
HINT: To run the current query, set citus.max_intermediate_result_size to a higher value or -1 to disable.
SET citus.max_intermediate_result_size TO 17;
SET citus.max_intermediate_result_size TO 9;
-- regular adaptive executor CTE should fail
WITH cte AS MATERIALIZED
(
SELECT
@ -38,20 +39,9 @@ FROM
ORDER BY
1,2
LIMIT 10;
user_id | value_2
---------------------------------------------------------------------
1 | 0
1 | 0
1 | 0
1 | 0
1 | 0
1 | 0
1 | 0
1 | 0
1 | 0
1 | 0
(10 rows)
ERROR: the intermediate result size exceeds citus.max_intermediate_result_size (currently 9 kB)
DETAIL: Citus restricts the size of intermediate results of complex subqueries and CTEs to avoid accidentally pulling large result sets into once place.
HINT: To run the current query, set citus.max_intermediate_result_size to a higher value or -1 to disable.
-- router queries should be able to get limitted too
SET citus.max_intermediate_result_size TO 2;
-- this should pass, since we fetch small portions in each subplan
@ -117,11 +107,9 @@ WITH cte AS MATERIALIZED (
AND EXISTS (select * from cte2, cte3)
)
SELECT count(*) FROM cte WHERE EXISTS (select * from cte);
count
---------------------------------------------------------------------
105
(1 row)
ERROR: the intermediate result size exceeds citus.max_intermediate_result_size (currently 4 kB)
DETAIL: Citus restricts the size of intermediate results of complex subqueries and CTEs to avoid accidentally pulling large result sets into once place.
HINT: To run the current query, set citus.max_intermediate_result_size to a higher value or -1 to disable.
SET citus.max_intermediate_result_size TO 3;
-- this should fail since the cte-subplan exceeds the limit even if the
-- cte2 and cte3 does not

View File

@ -1349,6 +1349,77 @@ SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.p
(schema,{test_schema_for_sequence_propagation},{})
(1 row)
-- Bug: https://github.com/citusdata/citus/issues/7378
-- Create a reference table
CREATE TABLE tbl_ref(row_id integer primary key);
INSERT INTO tbl_ref VALUES (1), (2);
SELECT create_reference_table('tbl_ref');
NOTICE: Copying data from local table...
NOTICE: copying the data has completed
DETAIL: The local data in the table is no longer visible, but is still on disk.
HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$multi_alter_table_statements.tbl_ref$$)
create_reference_table
---------------------------------------------------------------------
(1 row)
-- Create a distributed table
CREATE TABLE tbl_dist(series_id integer);
INSERT INTO tbl_dist VALUES (1), (1), (2), (2);
SELECT create_distributed_table('tbl_dist', 'series_id');
NOTICE: Copying data from local table...
NOTICE: copying the data has completed
DETAIL: The local data in the table is no longer visible, but is still on disk.
HINT: To remove the local data, run: SELECT truncate_local_data_after_distributing_table($$multi_alter_table_statements.tbl_dist$$)
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- Create a view that joins the distributed table with the reference table on the distribution key.
CREATE VIEW vw_citus_views as
SELECT d.series_id FROM tbl_dist d JOIN tbl_ref r ON d.series_id = r.row_id;
-- The view initially works fine
SELECT * FROM vw_citus_views ORDER BY 1;
series_id
---------------------------------------------------------------------
1
1
2
2
(4 rows)
-- Now, alter the table
ALTER TABLE tbl_ref ADD COLUMN category1 varchar(50);
SELECT * FROM vw_citus_views ORDER BY 1;
series_id
---------------------------------------------------------------------
1
1
2
2
(4 rows)
ALTER TABLE tbl_ref ADD COLUMN category2 varchar(50);
SELECT * FROM vw_citus_views ORDER BY 1;
series_id
---------------------------------------------------------------------
1
1
2
2
(4 rows)
ALTER TABLE tbl_ref DROP COLUMN category1;
SELECT * FROM vw_citus_views ORDER BY 1;
series_id
---------------------------------------------------------------------
1
1
2
2
(4 rows)
SET client_min_messages TO WARNING;
DROP SCHEMA test_schema_for_sequence_propagation CASCADE;
DROP TABLE table_without_sequence;

View File

@ -1420,10 +1420,15 @@ SELECT * FROM multi_extension.print_extension_changes();
-- Snapshot of state at 12.2-1
ALTER EXTENSION citus UPDATE TO '12.2-1';
SELECT * FROM multi_extension.print_extension_changes();
previous_object | current_object
previous_object | current_object
---------------------------------------------------------------------
| function citus_internal.commit_management_command_2pc() void
| function citus_internal.execute_command_on_remote_nodes_as_user(text,text) void
| function citus_internal.mark_object_distributed(oid,text,oid) void
| function citus_internal.start_management_transaction(xid8) void
| function citus_internal_acquire_citus_advisory_object_class_lock(integer,cstring) void
| function citus_internal_database_command(text) void
(1 row)
(6 rows)
DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff;
-- show running version
@ -1619,8 +1624,8 @@ CREATE EXTENSION citus;
-- Check that maintenance daemon can also be started in another database
CREATE DATABASE another;
NOTICE: Citus partially supports CREATE DATABASE for distributed databases
DETAIL: Citus does not propagate CREATE DATABASE command to workers
HINT: You can manually create a database and its extensions on workers.
DETAIL: Citus does not propagate CREATE DATABASE command to other nodes
HINT: You can manually create a database and its extensions on other nodes.
\c another
CREATE EXTENSION citus;
CREATE SCHEMA test;
@ -1678,13 +1683,13 @@ NOTICE: drop cascades to function test_daemon.maintenance_daemon_died(text)
-- create a test database, configure citus with single node
CREATE DATABASE another;
NOTICE: Citus partially supports CREATE DATABASE for distributed databases
DETAIL: Citus does not propagate CREATE DATABASE command to workers
HINT: You can manually create a database and its extensions on workers.
DETAIL: Citus does not propagate CREATE DATABASE command to other nodes
HINT: You can manually create a database and its extensions on other nodes.
\c - - - :worker_1_port
CREATE DATABASE another;
NOTICE: Citus partially supports CREATE DATABASE for distributed databases
DETAIL: Citus does not propagate CREATE DATABASE command to workers
HINT: You can manually create a database and its extensions on workers.
DETAIL: Citus does not propagate CREATE DATABASE command to other nodes
HINT: You can manually create a database and its extensions on other nodes.
\c - - - :master_port
\c another
CREATE EXTENSION citus;

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