Merge branch 'master' into velioglu/function_propagation

velioglu/tmpfuncprop
Burak Velioglu 2022-02-12 23:43:21 +03:00
commit e249aa2d62
No known key found for this signature in database
GPG Key ID: F6827E620F6549C6
130 changed files with 3212 additions and 1181 deletions

View File

@ -396,6 +396,15 @@ ReplicateAllObjectsToNodeCommandList(const char *nodeName, int nodePort)
ObjectAddress *dependency = NULL;
foreach_ptr(dependency, dependencies)
{
if (IsObjectAddressOwnedByExtension(dependency, NULL))
{
/*
* we expect extension-owned objects to be created as a result
* of the extension being created.
*/
continue;
}
ddlCommands = list_concat(ddlCommands,
GetDependencyCreateDDLCommands(dependency));
}

View File

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

View File

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

View File

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

View File

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

View File

@ -19,6 +19,7 @@
#include "access/hash.h"
#include "commands/dbcommands.h"
#include "distributed/backend_data.h"
#include "distributed/connection_management.h"
#include "distributed/errormessage.h"
#include "distributed/error_codes.h"
@ -1459,7 +1460,7 @@ IsRebalancerInternalBackend(void)
bool
IsCitusInternalBackend(void)
{
return application_name && strcmp(application_name, CITUS_APPLICATION_NAME) == 0;
return ExtractGlobalPID(application_name) != INVALID_CITUS_INTERNAL_BACKEND_GPID;
}

View File

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

View File

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

View File

@ -313,16 +313,6 @@ TryToDelegateFunctionCall(DistributedPlanningContext *planContext)
return NULL;
}
if (fromFuncExpr && !IsMultiStatementTransaction())
{
/*
* For now, let's not push the function from the FROM clause unless it's in a
* multistatement transaction with the forceDelegation flag ON.
*/
ereport(DEBUG2, (errmsg("function from the FROM clause is not pushed")));
return NULL;
}
/* dissuade the planner from trying a generic plan with parameters */
(void) expression_tree_walker((Node *) funcExpr->args, contain_param_walker,
&walkerParamContext);

View File

@ -458,6 +458,7 @@ StartupCitusBackend(void)
InitializeMaintenanceDaemonBackend();
InitializeBackendData();
RegisterConnectionCleanup();
AssignGlobalPID();
}
@ -1983,7 +1984,6 @@ NodeConninfoGucCheckHook(char **newval, void **extra, GucSource source)
{
/* this array _must_ be kept in an order usable by bsearch */
const char *allowedConninfoKeywords[] = {
"application_name",
"connect_timeout",
#if defined(ENABLE_GSS) && defined(ENABLE_SSPI)
"gsslib",

View File

@ -14,6 +14,25 @@
#include "udfs/worker_drop_sequence_dependency/11.0-1.sql"
#include "udfs/worker_drop_shell_table/11.0-1.sql"
#include "udfs/get_all_active_transactions/11.0-1.sql"
#include "udfs/get_global_active_transactions/11.0-1.sql"
#include "udfs/citus_worker_stat_activity/11.0-1.sql"
CREATE VIEW citus.citus_worker_stat_activity AS
SELECT * FROM pg_catalog.citus_worker_stat_activity();
ALTER VIEW citus.citus_worker_stat_activity SET SCHEMA pg_catalog;
GRANT SELECT ON pg_catalog.citus_worker_stat_activity TO PUBLIC;
#include "udfs/citus_dist_stat_activity/11.0-1.sql"
CREATE VIEW citus.citus_dist_stat_activity AS
SELECT * FROM pg_catalog.citus_dist_stat_activity();
ALTER VIEW citus.citus_dist_stat_activity SET SCHEMA pg_catalog;
GRANT SELECT ON pg_catalog.citus_dist_stat_activity TO PUBLIC;
-- we have to recreate this view because recreated citus_dist_stat_activity that this view depends
#include "udfs/citus_lock_waits/11.0-1.sql"
DROP FUNCTION IF EXISTS pg_catalog.master_apply_delete_command(text);
DROP FUNCTION pg_catalog.master_get_table_metadata(text);

View File

@ -84,3 +84,128 @@ DROP FUNCTION pg_catalog.citus_shards_on_worker();
DROP FUNCTION pg_catalog.citus_shard_indexes_on_worker();
#include "../udfs/create_distributed_function/9.0-1.sql"
ALTER TABLE citus.pg_dist_object DROP COLUMN force_delegation;
SET search_path = 'pg_catalog';
DROP FUNCTION IF EXISTS get_all_active_transactions();
CREATE OR REPLACE FUNCTION get_all_active_transactions(OUT datid oid, OUT process_id int, OUT initiator_node_identifier int4, OUT worker_query BOOL,
OUT transaction_number int8, OUT transaction_stamp timestamptz)
RETURNS SETOF RECORD
LANGUAGE C STRICT AS 'MODULE_PATHNAME',
$$get_all_active_transactions$$;
COMMENT ON FUNCTION get_all_active_transactions(OUT datid oid, OUT datname text, OUT process_id int, OUT initiator_node_identifier int4, OUT worker_query BOOL,
OUT transaction_number int8, OUT transaction_stamp timestamptz)
IS 'returns distributed transaction ids of active distributed transactions';
DROP FUNCTION IF EXISTS get_global_active_transactions();
CREATE FUNCTION get_global_active_transactions(OUT datid oid, OUT process_id int, OUT initiator_node_identifier int4, OUT worker_query BOOL, OUT transaction_number int8, OUT transaction_stamp timestamptz)
RETURNS SETOF RECORD
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$get_global_active_transactions$$;
COMMENT ON FUNCTION get_global_active_transactions(OUT database_id oid, OUT process_id int, OUT initiator_node_identifier int4, OUT transaction_number int8, OUT transaction_stamp timestamptz)
IS 'returns distributed transaction ids of active distributed transactions from each node of the cluster';
RESET search_path;
DROP FUNCTION pg_catalog.citus_dist_stat_activity CASCADE;
CREATE OR REPLACE FUNCTION pg_catalog.citus_dist_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int,
OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name,
OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET,
OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz,
OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text,
OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text)
RETURNS SETOF RECORD
LANGUAGE C STRICT AS 'MODULE_PATHNAME',
$$citus_dist_stat_activity$$;
COMMENT ON FUNCTION pg_catalog.citus_dist_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int,
OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name,
OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET,
OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz,
OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text,
OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text)
IS 'returns distributed transaction activity on distributed tables';
CREATE VIEW citus.citus_dist_stat_activity AS
SELECT * FROM pg_catalog.citus_dist_stat_activity();
ALTER VIEW citus.citus_dist_stat_activity SET SCHEMA pg_catalog;
GRANT SELECT ON pg_catalog.citus_dist_stat_activity TO PUBLIC;
SET search_path = 'pg_catalog';
-- we have to recreate this view because we drop citus_dist_stat_activity that this view depends
CREATE VIEW citus.citus_lock_waits AS
WITH
citus_dist_stat_activity AS
(
SELECT * FROM citus_dist_stat_activity
),
unique_global_wait_edges AS
(
SELECT DISTINCT ON(waiting_node_id, waiting_transaction_num, blocking_node_id, blocking_transaction_num) * FROM dump_global_wait_edges()
),
citus_dist_stat_activity_with_node_id AS
(
SELECT
citus_dist_stat_activity.*, (CASE citus_dist_stat_activity.distributed_query_host_name WHEN 'coordinator_host' THEN 0 ELSE pg_dist_node.nodeid END) as initiator_node_id
FROM
citus_dist_stat_activity LEFT JOIN pg_dist_node
ON
citus_dist_stat_activity.distributed_query_host_name = pg_dist_node.nodename AND
citus_dist_stat_activity.distributed_query_host_port = pg_dist_node.nodeport
)
SELECT
waiting.pid AS waiting_pid,
blocking.pid AS blocking_pid,
waiting.query AS blocked_statement,
blocking.query AS current_statement_in_blocking_process,
waiting.initiator_node_id AS waiting_node_id,
blocking.initiator_node_id AS blocking_node_id,
waiting.distributed_query_host_name AS waiting_node_name,
blocking.distributed_query_host_name AS blocking_node_name,
waiting.distributed_query_host_port AS waiting_node_port,
blocking.distributed_query_host_port AS blocking_node_port
FROM
unique_global_wait_edges
JOIN
citus_dist_stat_activity_with_node_id waiting ON (unique_global_wait_edges.waiting_transaction_num = waiting.transaction_number AND unique_global_wait_edges.waiting_node_id = waiting.initiator_node_id)
JOIN
citus_dist_stat_activity_with_node_id blocking ON (unique_global_wait_edges.blocking_transaction_num = blocking.transaction_number AND unique_global_wait_edges.blocking_node_id = blocking.initiator_node_id);
ALTER VIEW citus.citus_lock_waits SET SCHEMA pg_catalog;
GRANT SELECT ON pg_catalog.citus_lock_waits TO PUBLIC;
DROP FUNCTION citus_worker_stat_activity CASCADE;
CREATE OR REPLACE FUNCTION citus_worker_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int,
OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name,
OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET,
OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz,
OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text,
OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text)
RETURNS SETOF RECORD
LANGUAGE C STRICT AS 'MODULE_PATHNAME',
$$citus_worker_stat_activity$$;
COMMENT ON FUNCTION citus_worker_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int,
OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name,
OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET,
OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz,
OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text,
OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text)
IS 'returns distributed transaction activity on shards of distributed tables';
CREATE VIEW citus.citus_worker_stat_activity AS
SELECT * FROM pg_catalog.citus_worker_stat_activity();
ALTER VIEW citus.citus_worker_stat_activity SET SCHEMA pg_catalog;
GRANT SELECT ON pg_catalog.citus_worker_stat_activity TO PUBLIC;
RESET search_path;

View File

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

View File

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

View File

@ -0,0 +1,44 @@
SET search_path = 'pg_catalog';
CREATE VIEW citus.citus_lock_waits AS
WITH
citus_dist_stat_activity AS
(
SELECT * FROM citus_dist_stat_activity
),
unique_global_wait_edges AS
(
SELECT DISTINCT ON(waiting_node_id, waiting_transaction_num, blocking_node_id, blocking_transaction_num) * FROM dump_global_wait_edges()
),
citus_dist_stat_activity_with_node_id AS
(
SELECT
citus_dist_stat_activity.*, (CASE citus_dist_stat_activity.distributed_query_host_name WHEN 'coordinator_host' THEN 0 ELSE pg_dist_node.nodeid END) as initiator_node_id
FROM
citus_dist_stat_activity LEFT JOIN pg_dist_node
ON
citus_dist_stat_activity.distributed_query_host_name = pg_dist_node.nodename AND
citus_dist_stat_activity.distributed_query_host_port = pg_dist_node.nodeport
)
SELECT
waiting.pid AS waiting_pid,
blocking.pid AS blocking_pid,
waiting.query AS blocked_statement,
blocking.query AS current_statement_in_blocking_process,
waiting.initiator_node_id AS waiting_node_id,
blocking.initiator_node_id AS blocking_node_id,
waiting.distributed_query_host_name AS waiting_node_name,
blocking.distributed_query_host_name AS blocking_node_name,
waiting.distributed_query_host_port AS waiting_node_port,
blocking.distributed_query_host_port AS blocking_node_port
FROM
unique_global_wait_edges
JOIN
citus_dist_stat_activity_with_node_id waiting ON (unique_global_wait_edges.waiting_transaction_num = waiting.transaction_number AND unique_global_wait_edges.waiting_node_id = waiting.initiator_node_id)
JOIN
citus_dist_stat_activity_with_node_id blocking ON (unique_global_wait_edges.blocking_transaction_num = blocking.transaction_number AND unique_global_wait_edges.blocking_node_id = blocking.initiator_node_id);
ALTER VIEW citus.citus_lock_waits SET SCHEMA pg_catalog;
GRANT SELECT ON pg_catalog.citus_lock_waits TO PUBLIC;
RESET search_path;

View File

@ -0,0 +1,44 @@
SET search_path = 'pg_catalog';
CREATE VIEW citus.citus_lock_waits AS
WITH
citus_dist_stat_activity AS
(
SELECT * FROM citus_dist_stat_activity
),
unique_global_wait_edges AS
(
SELECT DISTINCT ON(waiting_node_id, waiting_transaction_num, blocking_node_id, blocking_transaction_num) * FROM dump_global_wait_edges()
),
citus_dist_stat_activity_with_node_id AS
(
SELECT
citus_dist_stat_activity.*, (CASE citus_dist_stat_activity.distributed_query_host_name WHEN 'coordinator_host' THEN 0 ELSE pg_dist_node.nodeid END) as initiator_node_id
FROM
citus_dist_stat_activity LEFT JOIN pg_dist_node
ON
citus_dist_stat_activity.distributed_query_host_name = pg_dist_node.nodename AND
citus_dist_stat_activity.distributed_query_host_port = pg_dist_node.nodeport
)
SELECT
waiting.pid AS waiting_pid,
blocking.pid AS blocking_pid,
waiting.query AS blocked_statement,
blocking.query AS current_statement_in_blocking_process,
waiting.initiator_node_id AS waiting_node_id,
blocking.initiator_node_id AS blocking_node_id,
waiting.distributed_query_host_name AS waiting_node_name,
blocking.distributed_query_host_name AS blocking_node_name,
waiting.distributed_query_host_port AS waiting_node_port,
blocking.distributed_query_host_port AS blocking_node_port
FROM
unique_global_wait_edges
JOIN
citus_dist_stat_activity_with_node_id waiting ON (unique_global_wait_edges.waiting_transaction_num = waiting.transaction_number AND unique_global_wait_edges.waiting_node_id = waiting.initiator_node_id)
JOIN
citus_dist_stat_activity_with_node_id blocking ON (unique_global_wait_edges.blocking_transaction_num = blocking.transaction_number AND unique_global_wait_edges.blocking_node_id = blocking.initiator_node_id);
ALTER VIEW citus.citus_lock_waits SET SCHEMA pg_catalog;
GRANT SELECT ON pg_catalog.citus_lock_waits TO PUBLIC;
RESET search_path;

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -367,6 +367,8 @@ extern ObjectAddress AlterRoleSetStmtObjectAddress(Node *node,
extern List * GenerateCreateOrAlterRoleCommand(Oid roleOid);
/* schema.c - forward declarations */
extern List * PreprocessCreateSchemaStmt(Node *node, const char *queryString,
ProcessUtilityContext processUtilityContext);
extern List * PreprocessDropSchemaStmt(Node *dropSchemaStatement,
const char *queryString,
ProcessUtilityContext processUtilityContext);
@ -376,6 +378,7 @@ extern List * PreprocessGrantOnSchemaStmt(Node *node, const char *queryString,
ProcessUtilityContext processUtilityContext);
extern List * PreprocessAlterSchemaRenameStmt(Node *node, const char *queryString,
ProcessUtilityContext processUtilityContext);
extern ObjectAddress CreateSchemaStmtObjectAddress(Node *node, bool missing_ok);
extern ObjectAddress AlterSchemaRenameStmtObjectAddress(Node *node, bool missing_ok);
/* sequence.c - forward declarations */

View File

@ -29,7 +29,7 @@
#define ERROR_BUFFER_SIZE 256
/* application name used for internal connections in Citus */
#define CITUS_APPLICATION_NAME "citus_internal"
#define CITUS_APPLICATION_NAME_PREFIX "citus_internal gpid="
/* application name used for internal connections in rebalancer */
#define CITUS_REBALANCER_NAME "citus_rebalancer"

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -1,11 +1,4 @@
CREATE SCHEMA collation_conflict;
SELECT run_command_on_workers($$CREATE SCHEMA collation_conflict;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"CREATE SCHEMA")
(localhost,57638,t,"CREATE SCHEMA")
(2 rows)
\c - - - :worker_1_port
SET search_path TO collation_conflict;
SET citus.enable_metadata_sync TO off;

View File

@ -469,8 +469,7 @@ ALTER FUNCTION eq(macaddr,macaddr) DEPENDS ON EXTENSION citus;
ERROR: distrtibuted functions are not allowed to depend on an extension
DETAIL: Function "function_tests.eq(pg_catalog.macaddr,pg_catalog.macaddr)" is already distributed. Functions from extensions are expected to be created on the workers by the extension they depend on.
SELECT create_distributed_function('pg_catalog.citus_drop_trigger()');
ERROR: unable to create a distributed function from functions owned by an extension
DETAIL: Function "pg_catalog.citus_drop_trigger()" has a dependency on extension "citus". Functions depending on an extension cannot be distributed. Create the function by creating the extension on the workers.
ERROR: Citus extension functions(citus_drop_trigger) cannot be distributed.
DROP FUNCTION eq(macaddr,macaddr);
-- call should fail as function should have been dropped
SELECT * FROM run_command_on_workers($$SELECT function_tests.eq('0123456789ab','ba9876543210');$$) ORDER BY 1,2;
@ -1068,16 +1067,9 @@ SELECT stop_metadata_sync_to_node(nodename,nodeport) FROM pg_dist_node WHERE isa
\c - - - :worker_1_port
UPDATE pg_dist_local_group SET groupid = 0;
TRUNCATE pg_dist_node;
SET client_min_messages TO error; -- suppress cascading objects dropping
DROP SCHEMA function_tests CASCADE;
DROP SCHEMA function_tests2 CASCADE;
SET search_path TO function_tests, function_tests2;
\c - - - :worker_2_port
UPDATE pg_dist_local_group SET groupid = 0;
TRUNCATE pg_dist_node;
SET client_min_messages TO error; -- suppress cascading objects dropping
DROP SCHEMA function_tests CASCADE;
DROP SCHEMA function_tests2 CASCADE;
\c - - - :master_port
SET client_min_messages TO ERROR;
DROP USER functionuser;

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -394,11 +394,4 @@ NOTICE: issuing ROLLBACK
NOTICE: issuing ROLLBACK
DROP SCHEMA drop_partitioned_table CASCADE;
NOTICE: drop cascades to 3 other objects
SELECT run_command_on_workers('DROP SCHEMA IF EXISTS drop_partitioned_table CASCADE');
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
SET search_path TO public;

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -1140,18 +1140,11 @@ SELECT * FROM forcepushdown_schema.test_subquery ORDER BY 1;
(5 rows)
-- Query with targetList greater than 1
-- Function from FROM clause is not delegated outside of a BEGIN (for now)
-- Function from FROM clause is delegated outside of a BEGIN
SELECT 1,2,3 FROM select_data(100);
DEBUG: generating subplan XXX_1 for subquery SELECT data FROM forcepushdown_schema.test_subquery WHERE (data OPERATOR(pg_catalog.=) 100)
CONTEXT: SQL statement "SELECT result FROM forcepushdown_schema.test_subquery WHERE data =
(SELECT data FROM forcepushdown_schema.test_subquery WHERE data = a)"
PL/pgSQL function select_data(integer) line XX at SQL statement
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT result FROM forcepushdown_schema.test_subquery WHERE (data OPERATOR(pg_catalog.=) (SELECT intermediate_result.data FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(data integer)))
CONTEXT: SQL statement "SELECT result FROM forcepushdown_schema.test_subquery WHERE data =
(SELECT data FROM forcepushdown_schema.test_subquery WHERE data = a)"
PL/pgSQL function select_data(integer) line XX at SQL statement
DEBUG: pushing down the function call
NOTICE: Result: -1
CONTEXT: PL/pgSQL function select_data(integer) line XX at RAISE
DETAIL: from localhost:xxxxx
?column? | ?column? | ?column?
---------------------------------------------------------------------
1 | 2 | 3

View File

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

View File

@ -15,16 +15,16 @@ step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
BEGIN;
step s3-begin:
BEGIN;
BEGIN;
step s1-alter-table:
ALTER TABLE test_table ADD COLUMN x INT;
step s2-sleep:
SELECT pg_sleep(0.5);
SELECT pg_sleep(0.5);
pg_sleep
---------------------------------------------------------------------
@ -32,7 +32,7 @@ pg_sleep
(1 row)
step s2-view-dist:
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' and query not ILIKE '%BEGIN%' and query NOT ILIKE '%pg_catalog.pg_isolation_test_session_is_blocked%' ORDER BY query DESC;
query |query_hostname |query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname
---------------------------------------------------------------------
@ -42,7 +42,7 @@ query |query_hostname |query_hostport|d
(1 row)
step s3-view-worker:
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname
---------------------------------------------------------------------
@ -61,13 +61,13 @@ SELECT worker_apply_shard_ddl_command (1300001, 'public', '
(4 rows)
step s2-rollback:
ROLLBACK;
ROLLBACK;
step s1-commit:
COMMIT;
step s3-rollback:
ROLLBACK;
ROLLBACK;
starting permutation: s1-cache-connections s1-begin s2-begin s3-begin s1-insert s2-sleep s2-view-dist s3-view-worker s2-rollback s1-commit s3-rollback
@ -85,16 +85,16 @@ step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
BEGIN;
step s3-begin:
BEGIN;
BEGIN;
step s1-insert:
INSERT INTO test_table VALUES (100, 100);
INSERT INTO test_table VALUES (100, 100);
step s2-sleep:
SELECT pg_sleep(0.5);
SELECT pg_sleep(0.5);
pg_sleep
---------------------------------------------------------------------
@ -102,17 +102,17 @@ pg_sleep
(1 row)
step s2-view-dist:
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' and query not ILIKE '%BEGIN%' and query NOT ILIKE '%pg_catalog.pg_isolation_test_session_is_blocked%' ORDER BY query DESC;
query |query_hostname |query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname
---------------------------------------------------------------------
INSERT INTO test_table VALUES (100, 100);
INSERT INTO test_table VALUES (100, 100);
|coordinator_host| 57636|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression
(1 row)
step s3-view-worker:
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname
---------------------------------------------------------------------
@ -120,13 +120,13 @@ INSERT INTO public.test_table_1300008 (column1, column2) VALUES (100, 100)|local
(1 row)
step s2-rollback:
ROLLBACK;
ROLLBACK;
step s1-commit:
COMMIT;
step s3-rollback:
ROLLBACK;
ROLLBACK;
starting permutation: s1-cache-connections s1-begin s2-begin s3-begin s1-select s2-sleep s2-view-dist s3-view-worker s2-rollback s1-commit s3-rollback
@ -144,10 +144,10 @@ step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
BEGIN;
step s3-begin:
BEGIN;
BEGIN;
step s1-select:
SELECT count(*) FROM test_table;
@ -158,7 +158,7 @@ count
(1 row)
step s2-sleep:
SELECT pg_sleep(0.5);
SELECT pg_sleep(0.5);
pg_sleep
---------------------------------------------------------------------
@ -166,7 +166,7 @@ pg_sleep
(1 row)
step s2-view-dist:
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' and query not ILIKE '%BEGIN%' and query NOT ILIKE '%pg_catalog.pg_isolation_test_session_is_blocked%' ORDER BY query DESC;
query |query_hostname |query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname
---------------------------------------------------------------------
@ -176,7 +176,7 @@ query |query_hostname |query_hostport|distribute
(1 row)
step s3-view-worker:
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname
---------------------------------------------------------------------
@ -187,13 +187,13 @@ SELECT count(*) AS count FROM public.test_table_1300011 test_table WHERE true|lo
(4 rows)
step s2-rollback:
ROLLBACK;
ROLLBACK;
step s1-commit:
COMMIT;
step s3-rollback:
ROLLBACK;
ROLLBACK;
starting permutation: s1-cache-connections s1-begin s2-begin s3-begin s1-select-router s2-sleep s2-view-dist s3-view-worker s2-rollback s1-commit s3-rollback
@ -211,10 +211,10 @@ step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
BEGIN;
step s3-begin:
BEGIN;
BEGIN;
step s1-select-router:
SELECT count(*) FROM test_table WHERE column1 = 55;
@ -225,7 +225,7 @@ count
(1 row)
step s2-sleep:
SELECT pg_sleep(0.5);
SELECT pg_sleep(0.5);
pg_sleep
---------------------------------------------------------------------
@ -233,7 +233,7 @@ pg_sleep
(1 row)
step s2-view-dist:
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' and query not ILIKE '%BEGIN%' and query NOT ILIKE '%pg_catalog.pg_isolation_test_session_is_blocked%' ORDER BY query DESC;
query |query_hostname |query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname
---------------------------------------------------------------------
@ -243,7 +243,7 @@ query |query_hostname |query_
(1 row)
step s3-view-worker:
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname
---------------------------------------------------------------------
@ -251,11 +251,11 @@ SELECT count(*) AS count FROM public.test_table_1300017 test_table WHERE (column
(1 row)
step s2-rollback:
ROLLBACK;
ROLLBACK;
step s1-commit:
COMMIT;
step s3-rollback:
ROLLBACK;
ROLLBACK;

View File

@ -94,7 +94,8 @@ step s1-verify-current-xact-is-on-worker:
get_current_transaction_id() as xact,
run_command_on_workers($$
SELECT row(initiator_node_identifier, transaction_number)
FROM get_all_active_transactions();
FROM get_all_active_transactions()
WHERE transaction_number != 0;
$$) as remote
ORDER BY remote.nodeport ASC;

View File

@ -1,6 +1,6 @@
Parsed test spec with 3 sessions
starting permutation: s1-print-distributed-objects s1-begin s1-add-worker s2-public-schema s2-create-table s1-commit s2-print-distributed-objects s3-drop-coordinator-schemas s3-drop-worker-schemas
starting permutation: s1-print-distributed-objects s1-begin s1-add-worker s2-public-schema s2-create-table s1-commit s2-print-distributed-objects s3-drop-coordinator-schemas
?column?
---------------------------------------------------------------------
1
@ -164,22 +164,6 @@ step s3-drop-coordinator-schemas:
DROP SCHEMA IF EXISTS myschema CASCADE;
DROP SCHEMA IF EXISTS myschema2 CASCADE;
step s3-drop-worker-schemas:
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema CASCADE;$$);
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema2 CASCADE;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
master_remove_node
---------------------------------------------------------------------
@ -187,7 +171,7 @@ master_remove_node
(2 rows)
starting permutation: s1-print-distributed-objects s1-begin s2-begin s1-add-worker s2-public-schema s2-create-table s1-commit s2-commit s2-print-distributed-objects s3-drop-coordinator-schemas s3-drop-worker-schemas
starting permutation: s1-print-distributed-objects s1-begin s2-begin s1-add-worker s2-public-schema s2-create-table s1-commit s2-commit s2-print-distributed-objects s3-drop-coordinator-schemas
?column?
---------------------------------------------------------------------
1
@ -357,22 +341,6 @@ step s3-drop-coordinator-schemas:
DROP SCHEMA IF EXISTS myschema CASCADE;
DROP SCHEMA IF EXISTS myschema2 CASCADE;
step s3-drop-worker-schemas:
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema CASCADE;$$);
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema2 CASCADE;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
master_remove_node
---------------------------------------------------------------------
@ -380,7 +348,7 @@ master_remove_node
(2 rows)
starting permutation: s1-print-distributed-objects s1-begin s2-begin s2-public-schema s2-create-table s1-add-worker s2-commit s1-commit s2-print-distributed-objects s3-drop-coordinator-schemas s3-drop-worker-schemas
starting permutation: s1-print-distributed-objects s1-begin s2-begin s2-public-schema s2-create-table s1-add-worker s2-commit s1-commit s2-print-distributed-objects s3-drop-coordinator-schemas
?column?
---------------------------------------------------------------------
1
@ -550,22 +518,6 @@ step s3-drop-coordinator-schemas:
DROP SCHEMA IF EXISTS myschema CASCADE;
DROP SCHEMA IF EXISTS myschema2 CASCADE;
step s3-drop-worker-schemas:
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema CASCADE;$$);
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema2 CASCADE;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
master_remove_node
---------------------------------------------------------------------
@ -573,7 +525,7 @@ master_remove_node
(2 rows)
starting permutation: s1-print-distributed-objects s1-begin s1-add-worker s2-create-schema s2-create-table s1-commit s2-print-distributed-objects s3-drop-coordinator-schemas s3-drop-worker-schemas
starting permutation: s1-print-distributed-objects s1-begin s1-add-worker s2-create-schema s2-create-table s1-commit s2-print-distributed-objects s3-drop-coordinator-schemas
?column?
---------------------------------------------------------------------
1
@ -739,22 +691,6 @@ step s3-drop-coordinator-schemas:
DROP SCHEMA IF EXISTS myschema CASCADE;
DROP SCHEMA IF EXISTS myschema2 CASCADE;
step s3-drop-worker-schemas:
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema CASCADE;$$);
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema2 CASCADE;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
master_remove_node
---------------------------------------------------------------------
@ -762,7 +698,7 @@ master_remove_node
(2 rows)
starting permutation: s1-print-distributed-objects s1-begin s2-begin s1-add-worker s2-create-schema s2-create-table s1-commit s2-commit s2-print-distributed-objects s3-drop-coordinator-schemas s3-drop-worker-schemas
starting permutation: s1-print-distributed-objects s1-begin s2-begin s1-add-worker s2-create-schema s2-create-table s1-commit s2-commit s2-print-distributed-objects s3-drop-coordinator-schemas
?column?
---------------------------------------------------------------------
1
@ -934,22 +870,6 @@ step s3-drop-coordinator-schemas:
DROP SCHEMA IF EXISTS myschema CASCADE;
DROP SCHEMA IF EXISTS myschema2 CASCADE;
step s3-drop-worker-schemas:
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema CASCADE;$$);
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema2 CASCADE;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
master_remove_node
---------------------------------------------------------------------
@ -957,7 +877,7 @@ master_remove_node
(2 rows)
starting permutation: s1-print-distributed-objects s1-begin s2-begin s2-create-schema s2-create-table s1-add-worker s2-commit s1-commit s2-print-distributed-objects s3-drop-coordinator-schemas s3-drop-worker-schemas
starting permutation: s1-print-distributed-objects s1-begin s2-begin s2-create-schema s2-create-table s1-add-worker s2-commit s1-commit s2-print-distributed-objects s3-drop-coordinator-schemas
?column?
---------------------------------------------------------------------
1
@ -1129,22 +1049,6 @@ step s3-drop-coordinator-schemas:
DROP SCHEMA IF EXISTS myschema CASCADE;
DROP SCHEMA IF EXISTS myschema2 CASCADE;
step s3-drop-worker-schemas:
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema CASCADE;$$);
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema2 CASCADE;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
master_remove_node
---------------------------------------------------------------------
@ -1152,7 +1056,7 @@ master_remove_node
(2 rows)
starting permutation: s1-print-distributed-objects s2-create-schema s1-begin s2-begin s1-add-worker s2-create-table s1-commit s2-commit s2-print-distributed-objects s3-drop-coordinator-schemas s3-drop-worker-schemas
starting permutation: s1-print-distributed-objects s2-create-schema s1-begin s2-begin s1-add-worker s2-create-table s1-commit s2-commit s2-print-distributed-objects s3-drop-coordinator-schemas
?column?
---------------------------------------------------------------------
1
@ -1324,22 +1228,6 @@ step s3-drop-coordinator-schemas:
DROP SCHEMA IF EXISTS myschema CASCADE;
DROP SCHEMA IF EXISTS myschema2 CASCADE;
step s3-drop-worker-schemas:
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema CASCADE;$$);
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema2 CASCADE;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
master_remove_node
---------------------------------------------------------------------
@ -1347,7 +1235,7 @@ master_remove_node
(2 rows)
starting permutation: s1-print-distributed-objects s1-add-worker s2-create-schema s2-begin s3-begin s3-use-schema s2-create-table s3-create-table s2-commit s3-commit s2-print-distributed-objects s3-drop-coordinator-schemas s3-drop-worker-schemas
starting permutation: s1-print-distributed-objects s1-add-worker s2-create-schema s2-begin s3-begin s3-use-schema s2-create-table s3-create-table s2-commit s3-commit s2-print-distributed-objects s3-drop-coordinator-schemas
?column?
---------------------------------------------------------------------
1
@ -1534,22 +1422,6 @@ step s3-drop-coordinator-schemas:
DROP SCHEMA IF EXISTS myschema CASCADE;
DROP SCHEMA IF EXISTS myschema2 CASCADE;
step s3-drop-worker-schemas:
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema CASCADE;$$);
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema2 CASCADE;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
master_remove_node
---------------------------------------------------------------------
@ -1557,7 +1429,7 @@ master_remove_node
(2 rows)
starting permutation: s1-print-distributed-objects s1-begin s2-begin s3-begin s1-add-worker s2-create-schema s3-create-schema2 s2-create-table s3-create-table s1-commit s3-commit s2-commit s2-print-distributed-objects s3-drop-coordinator-schemas s3-drop-worker-schemas
starting permutation: s1-print-distributed-objects s1-begin s2-begin s3-begin s1-add-worker s2-create-schema s3-create-schema2 s2-create-table s3-create-table s1-commit s3-commit s2-commit s2-print-distributed-objects s3-drop-coordinator-schemas
?column?
---------------------------------------------------------------------
1
@ -1753,22 +1625,6 @@ step s3-drop-coordinator-schemas:
DROP SCHEMA IF EXISTS myschema CASCADE;
DROP SCHEMA IF EXISTS myschema2 CASCADE;
step s3-drop-worker-schemas:
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema CASCADE;$$);
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema2 CASCADE;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
master_remove_node
---------------------------------------------------------------------
@ -1776,7 +1632,7 @@ master_remove_node
(2 rows)
starting permutation: s1-print-distributed-objects s1-begin s1-add-worker s2-public-schema s2-create-type s1-commit s2-print-distributed-objects s3-drop-coordinator-schemas s3-drop-worker-schemas
starting permutation: s1-print-distributed-objects s1-begin s1-add-worker s2-public-schema s2-create-type s1-commit s2-print-distributed-objects s3-drop-coordinator-schemas
?column?
---------------------------------------------------------------------
1
@ -1932,22 +1788,6 @@ step s3-drop-coordinator-schemas:
DROP SCHEMA IF EXISTS myschema CASCADE;
DROP SCHEMA IF EXISTS myschema2 CASCADE;
step s3-drop-worker-schemas:
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema CASCADE;$$);
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema2 CASCADE;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
master_remove_node
---------------------------------------------------------------------
@ -1955,7 +1795,7 @@ master_remove_node
(2 rows)
starting permutation: s1-print-distributed-objects s1-begin s2-public-schema s2-create-type s1-add-worker s1-commit s2-print-distributed-objects s3-drop-coordinator-schemas s3-drop-worker-schemas
starting permutation: s1-print-distributed-objects s1-begin s2-public-schema s2-create-type s1-add-worker s1-commit s2-print-distributed-objects s3-drop-coordinator-schemas
?column?
---------------------------------------------------------------------
1
@ -2110,22 +1950,6 @@ step s3-drop-coordinator-schemas:
DROP SCHEMA IF EXISTS myschema CASCADE;
DROP SCHEMA IF EXISTS myschema2 CASCADE;
step s3-drop-worker-schemas:
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema CASCADE;$$);
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema2 CASCADE;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
master_remove_node
---------------------------------------------------------------------
@ -2133,7 +1957,7 @@ master_remove_node
(2 rows)
starting permutation: s1-print-distributed-objects s1-begin s2-begin s2-create-schema s2-create-type s2-create-table-with-type s1-add-worker s2-commit s1-commit s2-print-distributed-objects s3-drop-coordinator-schemas s3-drop-worker-schemas
starting permutation: s1-print-distributed-objects s1-begin s2-begin s2-create-schema s2-create-type s2-create-table-with-type s1-add-worker s2-commit s1-commit s2-print-distributed-objects s3-drop-coordinator-schemas
?column?
---------------------------------------------------------------------
1
@ -2309,22 +2133,6 @@ step s3-drop-coordinator-schemas:
DROP SCHEMA IF EXISTS myschema CASCADE;
DROP SCHEMA IF EXISTS myschema2 CASCADE;
step s3-drop-worker-schemas:
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema CASCADE;$$);
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema2 CASCADE;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
master_remove_node
---------------------------------------------------------------------
@ -2332,7 +2140,7 @@ master_remove_node
(2 rows)
starting permutation: s1-print-distributed-objects s1-begin s1-add-worker s2-public-schema s2-distribute-function s1-commit s2-begin s2-commit s3-wait-for-metadata-sync s2-print-distributed-objects s3-drop-coordinator-schemas s3-drop-worker-schemas
starting permutation: s1-print-distributed-objects s1-begin s1-add-worker s2-public-schema s2-distribute-function s1-commit s2-begin s2-commit s3-wait-for-metadata-sync s2-print-distributed-objects s3-drop-coordinator-schemas
?column?
---------------------------------------------------------------------
1
@ -2508,22 +2316,6 @@ step s3-drop-coordinator-schemas:
DROP SCHEMA IF EXISTS myschema CASCADE;
DROP SCHEMA IF EXISTS myschema2 CASCADE;
step s3-drop-worker-schemas:
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema CASCADE;$$);
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema2 CASCADE;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
master_remove_node
---------------------------------------------------------------------
@ -2531,7 +2323,7 @@ master_remove_node
(2 rows)
starting permutation: s1-print-distributed-objects s1-begin s2-public-schema s2-distribute-function s2-begin s2-commit s3-wait-for-metadata-sync s1-add-worker s1-commit s3-wait-for-metadata-sync s2-print-distributed-objects s3-drop-coordinator-schemas s3-drop-worker-schemas
starting permutation: s1-print-distributed-objects s1-begin s2-public-schema s2-distribute-function s2-begin s2-commit s3-wait-for-metadata-sync s1-add-worker s1-commit s3-wait-for-metadata-sync s2-print-distributed-objects s3-drop-coordinator-schemas
?column?
---------------------------------------------------------------------
1
@ -2714,22 +2506,6 @@ step s3-drop-coordinator-schemas:
DROP SCHEMA IF EXISTS myschema CASCADE;
DROP SCHEMA IF EXISTS myschema2 CASCADE;
step s3-drop-worker-schemas:
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema CASCADE;$$);
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema2 CASCADE;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
master_remove_node
---------------------------------------------------------------------
@ -2737,7 +2513,7 @@ master_remove_node
(2 rows)
starting permutation: s1-print-distributed-objects s2-begin s2-create-schema s2-distribute-function s2-commit s3-wait-for-metadata-sync s1-begin s1-add-worker s1-commit s3-wait-for-metadata-sync s2-print-distributed-objects s3-drop-coordinator-schemas s3-drop-worker-schemas
starting permutation: s1-print-distributed-objects s2-begin s2-create-schema s2-distribute-function s2-commit s3-wait-for-metadata-sync s1-begin s1-add-worker s1-commit s3-wait-for-metadata-sync s2-print-distributed-objects s3-drop-coordinator-schemas
?column?
---------------------------------------------------------------------
1
@ -2922,22 +2698,6 @@ step s3-drop-coordinator-schemas:
DROP SCHEMA IF EXISTS myschema CASCADE;
DROP SCHEMA IF EXISTS myschema2 CASCADE;
step s3-drop-worker-schemas:
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema CASCADE;$$);
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema2 CASCADE;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
master_remove_node
---------------------------------------------------------------------

View File

@ -28,7 +28,7 @@ step s1-print:
count
---------------------------------------------------------------------
4
7
(1 row)
extname|extversion|nspname
@ -89,7 +89,7 @@ step s1-print:
count
---------------------------------------------------------------------
4
7
(1 row)
extname|extversion|nspname
@ -158,7 +158,7 @@ step s1-print:
count
---------------------------------------------------------------------
3
6
(1 row)
extname|extversion|nspname
@ -214,7 +214,7 @@ step s1-print:
count
---------------------------------------------------------------------
5
7
(1 row)
extname|extversion|nspname
@ -275,7 +275,7 @@ step s1-print:
count
---------------------------------------------------------------------
4
6
(1 row)
extname|extversion|nspname

View File

@ -35,8 +35,8 @@ step s2-begin-insert:
step s3-as-admin:
-- Admin should be able to see all transactions
SELECT count(*) FROM get_all_active_transactions();
SELECT count(*) FROM get_global_active_transactions();
SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0;
SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0;
count
---------------------------------------------------------------------
@ -51,8 +51,8 @@ count
step s3-as-user-1:
-- User should only be able to see its own transactions
SET ROLE test_user_1;
SELECT count(*) FROM get_all_active_transactions();
SELECT count(*) FROM get_global_active_transactions();
SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0;
SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0;
count
---------------------------------------------------------------------
@ -67,8 +67,8 @@ count
step s3-as-readonly:
-- Other user should not see transactions
SET ROLE test_readonly;
SELECT count(*) FROM get_all_active_transactions();
SELECT count(*) FROM get_global_active_transactions();
SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0;
SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0;
count
---------------------------------------------------------------------
@ -83,8 +83,8 @@ count
step s3-as-monitor:
-- Monitor should see all transactions
SET ROLE test_monitor;
SELECT count(*) FROM get_all_active_transactions();
SELECT count(*) FROM get_global_active_transactions();
SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0;
SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0;
count
---------------------------------------------------------------------

View File

@ -0,0 +1,145 @@
Parsed test spec with 2 sessions
starting permutation: s1-start-session-level-connection s1-worker-begin s1-worker-select s2-coordinator-citus_dist_stat_activity s2-coordinator-citus_worker_stat_activity s1-worker-commit s1-stop-session-level-connection
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-start-session-level-connection:
SELECT start_session_level_connection_to_node('localhost', 57637);
start_session_level_connection_to_node
---------------------------------------------------------------------
(1 row)
step s1-worker-begin:
SELECT run_commands_on_session_level_connection_to_node('BEGIN');
run_commands_on_session_level_connection_to_node
---------------------------------------------------------------------
(1 row)
step s1-worker-select:
SELECT run_commands_on_session_level_connection_to_node('SET citus.enable_local_execution TO off; SET citus.force_max_query_parallelization TO ON; SELECT * FROM dist_table');
run_commands_on_session_level_connection_to_node
---------------------------------------------------------------------
(1 row)
step s2-coordinator-citus_dist_stat_activity:
SELECT global_pid != 0 FROM citus_dist_stat_activity() WHERE query LIKE '%SELECT * FROM dist\_table%' and query NOT ILIKE '%run_commands_on_session_level_connection_to_node%';
?column?
---------------------------------------------------------------------
t
(1 row)
step s2-coordinator-citus_worker_stat_activity:
SELECT query FROM citus_worker_stat_activity() WHERE global_pid IN (
SELECT global_pid FROM citus_dist_stat_activity() WHERE query LIKE '%SELECT * FROM dist\_table%'
)
ORDER BY 1;
query
---------------------------------------------------------------------
SELECT a, b FROM public.dist_table_12345000 dist_table WHERE true
SELECT a, b FROM public.dist_table_12345001 dist_table WHERE true
SELECT a, b FROM public.dist_table_12345002 dist_table WHERE true
SELECT a, b FROM public.dist_table_12345003 dist_table WHERE true
(4 rows)
step s1-worker-commit:
SELECT run_commands_on_session_level_connection_to_node('COMMIT');
run_commands_on_session_level_connection_to_node
---------------------------------------------------------------------
(1 row)
step s1-stop-session-level-connection:
SELECT stop_session_level_connection_to_node();
stop_session_level_connection_to_node
---------------------------------------------------------------------
(1 row)
restore_isolation_tester_func
---------------------------------------------------------------------
(1 row)
starting permutation: s1-coordinator-begin s1-coordinator-select s2-coordinator-citus_dist_stat_activity s2-coordinator-citus_worker_stat_activity s2-coordinator-get_all_active_transactions s2-coordinator-get_global_active_transactions s1-coordinator-commit
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s1-coordinator-begin:
BEGIN;
step s1-coordinator-select:
SET citus.enable_local_execution TO off;
SET citus.force_max_query_parallelization TO ON;
SELECT * FROM dist_table;
a|b
---------------------------------------------------------------------
(0 rows)
step s2-coordinator-citus_dist_stat_activity:
SELECT global_pid != 0 FROM citus_dist_stat_activity() WHERE query LIKE '%SELECT * FROM dist\_table%' and query NOT ILIKE '%run_commands_on_session_level_connection_to_node%';
?column?
---------------------------------------------------------------------
t
(1 row)
step s2-coordinator-citus_worker_stat_activity:
SELECT query FROM citus_worker_stat_activity() WHERE global_pid IN (
SELECT global_pid FROM citus_dist_stat_activity() WHERE query LIKE '%SELECT * FROM dist\_table%'
)
ORDER BY 1;
query
---------------------------------------------------------------------
SELECT a, b FROM public.dist_table_12345000 dist_table WHERE true
SELECT a, b FROM public.dist_table_12345001 dist_table WHERE true
SELECT a, b FROM public.dist_table_12345002 dist_table WHERE true
SELECT a, b FROM public.dist_table_12345003 dist_table WHERE true
(4 rows)
step s2-coordinator-get_all_active_transactions:
SELECT count(*) FROM get_all_active_transactions() WHERE global_pid IN (
SELECT global_pid FROM citus_dist_stat_activity() WHERE query LIKE '%SELECT * FROM dist\_table%'
);
count
---------------------------------------------------------------------
1
(1 row)
step s2-coordinator-get_global_active_transactions:
SELECT count(*) FROM get_global_active_transactions() WHERE global_pid IN (
SELECT global_pid FROM citus_dist_stat_activity() WHERE query LIKE '%SELECT * FROM dist\_table%'
)
AND transaction_number != 0;
count
---------------------------------------------------------------------
5
(1 row)
step s1-coordinator-commit:
COMMIT;
restore_isolation_tester_func
---------------------------------------------------------------------
(1 row)

View File

@ -0,0 +1,716 @@
unused step name: s3-debug
Parsed test spec with 3 sessions
starting permutation: s3-compare-snapshot
step s3-compare-snapshot:
SELECT count(*) = 0 AS same_metadata_in_workers
FROM
(
(
SELECT unnest(activate_node_snapshot())
EXCEPT
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
)
UNION
(
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
EXCEPT
SELECT unnest(activate_node_snapshot())
)
) AS foo;
same_metadata_in_workers
---------------------------------------------------------------------
t
(1 row)
starting permutation: s1-begin s2-begin s1-start-metadata-sync s2-start-metadata-sync-to-same-node s1-commit s2-commit s3-compare-snapshot
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1-start-metadata-sync:
SELECT start_metadata_sync_to_node('localhost', 57638);
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
step s2-start-metadata-sync-to-same-node:
SELECT start_metadata_sync_to_node('localhost', 57638);
<waiting ...>
step s1-commit:
COMMIT;
step s2-start-metadata-sync-to-same-node: <... completed>
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
step s2-commit:
COMMIT;
step s3-compare-snapshot:
SELECT count(*) = 0 AS same_metadata_in_workers
FROM
(
(
SELECT unnest(activate_node_snapshot())
EXCEPT
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
)
UNION
(
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
EXCEPT
SELECT unnest(activate_node_snapshot())
)
) AS foo;
same_metadata_in_workers
---------------------------------------------------------------------
t
(1 row)
starting permutation: s1-begin s2-begin s1-start-metadata-sync s2-start-metadata-sync-to-another-node s1-commit s2-commit s3-compare-snapshot
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1-start-metadata-sync:
SELECT start_metadata_sync_to_node('localhost', 57638);
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
step s2-start-metadata-sync-to-another-node:
SELECT start_metadata_sync_to_node('localhost', 57637);
<waiting ...>
step s1-commit:
COMMIT;
step s2-start-metadata-sync-to-another-node: <... completed>
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
step s2-commit:
COMMIT;
step s3-compare-snapshot:
SELECT count(*) = 0 AS same_metadata_in_workers
FROM
(
(
SELECT unnest(activate_node_snapshot())
EXCEPT
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
)
UNION
(
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
EXCEPT
SELECT unnest(activate_node_snapshot())
)
) AS foo;
same_metadata_in_workers
---------------------------------------------------------------------
t
(1 row)
starting permutation: s1-begin s2-begin s1-start-metadata-sync s2-alter-table s1-commit s2-commit s3-compare-snapshot
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1-start-metadata-sync:
SELECT start_metadata_sync_to_node('localhost', 57638);
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
step s2-alter-table:
ALTER TABLE dist_table ADD COLUMN z int;
<waiting ...>
step s1-commit:
COMMIT;
step s2-alter-table: <... completed>
step s2-commit:
COMMIT;
step s3-compare-snapshot:
SELECT count(*) = 0 AS same_metadata_in_workers
FROM
(
(
SELECT unnest(activate_node_snapshot())
EXCEPT
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
)
UNION
(
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
EXCEPT
SELECT unnest(activate_node_snapshot())
)
) AS foo;
same_metadata_in_workers
---------------------------------------------------------------------
t
(1 row)
starting permutation: s1-begin s2-begin s1-start-metadata-sync s2-drop-table s1-commit s2-commit s3-compare-snapshot
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1-start-metadata-sync:
SELECT start_metadata_sync_to_node('localhost', 57638);
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
step s2-drop-table:
DROP TABLE dist_table;
<waiting ...>
step s1-commit:
COMMIT;
step s2-drop-table: <... completed>
step s2-commit:
COMMIT;
step s3-compare-snapshot:
SELECT count(*) = 0 AS same_metadata_in_workers
FROM
(
(
SELECT unnest(activate_node_snapshot())
EXCEPT
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
)
UNION
(
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
EXCEPT
SELECT unnest(activate_node_snapshot())
)
) AS foo;
same_metadata_in_workers
---------------------------------------------------------------------
t
(1 row)
starting permutation: s1-begin s2-begin s1-start-metadata-sync s2-create-dist-table s1-commit s2-commit s3-compare-snapshot
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1-start-metadata-sync:
SELECT start_metadata_sync_to_node('localhost', 57638);
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
step s2-create-dist-table:
CREATE TABLE new_dist_table(id int, data int);
SELECT create_distributed_table('new_dist_table', 'id');
<waiting ...>
step s1-commit:
COMMIT;
step s2-create-dist-table: <... completed>
create_distributed_table
---------------------------------------------------------------------
(1 row)
step s2-commit:
COMMIT;
step s3-compare-snapshot:
SELECT count(*) = 0 AS same_metadata_in_workers
FROM
(
(
SELECT unnest(activate_node_snapshot())
EXCEPT
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
)
UNION
(
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
EXCEPT
SELECT unnest(activate_node_snapshot())
)
) AS foo;
same_metadata_in_workers
---------------------------------------------------------------------
t
(1 row)
starting permutation: s1-begin s2-begin s1-start-metadata-sync s2-create-ref-table s1-commit s2-commit s3-compare-snapshot
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1-start-metadata-sync:
SELECT start_metadata_sync_to_node('localhost', 57638);
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
step s2-create-ref-table:
CREATE TABLE new_ref_table(id int, data int);
SELECT create_reference_table('new_ref_table');
<waiting ...>
step s1-commit:
COMMIT;
step s2-create-ref-table: <... completed>
create_reference_table
---------------------------------------------------------------------
(1 row)
step s2-commit:
COMMIT;
step s3-compare-snapshot:
SELECT count(*) = 0 AS same_metadata_in_workers
FROM
(
(
SELECT unnest(activate_node_snapshot())
EXCEPT
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
)
UNION
(
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
EXCEPT
SELECT unnest(activate_node_snapshot())
)
) AS foo;
same_metadata_in_workers
---------------------------------------------------------------------
t
(1 row)
starting permutation: s1-begin s2-begin s1-start-metadata-sync s2-attach-partition s1-commit s2-commit s3-compare-snapshot
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1-start-metadata-sync:
SELECT start_metadata_sync_to_node('localhost', 57638);
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
step s2-attach-partition:
ALTER TABLE dist_partitioned_table ATTACH PARTITION dist_partitioned_table_p1 FOR VALUES FROM (1) TO (9);
<waiting ...>
step s1-commit:
COMMIT;
step s2-attach-partition: <... completed>
step s2-commit:
COMMIT;
step s3-compare-snapshot:
SELECT count(*) = 0 AS same_metadata_in_workers
FROM
(
(
SELECT unnest(activate_node_snapshot())
EXCEPT
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
)
UNION
(
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
EXCEPT
SELECT unnest(activate_node_snapshot())
)
) AS foo;
same_metadata_in_workers
---------------------------------------------------------------------
t
(1 row)
starting permutation: s2-attach-partition s1-begin s2-begin s1-start-metadata-sync s2-detach-partition s1-commit s2-commit s3-compare-snapshot
step s2-attach-partition:
ALTER TABLE dist_partitioned_table ATTACH PARTITION dist_partitioned_table_p1 FOR VALUES FROM (1) TO (9);
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1-start-metadata-sync:
SELECT start_metadata_sync_to_node('localhost', 57638);
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
step s2-detach-partition:
ALTER TABLE dist_partitioned_table DETACH PARTITION dist_partitioned_table_p1;
<waiting ...>
step s1-commit:
COMMIT;
step s2-detach-partition: <... completed>
step s2-commit:
COMMIT;
step s3-compare-snapshot:
SELECT count(*) = 0 AS same_metadata_in_workers
FROM
(
(
SELECT unnest(activate_node_snapshot())
EXCEPT
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
)
UNION
(
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
EXCEPT
SELECT unnest(activate_node_snapshot())
)
) AS foo;
same_metadata_in_workers
---------------------------------------------------------------------
t
(1 row)
starting permutation: s2-attach-partition s1-begin s2-begin s1-start-metadata-sync s2-create-partition-of s1-commit s2-commit s3-compare-snapshot
step s2-attach-partition:
ALTER TABLE dist_partitioned_table ATTACH PARTITION dist_partitioned_table_p1 FOR VALUES FROM (1) TO (9);
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1-start-metadata-sync:
SELECT start_metadata_sync_to_node('localhost', 57638);
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
step s2-create-partition-of:
CREATE TABLE dist_partitioned_table_p2 PARTITION OF dist_partitioned_table FOR VALUES FROM (10) TO (20);
<waiting ...>
step s1-commit:
COMMIT;
step s2-create-partition-of: <... completed>
step s2-commit:
COMMIT;
step s3-compare-snapshot:
SELECT count(*) = 0 AS same_metadata_in_workers
FROM
(
(
SELECT unnest(activate_node_snapshot())
EXCEPT
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
)
UNION
(
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
EXCEPT
SELECT unnest(activate_node_snapshot())
)
) AS foo;
same_metadata_in_workers
---------------------------------------------------------------------
t
(1 row)
starting permutation: s1-begin s2-begin s1-start-metadata-sync s2-add-fk s1-commit s2-commit s3-compare-snapshot
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1-start-metadata-sync:
SELECT start_metadata_sync_to_node('localhost', 57638);
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
step s2-add-fk:
ALTER TABLE dist_table ADD CONSTRAINT y_fk FOREIGN KEY (y) REFERENCES ref_table(y);
<waiting ...>
step s1-commit:
COMMIT;
step s2-add-fk: <... completed>
step s2-commit:
COMMIT;
step s3-compare-snapshot:
SELECT count(*) = 0 AS same_metadata_in_workers
FROM
(
(
SELECT unnest(activate_node_snapshot())
EXCEPT
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
)
UNION
(
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
EXCEPT
SELECT unnest(activate_node_snapshot())
)
) AS foo;
same_metadata_in_workers
---------------------------------------------------------------------
t
(1 row)
starting permutation: s2-add-fk s1-begin s2-begin s1-start-metadata-sync s2-drop-fk s1-commit s2-commit s3-compare-snapshot
step s2-add-fk:
ALTER TABLE dist_table ADD CONSTRAINT y_fk FOREIGN KEY (y) REFERENCES ref_table(y);
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1-start-metadata-sync:
SELECT start_metadata_sync_to_node('localhost', 57638);
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
step s2-drop-fk:
ALTER TABLE dist_table DROP CONSTRAINT y_fk;
<waiting ...>
step s1-commit:
COMMIT;
step s2-drop-fk: <... completed>
step s2-commit:
COMMIT;
step s3-compare-snapshot:
SELECT count(*) = 0 AS same_metadata_in_workers
FROM
(
(
SELECT unnest(activate_node_snapshot())
EXCEPT
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
)
UNION
(
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
EXCEPT
SELECT unnest(activate_node_snapshot())
)
) AS foo;
same_metadata_in_workers
---------------------------------------------------------------------
t
(1 row)
starting permutation: s1-begin s2-begin s1-start-metadata-sync s2-create-type s1-commit s2-commit s3-compare-snapshot
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1-start-metadata-sync:
SELECT start_metadata_sync_to_node('localhost', 57638);
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
step s2-create-type:
CREATE TYPE my_type AS (a int, b int);
step s1-commit:
COMMIT;
step s2-commit:
COMMIT;
step s3-compare-snapshot:
SELECT count(*) = 0 AS same_metadata_in_workers
FROM
(
(
SELECT unnest(activate_node_snapshot())
EXCEPT
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
)
UNION
(
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
EXCEPT
SELECT unnest(activate_node_snapshot())
)
) AS foo;
same_metadata_in_workers
---------------------------------------------------------------------
t
(1 row)
starting permutation: s1-begin s2-begin s1-start-metadata-sync s2-create-dist-func s1-commit s2-commit s3-compare-snapshot
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1-start-metadata-sync:
SELECT start_metadata_sync_to_node('localhost', 57638);
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
step s2-create-dist-func:
CREATE FUNCTION squares(int) RETURNS SETOF RECORD
AS $$ SELECT i, i * i FROM generate_series(1, $1) i $$
LANGUAGE SQL;
SELECT create_distributed_function('squares(int)');
<waiting ...>
step s1-commit:
COMMIT;
step s2-create-dist-func: <... completed>
create_distributed_function
---------------------------------------------------------------------
(1 row)
step s2-commit:
COMMIT;
step s3-compare-snapshot:
SELECT count(*) = 0 AS same_metadata_in_workers
FROM
(
(
SELECT unnest(activate_node_snapshot())
EXCEPT
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
)
UNION
(
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
EXCEPT
SELECT unnest(activate_node_snapshot())
)
) AS foo;
same_metadata_in_workers
---------------------------------------------------------------------
t
(1 row)

View File

@ -83,14 +83,17 @@ pg_sleep
(1 row)
step s2-view-dist:
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND query NOT ILIKE '%pg_isolation_test_session_is_blocked%' AND query NOT ILIKE '%BEGIN%' ORDER BY query DESC;
query |query_hostname |query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname
query |query_hostname |query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname
---------------------------------------------------------------------
SELECT check_distributed_deadlocks();
|coordinator_host| 57636| | 0|idle |Client |ClientRead|postgres|regression
update ref_table set a = a + 1;
|coordinator_host| 57636|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression
(1 row)
|coordinator_host| 57636|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression
(2 rows)
step s2-view-worker:
SELECT query, query_hostname, query_hostport, distributed_query_host_name,
@ -101,7 +104,7 @@ step s2-view-worker:
query NOT ILIKE '%dump_local_wait_edges%'
ORDER BY query, query_hostport DESC;
query |query_hostname |query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname
query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname
---------------------------------------------------------------------
UPDATE public.ref_table_1500767 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57638|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression
UPDATE public.ref_table_1500767 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57637|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression
@ -136,8 +139,8 @@ step s1-update-ref-table:
step s2-active-transactions:
-- Admin should be able to see all transactions
SELECT count(*) FROM get_all_active_transactions();
SELECT count(*) FROM get_global_active_transactions();
SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0;
SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0;
count
---------------------------------------------------------------------

View File

@ -5,6 +5,13 @@ SET citus.next_placement_id TO 1500000;
-- supress notice messages to make sure that the tests
-- do not diverge with enterprise
SET client_min_messages TO WARNING;
SELECT run_command_on_workers($$CREATE ROLE metadata_sync_helper_role WITH LOGIN;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"CREATE ROLE")
(localhost,57638,t,"CREATE ROLE")
(2 rows)
CREATE ROLE metadata_sync_helper_role WITH LOGIN;
GRANT ALL ON SCHEMA metadata_sync_helpers TO metadata_sync_helper_role;
RESET client_min_messages;
@ -36,30 +43,14 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
SELECT citus_internal_add_partition_metadata ('test'::regclass, 'h', 'col_1', 0, 's');
ERROR: This is an internal Citus function can only be used in a distributed transaction
ROLLBACK;
-- connect back as super user, and then connect to the worker
-- with the superuser to make sure we can ingest metadata with
-- a regular user under the certain conditions
\c - postgres -
-- we don't need the table/schema anymore
SET client_min_messages TO ERROR;
DROP SCHEMA metadata_sync_helpers CASCADE;
DROP ROLE metadata_sync_helper_role;
\c - - - :worker_1_port
CREATE SCHEMA metadata_sync_helpers;
SET search_path TO metadata_sync_helpers;
CREATE TABLE test(col_1 int, col_2 int);
-- supress notice messages to make sure that the tests
-- do not diverge with enterprise
SET client_min_messages TO WARNING;
SET citus.enable_ddl_propagation TO OFF;
CREATE ROLE metadata_sync_helper_role WITH LOGIN;
GRANT ALL ON SCHEMA metadata_sync_helpers TO metadata_sync_helper_role;
RESET client_min_messages;
RESET citus.enable_ddl_propagation;
-- connect back with the regular user
\c - metadata_sync_helper_role - :worker_1_port
SET search_path TO metadata_sync_helpers;
@ -73,7 +64,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
SELECT citus_internal_add_partition_metadata ('test'::regclass, 'h', 'col_1', 0, 's');
ERROR: must be owner of table test
ROLLBACK;
@ -85,7 +76,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
SELECT citus_internal_update_relation_colocation ('test'::regclass, 10);
ERROR: must be owner of table test
ROLLBACK;
@ -99,7 +90,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 's');
citus_internal_add_partition_metadata
---------------------------------------------------------------------
@ -112,6 +103,54 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
1
(1 row)
ROLLBACK;
-- application_name with incorrect gpid
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
assign_distributed_transaction_id
---------------------------------------------------------------------
(1 row)
SET application_name to 'citus_internal gpid=not a correct gpid';
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 's');
ERROR: This is an internal Citus function can only be used in a distributed transaction
ROLLBACK;
-- application_name with empty gpid
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
assign_distributed_transaction_id
---------------------------------------------------------------------
(1 row)
SET application_name to 'citus_internal gpid=';
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 's');
ERROR: This is an internal Citus function can only be used in a distributed transaction
ROLLBACK;
-- empty application_name
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
assign_distributed_transaction_id
---------------------------------------------------------------------
(1 row)
SET application_name to '';
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 's');
ERROR: This is an internal Citus function can only be used in a distributed transaction
ROLLBACK;
-- application_name with incorrect prefix
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
assign_distributed_transaction_id
---------------------------------------------------------------------
(1 row)
SET application_name to 'citus gpid=10000000001';
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 's');
ERROR: This is an internal Citus function can only be used in a distributed transaction
ROLLBACK;
-- fails because there is no X distribution method
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
@ -121,7 +160,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'X', 'col_1', 0, 's');
ERROR: Metadata syncing is only allowed for hash, reference and local tables:X
ROLLBACK;
@ -133,7 +172,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'non_existing_col', 0, 's');
ERROR: column "non_existing_col" of relation "test_2" does not exist
ROLLBACK;
@ -145,7 +184,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
SELECT citus_internal_add_partition_metadata (NULL, 'h', 'non_existing_col', 0, 's');
ERROR: relation cannot be NULL
ROLLBACK;
@ -157,7 +196,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', -1, 's');
ERROR: Metadata syncing is only allowed for valid colocation id values.
ROLLBACK;
@ -169,7 +208,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'X', 'col_1', 0, 'X');
ERROR: Metadata syncing is only allowed for hash, reference and local tables:X
ROLLBACK;
@ -181,7 +220,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 's');
citus_internal_add_partition_metadata
@ -200,7 +239,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 0, 's');
citus_internal_add_partition_metadata
@ -219,7 +258,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', NULL, 0, 's');
ERROR: Distribution column cannot be NULL for relation "test_2"
@ -252,7 +291,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'X', 'col_1', 0, 's');
citus_internal_add_partition_metadata
---------------------------------------------------------------------
@ -268,7 +307,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
SELECT citus_internal_update_placement_metadata(1420007, 10000, 11111);
ERROR: could not find valid entry for shard xxxxx
@ -298,7 +337,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'X', 'col_1', 0, 's');
ERROR: role "non_existing_user" does not exist
ROLLBACK;
@ -329,7 +368,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
SELECT citus_internal_add_partition_metadata ('test_ref'::regclass, 'n', 'col_1', 0, 's');
ERROR: Reference or local tables cannot have distribution columns
ROLLBACK;
@ -341,7 +380,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
SELECT citus_internal_add_partition_metadata ('test_ref'::regclass, 'n', NULL, 0, 'A');
ERROR: Metadata syncing is only allowed for known replication models.
ROLLBACK;
@ -353,7 +392,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
SELECT citus_internal_add_partition_metadata ('test_ref'::regclass, 'n', NULL, 0, 'c');
ERROR: Local or references tables can only have 's' or 't' as the replication model.
ROLLBACK;
@ -368,7 +407,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
SELECT citus_internal_add_partition_metadata ('super_user_table'::regclass, 'h', 'col_1', 0, 's');
citus_internal_add_partition_metadata
---------------------------------------------------------------------
@ -387,7 +426,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('super_user_table'::regclass, 1420000::bigint, 't'::"char", '-2147483648'::text, '-1610612737'::text))
@ -402,7 +441,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '-2147483648'::text, '-1610612737'::text))
@ -417,7 +456,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
SELECT citus_internal_add_partition_metadata ('test_2'::regclass, 'h', 'col_1', 250, 's');
citus_internal_add_partition_metadata
---------------------------------------------------------------------
@ -445,7 +484,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
SELECT citus_internal_update_relation_colocation ('test_2'::regclass, 1231231232);
citus_internal_update_relation_colocation
---------------------------------------------------------------------
@ -461,7 +500,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, -1, 't'::"char", '-2147483648'::text, '-1610612737'::text))
@ -476,7 +515,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, 1420000, 'X'::"char", '-2147483648'::text, '-1610612737'::text))
@ -491,7 +530,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, 1420000, 't'::"char", NULL, '-1610612737'::text))
@ -506,7 +545,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", 'non-int'::text, '-1610612737'::text))
@ -521,7 +560,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '-1610612737'::text, '-2147483648'::text))
@ -536,7 +575,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '10'::text, '20'::text),
@ -548,13 +587,13 @@ ROLLBACK;
-- Now let's check valid pg_dist_object updates
-- check with non-existing object type
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
assign_distributed_transaction_id
---------------------------------------------------------------------
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation)
AS (VALUES ('non_existing_type', ARRAY['non_existing_user']::text[], ARRAY[]::text[], -1, 0, false))
@ -569,7 +608,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation)
AS (VALUES ('role', ARRAY['metadata_sync_helper_role']::text[], ARRAY[]::text[], -100, 0, false))
@ -583,7 +622,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation)
AS (VALUES ('role', ARRAY['metadata_sync_helper_role']::text[], ARRAY[]::text[], -1, -1, false))
@ -592,13 +631,13 @@ ERROR: colocationId must be a positive number
ROLLBACK;
-- check with non-existing object
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
assign_distributed_transaction_id
---------------------------------------------------------------------
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation)
AS (VALUES ('role', ARRAY['non_existing_user']::text[], ARRAY[]::text[], -1, 0, false))
@ -608,13 +647,13 @@ ROLLBACK;
-- since citus_internal_add_object_metadata is strict function returns NULL
-- if any parameter is NULL
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
assign_distributed_transaction_id
---------------------------------------------------------------------
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH distributed_object_data(typetext, objnames, objargs, distargumentindex, colocationid, force_delegation)
AS (VALUES ('role', ARRAY['metadata_sync_helper_role']::text[], ARRAY[]::text[], 0, NULL::int, false))
@ -635,7 +674,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
CREATE TABLE publication_test_table(id int);
CREATE PUBLICATION publication_test FOR TABLE publication_test_table;
@ -653,7 +692,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
CREATE FUNCTION distribution_test_function(int) RETURNS int
AS $$ SELECT $1 $$
@ -671,7 +710,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
CREATE TYPE distributed_test_type AS (a int, b int);
SET ROLE metadata_sync_helper_role;
@ -690,7 +729,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
UPDATE pg_dist_partition SET partmethod = 'X';
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
@ -709,7 +748,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '10'::text, '20'::text))
@ -736,7 +775,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_2'::regclass, 1420000::bigint, 't'::"char", '11'::text, '20'::text),
@ -767,7 +806,7 @@ BEGIN;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
SELECT citus_internal_update_relation_colocation('test_2'::regclass, 251);
ERROR: cannot colocate tables test_2 and test_3
ROLLBACK;
@ -779,7 +818,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_3'::regclass, 1420009::bigint, 't'::"char", '21'::text, '30'::text),
@ -806,7 +845,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_ref'::regclass, 1420003::bigint, 't'::"char", '-1610612737'::text, NULL))
@ -821,7 +860,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_ref'::regclass, 1420006::bigint, 't'::"char", NULL, NULL),
@ -837,7 +876,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('test_ref'::regclass, 1420006::bigint, 't'::"char", NULL, NULL))
@ -858,7 +897,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH shard_data(relationname, shardid, storagetype, shardminvalue, shardmaxvalue)
AS (VALUES ('super_user_table'::regclass, 1420007::bigint, 't'::"char", '11'::text, '20'::text))
@ -880,7 +919,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS
(VALUES (-10, 1, 0::bigint, 1::int, 1500000::bigint))
@ -895,7 +934,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS
(VALUES (1420000, 1, 0::bigint, 1::int, -10))
@ -910,7 +949,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS
(VALUES (1430100, 1, 0::bigint, 1::int, 10))
@ -925,7 +964,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS
(VALUES (1420000, 10, 0::bigint, 1::int, 1500000))
@ -940,7 +979,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS
(VALUES ( 1420000, 1, 0::bigint, 123123123::int, 1500000))
@ -970,7 +1009,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS
(VALUES (1420000, 1, 0::bigint, get_node_id(), 1500000),
@ -986,7 +1025,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS
(VALUES (1420007, 1, 0::bigint, get_node_id(), 1500000))
@ -1001,7 +1040,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH placement_data(shardid, shardstate, shardlength, groupid, placementid) AS
(VALUES (1420000, 1, 0::bigint, get_node_id(), 1500000),
@ -1042,7 +1081,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
SELECT citus_internal_update_relation_colocation('test_2'::regclass, 251);
citus_internal_update_relation_colocation
---------------------------------------------------------------------
@ -1059,7 +1098,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
SELECT citus_internal_update_placement_metadata(1420000, get_node_id(), get_node_id()+1000);
ERROR: Node with group id 1014 for shard placement xxxxx does not exist
@ -1072,7 +1111,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
SELECT citus_internal_update_placement_metadata(1420000, get_node_id()+10000, get_node_id());
ERROR: Active placement for shard xxxxx is not found on group:14
@ -1085,7 +1124,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
SELECT citus_internal_update_placement_metadata(0, get_node_id(), get_node_id()+1);
ERROR: Shard id does not exists: 0
@ -1098,7 +1137,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
SELECT citus_internal_update_placement_metadata(213123123123, get_node_id(), get_node_id()+1);
ERROR: Shard id does not exists: 213123123123
@ -1111,7 +1150,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
SELECT citus_internal_update_placement_metadata(1420007, get_node_id(), get_node_id()+1);
ERROR: must be owner of table super_user_table
@ -1124,7 +1163,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH shard_data(shardid)
AS (VALUES (1420007))
@ -1133,7 +1172,7 @@ ERROR: must be owner of table super_user_table
ROLLBACK;
-- the user only allowed to delete shards in a distributed transaction
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH shard_data(shardid)
AS (VALUES (1420007))
@ -1148,7 +1187,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH shard_data(shardid)
AS (VALUES (1420100))
@ -1175,7 +1214,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
WITH shard_data(shardid)
AS (VALUES (1420000))
@ -1209,7 +1248,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
-- with an ugly trick, update the repmodel
-- so that making two tables colocated fails
UPDATE pg_dist_partition SET repmodel = 't'
@ -1224,7 +1263,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
-- with an ugly trick, update the vartype of table from int to bigint
-- so that making two tables colocated fails
UPDATE pg_dist_partition SET partkey = '{VAR :varno 1 :varattno 1 :vartype 20 :vartypmod -1 :varcollid 0 :varlevelsup 0 :varnoold 1 :varoattno 1 :location -1}'
@ -1239,7 +1278,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
-- with an ugly trick, update the partmethod of the table to not-valid
-- so that making two tables colocated fails
UPDATE pg_dist_partition SET partmethod = ''
@ -1254,7 +1293,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
-- with an ugly trick, update the partmethod of the table to not-valid
-- so that making two tables colocated fails
UPDATE pg_dist_partition SET partmethod = 'a'
@ -1272,7 +1311,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
SELECT citus_internal_add_partition_metadata ('test_5'::regclass, 'h', 'int_col', 500, 's');
citus_internal_add_partition_metadata
@ -1295,7 +1334,7 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
(1 row)
SET application_name to 'citus_internal';
SET application_name to 'citus_internal gpid=10000000001';
\set VERBOSITY terse
SELECT citus_internal_add_partition_metadata ('test_7'::regclass, 'h', 'text_col', 500, 's');
citus_internal_add_partition_metadata
@ -1318,4 +1357,7 @@ SET client_min_messages TO ERROR;
SET citus.enable_ddl_propagation TO OFF;
DROP OWNED BY metadata_sync_helper_role;
DROP ROLE metadata_sync_helper_role;
\c - - - :master_port
-- cleanup
SET client_min_messages TO ERROR;
DROP SCHEMA metadata_sync_helpers CASCADE;

View File

@ -520,19 +520,11 @@ CONTEXT: PL/pgSQL function deparse_and_run_on_workers(text) line XX at RAISE
(localhost,57638,t,"DROP FUNCTION")
(2 rows)
set citus.enable_ddl_propagation to on;
-- create schema with weird names
CREATE SCHEMA "CiTuS.TeeN";
CREATE SCHEMA "CiTUS.TEEN2";
SELECT run_command_on_workers($$
CREATE SCHEMA IF NOT EXISTS "CiTuS.TeeN";
CREATE SCHEMA IF NOT EXISTS "CiTUS.TEEN2";
$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"CREATE SCHEMA")
(localhost,57638,t,"CREATE SCHEMA")
(2 rows)
set citus.enable_ddl_propagation to off;
-- create table with weird names
CREATE FUNCTION "CiTuS.TeeN"."TeeNFunCT10N.1!?!"() RETURNS TEXT
AS $$ SELECT 'test function without params' $$
@ -717,15 +709,4 @@ SET client_min_messages TO WARNING; -- suppress cascading objects dropping
DROP SCHEMA "CiTuS.TeeN" CASCADE;
DROP SCHEMA "CiTUS.TEEN2" CASCADE;
DROP SCHEMA function_tests CASCADE;
SELECT run_command_on_workers($$
DROP SCHEMA "CiTuS.TeeN" CASCADE;
DROP SCHEMA "CiTUS.TEEN2" CASCADE;
DROP SCHEMA function_tests CASCADE;
$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
DROP ROLE function_role;

View File

@ -741,10 +741,3 @@ SELECT citus_remove_node('localhost', :master_port);
(1 row)
SELECT run_command_on_workers($$ DROP SCHEMA IF EXISTS fix_idx_names CASCADE $$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)

View File

@ -440,6 +440,7 @@ drop cascades to constraint fkey_1 on table test_4
ROLLBACK;
-- Test schemas
BEGIN;
SET LOCAL citus.multi_shard_modify_mode TO 'sequential';
CREATE SCHEMA fkey_intermediate_schema_1;
CREATE SCHEMA fkey_intermediate_schema_2;
SET search_path TO fkey_graph, fkey_intermediate_schema_1, fkey_intermediate_schema_2;
@ -526,6 +527,7 @@ drop cascades to constraint test_8_id_fkey on table test_8
ROLLBACK;
BEGIN;
SET LOCAL citus.multi_shard_modify_mode TO 'sequential';
CREATE SCHEMA fkey_intermediate_schema_1;
CREATE SCHEMA fkey_intermediate_schema_2;
SET search_path TO fkey_graph, fkey_intermediate_schema_1, fkey_intermediate_schema_2;

View File

@ -53,6 +53,7 @@ SELECT * FROM table1 JOIN add(3,5) sum ON (id = sum) ORDER BY id ASC;
-- Check join of plpgsql functions
-- a function returning a single integer
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION increment(i integer) RETURNS integer AS $$
BEGIN
RETURN i + 1;
@ -76,7 +77,6 @@ BEGIN
RETURN QUERY SELECT x FROM generate_series(first_value, first_value+k-1) f(x);
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
SELECT *
FROM table1 JOIN next_k_integers(3,2) next_integers ON (id = next_integers.result)
ORDER BY id ASC;
@ -119,6 +119,7 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT f.f1, f.f2 FR
-- a stable function
CREATE OR REPLACE FUNCTION the_minimum_id()
RETURNS INTEGER STABLE AS 'SELECT min(id) FROM table1' LANGUAGE SQL;
RESET citus.enable_metadata_sync;
SELECT * FROM table1 JOIN the_minimum_id() min_id ON (id = min_id);
DEBUG: generating subplan XXX_1 for subquery SELECT min_id FROM functions_in_joins.the_minimum_id() min_id(min_id)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, table1.data, min_id.min_id FROM (functions_in_joins.table1 JOIN (SELECT intermediate_result.min_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(min_id integer)) min_id ON ((table1.id OPERATOR(pg_catalog.=) min_id.min_id)))
@ -171,6 +172,7 @@ CREATE TYPE min_and_max AS (
maximum INT
);
SET client_min_messages TO DEBUG1;
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION max_and_min () RETURNS
min_and_max AS $$
DECLARE
@ -180,6 +182,7 @@ begin
return result;
end;
$$ language plpgsql;
RESET citus.enable_metadata_sync;
SELECT * FROM table1 JOIN max_and_min() m ON (m.maximum = data OR m.minimum = data) ORDER BY 1,2,3,4;
DEBUG: generating subplan XXX_1 for subquery SELECT minimum, maximum FROM functions_in_joins.max_and_min() m(minimum, maximum)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, table1.data, m.minimum, m.maximum FROM (functions_in_joins.table1 JOIN (SELECT intermediate_result.minimum, intermediate_result.maximum FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(minimum integer, maximum integer)) m ON (((m.maximum OPERATOR(pg_catalog.=) table1.data) OR (m.minimum OPERATOR(pg_catalog.=) table1.data)))) ORDER BY table1.id, table1.data, m.minimum, m.maximum

View File

@ -57,6 +57,7 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, ta
-- Check join of plpgsql functions
-- a function returning a single integer
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION increment(i integer) RETURNS integer AS $$
BEGIN
RETURN i + 1;
@ -80,7 +81,6 @@ BEGIN
RETURN QUERY SELECT x FROM generate_series(first_value, first_value+k-1) f(x);
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
SELECT *
FROM table1 JOIN next_k_integers(3,2) next_integers ON (id = next_integers.result)
ORDER BY id ASC;
@ -123,6 +123,7 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT f.f1, f.f2 FR
-- a stable function
CREATE OR REPLACE FUNCTION the_minimum_id()
RETURNS INTEGER STABLE AS 'SELECT min(id) FROM table1' LANGUAGE SQL;
RESET citus.enable_metadata_sync;
SELECT * FROM table1 JOIN the_minimum_id() min_id ON (id = min_id);
DEBUG: generating subplan XXX_1 for subquery SELECT min_id FROM functions_in_joins.the_minimum_id() min_id(min_id)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, table1.data, min_id.min_id FROM (functions_in_joins.table1 JOIN (SELECT intermediate_result.min_id FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(min_id integer)) min_id ON ((table1.id OPERATOR(pg_catalog.=) min_id.min_id)))
@ -175,6 +176,7 @@ CREATE TYPE min_and_max AS (
maximum INT
);
SET client_min_messages TO DEBUG1;
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION max_and_min () RETURNS
min_and_max AS $$
DECLARE
@ -184,6 +186,7 @@ begin
return result;
end;
$$ language plpgsql;
RESET citus.enable_metadata_sync;
SELECT * FROM table1 JOIN max_and_min() m ON (m.maximum = data OR m.minimum = data) ORDER BY 1,2,3,4;
DEBUG: generating subplan XXX_1 for subquery SELECT minimum, maximum FROM functions_in_joins.max_and_min() m(minimum, maximum)
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, table1.data, m.minimum, m.maximum FROM (functions_in_joins.table1 JOIN (SELECT intermediate_result.minimum, intermediate_result.maximum FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(minimum integer, maximum integer)) m ON (((m.maximum OPERATOR(pg_catalog.=) table1.data) OR (m.minimum OPERATOR(pg_catalog.=) table1.data)))) ORDER BY table1.id, table1.data, m.minimum, m.maximum

View File

@ -75,11 +75,6 @@ GRANT ALL ON TABLE test_1420000 TO full_access;
GRANT SELECT ON TABLE test_1420000 TO read_access;
GRANT ALL ON TABLE test_1420002 TO full_access;
GRANT SELECT ON TABLE test_1420002 TO read_access;
CREATE SCHEMA full_access_user_schema;
REVOKE ALL ON SCHEMA full_access_user_schema FROM PUBLIC;
GRANT USAGE ON SCHEMA full_access_user_schema TO full_access;
GRANT ALL ON SCHEMA full_access_user_schema TO full_access;
GRANT USAGE ON SCHEMA full_access_user_schema TO usage_access;
\c - - - :worker_2_port
CREATE USER full_access;
NOTICE: not propagating CREATE ROLE/USER commands to worker nodes
@ -102,11 +97,6 @@ GRANT ALL ON TABLE test_1420001 TO full_access;
GRANT SELECT ON TABLE test_1420001 TO read_access;
GRANT ALL ON TABLE test_1420003 TO full_access;
GRANT SELECT ON TABLE test_1420003 TO read_access;
CREATE SCHEMA full_access_user_schema;
REVOKE ALL ON SCHEMA full_access_user_schema FROM PUBLIC;
GRANT USAGE ON SCHEMA full_access_user_schema TO full_access;
GRANT ALL ON SCHEMA full_access_user_schema TO full_access;
GRANT USAGE ON SCHEMA full_access_user_schema TO usage_access;
\c - - - :master_port
SET citus.shard_replication_factor TO 1;
-- create prepare tests

View File

@ -63,9 +63,6 @@ CREATE TABLE citus_local_table(a int);
SELECT citus_add_local_table_to_metadata('citus_local_table');
ERROR: operation is not allowed on this node
HINT: Connect to the coordinator and run it again.
-- create schema to test schema support
CREATE SCHEMA citus_mx_test_schema_join_1;
CREATE SCHEMA citus_mx_test_schema_join_2;
SET search_path TO citus_mx_test_schema;
-- create operator
CREATE OPERATOR citus_mx_test_schema.=== (
@ -78,9 +75,6 @@ CREATE OPERATOR citus_mx_test_schema.=== (
);
-- now create required stuff in the worker 2
\c - - - :worker_2_port
-- create schema to test schema support
CREATE SCHEMA citus_mx_test_schema_join_1;
CREATE SCHEMA citus_mx_test_schema_join_2;
SET search_path TO citus_mx_test_schema;
-- create operator
CREATE OPERATOR citus_mx_test_schema.=== (

View File

@ -304,13 +304,6 @@ SELECT * FROM seg_test;
\c - - - :master_port
CREATE SCHEMA ext_owned_tables;
SELECT run_command_on_workers($$CREATE SCHEMA ext_owned_tables;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"CREATE SCHEMA")
(localhost,57638,t,"CREATE SCHEMA")
(2 rows)
SET search_path TO ext_owned_tables;
CREATE sequence my_seq_ext_1;
SELECT run_command_on_workers($$CREATE sequence ext_owned_tables.my_seq_ext_1;$$);

View File

@ -622,18 +622,6 @@ PL/pgSQL function mx_call_func(integer,integer) line XX at assignment
(1 row)
-- test forms we don't distribute
select * from mx_call_func(2, 0);
DEBUG: generating subplan XXX_1 for subquery SELECT sum((t1.val OPERATOR(pg_catalog.+) t2.val)) AS sum FROM (multi_mx_function_call_delegation.mx_call_dist_table_1 t1 JOIN multi_mx_function_call_delegation.mx_call_dist_table_2 t2 ON ((t1.id OPERATOR(pg_catalog.=) t2.id)))
CONTEXT: PL/pgSQL assignment "y := y + (select sum(t1.val + t2.val) from multi_mx_function_call_delegation.mx_call_dist_table_1 t1 join multi_mx_function_call_delegation.mx_call_dist_table_2 t2 on t1.id = t2.id)"
PL/pgSQL function mx_call_func(integer,integer) line XX at assignment
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT ((3 OPERATOR(pg_catalog.+) (SELECT intermediate_result.sum FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(sum bigint))))::integer
CONTEXT: PL/pgSQL assignment "y := y + (select sum(t1.val + t2.val) from multi_mx_function_call_delegation.mx_call_dist_table_1 t1 join multi_mx_function_call_delegation.mx_call_dist_table_2 t2 on t1.id = t2.id)"
PL/pgSQL function mx_call_func(integer,integer) line XX at assignment
y
---------------------------------------------------------------------
29
(1 row)
select mx_call_func(2, 0) where mx_call_func(0, 2) = 0;
DEBUG: generating subplan XXX_1 for subquery SELECT sum((t1.val OPERATOR(pg_catalog.+) t2.val)) AS sum FROM (multi_mx_function_call_delegation.mx_call_dist_table_1 t1 JOIN multi_mx_function_call_delegation.mx_call_dist_table_2 t2 ON ((t1.id OPERATOR(pg_catalog.=) t2.id)))
CONTEXT: PL/pgSQL assignment "y := y + (select sum(t1.val + t2.val) from multi_mx_function_call_delegation.mx_call_dist_table_1 t1 join multi_mx_function_call_delegation.mx_call_dist_table_2 t2 on t1.id = t2.id)"
@ -663,6 +651,58 @@ PL/pgSQL function mx_call_func(integer,integer) line XX at assignment
29 | 27
(1 row)
-- regular call in FROM can be pushed down
select * from mx_call_func(2, 0);
DEBUG: pushing down the function call
y
---------------------------------------------------------------------
28
(1 row)
-- prepared statement with 6 invocations to trigger generic plan
prepare call_func(int, int) as select $1 from mx_call_func($1, $2);
execute call_func(2, 0);
DEBUG: pushing down the function call
?column?
---------------------------------------------------------------------
2
(1 row)
execute call_func(2, 0);
DEBUG: pushing down the function call
?column?
---------------------------------------------------------------------
2
(1 row)
execute call_func(2, 0);
DEBUG: pushing down the function call
?column?
---------------------------------------------------------------------
2
(1 row)
execute call_func(2, 0);
DEBUG: pushing down the function call
?column?
---------------------------------------------------------------------
2
(1 row)
execute call_func(2, 0);
DEBUG: pushing down the function call
?column?
---------------------------------------------------------------------
2
(1 row)
execute call_func(2, 0);
DEBUG: pushing down the function call
?column?
---------------------------------------------------------------------
2
(1 row)
-- we do not delegate the call, but do push down the query
-- that result in remote execution from workers
select mx_call_func(id, 0) from mx_call_dist_table_1;

View File

@ -622,18 +622,6 @@ PL/pgSQL function mx_call_func(integer,integer) line XX at assignment
(1 row)
-- test forms we don't distribute
select * from mx_call_func(2, 0);
DEBUG: generating subplan XXX_1 for subquery SELECT sum((t1.val OPERATOR(pg_catalog.+) t2.val)) AS sum FROM (multi_mx_function_call_delegation.mx_call_dist_table_1 t1 JOIN multi_mx_function_call_delegation.mx_call_dist_table_2 t2 ON ((t1.id OPERATOR(pg_catalog.=) t2.id)))
CONTEXT: SQL statement "SELECT y + (select sum(t1.val + t2.val) from multi_mx_function_call_delegation.mx_call_dist_table_1 t1 join multi_mx_function_call_delegation.mx_call_dist_table_2 t2 on t1.id = t2.id)"
PL/pgSQL function mx_call_func(integer,integer) line XX at assignment
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT (3 OPERATOR(pg_catalog.+) (SELECT intermediate_result.sum FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(sum bigint)))
CONTEXT: SQL statement "SELECT y + (select sum(t1.val + t2.val) from multi_mx_function_call_delegation.mx_call_dist_table_1 t1 join multi_mx_function_call_delegation.mx_call_dist_table_2 t2 on t1.id = t2.id)"
PL/pgSQL function mx_call_func(integer,integer) line XX at assignment
y
---------------------------------------------------------------------
29
(1 row)
select mx_call_func(2, 0) where mx_call_func(0, 2) = 0;
DEBUG: generating subplan XXX_1 for subquery SELECT sum((t1.val OPERATOR(pg_catalog.+) t2.val)) AS sum FROM (multi_mx_function_call_delegation.mx_call_dist_table_1 t1 JOIN multi_mx_function_call_delegation.mx_call_dist_table_2 t2 ON ((t1.id OPERATOR(pg_catalog.=) t2.id)))
CONTEXT: SQL statement "SELECT y + (select sum(t1.val + t2.val) from multi_mx_function_call_delegation.mx_call_dist_table_1 t1 join multi_mx_function_call_delegation.mx_call_dist_table_2 t2 on t1.id = t2.id)"
@ -663,6 +651,58 @@ PL/pgSQL function mx_call_func(integer,integer) line XX at assignment
29 | 27
(1 row)
-- regular call in FROM can be pushed down
select * from mx_call_func(2, 0);
DEBUG: pushing down the function call
y
---------------------------------------------------------------------
28
(1 row)
-- prepared statement with 6 invocations to trigger generic plan
prepare call_func(int, int) as select $1 from mx_call_func($1, $2);
execute call_func(2, 0);
DEBUG: pushing down the function call
?column?
---------------------------------------------------------------------
2
(1 row)
execute call_func(2, 0);
DEBUG: pushing down the function call
?column?
---------------------------------------------------------------------
2
(1 row)
execute call_func(2, 0);
DEBUG: pushing down the function call
?column?
---------------------------------------------------------------------
2
(1 row)
execute call_func(2, 0);
DEBUG: pushing down the function call
?column?
---------------------------------------------------------------------
2
(1 row)
execute call_func(2, 0);
DEBUG: pushing down the function call
?column?
---------------------------------------------------------------------
2
(1 row)
execute call_func(2, 0);
DEBUG: pushing down the function call
?column?
---------------------------------------------------------------------
2
(1 row)
-- we do not delegate the call, but do push down the query
-- that result in remote execution from workers
select mx_call_func(id, 0) from mx_call_dist_table_1;

View File

@ -444,11 +444,9 @@ ALTER TABLE "CiTuS.TeAeN"."TeeNTabLE.1!?!" ADD COLUMN new_col INT;
-- set the search_path to not existing schema
SET search_path TO not_existing_schema;
ALTER TABLE "CiTuS.TeAeN"."TeeNTabLE.1!?!" DROP COLUMN new_col;
SET client_min_messages TO ERROR;
DROP SCHEMA mx_ddl_schema_1, mx_ddl_schema_2, "CiTuS.TeAeN" CASCADE;
NOTICE: drop cascades to 3 other objects
DETAIL: drop cascades to table "CiTuS.TeAeN"."TeeNTabLE.1!?!"
drop cascades to table mx_ddl_schema_2.table_2
drop cascades to table mx_ddl_schema_1.table_1
RESET client_min_messages;
-- test if ALTER TABLE SET SCHEMA sets the original table in the worker
SET search_path TO public;
CREATE SCHEMA mx_old_schema;
@ -462,11 +460,13 @@ SELECT create_distributed_table('mx_old_schema.table_set_schema', 'id');
CREATE SCHEMA mx_new_schema;
SELECT objid::oid::regnamespace as "Distributed Schemas"
FROM citus.pg_dist_object
WHERE objid::oid::regnamespace IN ('mx_old_schema', 'mx_new_schema');
WHERE objid::oid::regnamespace IN ('mx_old_schema', 'mx_new_schema')
ORDER BY "Distributed Schemas";
Distributed Schemas
---------------------------------------------------------------------
mx_old_schema
(1 row)
mx_new_schema
(2 rows)
\c - - - :worker_1_port
SELECT table_schema AS "Table's Schema" FROM information_schema.tables WHERE table_name='table_set_schema';

View File

@ -889,7 +889,8 @@ SELECT objid::oid::regnamespace as "Distributed Schemas"
Distributed Schemas
---------------------------------------------------------------------
old_schema
(1 row)
new_schema
(2 rows)
\c - - - :worker_1_port
SELECT table_schema AS "Shards' Schema"
@ -947,7 +948,8 @@ SELECT objid::oid::regnamespace as "Distributed Schemas"
WHERE objid='new_schema'::regnamespace::oid;
Distributed Schemas
---------------------------------------------------------------------
(0 rows)
new_schema
(1 row)
\c - - - :worker_1_port
SELECT table_schema AS "Shards' Schema"
@ -1013,7 +1015,8 @@ SELECT objid::oid::regnamespace as "Distributed Schemas"
Distributed Schemas
---------------------------------------------------------------------
old_schema
(1 row)
new_schema
(2 rows)
\c - - - :worker_1_port
SELECT table_schema AS "Shards' Schema", COUNT(*) AS "Counts"
@ -1374,7 +1377,7 @@ BEGIN;
ALTER SCHEMA bar RENAME TO foo;
ERROR: cannot create or modify schema because there was a parallel operation on a distributed table in the transaction
DETAIL: When creating or altering a schema, Citus needs to perform all operations over a single connection per node to ensure consistency.
DETAIL: When creating, altering, or dropping a schema, Citus needs to perform all operations over a single connection per node to ensure consistency.
HINT: Try re-running the transaction with "SET LOCAL citus.multi_shard_modify_mode TO 'sequential';"
ROLLBACK;
BEGIN;
@ -1389,10 +1392,18 @@ BEGIN;
ROLLBACK;
-- Clean up the created schema
SET client_min_messages TO WARNING;
DROP SCHEMA run_test_schema CASCADE;
DROP SCHEMA test_schema_support_join_1 CASCADE;
DROP SCHEMA test_schema_support_join_2 CASCADE;
DROP SCHEMA "Citus'Teen123" CASCADE;
DROP SCHEMA "CiTUS.TEEN2" CASCADE;
DROP SCHEMA bar CASCADE;
DROP SCHEMA test_schema_support CASCADE;
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object
WHERE classid=2615 and objid IN (select oid from pg_namespace where nspname='run_test_schema');
pg_identify_object_as_address
---------------------------------------------------------------------
(schema,{run_test_schema},{})
(1 row)
DROP SCHEMA run_test_schema, test_schema_support_join_1, test_schema_support_join_2, "Citus'Teen123", "CiTUS.TEEN2", bar, test_schema_support CASCADE;
-- verify that the dropped schema is removed from worker's pg_dist_object
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object
WHERE classid=2615 and objid IN (select oid from pg_namespace where nspname='run_test_schema');
pg_identify_object_as_address
---------------------------------------------------------------------
(0 rows)

View File

@ -370,70 +370,6 @@ SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
DROP SEQUENCE sequence_default_0.sequence_3 CASCADE;
NOTICE: drop cascades to default value for column y of table seq_test_3
DROP SCHEMA sequence_default_0;
-- DROP SCHEMA problem: expected since we don't propagate DROP SCHEMA
CREATE TABLE seq_test_5 (x int, y int);
SELECT create_distributed_table('seq_test_5','x');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE SCHEMA sequence_default_1;
CREATE SEQUENCE sequence_default_1.seq_5;
ALTER TABLE seq_test_5 ADD COLUMN a bigint DEFAULT nextval('sequence_default_1.seq_5');
DROP SCHEMA sequence_default_1 CASCADE;
NOTICE: drop cascades to 2 other objects
DETAIL: drop cascades to sequence sequence_default_1.seq_5
drop cascades to default value for column a of table seq_test_5
-- sequence is gone from coordinator
INSERT INTO seq_test_5 VALUES (1, 2) RETURNING *;
x | y | a
---------------------------------------------------------------------
1 | 2 |
(1 row)
-- but is still present on worker
\c - - - :worker_1_port
INSERT INTO sequence_default.seq_test_5 VALUES (1, 2) RETURNING *;
x | y | a
---------------------------------------------------------------------
1 | 2 | 281474976710657
(1 row)
\c - - - :master_port
SET citus.shard_replication_factor TO 1;
SET search_path = sequence_default, public;
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
-- apply workaround
SELECT run_command_on_workers('DROP SCHEMA sequence_default_1 CASCADE');
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
-- now the sequence is gone from the worker as well
\c - - - :worker_1_port
INSERT INTO sequence_default.seq_test_5 VALUES (1, 2) RETURNING *;
x | y | a
---------------------------------------------------------------------
1 | 2 |
(1 row)
\c - - - :master_port
SET citus.shard_replication_factor TO 1;
SET search_path = sequence_default, public;
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
start_metadata_sync_to_node
---------------------------------------------------------------------
(1 row)
-- check some more complex cases
CREATE SEQUENCE seq_6;
CREATE TABLE seq_test_6 (x int, t timestamptz DEFAULT now(), s int DEFAULT nextval('seq_6'), m int) PARTITION BY RANGE (t);
@ -569,14 +505,7 @@ SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
(1 row)
DROP SCHEMA sequence_default_8;
SELECT run_command_on_workers('DROP SCHEMA IF EXISTS sequence_default_8 CASCADE');
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
DROP SCHEMA sequence_default_8 CASCADE;
-- cannot use more than one sequence in a column default
CREATE SEQUENCE seq_9;
CREATE SEQUENCE seq_10;
@ -914,17 +843,6 @@ ERROR: nextval: reached maximum value of sequence "seq_14" (32767)
-- table with default column is added
CREATE SCHEMA test_schema_for_sequence_default_propagation;
CREATE SEQUENCE test_schema_for_sequence_default_propagation.seq_10;
-- Both should return 0 rows
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation.seq_10'::regclass);
pg_identify_object_as_address
---------------------------------------------------------------------
(0 rows)
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation'::regnamespace);
pg_identify_object_as_address
---------------------------------------------------------------------
(0 rows)
-- Create distributed table with default column to propagate dependencies
CREATE TABLE test_seq_dist(a int, x BIGINT DEFAULT nextval('test_schema_for_sequence_default_propagation.seq_10'));
SELECT create_distributed_table('test_seq_dist', 'a');
@ -998,13 +916,6 @@ DROP TABLE test_seq_dist;
DROP TABLE sequence_default.seq_test_7_par;
SET client_min_messages TO error; -- suppress cascading objects dropping
DROP SCHEMA sequence_default CASCADE;
SELECT run_command_on_workers('DROP SCHEMA IF EXISTS sequence_default CASCADE');
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"DROP SCHEMA")
(localhost,57638,t,"DROP SCHEMA")
(2 rows)
SELECT master_remove_node('localhost', :master_port);
master_remove_node
---------------------------------------------------------------------

View File

@ -174,16 +174,6 @@ ERROR: unsupported lockmode 8
\c - postgres - :master_port;
SET client_min_messages TO ERROR;
DROP SCHEMA "Mx Super User" CASCADE;
\c - postgres - :worker_1_port;
SET client_min_messages TO ERROR;
SET citus.enable_ddl_propagation TO OFF;
CREATE SCHEMA "Mx Regular User";
GRANT ALL ON SCHEMA "Mx Regular User" TO regular_mx_user;
\c - postgres - :worker_2_port;
SET client_min_messages TO ERROR;
SET citus.enable_ddl_propagation TO OFF;
CREATE SCHEMA "Mx Regular User";
GRANT ALL ON SCHEMA "Mx Regular User" TO regular_mx_user;
-- now connect with that user
\c - regular_mx_user - :master_port
SET search_path TO "Mx Regular User";

View File

@ -76,6 +76,10 @@ select count(*) from test where a = 0;
(1 row)
ALTER SYSTEM SET citus.node_conninfo = 'sslmode=doesnotexist';
-- we cannot set application name
ALTER SYSTEM SET citus.node_conninfo = 'application_name=XXX';
ERROR: invalid value for parameter "citus.node_conninfo": "application_name=XXX"
DETAIL: Prohibited conninfo keyword detected: application_name
BEGIN;
-- Should still work (no SIGHUP yet);
select count(*) from test where a = 0;

View File

@ -12,6 +12,7 @@ SET log_error_verbosity TO TERSE;
SET citus.enable_repartition_joins TO ON;
-- Function that parses explain output as JSON
-- copied from multi_explain.sql
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION explain_json(query text)
RETURNS jsonb
AS $BODY$
@ -22,6 +23,7 @@ BEGIN
RETURN result;
END;
$BODY$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
SHOW log_error_verbosity;
log_error_verbosity
---------------------------------------------------------------------

View File

@ -412,7 +412,7 @@ SELECT * FROM run_command_on_workers($$ SELECT distribution_argument_index FROM
localhost | 57638 | t | 1
(2 rows)
-- Show that dropping schema doesn't affect the worker node
-- Show that the schema is dropped on worker node as well
DROP SCHEMA local_schema CASCADE;
NOTICE: drop cascades to 3 other objects
DETAIL: drop cascades to function plpgsql_dist_function(text)
@ -426,8 +426,8 @@ SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{local_schema}%';$$) ORDER BY 1,2;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | (schema,{local_schema},{})
localhost | 57638 | t | (schema,{local_schema},{})
localhost | 57637 | t |
localhost | 57638 | t |
(2 rows)
-- Show that extension and dependent sequence also created and marked as distributed

View File

@ -81,22 +81,5 @@ ON TRUE
---------------------------------------------------------------------
(0 rows)
-- find non-distributed dependencies of the local table test2
CREATE SCHEMA objectprop2;
create table objectprop2.test2(a int, b t1);
SET search_path TO objectprop2;
SELECT
pg_identify_object(t.classid, t.objid, t.objsubid)
FROM
(SELECT * FROM pg_get_object_address('table', '{test2}', '{}')) as addr
JOIN LATERAL
"object prop".citus_get_dependencies_for_object(addr.classid, addr.objid, addr.objsubid) as t(classid oid, objid oid, objsubid int)
ON TRUE
ORDER BY 1;
pg_identify_object
---------------------------------------------------------------------
(schema,,objectprop2,objectprop2)
(1 row)
SET client_min_messages TO ERROR;
DROP SCHEMA "object prop", objectprop2 CASCADE;
DROP SCHEMA "object prop" CASCADE;

View File

@ -382,10 +382,34 @@ SELECT 1 from master_remove_node('localhost', :worker_2_port);
1
(1 row)
-- Test extension function incorrect distribution argument
CREATE TABLE test_extension_function(col varchar);
CREATE EXTENSION seg;
-- Missing distribution argument
SELECT create_distributed_function('seg_in(cstring)');
ERROR: Extension functions(seg_in) without distribution argument are not supported.
-- Missing colocation argument
SELECT create_distributed_function('seg_in(cstring)', '$1');
ERROR: cannot distribute the function "seg_in" since there is no table to colocate with
HINT: Provide a distributed table via "colocate_with" option to create_distributed_function()
-- Incorrect distribution argument
SELECT create_distributed_function('seg_in(cstring)', '$2', colocate_with:='test_extension_function');
ERROR: cannot distribute the function "seg_in" since the distribution argument is not valid
HINT: Either provide a valid function argument name or a valid "$paramIndex" to create_distributed_function()
-- Colocated table is not distributed
SELECT create_distributed_function('seg_in(cstring)', '$1', 'test_extension_function');
ERROR: relation test_extension_function is not distributed
DROP EXTENSION seg;
SET citus.shard_replication_factor TO 1;
SELECT create_distributed_table('test_extension_function', 'col', colocate_with := 'none');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- now, create a type that depends on another type, which
-- finally depends on an extension
BEGIN;
SET citus.shard_replication_factor TO 1;
CREATE EXTENSION seg;
CREATE EXTENSION isn;
CREATE TYPE test_type AS (a int, b seg);
@ -403,9 +427,39 @@ BEGIN;
create_reference_table
---------------------------------------------------------------------
(1 row)
-- Distribute an extension-function
SELECT create_distributed_function('seg_in(cstring)', '$1', 'test_extension_function');
create_distributed_function
---------------------------------------------------------------------
(1 row)
COMMIT;
-- Check the pg_dist_object
SELECT pg_proc.proname as DistributedFunction
FROM citus.pg_dist_object, pg_proc
WHERE pg_proc.proname = 'seg_in' and
pg_proc.oid = citus.pg_dist_object.objid and
classid = 'pg_proc'::regclass;
distributedfunction
---------------------------------------------------------------------
seg_in
(1 row)
SELECT run_command_on_workers($$
SELECT count(*)
FROM citus.pg_dist_object, pg_proc
WHERE pg_proc.proname = 'seg_in' and
pg_proc.oid = citus.pg_dist_object.objid and
classid = 'pg_proc'::regclass;
$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,1)
(1 row)
-- add the node back
SELECT 1 from master_add_node('localhost', :worker_2_port);
?column?
@ -427,5 +481,145 @@ SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname
(localhost,57638,t,2)
(2 rows)
-- Check the pg_dist_object on the both nodes
SELECT run_command_on_workers($$
SELECT count(*)
FROM citus.pg_dist_object, pg_proc
WHERE pg_proc.proname = 'seg_in' and
pg_proc.oid = citus.pg_dist_object.objid and
classid = 'pg_proc'::regclass;
$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,1)
(localhost,57638,t,1)
(2 rows)
DROP EXTENSION seg CASCADE;
-- Recheck the pg_dist_object
SELECT pg_proc.proname as DistributedFunction
FROM citus.pg_dist_object, pg_proc
WHERE pg_proc.proname = 'seg_in' and
pg_proc.oid = citus.pg_dist_object.objid and
classid = 'pg_proc'::regclass;
distributedfunction
---------------------------------------------------------------------
(0 rows)
SELECT run_command_on_workers($$
SELECT count(*)
FROM citus.pg_dist_object, pg_proc
WHERE pg_proc.proname = 'seg_in' and
pg_proc.oid = citus.pg_dist_object.objid and
classid = 'pg_proc'::regclass;
$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,0)
(localhost,57638,t,0)
(2 rows)
-- Distribute an extension-function where extension is not in pg_dist_object
SET citus.enable_ddl_propagation TO false;
CREATE EXTENSION seg;
SET citus.enable_ddl_propagation TO true;
-- Check the extension in pg_dist_object
SELECT count(*) FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND
objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
count
---------------------------------------------------------------------
0
(1 row)
SELECT run_command_on_workers($$
SELECT count(*)
FROM citus.pg_dist_object, pg_proc
WHERE pg_proc.proname = 'seg_in' and
pg_proc.oid = citus.pg_dist_object.objid and
classid = 'pg_proc'::regclass;
$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,0)
(localhost,57638,t,0)
(2 rows)
SELECT create_distributed_function('seg_in(cstring)', '$1', 'test_extension_function');
create_distributed_function
---------------------------------------------------------------------
(1 row)
-- Recheck the extension in pg_dist_object
SELECT count(*) FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND
objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
count
---------------------------------------------------------------------
1
(1 row)
SELECT pg_proc.proname as DistributedFunction
FROM citus.pg_dist_object, pg_proc
WHERE pg_proc.proname = 'seg_in' and
pg_proc.oid = citus.pg_dist_object.objid and
classid = 'pg_proc'::regclass;
distributedfunction
---------------------------------------------------------------------
seg_in
(1 row)
SELECT run_command_on_workers($$
SELECT count(*)
FROM citus.pg_dist_object, pg_proc
WHERE pg_proc.proname = 'seg_in' and
pg_proc.oid = citus.pg_dist_object.objid and
classid = 'pg_proc'::regclass;
$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,1)
(localhost,57638,t,1)
(2 rows)
DROP EXTENSION seg;
DROP TABLE test_extension_function;
-- Test extension function altering distribution argument
BEGIN;
SET citus.shard_replication_factor = 1;
CREATE TABLE test_extension_function(col1 float8[], col2 float8[]);
SELECT create_distributed_table('test_extension_function', 'col1', colocate_with := 'none');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE EXTENSION cube;
SELECT create_distributed_function('cube(float8[], float8[])', '$1', 'test_extension_function');
create_distributed_function
---------------------------------------------------------------------
(1 row)
SELECT distribution_argument_index FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND
objid = (SELECT oid FROM pg_proc WHERE prosrc = 'cube_a_f8_f8');
distribution_argument_index
---------------------------------------------------------------------
0
(1 row)
SELECT create_distributed_function('cube(float8[], float8[])', '$2', 'test_extension_function');
create_distributed_function
---------------------------------------------------------------------
(1 row)
SELECT distribution_argument_index FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND
objid = (SELECT oid FROM pg_proc WHERE prosrc = 'cube_a_f8_f8');
distribution_argument_index
---------------------------------------------------------------------
1
(1 row)
ROLLBACK;
-- drop the schema and all the objects
DROP SCHEMA "extension'test" CASCADE;

View File

@ -381,10 +381,34 @@ SELECT 1 from master_remove_node('localhost', :worker_2_port);
1
(1 row)
-- Test extension function incorrect distribution argument
CREATE TABLE test_extension_function(col varchar);
CREATE EXTENSION seg;
-- Missing distribution argument
SELECT create_distributed_function('seg_in(cstring)');
ERROR: Extension functions(seg_in) without distribution argument are not supported.
-- Missing colocation argument
SELECT create_distributed_function('seg_in(cstring)', '$1');
ERROR: cannot distribute the function "seg_in" since there is no table to colocate with
HINT: Provide a distributed table via "colocate_with" option to create_distributed_function()
-- Incorrect distribution argument
SELECT create_distributed_function('seg_in(cstring)', '$2', colocate_with:='test_extension_function');
ERROR: cannot distribute the function "seg_in" since the distribution argument is not valid
HINT: Either provide a valid function argument name or a valid "$paramIndex" to create_distributed_function()
-- Colocated table is not distributed
SELECT create_distributed_function('seg_in(cstring)', '$1', 'test_extension_function');
ERROR: relation test_extension_function is not distributed
DROP EXTENSION seg;
SET citus.shard_replication_factor TO 1;
SELECT create_distributed_table('test_extension_function', 'col', colocate_with := 'none');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- now, create a type that depends on another type, which
-- finally depends on an extension
BEGIN;
SET citus.shard_replication_factor TO 1;
CREATE EXTENSION seg;
CREATE EXTENSION isn;
CREATE TYPE test_type AS (a int, b seg);
@ -402,9 +426,39 @@ BEGIN;
create_reference_table
---------------------------------------------------------------------
(1 row)
-- Distribute an extension-function
SELECT create_distributed_function('seg_in(cstring)', '$1', 'test_extension_function');
create_distributed_function
---------------------------------------------------------------------
(1 row)
COMMIT;
-- Check the pg_dist_object
SELECT pg_proc.proname as DistributedFunction
FROM citus.pg_dist_object, pg_proc
WHERE pg_proc.proname = 'seg_in' and
pg_proc.oid = citus.pg_dist_object.objid and
classid = 'pg_proc'::regclass;
distributedfunction
---------------------------------------------------------------------
seg_in
(1 row)
SELECT run_command_on_workers($$
SELECT count(*)
FROM citus.pg_dist_object, pg_proc
WHERE pg_proc.proname = 'seg_in' and
pg_proc.oid = citus.pg_dist_object.objid and
classid = 'pg_proc'::regclass;
$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,1)
(1 row)
-- add the node back
SELECT 1 from master_add_node('localhost', :worker_2_port);
?column?
@ -426,5 +480,145 @@ SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname
(localhost,57638,t,2)
(2 rows)
-- Check the pg_dist_object on the both nodes
SELECT run_command_on_workers($$
SELECT count(*)
FROM citus.pg_dist_object, pg_proc
WHERE pg_proc.proname = 'seg_in' and
pg_proc.oid = citus.pg_dist_object.objid and
classid = 'pg_proc'::regclass;
$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,1)
(localhost,57638,t,1)
(2 rows)
DROP EXTENSION seg CASCADE;
-- Recheck the pg_dist_object
SELECT pg_proc.proname as DistributedFunction
FROM citus.pg_dist_object, pg_proc
WHERE pg_proc.proname = 'seg_in' and
pg_proc.oid = citus.pg_dist_object.objid and
classid = 'pg_proc'::regclass;
distributedfunction
---------------------------------------------------------------------
(0 rows)
SELECT run_command_on_workers($$
SELECT count(*)
FROM citus.pg_dist_object, pg_proc
WHERE pg_proc.proname = 'seg_in' and
pg_proc.oid = citus.pg_dist_object.objid and
classid = 'pg_proc'::regclass;
$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,0)
(localhost,57638,t,0)
(2 rows)
-- Distribute an extension-function where extension is not in pg_dist_object
SET citus.enable_ddl_propagation TO false;
CREATE EXTENSION seg;
SET citus.enable_ddl_propagation TO true;
-- Check the extension in pg_dist_object
SELECT count(*) FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND
objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
count
---------------------------------------------------------------------
0
(1 row)
SELECT run_command_on_workers($$
SELECT count(*)
FROM citus.pg_dist_object, pg_proc
WHERE pg_proc.proname = 'seg_in' and
pg_proc.oid = citus.pg_dist_object.objid and
classid = 'pg_proc'::regclass;
$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,0)
(localhost,57638,t,0)
(2 rows)
SELECT create_distributed_function('seg_in(cstring)', '$1', 'test_extension_function');
create_distributed_function
---------------------------------------------------------------------
(1 row)
-- Recheck the extension in pg_dist_object
SELECT count(*) FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND
objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
count
---------------------------------------------------------------------
1
(1 row)
SELECT pg_proc.proname as DistributedFunction
FROM citus.pg_dist_object, pg_proc
WHERE pg_proc.proname = 'seg_in' and
pg_proc.oid = citus.pg_dist_object.objid and
classid = 'pg_proc'::regclass;
distributedfunction
---------------------------------------------------------------------
seg_in
(1 row)
SELECT run_command_on_workers($$
SELECT count(*)
FROM citus.pg_dist_object, pg_proc
WHERE pg_proc.proname = 'seg_in' and
pg_proc.oid = citus.pg_dist_object.objid and
classid = 'pg_proc'::regclass;
$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,1)
(localhost,57638,t,1)
(2 rows)
DROP EXTENSION seg;
DROP TABLE test_extension_function;
-- Test extension function altering distribution argument
BEGIN;
SET citus.shard_replication_factor = 1;
CREATE TABLE test_extension_function(col1 float8[], col2 float8[]);
SELECT create_distributed_table('test_extension_function', 'col1', colocate_with := 'none');
create_distributed_table
---------------------------------------------------------------------
(1 row)
CREATE EXTENSION cube;
SELECT create_distributed_function('cube(float8[], float8[])', '$1', 'test_extension_function');
create_distributed_function
---------------------------------------------------------------------
(1 row)
SELECT distribution_argument_index FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND
objid = (SELECT oid FROM pg_proc WHERE prosrc = 'cube_a_f8_f8');
distribution_argument_index
---------------------------------------------------------------------
0
(1 row)
SELECT create_distributed_function('cube(float8[], float8[])', '$2', 'test_extension_function');
create_distributed_function
---------------------------------------------------------------------
(1 row)
SELECT distribution_argument_index FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND
objid = (SELECT oid FROM pg_proc WHERE prosrc = 'cube_a_f8_f8');
distribution_argument_index
---------------------------------------------------------------------
1
(1 row)
ROLLBACK;
-- drop the schema and all the objects
DROP SCHEMA "extension'test" CASCADE;

View File

@ -12,9 +12,11 @@ SELECT create_distributed_table('dist_table', 'id', colocate_with => 'users_tabl
(1 row)
INSERT INTO dist_table (id, value) VALUES(1, 2),(2, 3),(3,4);
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION func() RETURNS TABLE (id int, value int) AS $$
SELECT 1, 2
$$ LANGUAGE SQL;
RESET citus.enable_metadata_sync;
SET client_min_messages TO DEBUG1;
-- CTEs are recursively planned, and subquery foo is also recursively planned
-- final plan becomes a router plan

View File

@ -2,13 +2,6 @@
-- test recursive planning functionality on prepared statements
-- ===================================================================
CREATE SCHEMA subquery_prepared_statements;
SELECT run_command_on_workers('CREATE SCHEMA subquery_prepared_statements;');
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"CREATE SCHEMA")
(localhost,57638,t,"CREATE SCHEMA")
(2 rows)
SET search_path TO subquery_prepared_statements, public;
CREATE TYPE subquery_prepared_statements.xy AS (x int, y int);
SET client_min_messages TO DEBUG1;

View File

@ -648,10 +648,6 @@ SELECT create_distributed_table('table_without_sequence', 'a');
CREATE SCHEMA test_schema_for_sequence_propagation;
CREATE SEQUENCE test_schema_for_sequence_propagation.seq_10;
-- Both should have zero rows
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation.seq_10'::regclass);
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation'::regnamespace);
ALTER TABLE table_without_sequence ADD COLUMN x BIGINT DEFAULT nextval('test_schema_for_sequence_propagation.seq_10');
-- Should be distributed along with the sequence

View File

@ -63,6 +63,7 @@ test: shared_connection_waits
test: isolation_cancellation
test: isolation_undistribute_table
test: isolation_fix_partition_shard_index_names
test: isolation_global_pid
# Rebalancer
test: isolation_blocking_move_single_shard_commands
@ -91,6 +92,7 @@ test: isolation_metadata_sync_deadlock
test: isolation_replicated_dist_on_mx
test: isolation_replicate_reference_tables_to_coordinator
test: isolation_multiuser_locking
test: isolation_metadata_sync_vs_all
# MXless tests
test: isolation_check_mx

View File

@ -268,7 +268,6 @@ test: multi_foreign_key_relation_graph
# and rerun some of the tests.
# --------
test: add_coordinator
test: foreign_key_to_reference_table
test: replicate_reference_tables_to_coordinator
test: citus_local_tables
test: mixed_relkind_tests

View File

@ -1273,17 +1273,6 @@ SELECT create_distributed_table('table_without_sequence', 'a');
CREATE SCHEMA test_schema_for_sequence_propagation;
CREATE SEQUENCE test_schema_for_sequence_propagation.seq_10;
-- Both should have zero rows
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation.seq_10'::regclass);
pg_identify_object_as_address
-------------------------------
(0 rows)
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation'::regnamespace);
pg_identify_object_as_address
-------------------------------
(0 rows)
ALTER TABLE table_without_sequence ADD COLUMN x BIGINT DEFAULT nextval('test_schema_for_sequence_propagation.seq_10');
-- Should be distributed along with the sequence
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation.seq_10'::regclass);

View File

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

View File

@ -71,7 +71,7 @@ step "s2-sleep"
step "s2-view-dist"
{
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' and query not ILIKE '%BEGIN%' and query NOT ILIKE '%pg_catalog.pg_isolation_test_session_is_blocked%' ORDER BY query DESC;
}

View File

@ -54,7 +54,8 @@ step "s1-verify-current-xact-is-on-worker"
get_current_transaction_id() as xact,
run_command_on_workers($$
SELECT row(initiator_node_identifier, transaction_number)
FROM get_all_active_transactions();
FROM get_all_active_transactions()
WHERE transaction_number != 0;
$$) as remote
ORDER BY remote.nodeport ASC;
}

View File

@ -188,40 +188,34 @@ step "s3-drop-coordinator-schemas"
DROP SCHEMA IF EXISTS myschema2 CASCADE;
}
step "s3-drop-worker-schemas"
{
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema CASCADE;$$);
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS myschema2 CASCADE;$$);
}
// schema only tests
permutation "s1-print-distributed-objects" "s1-begin" "s1-add-worker" "s2-public-schema" "s2-create-table" "s1-commit" "s2-print-distributed-objects" "s3-drop-coordinator-schemas" "s3-drop-worker-schemas"
permutation "s1-print-distributed-objects" "s1-begin" "s2-begin" "s1-add-worker" "s2-public-schema" "s2-create-table" "s1-commit" "s2-commit" "s2-print-distributed-objects" "s3-drop-coordinator-schemas" "s3-drop-worker-schemas"
permutation "s1-print-distributed-objects" "s1-begin" "s2-begin" "s2-public-schema" "s2-create-table" "s1-add-worker" "s2-commit" "s1-commit" "s2-print-distributed-objects" "s3-drop-coordinator-schemas" "s3-drop-worker-schemas"
permutation "s1-print-distributed-objects" "s1-begin" "s1-add-worker" "s2-create-schema" "s2-create-table" "s1-commit" "s2-print-distributed-objects" "s3-drop-coordinator-schemas" "s3-drop-worker-schemas"
permutation "s1-print-distributed-objects" "s1-begin" "s2-begin" "s1-add-worker" "s2-create-schema" "s2-create-table" "s1-commit" "s2-commit" "s2-print-distributed-objects" "s3-drop-coordinator-schemas" "s3-drop-worker-schemas"
permutation "s1-print-distributed-objects" "s1-begin" "s2-begin" "s2-create-schema" "s2-create-table" "s1-add-worker" "s2-commit" "s1-commit" "s2-print-distributed-objects" "s3-drop-coordinator-schemas" "s3-drop-worker-schemas"
permutation "s1-print-distributed-objects" "s1-begin" "s1-add-worker" "s2-public-schema" "s2-create-table" "s1-commit" "s2-print-distributed-objects" "s3-drop-coordinator-schemas"
permutation "s1-print-distributed-objects" "s1-begin" "s2-begin" "s1-add-worker" "s2-public-schema" "s2-create-table" "s1-commit" "s2-commit" "s2-print-distributed-objects" "s3-drop-coordinator-schemas"
permutation "s1-print-distributed-objects" "s1-begin" "s2-begin" "s2-public-schema" "s2-create-table" "s1-add-worker" "s2-commit" "s1-commit" "s2-print-distributed-objects" "s3-drop-coordinator-schemas"
permutation "s1-print-distributed-objects" "s1-begin" "s1-add-worker" "s2-create-schema" "s2-create-table" "s1-commit" "s2-print-distributed-objects" "s3-drop-coordinator-schemas"
permutation "s1-print-distributed-objects" "s1-begin" "s2-begin" "s1-add-worker" "s2-create-schema" "s2-create-table" "s1-commit" "s2-commit" "s2-print-distributed-objects" "s3-drop-coordinator-schemas"
permutation "s1-print-distributed-objects" "s1-begin" "s2-begin" "s2-create-schema" "s2-create-table" "s1-add-worker" "s2-commit" "s1-commit" "s2-print-distributed-objects" "s3-drop-coordinator-schemas"
// concurrency tests with multi schema distribution
permutation "s1-print-distributed-objects" "s2-create-schema" "s1-begin" "s2-begin" "s1-add-worker" "s2-create-table" "s1-commit" "s2-commit" "s2-print-distributed-objects" "s3-drop-coordinator-schemas" "s3-drop-worker-schemas"
permutation "s1-print-distributed-objects" "s1-add-worker" "s2-create-schema" "s2-begin" "s3-begin" "s3-use-schema" "s2-create-table" "s3-create-table" "s2-commit" "s3-commit" "s2-print-distributed-objects" "s3-drop-coordinator-schemas" "s3-drop-worker-schemas"
permutation "s1-print-distributed-objects" "s1-begin" "s2-begin" "s3-begin" "s1-add-worker" "s2-create-schema" "s3-create-schema2" "s2-create-table" "s3-create-table" "s1-commit" "s3-commit" "s2-commit" "s2-print-distributed-objects" "s3-drop-coordinator-schemas" "s3-drop-worker-schemas"
permutation "s1-print-distributed-objects" "s2-create-schema" "s1-begin" "s2-begin" "s1-add-worker" "s2-create-table" "s1-commit" "s2-commit" "s2-print-distributed-objects" "s3-drop-coordinator-schemas"
permutation "s1-print-distributed-objects" "s1-add-worker" "s2-create-schema" "s2-begin" "s3-begin" "s3-use-schema" "s2-create-table" "s3-create-table" "s2-commit" "s3-commit" "s2-print-distributed-objects" "s3-drop-coordinator-schemas"
permutation "s1-print-distributed-objects" "s1-begin" "s2-begin" "s3-begin" "s1-add-worker" "s2-create-schema" "s3-create-schema2" "s2-create-table" "s3-create-table" "s1-commit" "s3-commit" "s2-commit" "s2-print-distributed-objects" "s3-drop-coordinator-schemas"
// type and schema tests
permutation "s1-print-distributed-objects" "s1-begin" "s1-add-worker" "s2-public-schema" "s2-create-type" "s1-commit" "s2-print-distributed-objects" "s3-drop-coordinator-schemas" "s3-drop-worker-schemas"
permutation "s1-print-distributed-objects" "s1-begin" "s2-public-schema" "s2-create-type" "s1-add-worker" "s1-commit" "s2-print-distributed-objects" "s3-drop-coordinator-schemas" "s3-drop-worker-schemas"
permutation "s1-print-distributed-objects" "s1-begin" "s2-begin" "s2-create-schema" "s2-create-type" "s2-create-table-with-type" "s1-add-worker" "s2-commit" "s1-commit" "s2-print-distributed-objects" "s3-drop-coordinator-schemas" "s3-drop-worker-schemas"
permutation "s1-print-distributed-objects" "s1-begin" "s1-add-worker" "s2-public-schema" "s2-create-type" "s1-commit" "s2-print-distributed-objects" "s3-drop-coordinator-schemas"
permutation "s1-print-distributed-objects" "s1-begin" "s2-public-schema" "s2-create-type" "s1-add-worker" "s1-commit" "s2-print-distributed-objects" "s3-drop-coordinator-schemas"
permutation "s1-print-distributed-objects" "s1-begin" "s2-begin" "s2-create-schema" "s2-create-type" "s2-create-table-with-type" "s1-add-worker" "s2-commit" "s1-commit" "s2-print-distributed-objects" "s3-drop-coordinator-schemas"
// distributed function tests
// isolation tests are not very simple psql, so trigger NOTIFY reliably for
// s3-wait-for-metadata-sync step, we do "s2-begin" followed directly by
// "s2-commit", because "COMMIT" syncs the messages
permutation "s1-print-distributed-objects" "s1-begin" "s1-add-worker" "s2-public-schema" "s2-distribute-function" "s1-commit" "s2-begin" "s2-commit" "s3-wait-for-metadata-sync" "s2-print-distributed-objects" "s3-drop-coordinator-schemas" "s3-drop-worker-schemas"
permutation "s1-print-distributed-objects" "s1-begin" "s2-public-schema" "s2-distribute-function" "s2-begin" "s2-commit" "s3-wait-for-metadata-sync" "s1-add-worker" "s1-commit" "s3-wait-for-metadata-sync" "s2-print-distributed-objects" "s3-drop-coordinator-schemas" "s3-drop-worker-schemas"
permutation "s1-print-distributed-objects" "s1-begin" "s1-add-worker" "s2-public-schema" "s2-distribute-function" "s1-commit" "s2-begin" "s2-commit" "s3-wait-for-metadata-sync" "s2-print-distributed-objects" "s3-drop-coordinator-schemas"
permutation "s1-print-distributed-objects" "s1-begin" "s2-public-schema" "s2-distribute-function" "s2-begin" "s2-commit" "s3-wait-for-metadata-sync" "s1-add-worker" "s1-commit" "s3-wait-for-metadata-sync" "s2-print-distributed-objects" "s3-drop-coordinator-schemas"
// we cannot run the following operations concurrently
// the problem is that NOTIFY event doesn't (reliably) happen before COMMIT
// so we have to commit s2 before s1 starts
permutation "s1-print-distributed-objects" "s2-begin" "s2-create-schema" "s2-distribute-function" "s2-commit" "s3-wait-for-metadata-sync" "s1-begin" "s1-add-worker" "s1-commit" "s3-wait-for-metadata-sync" "s2-print-distributed-objects" "s3-drop-coordinator-schemas" "s3-drop-worker-schemas"
permutation "s1-print-distributed-objects" "s2-begin" "s2-create-schema" "s2-distribute-function" "s2-commit" "s3-wait-for-metadata-sync" "s1-begin" "s1-add-worker" "s1-commit" "s3-wait-for-metadata-sync" "s2-print-distributed-objects" "s3-drop-coordinator-schemas"

View File

@ -71,32 +71,32 @@ session "s3"
step "s3-as-admin"
{
-- Admin should be able to see all transactions
SELECT count(*) FROM get_all_active_transactions();
SELECT count(*) FROM get_global_active_transactions();
SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0;
SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0;
}
step "s3-as-user-1"
{
-- User should only be able to see its own transactions
SET ROLE test_user_1;
SELECT count(*) FROM get_all_active_transactions();
SELECT count(*) FROM get_global_active_transactions();
SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0;
SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0;
}
step "s3-as-readonly"
{
-- Other user should not see transactions
SET ROLE test_readonly;
SELECT count(*) FROM get_all_active_transactions();
SELECT count(*) FROM get_global_active_transactions();
SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0;
SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0;
}
step "s3-as-monitor"
{
-- Monitor should see all transactions
SET ROLE test_monitor;
SELECT count(*) FROM get_all_active_transactions();
SELECT count(*) FROM get_global_active_transactions();
SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0;
SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0;
}
permutation "s1-grant" "s1-begin-insert" "s2-begin-insert" "s3-as-admin" "s3-as-user-1" "s3-as-readonly" "s3-as-monitor" "s1-commit" "s2-commit"

View File

@ -0,0 +1,96 @@
#include "isolation_mx_common.include.spec"
setup
{
SET citus.next_shard_id TO 12345000;
CREATE TABLE dist_table (a INT, b INT);
SELECT create_distributed_table('dist_table', 'a', shard_count:=4);
}
teardown
{
DROP TABLE dist_table;
SELECT citus_internal.restore_isolation_tester_func();
}
session "s1"
step "s1-coordinator-begin"
{
BEGIN;
}
step "s1-coordinator-select"
{
SET citus.enable_local_execution TO off;
SET citus.force_max_query_parallelization TO ON;
SELECT * FROM dist_table;
}
step "s1-coordinator-commit"
{
COMMIT;
}
step "s1-start-session-level-connection"
{
SELECT start_session_level_connection_to_node('localhost', 57637);
}
step "s1-worker-begin"
{
SELECT run_commands_on_session_level_connection_to_node('BEGIN');
}
step "s1-worker-select"
{
SELECT run_commands_on_session_level_connection_to_node('SET citus.enable_local_execution TO off; SET citus.force_max_query_parallelization TO ON; SELECT * FROM dist_table');
}
step "s1-worker-commit"
{
SELECT run_commands_on_session_level_connection_to_node('COMMIT');
}
step "s1-stop-session-level-connection"
{
SELECT stop_session_level_connection_to_node();
}
session "s2"
step "s2-coordinator-citus_dist_stat_activity"
{
SELECT global_pid != 0 FROM citus_dist_stat_activity() WHERE query LIKE '%SELECT * FROM dist\_table%' and query NOT ILIKE '%run_commands_on_session_level_connection_to_node%';
}
step "s2-coordinator-citus_worker_stat_activity"
{
SELECT query FROM citus_worker_stat_activity() WHERE global_pid IN (
SELECT global_pid FROM citus_dist_stat_activity() WHERE query LIKE '%SELECT * FROM dist\_table%'
)
ORDER BY 1;
}
step "s2-coordinator-get_all_active_transactions"
{
SELECT count(*) FROM get_all_active_transactions() WHERE global_pid IN (
SELECT global_pid FROM citus_dist_stat_activity() WHERE query LIKE '%SELECT * FROM dist\_table%'
);
}
step "s2-coordinator-get_global_active_transactions"
{
SELECT count(*) FROM get_global_active_transactions() WHERE global_pid IN (
SELECT global_pid FROM citus_dist_stat_activity() WHERE query LIKE '%SELECT * FROM dist\_table%'
)
AND transaction_number != 0;
}
// worker - coordinator
permutation "s1-start-session-level-connection" "s1-worker-begin" "s1-worker-select" "s2-coordinator-citus_dist_stat_activity" "s2-coordinator-citus_worker_stat_activity" "s1-worker-commit" "s1-stop-session-level-connection"
// coordinator - coordinator
permutation "s1-coordinator-begin" "s1-coordinator-select" "s2-coordinator-citus_dist_stat_activity" "s2-coordinator-citus_worker_stat_activity" "s2-coordinator-get_all_active_transactions" "s2-coordinator-get_global_active_transactions" "s1-coordinator-commit"

View File

@ -0,0 +1,199 @@
setup
{
SET citus.shard_replication_factor to 1;
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 123000;
ALTER SEQUENCE pg_catalog.pg_dist_groupid_seq RESTART 123000;
ALTER SEQUENCE pg_catalog.pg_dist_node_nodeid_seq RESTART 123000;
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 123000;
-- Create the necessary test utility function
CREATE OR REPLACE FUNCTION activate_node_snapshot()
RETURNS text[]
LANGUAGE C STRICT
AS 'citus';
SELECT create_distributed_function('activate_node_snapshot()');
-- Create distributed tables
CREATE TABLE ref_table (test_id integer, y int unique);
SELECT create_reference_table('ref_table');
CREATE TABLE dist_table (x int, y int);
SELECT create_distributed_table('dist_table', 'x');
CREATE TABLE dist_partitioned_table (x int, y int) PARTITION BY RANGE(y);
SELECT create_distributed_table('dist_partitioned_table', 'x');
CREATE TABLE dist_partitioned_table_p1(x int, y int);
}
teardown
{
// drop all distributed tables
DROP TABLE IF EXISTS ref_table,
dist_table,
dist_partitioned_table,
dist_partitioned_table_p1,
dist_partitioned_table_p2,
new_dist_table,
new_ref_table;
// drop all distributed objects
DROP FUNCTION activate_node_snapshot();
DROP FUNCTION IF EXISTS squares(int);
DROP TYPE IF EXISTS my_type;
}
session "s1"
step "s1-begin"
{
BEGIN;
}
step "s1-commit"
{
COMMIT;
}
step "s1-start-metadata-sync"
{
SELECT start_metadata_sync_to_node('localhost', 57638);
}
session "s2"
step "s2-begin"
{
BEGIN;
}
step "s2-commit"
{
COMMIT;
}
step "s2-start-metadata-sync-to-same-node"
{
SELECT start_metadata_sync_to_node('localhost', 57638);
}
step "s2-start-metadata-sync-to-another-node"
{
SELECT start_metadata_sync_to_node('localhost', 57637);
}
step "s2-alter-table"
{
ALTER TABLE dist_table ADD COLUMN z int;
}
step "s2-add-fk"
{
ALTER TABLE dist_table ADD CONSTRAINT y_fk FOREIGN KEY (y) REFERENCES ref_table(y);
}
step "s2-drop-fk"
{
ALTER TABLE dist_table DROP CONSTRAINT y_fk;
}
step "s2-drop-table"
{
DROP TABLE dist_table;
}
step "s2-create-dist-table"
{
CREATE TABLE new_dist_table(id int, data int);
SELECT create_distributed_table('new_dist_table', 'id');
}
step "s2-create-ref-table"
{
CREATE TABLE new_ref_table(id int, data int);
SELECT create_reference_table('new_ref_table');
}
step "s2-attach-partition"
{
ALTER TABLE dist_partitioned_table ATTACH PARTITION dist_partitioned_table_p1 FOR VALUES FROM (1) TO (9);
}
step "s2-detach-partition"
{
ALTER TABLE dist_partitioned_table DETACH PARTITION dist_partitioned_table_p1;
}
step "s2-create-partition-of"
{
CREATE TABLE dist_partitioned_table_p2 PARTITION OF dist_partitioned_table FOR VALUES FROM (10) TO (20);
}
step "s2-create-type"
{
CREATE TYPE my_type AS (a int, b int);
}
step "s2-create-dist-func"
{
CREATE FUNCTION squares(int) RETURNS SETOF RECORD
AS $$ SELECT i, i * i FROM generate_series(1, $1) i $$
LANGUAGE SQL;
SELECT create_distributed_function('squares(int)');
}
session "s3"
step "s3-compare-snapshot"
{
SELECT count(*) = 0 AS same_metadata_in_workers
FROM
(
(
SELECT unnest(activate_node_snapshot())
EXCEPT
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
)
UNION
(
SELECT unnest(result::text[]) AS unnested_result
FROM run_command_on_workers($$SELECT activate_node_snapshot()$$)
EXCEPT
SELECT unnest(activate_node_snapshot())
)
) AS foo;
}
step "s3-debug"
{
SELECT unnest(activate_node_snapshot());
SELECT unnest(result::text[])
FROM run_command_on_workers('SELECT activate_node_snapshot()');
}
// before running any updates to metadata, make sure all nodes have same metadata in the cluster
permutation "s3-compare-snapshot"
// concurrent metadata syncing operations get blocked
permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-start-metadata-sync-to-same-node" "s1-commit" "s2-commit" "s3-compare-snapshot"
permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-start-metadata-sync-to-another-node" "s1-commit" "s2-commit" "s3-compare-snapshot"
// the following operations get blocked when a concurrent metadata sync is in progress
permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-alter-table" "s1-commit" "s2-commit" "s3-compare-snapshot"
permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-drop-table" "s1-commit" "s2-commit" "s3-compare-snapshot"
permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-create-dist-table" "s1-commit" "s2-commit" "s3-compare-snapshot"
permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-create-ref-table" "s1-commit" "s2-commit" "s3-compare-snapshot"
permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-attach-partition" "s1-commit" "s2-commit" "s3-compare-snapshot"
permutation "s2-attach-partition" "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-detach-partition" "s1-commit" "s2-commit" "s3-compare-snapshot"
permutation "s2-attach-partition" "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-create-partition-of" "s1-commit" "s2-commit" "s3-compare-snapshot"
permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-add-fk" "s1-commit" "s2-commit" "s3-compare-snapshot"
permutation "s2-add-fk" "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-drop-fk" "s1-commit" "s2-commit" "s3-compare-snapshot"
// the following operations do not get blocked
permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-create-type" "s1-commit" "s2-commit" "s3-compare-snapshot"
permutation "s1-begin" "s2-begin" "s1-start-metadata-sync" "s2-create-dist-func" "s1-commit" "s2-commit" "s3-compare-snapshot"

View File

@ -83,7 +83,7 @@ step "s2-lock-ref-table-placement-on-coordinator"
step "s2-view-dist"
{
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND query NOT ILIKE '%pg_isolation_test_session_is_blocked%' AND query NOT ILIKE '%BEGIN%' ORDER BY query DESC;
}
step "s2-view-worker"
@ -106,14 +106,15 @@ step "s2-sleep"
step "s2-active-transactions"
{
-- Admin should be able to see all transactions
SELECT count(*) FROM get_all_active_transactions();
SELECT count(*) FROM get_global_active_transactions();
SELECT count(*) FROM get_all_active_transactions() WHERE transaction_number != 0;
SELECT count(*) FROM get_global_active_transactions() WHERE transaction_number != 0;
}
// we disable the daemon during the regression tests in order to get consistent results
// thus we manually issue the deadlock detection
session "deadlock-checker"
// we issue the checker not only when there are deadlocks to ensure that we never cancel
// backend inappropriately
step "deadlock-checker-call"

View File

@ -160,8 +160,9 @@ CREATE TRIGGER "trigger\'name"
BEFORE INSERT ON "interesting!schema"."citus_local!_table"
FOR EACH STATEMENT EXECUTE FUNCTION dummy_function();
CREATE EXTENSION seg;
BEGIN;
CREATE EXTENSION seg;
-- ALTER TRIGGER DEPENDS ON
ALTER TRIGGER "trigger\'name" ON "interesting!schema"."citus_local!_table" DEPENDS ON EXTENSION seg;
@ -384,4 +385,5 @@ BEGIN;
ROLLBACK;
-- cleanup at exit
SET client_min_messages TO ERROR;
DROP SCHEMA citus_local_table_triggers, "interesting!schema" CASCADE;

View File

@ -128,6 +128,7 @@ set columnar.compression = 'pglz';
INSERT INTO truncate_schema.truncate_tbl SELECT generate_series(1, 100);
set columnar.compression to default;
-- create a user that can not truncate
SELECT run_command_on_workers($$CREATE USER truncate_user;$$);
CREATE USER truncate_user;
GRANT USAGE ON SCHEMA truncate_schema TO truncate_user;
GRANT SELECT ON TABLE truncate_schema.truncate_tbl TO truncate_user;

View File

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

View File

@ -1,5 +1,4 @@
CREATE SCHEMA collation_conflict;
SELECT run_command_on_workers($$CREATE SCHEMA collation_conflict;$$);
\c - - - :worker_1_port
SET search_path TO collation_conflict;

View File

@ -674,16 +674,9 @@ SELECT stop_metadata_sync_to_node(nodename,nodeport) FROM pg_dist_node WHERE isa
\c - - - :worker_1_port
UPDATE pg_dist_local_group SET groupid = 0;
TRUNCATE pg_dist_node;
SET client_min_messages TO error; -- suppress cascading objects dropping
DROP SCHEMA function_tests CASCADE;
DROP SCHEMA function_tests2 CASCADE;
SET search_path TO function_tests, function_tests2;
\c - - - :worker_2_port
UPDATE pg_dist_local_group SET groupid = 0;
TRUNCATE pg_dist_node;
SET client_min_messages TO error; -- suppress cascading objects dropping
DROP SCHEMA function_tests CASCADE;
DROP SCHEMA function_tests2 CASCADE;
\c - - - :master_port
SET client_min_messages TO ERROR;

View File

@ -2,7 +2,6 @@
-- Note in PG12 we use CREATE OR REPLACE AGGREGATE, thus the renaming does not occur
CREATE SCHEMA proc_conflict;
SELECT run_command_on_workers($$CREATE SCHEMA proc_conflict;$$);
\c - - - :worker_1_port
SET search_path TO proc_conflict;

View File

@ -79,9 +79,7 @@ SELECT * FROM run_command_on_workers($$CALL procedure_tests.raise_info('hello');
SET client_min_messages TO error; -- suppress cascading objects dropping
DROP SCHEMA procedure_tests CASCADE;
SELECT run_command_on_workers($$DROP SCHEMA procedure_tests CASCADE;$$);
DROP SCHEMA procedure_tests2 CASCADE;
SELECT run_command_on_workers($$DROP SCHEMA procedure_tests2 CASCADE;$$);
DROP USER procedureuser;
SELECT 1 FROM run_command_on_workers($$DROP USER procedureuser;$$);

View File

@ -284,8 +284,6 @@ SELECT * FROM field_indirection_test_2 ORDER BY 1,2,3;
-- clear objects
SET client_min_messages TO error; -- suppress cascading objects dropping
DROP SCHEMA type_tests CASCADE;
SELECT run_command_on_workers($$DROP SCHEMA type_tests CASCADE;$$);
DROP SCHEMA type_tests2 CASCADE;
SELECT run_command_on_workers($$DROP SCHEMA type_tests2 CASCADE;$$);
DROP USER typeuser;
SELECT run_command_on_workers($$DROP USER typeuser;$$);

View File

@ -1,7 +1,6 @@
SET citus.next_shard_id TO 20020000;
CREATE SCHEMA type_conflict;
SELECT run_command_on_workers($$CREATE SCHEMA type_conflict;$$);
-- create a type on a worker that should not cause data loss once overwritten with a type
-- from the coordinator

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