Distribute Types to worker nodes (#2893)

DESCRIPTION: Distribute Types to worker nodes

When to propagate
==============

There are two logical moments that types could be distributed to the worker nodes
 - When they get used ( just in time distribution )
 - When they get created ( proactive distribution )

The just in time distribution follows the model used by how schema's get created right before we are going to create a table in that schema, for types this would be when the table uses a type as its column.

The proactive distribution is suitable for situations where it is benificial to have the type on the worker nodes directly. They can later on be used in queries where an intermediate result gets created with a cast to this type.

Just in time creation is always the last resort, you cannot create a distributed table before the type gets created. A good example use case is; you have an existing postgres server that needs to scale out. By adding the citus extension, add some nodes to the cluster, and distribute the table. The type got created before citus existed. There was no moment where citus could have propagated the creation of a type.

Proactive is almost always a good option. Types are not resource intensive objects, there is no performance overhead of having 100's of types. If you want to use them in a query to represent an intermediate result (which happens in our test suite) they just work.

There is however a moment when proactive type distribution is not beneficial; in transactions where the type is used in a distributed table.

Lets assume the following transaction:

```sql
BEGIN;
CREATE TYPE tt1 AS (a int, b int);
CREATE TABLE t1 AS (a int PRIMARY KEY, b tt1);
SELECT create_distributed_table('t1', 'a');
\copy t1 FROM bigdata.csv
```

Types are node scoped objects; meaning the type exists once per worker. Shards however have best performance when they are created over their own connection. For the type to be visible on all connections it needs to be created and committed before we try to create the shards. Here the just in time situation is most beneficial and follows how we create schema's on the workers. Outside of a transaction block we will just use 1 connection to propagate the creation.

How propagation works
=================

Just in time
-----------

Just in time propagation hooks into the infrastructure introduced in #2882. It adds types as a supported object in `SupportedDependencyByCitus`. This will make sure that any object being distributed by citus that depends on types will now cascade into types. When types are depending them self on other objects they will get created first.

Creation later works by getting the ddl commands to create the object by its `ObjectAddress` in `GetDependencyCreateDDLCommands` which will dispatch types to `CreateTypeDDLCommandsIdempotent`.

For the correct walking of the graph we follow array types, when later asked for the ddl commands for array types we return `NIL` (empty list) which makes that the object will not be recorded as distributed, (its an internal type, dependant on the user type).

Proactive distribution
---------------------

When the user creates a type (composite or enum) we will have a hook running in `multi_ProcessUtility` after the command has been applied locally. Running after running locally makes that we already have an `ObjectAddress` for the type. This is required to mark the type as being distributed.

Keeping the type up to date
====================

For types that are recorded in `pg_dist_object` (eg. `IsObjectDistributed` returns true for the `ObjectAddress`) we will intercept the utility commands that alter the type.
 - `AlterTableStmt` with `relkind` set to `OBJECT_TYPE` encapsulate changes to the fields of a composite type.
 - `DropStmt` with removeType set to `OBJECT_TYPE` encapsulate `DROP TYPE`.
 - `AlterEnumStmt` encapsulates changes to enum values.
    Enum types can not be changed transactionally. When the execution on a worker fails a warning will be shown to the user the propagation was incomplete due to worker communication failure. An idempotent command is shown for the user to re-execute when the worker communication is fixed.

Keeping types up to date is done via the executor. Before the statement is executed locally we create a plan on how to apply it on the workers. This plan is executed after we have applied the statement locally.

All changes to types need to be done in the same transaction for types that have already been distributed and will fail with an error if parallel queries have already been executed in the same transaction. Much like foreign keys to reference tables.
pull/2951/head
Nils Dijk 2019-09-13 17:46:07 +02:00 committed by GitHub
parent 6e4fbeb8b9
commit 2879689441
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
98 changed files with 5790 additions and 580 deletions

View File

@ -10,7 +10,7 @@ EXTENSION = citus
DATA = $(patsubst $(citus_abs_srcdir)/%.sql,%.sql,$(wildcard $(citus_abs_srcdir)/$(EXTENSION)--*.sql)) DATA = $(patsubst $(citus_abs_srcdir)/%.sql,%.sql,$(wildcard $(citus_abs_srcdir)/$(EXTENSION)--*.sql))
# directories with source files # directories with source files
SUBDIRS = . commands connection ddl executor master metadata planner progress relay test transaction utils worker SUBDIRS = . commands connection ddl deparser executor master metadata planner progress relay test transaction utils worker
# That patsubst rule searches all directories listed in SUBDIRS for .c # That patsubst rule searches all directories listed in SUBDIRS for .c
# files, and adds the corresponding .o files to OBJS # files, and adds the corresponding .o files to OBJS

View File

@ -42,6 +42,13 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_unmark_object_distributed(classid o
COMMENT ON FUNCTION pg_catalog.master_unmark_object_distributed(classid oid, objid oid, objsubid int) COMMENT ON FUNCTION pg_catalog.master_unmark_object_distributed(classid oid, objid oid, objsubid int)
IS 'remove an object address from citus.pg_dist_object once the object has been deleted'; IS 'remove an object address from citus.pg_dist_object once the object has been deleted';
CREATE OR REPLACE FUNCTION pg_catalog.worker_create_or_replace_object(statement text)
RETURNS bool
LANGUAGE C STRICT
AS 'MODULE_PATHNAME', $$worker_create_or_replace_object$$;
COMMENT ON FUNCTION pg_catalog.worker_create_or_replace_object(statement text)
IS 'takes a sql CREATE statement, before executing the create it will check if an object with that name already exists and safely replaces that named object with the new object';
CREATE TABLE citus.pg_dist_object ( CREATE TABLE citus.pg_dist_object (
classid oid NOT NULL, classid oid NOT NULL,
objid oid NOT NULL, objid oid NOT NULL,

View File

@ -19,7 +19,9 @@
#include "distributed/metadata_sync.h" #include "distributed/metadata_sync.h"
#include "distributed/remote_commands.h" #include "distributed/remote_commands.h"
#include "distributed/worker_manager.h" #include "distributed/worker_manager.h"
#include "distributed/worker_transaction.h"
#include "storage/lmgr.h" #include "storage/lmgr.h"
#include "utils/lsyscache.h"
static List * GetDependencyCreateDDLCommands(const ObjectAddress *dependency); static List * GetDependencyCreateDDLCommands(const ObjectAddress *dependency);
@ -42,10 +44,9 @@ static List * GetDependencyCreateDDLCommands(const ObjectAddress *dependency);
void void
EnsureDependenciesExistsOnAllNodes(const ObjectAddress *target) EnsureDependenciesExistsOnAllNodes(const ObjectAddress *target)
{ {
const uint32 connectionFlag = FORCE_NEW_CONNECTION;
/* local variables to work with dependencies */ /* local variables to work with dependencies */
List *dependencies = NIL; List *dependencies = NIL;
List *dependenciesWithCommands = NIL;
ListCell *dependencyCell = NULL; ListCell *dependencyCell = NULL;
/* local variables to collect ddl commands */ /* local variables to collect ddl commands */
@ -54,8 +55,6 @@ EnsureDependenciesExistsOnAllNodes(const ObjectAddress *target)
/* local variables to work with worker nodes */ /* local variables to work with worker nodes */
List *workerNodeList = NULL; List *workerNodeList = NULL;
ListCell *workerNodeCell = NULL; ListCell *workerNodeCell = NULL;
List *connections = NULL;
ListCell *connectionCell = NULL;
/* /*
* collect all dependencies in creation order and get their ddl commands * collect all dependencies in creation order and get their ddl commands
@ -64,8 +63,14 @@ EnsureDependenciesExistsOnAllNodes(const ObjectAddress *target)
foreach(dependencyCell, dependencies) foreach(dependencyCell, dependencies)
{ {
ObjectAddress *dependency = (ObjectAddress *) lfirst(dependencyCell); ObjectAddress *dependency = (ObjectAddress *) lfirst(dependencyCell);
ddlCommands = list_concat(ddlCommands, List *dependencyCommands = GetDependencyCreateDDLCommands(dependency);
GetDependencyCreateDDLCommands(dependency)); ddlCommands = list_concat(ddlCommands, dependencyCommands);
/* create a new list with dependencies that actually created commands */
if (list_length(dependencyCommands) > 0)
{
dependenciesWithCommands = lappend(dependenciesWithCommands, dependency);
}
} }
if (list_length(ddlCommands) <= 0) if (list_length(ddlCommands) <= 0)
{ {
@ -73,6 +78,9 @@ EnsureDependenciesExistsOnAllNodes(const ObjectAddress *target)
return; return;
} }
/* since we are executing ddl commands lets disable propagation, primarily for mx */
ddlCommands = list_concat(list_make1(DISABLE_DDL_PROPAGATION), ddlCommands);
/* /*
* Make sure that no new nodes are added after this point until the end of the * Make sure that no new nodes are added after this point until the end of the
* transaction by taking a RowShareLock on pg_dist_node, which conflicts with the * transaction by taking a RowShareLock on pg_dist_node, which conflicts with the
@ -93,7 +101,7 @@ EnsureDependenciesExistsOnAllNodes(const ObjectAddress *target)
* to the nodes before marking the objects as distributed these objects would never be * to the nodes before marking the objects as distributed these objects would never be
* created on the workers when they get added, causing shards to fail to create. * created on the workers when they get added, causing shards to fail to create.
*/ */
foreach(dependencyCell, dependencies) foreach(dependencyCell, dependenciesWithCommands)
{ {
ObjectAddress *dependency = (ObjectAddress *) lfirst(dependencyCell); ObjectAddress *dependency = (ObjectAddress *) lfirst(dependencyCell);
MarkObjectDistributed(dependency); MarkObjectDistributed(dependency);
@ -108,37 +116,18 @@ EnsureDependenciesExistsOnAllNodes(const ObjectAddress *target)
/* no nodes to execute on */ /* no nodes to execute on */
return; return;
} }
foreach(workerNodeCell, workerNodeList) foreach(workerNodeCell, workerNodeList)
{ {
WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell); WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell);
MultiConnection *connection = NULL;
char *nodeName = workerNode->workerName; const char *nodeName = workerNode->workerName;
uint32 nodePort = workerNode->workerPort; uint32 nodePort = workerNode->workerPort;
connection = StartNodeUserDatabaseConnection(connectionFlag, nodeName, nodePort, SendCommandListToWorkerInSingleTransaction(nodeName, nodePort,
CitusExtensionOwnerName(), NULL); CitusExtensionOwnerName(),
ddlCommands);
connections = lappend(connections, connection);
}
FinishConnectionListEstablishment(connections);
/*
* create dependency on all nodes
*/
foreach(connectionCell, connections)
{
MultiConnection *connection = (MultiConnection *) lfirst(connectionCell);
ExecuteCriticalRemoteCommandList(connection, ddlCommands);
}
/*
* disconnect from nodes
*/
foreach(connectionCell, connections)
{
MultiConnection *connection = (MultiConnection *) lfirst(connectionCell);
CloseConnection(connection);
} }
} }
@ -165,6 +154,26 @@ GetDependencyCreateDDLCommands(const ObjectAddress *dependency)
return list_make1((void *) schemaDDLCommand); return list_make1((void *) schemaDDLCommand);
} }
case OCLASS_TYPE:
{
return CreateTypeDDLCommandsIdempotent(dependency);
}
case OCLASS_CLASS:
{
/*
* types have an intermediate dependency on a relation (aka class), so we do
* support classes when the relkind is composite
*/
if (get_rel_relkind(dependency->objectId) == RELKIND_COMPOSITE_TYPE)
{
return NIL;
}
/* if this relation is not supported, break to the error at the end */
break;
}
default: default:
{ {
break; break;
@ -181,7 +190,6 @@ GetDependencyCreateDDLCommands(const ObjectAddress *dependency)
errdetail( errdetail(
"citus tries to recreate an unsupported object on its workers"), "citus tries to recreate an unsupported object on its workers"),
errhint("please report a bug as this should not be happening"))); errhint("please report a bug as this should not be happening")));
return NIL;
} }
@ -191,11 +199,9 @@ GetDependencyCreateDDLCommands(const ObjectAddress *dependency)
void void
ReplicateAllDependenciesToNode(const char *nodeName, int nodePort) ReplicateAllDependenciesToNode(const char *nodeName, int nodePort)
{ {
const uint32 connectionFlag = FORCE_NEW_CONNECTION;
ListCell *dependencyCell = NULL; ListCell *dependencyCell = NULL;
List *dependencies = NIL; List *dependencies = NIL;
List *ddlCommands = NIL; List *ddlCommands = NIL;
MultiConnection *connection = NULL;
/* /*
* collect all dependencies in creation order and get their ddl commands * collect all dependencies in creation order and get their ddl commands
@ -230,11 +236,9 @@ ReplicateAllDependenciesToNode(const char *nodeName, int nodePort)
return; return;
} }
/* /* since we are executing ddl commands lets disable propagation, primarily for mx */
* connect to the new host and create all applicable dependencies ddlCommands = list_concat(list_make1(DISABLE_DDL_PROPAGATION), ddlCommands);
*/
connection = GetNodeUserDatabaseConnection(connectionFlag, nodeName, nodePort, SendCommandListToWorkerInSingleTransaction(nodeName, nodePort,
CitusExtensionOwnerName(), NULL); CitusExtensionOwnerName(), ddlCommands);
ExecuteCriticalRemoteCommandList(connection, ddlCommands);
CloseConnection(connection);
} }

View File

@ -104,25 +104,58 @@ ProcessDropSchemaStmt(DropStmt *dropStatement)
/* /*
* PlanAlterObjectSchemaStmt determines whether a given ALTER ... SET SCHEMA * PlanAlterObjectSchemaStmt is called by citus' utility hook for AlterObjectSchemaStmt
* parsetrees. It dispatches the statement based on the object type for which the schema
* is being altered.
*
* A (potentially empty) list of DDLJobs is being returned with the jobs on how to
* distribute the change into the cluster.
*/
List *
PlanAlterObjectSchemaStmt(AlterObjectSchemaStmt *stmt, const char *queryString)
{
switch (stmt->objectType)
{
case OBJECT_TYPE:
{
return PlanAlterTypeSchemaStmt(stmt, queryString);
}
default:
{
/* do nothing for unsupported objects */
break;
}
}
/*
* old behaviour, needs to be reconciled to the above switch statement for all
* objectType's relating to tables. Maybe it is as easy to support
* ALTER TABLE ... SET SCHEMA
*/
return PlanAlterTableSchemaStmt(stmt, queryString);
}
/*
* PlanAlterTableSchemaStmt determines whether a given ALTER ... SET SCHEMA
* statement involves a distributed table and issues a warning if so. Because * statement involves a distributed table and issues a warning if so. Because
* we do not support distributed ALTER ... SET SCHEMA, this function always * we do not support distributed ALTER ... SET SCHEMA, this function always
* returns NIL. * returns NIL.
*/ */
List * List *
PlanAlterObjectSchemaStmt(AlterObjectSchemaStmt *alterObjectSchemaStmt, PlanAlterTableSchemaStmt(AlterObjectSchemaStmt *stmt, const char *queryString)
const char *alterObjectSchemaCommand)
{ {
Oid relationId = InvalidOid; Oid relationId = InvalidOid;
if (alterObjectSchemaStmt->relation == NULL) if (stmt->relation == NULL)
{ {
return NIL; return NIL;
} }
relationId = RangeVarGetRelid(alterObjectSchemaStmt->relation, relationId = RangeVarGetRelid(stmt->relation,
AccessExclusiveLock, AccessExclusiveLock,
alterObjectSchemaStmt->missing_ok); stmt->missing_ok);
/* first check whether a distributed relation is affected */ /* first check whether a distributed relation is affected */
if (!OidIsValid(relationId) || !IsDistributedTable(relationId)) if (!OidIsValid(relationId) || !IsDistributedTable(relationId))
@ -138,3 +171,29 @@ PlanAlterObjectSchemaStmt(AlterObjectSchemaStmt *alterObjectSchemaStmt,
return NIL; return NIL;
} }
/*
* ProcessAlterObjectSchemaStmt is called by multi_ProcessUtility _after_ the command has
* been applied to the local postgres. It is useful to create potentially new dependencies
* of this object (the new schema) on the workers before the command gets applied to the
* remote objects.
*/
void
ProcessAlterObjectSchemaStmt(AlterObjectSchemaStmt *stmt, const char *queryString)
{
switch (stmt->objectType)
{
case OBJECT_TYPE:
{
ProcessAlterTypeSchemaStmt(stmt, queryString);
return;
}
default:
{
/* do nothing for unsupported objects */
return;
}
}
}

View File

@ -613,6 +613,19 @@ PostProcessAlterTableStmt(AlterTableStmt *alterTableStatement)
{ {
List *commandList = alterTableStatement->cmds; List *commandList = alterTableStatement->cmds;
ListCell *commandCell = NULL; ListCell *commandCell = NULL;
LOCKMODE lockmode = NoLock;
Oid relationId = InvalidOid;
lockmode = AlterTableGetLockLevel(alterTableStatement->cmds);
relationId = AlterTableLookupRelation(alterTableStatement, lockmode);
if (relationId != InvalidOid)
{
/* changing a relation could introduce new dependencies */
ObjectAddress tableAddress = { 0 };
ObjectAddressSet(tableAddress, RelationRelationId, relationId);
EnsureDependenciesExistsOnAllNodes(&tableAddress);
}
foreach(commandCell, commandList) foreach(commandCell, commandList)
{ {
@ -621,17 +634,12 @@ PostProcessAlterTableStmt(AlterTableStmt *alterTableStatement)
if (alterTableType == AT_AddConstraint) if (alterTableType == AT_AddConstraint)
{ {
LOCKMODE lockmode = NoLock;
Oid relationId = InvalidOid;
Constraint *constraint = NULL; Constraint *constraint = NULL;
Assert(list_length(commandList) == 1); Assert(list_length(commandList) == 1);
ErrorIfUnsupportedAlterAddConstraintStmt(alterTableStatement); ErrorIfUnsupportedAlterAddConstraintStmt(alterTableStatement);
lockmode = AlterTableGetLockLevel(alterTableStatement->cmds);
relationId = AlterTableLookupRelation(alterTableStatement, lockmode);
if (!OidIsValid(relationId)) if (!OidIsValid(relationId))
{ {
continue; continue;
@ -647,8 +655,6 @@ PostProcessAlterTableStmt(AlterTableStmt *alterTableStatement)
{ {
List *columnConstraints = NIL; List *columnConstraints = NIL;
ListCell *columnConstraint = NULL; ListCell *columnConstraint = NULL;
Oid relationId = InvalidOid;
LOCKMODE lockmode = NoLock;
ColumnDef *columnDefinition = (ColumnDef *) command->def; ColumnDef *columnDefinition = (ColumnDef *) command->def;
columnConstraints = columnDefinition->constraints; columnConstraints = columnDefinition->constraints;
@ -657,8 +663,6 @@ PostProcessAlterTableStmt(AlterTableStmt *alterTableStatement)
ErrorIfUnsupportedAlterAddConstraintStmt(alterTableStatement); ErrorIfUnsupportedAlterAddConstraintStmt(alterTableStatement);
} }
lockmode = AlterTableGetLockLevel(alterTableStatement->cmds);
relationId = AlterTableLookupRelation(alterTableStatement, lockmode);
if (!OidIsValid(relationId)) if (!OidIsValid(relationId))
{ {
continue; continue;

File diff suppressed because it is too large Load Diff

View File

@ -41,6 +41,7 @@
#include "distributed/commands.h" #include "distributed/commands.h"
#include "distributed/commands/multi_copy.h" #include "distributed/commands/multi_copy.h"
#include "distributed/commands/utility_hook.h" /* IWYU pragma: keep */ #include "distributed/commands/utility_hook.h" /* IWYU pragma: keep */
#include "distributed/listutils.h"
#include "distributed/local_executor.h" #include "distributed/local_executor.h"
#include "distributed/maintenanced.h" #include "distributed/maintenanced.h"
#include "distributed/master_protocol.h" #include "distributed/master_protocol.h"
@ -52,6 +53,7 @@
#include "distributed/version_compat.h" #include "distributed/version_compat.h"
#include "distributed/worker_transaction.h" #include "distributed/worker_transaction.h"
#include "lib/stringinfo.h" #include "lib/stringinfo.h"
#include "nodes/pg_list.h"
#include "tcop/utility.h" #include "tcop/utility.h"
#include "utils/builtins.h" #include "utils/builtins.h"
#include "utils/lsyscache.h" #include "utils/lsyscache.h"
@ -69,6 +71,10 @@ static void ExecuteDistributedDDLJob(DDLJob *ddlJob);
static char * SetSearchPathToCurrentSearchPathCommand(void); static char * SetSearchPathToCurrentSearchPathCommand(void);
static char * CurrentSearchPath(void); static char * CurrentSearchPath(void);
static void PostProcessUtility(Node *parsetree); static void PostProcessUtility(Node *parsetree);
static List * PlanRenameAttributeStmt(RenameStmt *stmt, const char *queryString);
static List * PlanAlterOwnerStmt(AlterOwnerStmt *stmt, const char *queryString);
static void ExecuteNodeBaseDDLCommands(List *taskList);
/* /*
@ -337,27 +343,50 @@ multi_ProcessUtility(PlannedStmt *pstmt,
if (IsA(parsetree, DropStmt)) if (IsA(parsetree, DropStmt))
{ {
DropStmt *dropStatement = (DropStmt *) parsetree; DropStmt *dropStatement = (DropStmt *) parsetree;
if (dropStatement->removeType == OBJECT_INDEX) switch (dropStatement->removeType)
{ {
ddlJobs = PlanDropIndexStmt(dropStatement, queryString); case OBJECT_INDEX:
} {
ddlJobs = PlanDropIndexStmt(dropStatement, queryString);
break;
}
if (dropStatement->removeType == OBJECT_TABLE) case OBJECT_TABLE:
{ {
ProcessDropTableStmt(dropStatement); ProcessDropTableStmt(dropStatement);
} break;
}
if (dropStatement->removeType == OBJECT_SCHEMA) case OBJECT_SCHEMA:
{ {
ProcessDropSchemaStmt(dropStatement); ProcessDropSchemaStmt(dropStatement);
} break;
}
if (dropStatement->removeType == OBJECT_POLICY) case OBJECT_POLICY:
{ {
ddlJobs = PlanDropPolicyStmt(dropStatement, queryString); ddlJobs = PlanDropPolicyStmt(dropStatement, queryString);
break;
}
case OBJECT_TYPE:
{
ddlJobs = PlanDropTypeStmt(dropStatement, queryString);
break;
}
default:
{
/* unsupported type, skipping*/
}
} }
} }
if (IsA(parsetree, AlterEnumStmt))
{
ddlJobs = PlanAlterEnumStmt(castNode(AlterEnumStmt, parsetree), queryString);
}
if (IsA(parsetree, AlterTableStmt)) if (IsA(parsetree, AlterTableStmt))
{ {
AlterTableStmt *alterTableStmt = (AlterTableStmt *) parsetree; AlterTableStmt *alterTableStmt = (AlterTableStmt *) parsetree;
@ -367,6 +396,11 @@ multi_ProcessUtility(PlannedStmt *pstmt,
{ {
ddlJobs = PlanAlterTableStmt(alterTableStmt, queryString); ddlJobs = PlanAlterTableStmt(alterTableStmt, queryString);
} }
if (alterTableStmt->relkind == OBJECT_TYPE)
{
ddlJobs = PlanAlterTypeStmt(alterTableStmt, queryString);
}
} }
/* /*
@ -375,7 +409,28 @@ multi_ProcessUtility(PlannedStmt *pstmt,
*/ */
if (IsA(parsetree, RenameStmt)) if (IsA(parsetree, RenameStmt))
{ {
ddlJobs = PlanRenameStmt((RenameStmt *) parsetree, queryString); RenameStmt *renameStmt = (RenameStmt *) parsetree;
switch (renameStmt->renameType)
{
case OBJECT_TYPE:
{
ddlJobs = PlanRenameTypeStmt(renameStmt, queryString);
break;
}
case OBJECT_ATTRIBUTE:
{
ddlJobs = PlanRenameAttributeStmt(renameStmt, queryString);
break;
}
default:
{
ddlJobs = PlanRenameStmt(renameStmt, queryString);
break;
}
}
} }
/* handle distributed CLUSTER statements */ /* handle distributed CLUSTER statements */
@ -390,8 +445,8 @@ multi_ProcessUtility(PlannedStmt *pstmt,
*/ */
if (IsA(parsetree, AlterObjectSchemaStmt)) if (IsA(parsetree, AlterObjectSchemaStmt))
{ {
AlterObjectSchemaStmt *setSchemaStmt = (AlterObjectSchemaStmt *) parsetree; ddlJobs = PlanAlterObjectSchemaStmt(
ddlJobs = PlanAlterObjectSchemaStmt(setSchemaStmt, queryString); castNode(AlterObjectSchemaStmt, parsetree), queryString);
} }
if (IsA(parsetree, GrantStmt)) if (IsA(parsetree, GrantStmt))
@ -399,6 +454,12 @@ multi_ProcessUtility(PlannedStmt *pstmt,
ddlJobs = PlanGrantStmt((GrantStmt *) parsetree); ddlJobs = PlanGrantStmt((GrantStmt *) parsetree);
} }
if (IsA(parsetree, AlterOwnerStmt))
{
ddlJobs = PlanAlterOwnerStmt(castNode(AlterOwnerStmt, parsetree),
queryString);
}
if (IsA(parsetree, CreatePolicyStmt)) if (IsA(parsetree, CreatePolicyStmt))
{ {
ddlJobs = PlanCreatePolicyStmt((CreatePolicyStmt *) parsetree); ddlJobs = PlanCreatePolicyStmt((CreatePolicyStmt *) parsetree);
@ -409,6 +470,18 @@ multi_ProcessUtility(PlannedStmt *pstmt,
ddlJobs = PlanAlterPolicyStmt((AlterPolicyStmt *) parsetree); ddlJobs = PlanAlterPolicyStmt((AlterPolicyStmt *) parsetree);
} }
if (IsA(parsetree, CompositeTypeStmt))
{
ddlJobs = PlanCompositeTypeStmt(castNode(CompositeTypeStmt, parsetree),
queryString);
}
if (IsA(parsetree, CreateEnumStmt))
{
ddlJobs = PlanCreateEnumStmt(castNode(CreateEnumStmt, parsetree),
queryString);
}
/* /*
* ALTER TABLE ALL IN TABLESPACE statements have their node type as * ALTER TABLE ALL IN TABLESPACE statements have their node type as
* AlterTableMoveAllStmt. At the moment we do not support this functionality in * AlterTableMoveAllStmt. At the moment we do not support this functionality in
@ -504,6 +577,22 @@ multi_ProcessUtility(PlannedStmt *pstmt,
standard_ProcessUtility(pstmt, queryString, context, standard_ProcessUtility(pstmt, queryString, context,
params, queryEnv, dest, completionTag); params, queryEnv, dest, completionTag);
/*
* Postgres added the following CommandCounterIncrement as a patch in:
* - 10.7 -> 10.8
* - 11.2 -> 11.3
* The patch was a response to bug #15631.
*
* CommandCounterIncrement is used to make changes to the catalog visible for post
* processing of create commands (eg. create type). It is safe to call
* CommandCounterIncrement twice, as the call is a no-op if the command id is not
* used yet.
*
* Once versions older then above are not deemed important anymore this patch can
* be remove from citus.
*/
CommandCounterIncrement();
if (IsA(parsetree, AlterTableStmt)) if (IsA(parsetree, AlterTableStmt))
{ {
activeAlterTables--; activeAlterTables--;
@ -520,6 +609,33 @@ multi_ProcessUtility(PlannedStmt *pstmt,
} }
PG_END_TRY(); PG_END_TRY();
/*
* Post process for ddl statements
*/
if (EnableDDLPropagation)
{
if (IsA(parsetree, CompositeTypeStmt))
{
ProcessCompositeTypeStmt(castNode(CompositeTypeStmt, parsetree), queryString);
}
if (IsA(parsetree, CreateEnumStmt))
{
ProcessCreateEnumStmt(castNode(CreateEnumStmt, parsetree), queryString);
}
if (IsA(parsetree, AlterObjectSchemaStmt))
{
ProcessAlterObjectSchemaStmt(castNode(AlterObjectSchemaStmt, parsetree),
queryString);
}
if (IsA(parsetree, AlterEnumStmt))
{
ProcessAlterEnumStmt(castNode(AlterEnumStmt, parsetree), queryString);
}
}
/* /*
* We only process CREATE TABLE ... PARTITION OF commands in the function below * We only process CREATE TABLE ... PARTITION OF commands in the function below
* to handle the case when user creates a table as a partition of distributed table. * to handle the case when user creates a table as a partition of distributed table.
@ -592,6 +708,57 @@ multi_ProcessUtility(PlannedStmt *pstmt,
} }
/*
* PlanRenameAttributeStmt called for RenameStmt's that are targetting an attribute eg.
* type attributes. Based on the relation type the attribute gets renamed it dispatches to
* a specialized implementation if present, otherwise return an empty list for its DDLJobs
*/
static List *
PlanRenameAttributeStmt(RenameStmt *stmt, const char *queryString)
{
Assert(stmt->renameType == OBJECT_ATTRIBUTE);
switch (stmt->relationType)
{
case OBJECT_TYPE:
{
return PlanRenameTypeAttributeStmt(stmt, queryString);
}
default:
{
/* unsupported relation for attribute rename, do nothing */
return NIL;
}
}
}
/*
* PlanAlterOwnerStmt gets called for statements that change the ownership of an object.
* Based on the type of object the ownership gets changed for it dispatches to a
* specialized implementation or returns an empty list of DDLJobs for objects that do not
* have an implementation provided.
*/
static List *
PlanAlterOwnerStmt(AlterOwnerStmt *stmt, const char *queryString)
{
switch (stmt->objectType)
{
case OBJECT_TYPE:
{
return PlanAlterTypeOwnerStmt(stmt, queryString);
}
default:
{
/* do nothing for unsupported alter owner statements */
return NIL;
}
}
}
/* /*
* ExecuteDistributedDDLJob simply executes a provided DDLJob in a distributed trans- * ExecuteDistributedDDLJob simply executes a provided DDLJob in a distributed trans-
* action, including metadata sync if needed. If the multi shard commit protocol is * action, including metadata sync if needed. If the multi shard commit protocol is
@ -607,12 +774,33 @@ multi_ProcessUtility(PlannedStmt *pstmt,
static void static void
ExecuteDistributedDDLJob(DDLJob *ddlJob) ExecuteDistributedDDLJob(DDLJob *ddlJob)
{ {
bool shouldSyncMetadata = ShouldSyncTableMetadata(ddlJob->targetRelationId); bool shouldSyncMetadata = false;
EnsureCoordinator(); EnsureCoordinator();
EnsurePartitionTableNotReplicated(ddlJob->targetRelationId);
if (!ddlJob->concurrentIndexCmd) if (ddlJob->targetRelationId != InvalidOid)
{
/*
* Only for ddlJobs that are targetting a relation (table) we want to sync its
* metadata and verify some properties around the table.
*/
shouldSyncMetadata = ShouldSyncTableMetadata(ddlJob->targetRelationId);
EnsurePartitionTableNotReplicated(ddlJob->targetRelationId);
}
if (TaskExecutorType != MULTI_EXECUTOR_ADAPTIVE &&
ddlJob->targetRelationId == InvalidOid)
{
/*
* Some ddl jobs can only be run by the adaptive executor and not our legacy ones.
*
* These are tasks that are not pinned to any relation nor shards. We can execute
* these very naively with a simple for loop that sends them to the target worker.
*/
ExecuteNodeBaseDDLCommands(ddlJob->taskList);
}
else if (!ddlJob->concurrentIndexCmd)
{ {
if (shouldSyncMetadata) if (shouldSyncMetadata)
{ {
@ -680,6 +868,34 @@ ExecuteDistributedDDLJob(DDLJob *ddlJob)
} }
/*
* ExecuteNodeBaseDDLCommands executes ddl commands naively only when we are not using the
* adaptive executor. It gets connections to the target placements and executes the
* commands.
*/
static void
ExecuteNodeBaseDDLCommands(List *taskList)
{
ListCell *taskCell = NULL;
foreach(taskCell, taskList)
{
Task *task = (Task *) lfirst(taskCell);
ListCell *taskPlacementCell = NULL;
/* these tasks should not be pinned to any shard */
Assert(task->anchorShardId == INVALID_SHARD_ID);
foreach(taskPlacementCell, task->taskPlacementList)
{
ShardPlacement *placement = (ShardPlacement *) lfirst(taskPlacementCell);
SendCommandToWorkerAsUser(placement->nodeName, placement->nodePort, NULL,
task->queryString);
}
}
}
/* /*
* SetSearchPathToCurrentSearchPathCommand generates a command which can * SetSearchPathToCurrentSearchPathCommand generates a command which can
* set the search path to the exact same search path that the issueing node * set the search path to the exact same search path that the issueing node
@ -843,6 +1059,55 @@ DDLTaskList(Oid relationId, const char *commandString)
} }
/*
* NodeDDLTaskList builds a list of tasks to execute a DDL command on a
* given target set of nodes.
*/
List *
NodeDDLTaskList(TargetWorkerSet targets, List *commands)
{
List *workerNodes = TargetWorkerSetNodeList(targets);
char *concatenatedCommands = StringJoin(commands, ';');
DDLJob *ddlJob = NULL;
ListCell *workerNodeCell = NULL;
Task *task = NULL;
if (list_length(workerNodes) <= 0)
{
/*
* if there are no nodes we don't have to plan any ddl tasks. Planning them would
* cause a hang in the executor.
*/
return NIL;
}
task = CitusMakeNode(Task);
task->taskType = DDL_TASK;
task->queryString = concatenatedCommands;
foreach(workerNodeCell, workerNodes)
{
WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell);
ShardPlacement *targetPlacement = NULL;
targetPlacement = CitusMakeNode(ShardPlacement);
targetPlacement->nodeName = workerNode->workerName;
targetPlacement->nodePort = workerNode->workerPort;
targetPlacement->groupId = workerNode->groupId;
task->taskPlacementList = lappend(task->taskPlacementList, targetPlacement);
}
ddlJob = palloc0(sizeof(DDLJob));
ddlJob->targetRelationId = InvalidOid;
ddlJob->concurrentIndexCmd = false;
ddlJob->commandString = NULL;
ddlJob->taskList = list_make1(task);
return list_make1(ddlJob);
}
/* /*
* AlterTableInProgress returns true if we're processing an ALTER TABLE command * AlterTableInProgress returns true if we're processing an ALTER TABLE command
* right now. * right now.

View File

@ -356,7 +356,8 @@ LogRemoteCommand(MultiConnection *connection, const char *command)
} }
ereport(LOG, (errmsg("issuing %s", ApplyLogRedaction(command)), ereport(LOG, (errmsg("issuing %s", ApplyLogRedaction(command)),
errdetail("on server %s:%d", connection->hostname, connection->port))); errdetail("on server %s@%s:%d", connection->user, connection->hostname,
connection->port)));
} }
@ -440,8 +441,20 @@ ExecuteOptionalRemoteCommand(MultiConnection *connection, const char *command,
return RESPONSE_NOT_OKAY; return RESPONSE_NOT_OKAY;
} }
*result = localResult; /*
return 0; * store result if result has been set, when the user is not interested in the result
* a NULL pointer could be passed and the result will be cleared
*/
if (result)
{
*result = localResult;
}
else
{
PQclear(localResult);
ForgetResults(connection);
}
return RESPONSE_OKAY;
} }

View File

@ -0,0 +1,204 @@
/*-------------------------------------------------------------------------
*
* deparse.c
* Entrypoint for deparsing parsetrees.
*
* The goal of deparsing parsetrees is to reconstruct sql statements
* from any parsed sql statement by ParseTreeNode. Deparsed statements
* can be used to reapply them on remote postgres nodes like the citus
* workers.
*
* Copyright (c) 2019, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "distributed/deparser.h"
static const char * DeparseDropStmt(DropStmt *stmt);
static const char * DeparseAlterTableStmt(AlterTableStmt *stmt);
static const char * DeparseRenameStmt(RenameStmt *stmt);
static const char * DeparseRenameAttributeStmt(RenameStmt *stmt);
static const char * DeparseAlterObjectSchemaStmt(AlterObjectSchemaStmt *stmt);
static const char * DeparseAlterOwnerStmt(AlterOwnerStmt *stmt);
/*
* DeparseTreeNode aims to be the inverse of postgres' ParseTreeNode. Currently with
* limited support. Check support before using, and add support for new statements as
* required.
*
* Currently supported:
* - CREATE TYPE
* - ALTER TYPE
* - DROP TYPE
*/
const char *
DeparseTreeNode(Node *stmt)
{
switch (nodeTag(stmt))
{
case T_DropStmt:
{
return DeparseDropStmt(castNode(DropStmt, stmt));
}
case T_CompositeTypeStmt:
{
return DeparseCompositeTypeStmt(castNode(CompositeTypeStmt, stmt));
}
case T_CreateEnumStmt:
{
return DeparseCreateEnumStmt(castNode(CreateEnumStmt, stmt));
}
case T_AlterTableStmt:
{
return DeparseAlterTableStmt(castNode(AlterTableStmt, stmt));
}
case T_AlterEnumStmt:
{
return DeparseAlterEnumStmt(castNode(AlterEnumStmt, stmt));
}
case T_RenameStmt:
{
return DeparseRenameStmt(castNode(RenameStmt, stmt));
}
case T_AlterObjectSchemaStmt:
{
return DeparseAlterObjectSchemaStmt(castNode(AlterObjectSchemaStmt, stmt));
}
case T_AlterOwnerStmt:
{
return DeparseAlterOwnerStmt(castNode(AlterOwnerStmt, stmt));
}
default:
{
ereport(ERROR, (errmsg("unsupported statement for deparsing")));
}
}
}
static const char *
DeparseDropStmt(DropStmt *stmt)
{
switch (stmt->removeType)
{
case OBJECT_TYPE:
{
return DeparseDropTypeStmt(stmt);
}
default:
{
ereport(ERROR, (errmsg("unsupported drop statement for deparsing")));
}
}
}
static const char *
DeparseAlterTableStmt(AlterTableStmt *stmt)
{
switch (stmt->relkind)
{
case OBJECT_TYPE:
{
return DeparseAlterTypeStmt(stmt);
}
default:
{
ereport(ERROR, (errmsg("unsupported alter statement for deparsing")));
}
}
}
static const char *
DeparseRenameStmt(RenameStmt *stmt)
{
switch (stmt->renameType)
{
case OBJECT_TYPE:
{
return DeparseRenameTypeStmt(stmt);
}
case OBJECT_ATTRIBUTE:
{
return DeparseRenameAttributeStmt(stmt);
}
default:
{
ereport(ERROR, (errmsg("unsupported rename statement for deparsing")));
}
}
}
static const char *
DeparseRenameAttributeStmt(RenameStmt *stmt)
{
Assert(stmt->renameType == OBJECT_ATTRIBUTE);
switch (stmt->relationType)
{
case OBJECT_TYPE:
{
return DeparseRenameTypeAttributeStmt(stmt);
}
default:
{
ereport(ERROR, (errmsg("unsupported rename attribute statement for"
" deparsing")));
}
}
}
static const char *
DeparseAlterObjectSchemaStmt(AlterObjectSchemaStmt *stmt)
{
switch (stmt->objectType)
{
case OBJECT_TYPE:
{
return DeparseAlterTypeSchemaStmt(stmt);
}
default:
{
ereport(ERROR, (errmsg("unsupported rename statement for deparsing")));
}
}
}
static const char *
DeparseAlterOwnerStmt(AlterOwnerStmt *stmt)
{
switch (stmt->objectType)
{
case OBJECT_TYPE:
{
return DeparseAlterTypeOwnerStmt(stmt);
}
default:
{
ereport(ERROR, (errmsg("unsupported alter owner statement for deparsing")));
}
}
}

View File

@ -0,0 +1,509 @@
/*-------------------------------------------------------------------------
*
* deparse_type_stmts.c
* All routines to deparse type statements.
* This file contains all entry points specific for type statement deparsing as well as
* functions that are currently only used for deparsing of the type statements.
*
* Functions that could move later are AppendColumnDef, AppendColumnDefList, etc. These
* should be reused across multiple statements and should live in their own deparse
* file.
*
* Copyright (c) 2019, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "catalog/namespace.h"
#include "lib/stringinfo.h"
#include "nodes/makefuncs.h"
#include "nodes/parsenodes.h"
#include "parser/parse_type.h"
#include "utils/builtins.h"
#include "distributed/citus_ruleutils.h"
#include "distributed/commands.h"
#include "distributed/deparser.h"
#include "distributed/metadata/namespace.h"
#define AlterEnumIsRename(stmt) (stmt->oldVal != NULL)
#define AlterEnumIsAddValue(stmt) (stmt->oldVal == NULL)
/* forward declaration for deparse functions */
static void AppendCompositeTypeStmt(StringInfo str, CompositeTypeStmt *stmt);
static void AppendColumnDef(StringInfo str, ColumnDef *columnDef);
static void AppendColumnDefList(StringInfo str, List *columnDefs);
static void AppendCreateEnumStmt(StringInfo str, CreateEnumStmt *stmt);
static void AppendStringList(StringInfo str, List *strings);
static void AppendDropTypeStmt(StringInfo buf, DropStmt *stmt);
static void AppendTypeNameList(StringInfo buf, List *objects);
static void AppendAlterEnumStmt(StringInfo buf, AlterEnumStmt *stmt);
static void AppendAlterTypeStmt(StringInfo buf, AlterTableStmt *stmt);
static void AppendAlterTypeCmd(StringInfo buf, AlterTableCmd *alterTableCmd);
static void AppendAlterTypeCmdAddColumn(StringInfo buf, AlterTableCmd *alterTableCmd);
static void AppendAlterTypeCmdDropColumn(StringInfo buf, AlterTableCmd *alterTableCmd);
static void AppendAlterTypeCmdAlterColumnType(StringInfo buf,
AlterTableCmd *alterTableCmd);
static void AppendRenameTypeStmt(StringInfo buf, RenameStmt *stmt);
static void AppendRenameTypeAttributeStmt(StringInfo buf, RenameStmt *stmt);
static void AppendAlterTypeSchemaStmt(StringInfo buf, AlterObjectSchemaStmt *stmt);
static void AppendAlterTypeOwnerStmt(StringInfo buf, AlterOwnerStmt *stmt);
/*
* DeparseCompositeTypeStmt builds and returns a string representing the
* CompositeTypeStmt for application on a remote server.
*/
const char *
DeparseCompositeTypeStmt(CompositeTypeStmt *stmt)
{
StringInfoData sql = { 0 };
initStringInfo(&sql);
AppendCompositeTypeStmt(&sql, stmt);
return sql.data;
}
const char *
DeparseCreateEnumStmt(CreateEnumStmt *stmt)
{
StringInfoData sql = { 0 };
initStringInfo(&sql);
AppendCreateEnumStmt(&sql, stmt);
return sql.data;
}
const char *
DeparseAlterEnumStmt(AlterEnumStmt *stmt)
{
StringInfoData sql = { 0 };
initStringInfo(&sql);
AppendAlterEnumStmt(&sql, stmt);
return sql.data;
}
const char *
DeparseDropTypeStmt(DropStmt *stmt)
{
StringInfoData str = { 0 };
initStringInfo(&str);
Assert(stmt->removeType == OBJECT_TYPE);
AppendDropTypeStmt(&str, stmt);
return str.data;
}
const char *
DeparseAlterTypeStmt(AlterTableStmt *stmt)
{
StringInfoData str = { 0 };
initStringInfo(&str);
Assert(stmt->relkind == OBJECT_TYPE);
AppendAlterTypeStmt(&str, stmt);
return str.data;
}
static void
AppendAlterTypeStmt(StringInfo buf, AlterTableStmt *stmt)
{
const char *identifier = quote_qualified_identifier(stmt->relation->schemaname,
stmt->relation->relname);
ListCell *cmdCell = NULL;
Assert(stmt->relkind = OBJECT_TYPE);
appendStringInfo(buf, "ALTER TYPE %s", identifier);
foreach(cmdCell, stmt->cmds)
{
AlterTableCmd *alterTableCmd = NULL;
if (cmdCell != list_head(stmt->cmds))
{
appendStringInfoString(buf, ", ");
}
alterTableCmd = castNode(AlterTableCmd, lfirst(cmdCell));
AppendAlterTypeCmd(buf, alterTableCmd);
}
appendStringInfoString(buf, ";");
}
static void
AppendAlterTypeCmd(StringInfo buf, AlterTableCmd *alterTableCmd)
{
switch (alterTableCmd->subtype)
{
case AT_AddColumn:
{
AppendAlterTypeCmdAddColumn(buf, alterTableCmd);
break;
}
case AT_DropColumn:
{
AppendAlterTypeCmdDropColumn(buf, alterTableCmd);
break;
}
case AT_AlterColumnType:
{
AppendAlterTypeCmdAlterColumnType(buf, alterTableCmd);
break;
}
default:
{
ereport(ERROR, (errmsg("unsupported subtype for alter table command"),
errdetail("sub command type: %d", alterTableCmd->subtype)));
}
}
}
static void
AppendAlterTypeCmdAddColumn(StringInfo buf, AlterTableCmd *alterTableCmd)
{
Assert(alterTableCmd->subtype == AT_AddColumn);
appendStringInfoString(buf, " ADD ATTRIBUTE ");
AppendColumnDef(buf, castNode(ColumnDef, alterTableCmd->def));
}
static void
AppendAlterTypeCmdDropColumn(StringInfo buf, AlterTableCmd *alterTableCmd)
{
Assert(alterTableCmd->subtype == AT_DropColumn);
appendStringInfo(buf, " DROP ATTRIBUTE %s", quote_identifier(alterTableCmd->name));
if (alterTableCmd->behavior == DROP_CASCADE)
{
appendStringInfoString(buf, " CASCADE");
}
}
static void
AppendAlterTypeCmdAlterColumnType(StringInfo buf, AlterTableCmd *alterTableCmd)
{
Assert(alterTableCmd->subtype == AT_AlterColumnType);
appendStringInfo(buf, " ALTER ATTRIBUTE %s SET DATA TYPE ", quote_identifier(
alterTableCmd->name));
AppendColumnDef(buf, castNode(ColumnDef, alterTableCmd->def));
if (alterTableCmd->behavior == DROP_CASCADE)
{
appendStringInfoString(buf, " CASCADE");
}
}
static void
AppendAlterEnumStmt(StringInfo buf, AlterEnumStmt *stmt)
{
appendStringInfo(buf, "ALTER TYPE %s", NameListToQuotedString(stmt->typeName));
if (AlterEnumIsRename(stmt))
{
/* Rename an existing label */
appendStringInfo(buf, " RENAME VALUE %s TO %s;",
quote_literal_cstr(stmt->oldVal),
quote_literal_cstr(stmt->newVal));
}
else if (AlterEnumIsAddValue(stmt))
{
/* Add a new label */
appendStringInfoString(buf, " ADD VALUE ");
if (stmt->skipIfNewValExists)
{
appendStringInfoString(buf, "IF NOT EXISTS ");
}
appendStringInfoString(buf, quote_literal_cstr(stmt->newVal));
if (stmt->newValNeighbor)
{
appendStringInfo(buf, " %s %s",
stmt->newValIsAfter ? "AFTER" : "BEFORE",
quote_literal_cstr(stmt->newValNeighbor));
}
appendStringInfoString(buf, ";");
}
}
static void
AppendDropTypeStmt(StringInfo buf, DropStmt *stmt)
{
/*
* already tested at call site, but for future it might be collapsed in a
* DeparseDropStmt so be safe and check again
*/
Assert(stmt->removeType == OBJECT_TYPE);
appendStringInfo(buf, "DROP TYPE ");
if (stmt->missing_ok)
{
appendStringInfoString(buf, "IF EXISTS ");
}
AppendTypeNameList(buf, stmt->objects);
if (stmt->behavior == DROP_CASCADE)
{
appendStringInfoString(buf, " CASCADE");
}
appendStringInfoString(buf, ";");
}
static void
AppendTypeNameList(StringInfo buf, List *objects)
{
ListCell *objectCell = NULL;
foreach(objectCell, objects)
{
TypeName *typeName = castNode(TypeName, lfirst(objectCell));
Oid typeOid = LookupTypeNameOid(NULL, typeName, false);
const char *identifier = format_type_be_qualified(typeOid);
if (objectCell != list_head(objects))
{
appendStringInfo(buf, ", ");
}
appendStringInfoString(buf, identifier);
}
}
/*
* AppendCompositeTypeStmt appends the sql string to recreate a CompositeTypeStmt to the
* provided buffer, ending in a ; for concatination of multiple statements.
*/
static void
AppendCompositeTypeStmt(StringInfo str, CompositeTypeStmt *stmt)
{
const char *identifier = quote_qualified_identifier(stmt->typevar->schemaname,
stmt->typevar->relname);
appendStringInfo(str, "CREATE TYPE %s AS (", identifier);
AppendColumnDefList(str, stmt->coldeflist);
appendStringInfo(str, ");");
}
static void
AppendCreateEnumStmt(StringInfo str, CreateEnumStmt *stmt)
{
RangeVar *typevar = NULL;
const char *identifier = NULL;
typevar = makeRangeVarFromNameList(stmt->typeName);
/* create the identifier from the fully qualified rangevar */
identifier = quote_qualified_identifier(typevar->schemaname, typevar->relname);
appendStringInfo(str, "CREATE TYPE %s AS ENUM (", identifier);
AppendStringList(str, stmt->vals);
appendStringInfo(str, ");");
}
static void
AppendStringList(StringInfo str, List *strings)
{
ListCell *stringCell = NULL;
foreach(stringCell, strings)
{
const char *string = strVal(lfirst(stringCell));
if (stringCell != list_head(strings))
{
appendStringInfoString(str, ", ");
}
string = quote_literal_cstr(string);
appendStringInfoString(str, string);
}
}
/*
* AppendColumnDefList appends the definition of a list of ColumnDef items to the provided
* buffer, adding separators as necessary.
*/
static void
AppendColumnDefList(StringInfo str, List *columnDefs)
{
ListCell *columnDefCell = NULL;
foreach(columnDefCell, columnDefs)
{
if (columnDefCell != list_head(columnDefs))
{
appendStringInfoString(str, ", ");
}
AppendColumnDef(str, castNode(ColumnDef, lfirst(columnDefCell)));
}
}
/*
* AppendColumnDef appends the definition of one ColumnDef completely qualified to the
* provided buffer.
*
* If the colname is not set that part is ommitted. This is the case in alter column type
* statements.
*/
static void
AppendColumnDef(StringInfo str, ColumnDef *columnDef)
{
Oid typeOid = LookupTypeNameOid(NULL, columnDef->typeName, false);
Oid collationOid = GetColumnDefCollation(NULL, columnDef, typeOid);
Assert(!columnDef->is_not_null); /* not null is not supported on composite types */
if (columnDef->colname)
{
appendStringInfo(str, "%s ", quote_identifier(columnDef->colname));
}
appendStringInfo(str, "%s", format_type_be_qualified(typeOid));
if (OidIsValid(collationOid))
{
const char *identifier = FormatCollateBEQualified(collationOid);
appendStringInfo(str, " COLLATE %s", identifier);
}
}
const char *
DeparseRenameTypeStmt(RenameStmt *stmt)
{
StringInfoData str = { 0 };
initStringInfo(&str);
Assert(stmt->renameType == OBJECT_TYPE);
AppendRenameTypeStmt(&str, stmt);
return str.data;
}
static void
AppendRenameTypeStmt(StringInfo buf, RenameStmt *stmt)
{
List *names = (List *) stmt->object;
appendStringInfo(buf, "ALTER TYPE %s RENAME TO %s;", NameListToQuotedString(names),
quote_identifier(stmt->newname));
}
const char *
DeparseRenameTypeAttributeStmt(RenameStmt *stmt)
{
StringInfoData str = { 0 };
initStringInfo(&str);
Assert(stmt->renameType == OBJECT_ATTRIBUTE);
Assert(stmt->relationType == OBJECT_TYPE);
AppendRenameTypeAttributeStmt(&str, stmt);
return str.data;
}
static void
AppendRenameTypeAttributeStmt(StringInfo buf, RenameStmt *stmt)
{
appendStringInfo(buf, "ALTER TYPE %s RENAME ATTRIBUTE %s TO %s",
quote_qualified_identifier(stmt->relation->schemaname,
stmt->relation->relname),
quote_identifier(stmt->subname),
quote_identifier(stmt->newname));
if (stmt->behavior == DROP_CASCADE)
{
appendStringInfoString(buf, " CASCADE");
}
appendStringInfoString(buf, ";");
}
const char *
DeparseAlterTypeSchemaStmt(AlterObjectSchemaStmt *stmt)
{
StringInfoData str = { 0 };
initStringInfo(&str);
Assert(stmt->objectType == OBJECT_TYPE);
AppendAlterTypeSchemaStmt(&str, stmt);
return str.data;
}
static void
AppendAlterTypeSchemaStmt(StringInfo buf, AlterObjectSchemaStmt *stmt)
{
List *names = NIL;
Assert(stmt->objectType == OBJECT_TYPE);
names = (List *) stmt->object;
appendStringInfo(buf, "ALTER TYPE %s SET SCHEMA %s;", NameListToQuotedString(names),
quote_identifier(stmt->newschema));
}
const char *
DeparseAlterTypeOwnerStmt(AlterOwnerStmt *stmt)
{
StringInfoData str = { 0 };
initStringInfo(&str);
Assert(stmt->objectType == OBJECT_TYPE);
AppendAlterTypeOwnerStmt(&str, stmt);
return str.data;
}
static void
AppendAlterTypeOwnerStmt(StringInfo buf, AlterOwnerStmt *stmt)
{
List *names = NIL;
Assert(stmt->objectType == OBJECT_TYPE);
names = (List *) stmt->object;
appendStringInfo(buf, "ALTER TYPE %s OWNER TO %s;", NameListToQuotedString(names),
RoleSpecString(stmt->newowner));
}

View File

@ -0,0 +1,104 @@
/*-------------------------------------------------------------------------
*
* format_collate.c
* Display collate names "nicely".
*
* This file is modeled after postgres' utils/adt/format_*.c files
*
* Copyright (c) 2019, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "access/htup_details.h"
#include "catalog/namespace.h"
#include "catalog/pg_collation.h"
#include "utils/builtins.h"
#include "utils/lsyscache.h"
#include "utils/syscache.h"
#include "distributed/deparser.h"
/*
* This version is for use within the backend in error messages, etc.
* One difference is that it will fail for an invalid collate.
*
* The result is always a palloc'd string.
*/
char *
FormatCollateBE(Oid collate_oid)
{
return FormatCollateExtended(collate_oid, 0);
}
/*
* This version returns a name that is always qualified.
*/
char *
FormatCollateBEQualified(Oid collate_oid)
{
return FormatCollateExtended(collate_oid, FORMAT_COLLATE_FORCE_QUALIFY);
}
/*
* FormatCollateExtended - inspired by format_type_extended
* Generate a possibly-qualified collate name.
*
* The default behavior is to only qualify if the type is not in the search
* path, and to raise an error if a non-existent collate_oid is given.
*
* The following bits in 'flags' modify the behavior:
* - FORMAT_COLLATE_FORCE_QUALIFY
* always schema-qualify collate names, regardless of search_path
*
* Returns a palloc'd string.
*/
char *
FormatCollateExtended(Oid collid, bits16 flags)
{
HeapTuple tuple = NULL;
Form_pg_collation collform = NULL;
char *buf = NULL;
char *nspname = NULL;
char *typname = NULL;
if (collid == InvalidOid && (flags & FORMAT_COLLATE_ALLOW_INVALID) != 0)
{
return pstrdup("-");
}
tuple = SearchSysCache1(COLLOID, ObjectIdGetDatum(collid));
if (!HeapTupleIsValid(tuple))
{
if ((flags & FORMAT_COLLATE_ALLOW_INVALID) != 0)
{
return pstrdup("???");
}
else
{
elog(ERROR, "cache lookup failed for collate %u", collid);
}
}
collform = (Form_pg_collation) GETSTRUCT(tuple);
if ((flags & FORMAT_COLLATE_FORCE_QUALIFY) == 0 && CollationIsVisible(collid))
{
nspname = NULL;
}
else
{
nspname = get_namespace_name_or_temp(collform->collnamespace);
}
typname = NameStr(collform->collname);
buf = quote_qualified_identifier(nspname, typname);
ReleaseSysCache(tuple);
return buf;
}

View File

@ -0,0 +1,191 @@
/*-------------------------------------------------------------------------
*
* objectaddress.c
* Parstrees almost always target a object that postgres can address by
* an ObjectAddress. Here we have a walker for parsetrees to find the
* address of the object targeted.
*
* Copyright (c) 2019, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "distributed/commands.h"
#include "distributed/deparser.h"
static const ObjectAddress * AlterTableStmtObjectAddress(AlterTableStmt *stmt,
bool missing_ok);
static const ObjectAddress * RenameStmtObjectAddress(RenameStmt *stmt, bool missing_ok);
static const ObjectAddress * AlterObjectSchemaStmtObjectAddress(
AlterObjectSchemaStmt *stmt, bool missing_ok);
static const ObjectAddress * RenameAttributeStmtObjectAddress(RenameStmt *stmt,
bool missing_ok);
static const ObjectAddress * AlterOwnerStmtObjectAddress(AlterOwnerStmt *stmt,
bool missing_ok);
/*
* GetObjectAddressFromParseTree returns the ObjectAdderss of the main target of the parse
* tree.
*/
const ObjectAddress *
GetObjectAddressFromParseTree(Node *parseTree, bool missing_ok)
{
switch (parseTree->type)
{
case T_CompositeTypeStmt:
{
return CompositeTypeStmtObjectAddress(castNode(CompositeTypeStmt, parseTree),
missing_ok);
}
case T_AlterTableStmt:
{
return AlterTableStmtObjectAddress(castNode(AlterTableStmt, parseTree),
missing_ok);
}
case T_CreateEnumStmt:
{
return CreateEnumStmtObjectAddress(castNode(CreateEnumStmt, parseTree),
missing_ok);
}
case T_AlterEnumStmt:
{
return AlterEnumStmtObjectAddress(castNode(AlterEnumStmt, parseTree),
missing_ok);
}
case T_RenameStmt:
{
return RenameStmtObjectAddress(castNode(RenameStmt, parseTree), missing_ok);
}
case T_AlterObjectSchemaStmt:
{
return AlterObjectSchemaStmtObjectAddress(castNode(AlterObjectSchemaStmt,
parseTree), missing_ok);
}
case T_AlterOwnerStmt:
{
return AlterOwnerStmtObjectAddress(castNode(AlterOwnerStmt, parseTree),
missing_ok);
}
default:
{
/*
* should not be reached, indicates the coordinator is sending unsupported
* statements
*/
ereport(ERROR, (errmsg("unsupported statement to get object address for")));
return NULL;
}
}
}
static const ObjectAddress *
AlterTableStmtObjectAddress(AlterTableStmt *stmt, bool missing_ok)
{
switch (stmt->relkind)
{
case OBJECT_TYPE:
{
return AlterTypeStmtObjectAddress(stmt, missing_ok);
}
default:
{
ereport(ERROR, (errmsg("unsupported alter statement to get object address for"
)));
}
}
}
static const ObjectAddress *
RenameStmtObjectAddress(RenameStmt *stmt, bool missing_ok)
{
switch (stmt->renameType)
{
case OBJECT_TYPE:
{
return RenameTypeStmtObjectAddress(stmt, missing_ok);
}
case OBJECT_ATTRIBUTE:
{
return RenameAttributeStmtObjectAddress(stmt, missing_ok);
}
default:
{
ereport(ERROR, (errmsg("unsupported rename statement to get object address "
"for")));
}
}
}
static const ObjectAddress *
AlterObjectSchemaStmtObjectAddress(AlterObjectSchemaStmt *stmt, bool missing_ok)
{
switch (stmt->objectType)
{
case OBJECT_TYPE:
{
return AlterTypeSchemaStmtObjectAddress(stmt, missing_ok);
}
default:
{
ereport(ERROR, (errmsg("unsupported alter schema statement to get object "
"address for")));
}
}
}
static const ObjectAddress *
RenameAttributeStmtObjectAddress(RenameStmt *stmt, bool missing_ok)
{
Assert(stmt->renameType == OBJECT_ATTRIBUTE);
switch (stmt->relationType)
{
case OBJECT_TYPE:
{
return RenameTypeAttributeStmtObjectAddress(stmt, missing_ok);
}
default:
{
ereport(ERROR, (errmsg("unsupported alter rename attribute statement to get "
"object address for")));
}
}
}
static const ObjectAddress *
AlterOwnerStmtObjectAddress(AlterOwnerStmt *stmt, bool missing_ok)
{
switch (stmt->objectType)
{
case OBJECT_TYPE:
{
return AlterTypeOwnerObjectAddress(stmt, missing_ok);
}
default:
{
ereport(ERROR, (errmsg("unsupported alter owner statement to get object "
"address for")));
}
}
}

View File

@ -0,0 +1,198 @@
/*-------------------------------------------------------------------------
*
* qualify.c
* The deparser relies on fully qualified names on all statements to
* work transparently on a remote worker node. Here we have helpers to
* fully qualify parsetrees.
*
* Fully qualified parsetrees contain names for all identifiers that
* are search_path agnostic. Meaning we need to include the schema name
* for each and every identifier in the parsetree.
*
* This file contains mostly the distpatching functions to specialized
* functions for each class of objects. eg qualify_type_stmt.c contains
* all functions related to fully qualifying parsetrees that interact
* with types.
*
* Copyright (c) 2019, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "distributed/deparser.h"
static void QualifyRenameStmt(RenameStmt *stmt);
static void QualifyRenameAttributeStmt(RenameStmt *stmt);
static void QualifyAlterTableStmt(AlterTableStmt *stmt);
static void QualifyAlterObjectSchemaStmt(AlterObjectSchemaStmt *stmt);
static void QualifyAlterOwnerStmt(AlterOwnerStmt *stmt);
/*
* QualifyTreeNode transforms the statement in place and makes all (supported) statements
* fully qualified. Fully qualified statements allow for application on a remote postgres
* server irregardless of their search_path.
*/
void
QualifyTreeNode(Node *stmt)
{
switch (nodeTag(stmt))
{
case T_RenameStmt:
{
QualifyRenameStmt(castNode(RenameStmt, stmt));
return;
}
case T_AlterEnumStmt:
{
QualifyAlterEnumStmt(castNode(AlterEnumStmt, stmt));
return;
}
case T_AlterTableStmt:
{
QualifyAlterTableStmt(castNode(AlterTableStmt, stmt));
return;
}
case T_CompositeTypeStmt:
{
QualifyCompositeTypeStmt(castNode(CompositeTypeStmt, stmt));
return;
}
case T_CreateEnumStmt:
{
QualifyCreateEnumStmt(castNode(CreateEnumStmt, stmt));
return;
}
case T_AlterObjectSchemaStmt:
{
QualifyAlterObjectSchemaStmt(castNode(AlterObjectSchemaStmt, stmt));
return;
}
case T_AlterOwnerStmt:
{
QualifyAlterOwnerStmt(castNode(AlterOwnerStmt, stmt));
return;
}
default:
{
/* skip unsupported statements */
break;
}
}
}
static void
QualifyRenameStmt(RenameStmt *stmt)
{
switch (stmt->renameType)
{
case OBJECT_TYPE:
{
QualifyRenameTypeStmt(stmt);
return;
}
case OBJECT_ATTRIBUTE:
{
QualifyRenameAttributeStmt(stmt);
return;
}
default:
{
/* skip unsupported statements */
break;
}
}
}
static void
QualifyRenameAttributeStmt(RenameStmt *stmt)
{
Assert(stmt->renameType == OBJECT_ATTRIBUTE);
switch (stmt->relationType)
{
case OBJECT_TYPE:
{
QualifyRenameTypeAttributeStmt(stmt);
return;
}
default:
{
return;
}
}
}
static void
QualifyAlterTableStmt(AlterTableStmt *stmt)
{
switch (stmt->relkind)
{
case OBJECT_TYPE:
{
QualifyAlterTypeStmt(stmt);
return;
}
default:
{
/* skip unsupported statements */
break;
}
}
}
static void
QualifyAlterObjectSchemaStmt(AlterObjectSchemaStmt *stmt)
{
switch (stmt->objectType)
{
case OBJECT_TYPE:
{
QualifyAlterTypeSchemaStmt(stmt);
return;
}
default:
{
/* skip unsupported statements */
break;
}
}
}
static void
QualifyAlterOwnerStmt(AlterOwnerStmt *stmt)
{
switch (stmt->objectType)
{
case OBJECT_TYPE:
{
QualifyAlterTypeOwnerStmt(stmt);
return;
}
default:
{
return;
}
}
}

View File

@ -0,0 +1,190 @@
/*-------------------------------------------------------------------------
*
* qualify_type_stmt.c
* Functions specialized in fully qualifying all type statements. These
* functions are dispatched from qualify.c
*
* Fully qualifying type statements consists of adding the schema name
* to the subject of the types as well as any other branch of the
* parsetree.
*
* Goal would be that the deparser functions for these statements can
* serialize the statement without any external lookups.
*
* Copyright (c) 2019, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "access/heapam.h"
#include "access/htup_details.h"
#include "catalog/namespace.h"
#include "catalog/objectaddress.h"
#include "catalog/pg_type.h"
#include "distributed/deparser.h"
#include "distributed/metadata/namespace.h"
#include "nodes/makefuncs.h"
#include "parser/parse_type.h"
#include "utils/lsyscache.h"
static char * GetTypeNamespaceNameByNameList(List *names);
static Oid TypeOidGetNamespaceOid(Oid typeOid);
/*
* GetTypeNamespaceNameByNameList resolved the schema name of a type by its namelist.
*/
static char *
GetTypeNamespaceNameByNameList(List *names)
{
TypeName *typeName = makeTypeNameFromNameList(names);
Oid typeOid = LookupTypeNameOid(NULL, typeName, false);
Oid namespaceOid = TypeOidGetNamespaceOid(typeOid);
char *nspname = get_namespace_name_or_temp(namespaceOid);
return nspname;
}
/*
* TypeOidGetNamespaceOid resolves the namespace oid for a type identified by its type oid
*/
static Oid
TypeOidGetNamespaceOid(Oid typeOid)
{
Form_pg_type typeData = NULL;
Relation catalog = heap_open(TypeRelationId, AccessShareLock);
#if PG_VERSION_NUM >= 120000
HeapTuple typeTuple = get_catalog_object_by_oid(catalog, Anum_pg_type_oid, typeOid);
#else
HeapTuple typeTuple = get_catalog_object_by_oid(catalog, typeOid);
#endif
heap_close(catalog, AccessShareLock);
typeData = (Form_pg_type) GETSTRUCT(typeTuple);
return typeData->typnamespace;
}
void
QualifyRenameTypeStmt(RenameStmt *stmt)
{
List *names = (List *) stmt->object;
Assert(stmt->renameType == OBJECT_TYPE);
if (list_length(names) == 1)
{
/* not qualified, lookup name and add namespace name to names */
char *nspname = GetTypeNamespaceNameByNameList(names);
names = list_make2(makeString(nspname), linitial(names));
stmt->object = (Node *) names;
}
}
void
QualifyRenameTypeAttributeStmt(RenameStmt *stmt)
{
Assert(stmt->renameType == OBJECT_ATTRIBUTE);
Assert(stmt->relationType == OBJECT_TYPE);
if (stmt->relation->schemaname == NULL)
{
List *names = list_make1(makeString(stmt->relation->relname));
char *nspname = GetTypeNamespaceNameByNameList(names);
stmt->relation->schemaname = nspname;
}
}
void
QualifyAlterEnumStmt(AlterEnumStmt *stmt)
{
List *names = stmt->typeName;
if (list_length(names) == 1)
{
/* not qualified, lookup name and add namespace name to names */
char *nspname = GetTypeNamespaceNameByNameList(names);
names = list_make2(makeString(nspname), linitial(names));
stmt->typeName = names;
}
}
void
QualifyAlterTypeStmt(AlterTableStmt *stmt)
{
Assert(stmt->relkind == OBJECT_TYPE);
if (stmt->relation->schemaname == NULL)
{
List *names = MakeNameListFromRangeVar(stmt->relation);
char *nspname = GetTypeNamespaceNameByNameList(names);
stmt->relation->schemaname = nspname;
}
}
void
QualifyCompositeTypeStmt(CompositeTypeStmt *stmt)
{
if (stmt->typevar->schemaname == NULL)
{
Oid creationSchema = RangeVarGetCreationNamespace(stmt->typevar);
stmt->typevar->schemaname = get_namespace_name(creationSchema);
}
}
void
QualifyCreateEnumStmt(CreateEnumStmt *stmt)
{
if (list_length(stmt->typeName) == 1)
{
char *objname = NULL;
Oid creationSchema = QualifiedNameGetCreationNamespace(stmt->typeName, &objname);
stmt->typeName = list_make2(makeString(get_namespace_name(creationSchema)),
linitial(stmt->typeName));
}
}
void
QualifyAlterTypeSchemaStmt(AlterObjectSchemaStmt *stmt)
{
List *names = NIL;
Assert(stmt->objectType == OBJECT_TYPE);
names = (List *) stmt->object;
if (list_length(names) == 1)
{
/* not qualified with schema, lookup type and its schema s*/
char *nspname = GetTypeNamespaceNameByNameList(names);
names = list_make2(makeString(nspname), linitial(names));
stmt->object = (Node *) names;
}
}
void
QualifyAlterTypeOwnerStmt(AlterOwnerStmt *stmt)
{
List *names = NIL;
Assert(stmt->objectType == OBJECT_TYPE);
names = (List *) stmt->object;
if (list_length(names) == 1)
{
/* not qualified with schema, lookup type and its schema s*/
char *nspname = GetTypeNamespaceNameByNameList(names);
names = list_make2(makeString(nspname), linitial(names));
stmt->object = (Node *) names;
}
}

View File

@ -373,6 +373,12 @@ AcquireExecutorMultiShardLocks(List *taskList)
Task *task = (Task *) lfirst(taskCell); Task *task = (Task *) lfirst(taskCell);
LOCKMODE lockMode = NoLock; LOCKMODE lockMode = NoLock;
if (task->anchorShardId == INVALID_SHARD_ID)
{
/* no shard locks to take if the task is not anchored to a shard */
continue;
}
if (AllModificationsCommutative || list_length(task->taskPlacementList) == 1) if (AllModificationsCommutative || list_length(task->taskPlacementList) == 1)
{ {
/* /*
@ -1465,7 +1471,6 @@ ExecuteModifyTasks(List *taskList, bool expectResults, ParamListInfo paramListIn
int64 totalAffectedTupleCount = 0; int64 totalAffectedTupleCount = 0;
ListCell *taskCell = NULL; ListCell *taskCell = NULL;
Task *firstTask = NULL; Task *firstTask = NULL;
ShardInterval *firstShardInterval = NULL;
int connectionFlags = 0; int connectionFlags = 0;
List *affectedTupleCountList = NIL; List *affectedTupleCountList = NIL;
HTAB *shardConnectionHash = NULL; HTAB *shardConnectionHash = NULL;
@ -1486,11 +1491,15 @@ ExecuteModifyTasks(List *taskList, bool expectResults, ParamListInfo paramListIn
* ProcessUtility, so we only need to do this for DML commands. * ProcessUtility, so we only need to do this for DML commands.
*/ */
firstTask = (Task *) linitial(taskList); firstTask = (Task *) linitial(taskList);
firstShardInterval = LoadShardInterval(firstTask->anchorShardId); if (firstTask->taskType == MODIFY_TASK)
if (PartitionedTable(firstShardInterval->relationId) &&
firstTask->taskType == MODIFY_TASK)
{ {
LockPartitionRelations(firstShardInterval->relationId, RowExclusiveLock); ShardInterval *firstShardInterval = NULL;
firstShardInterval = LoadShardInterval(firstTask->anchorShardId);
if (PartitionedTable(firstShardInterval->relationId))
{
LockPartitionRelations(firstShardInterval->relationId, RowExclusiveLock);
}
} }
/* /*

View File

@ -322,6 +322,7 @@ RepairShardPlacement(int64 shardId, char *sourceNodeName, int32 sourceNodePort,
ddlCommandList = lappend(ddlCommandList, copyShardDataCommand->data); ddlCommandList = lappend(ddlCommandList, copyShardDataCommand->data);
} }
EnsureNoModificationsHaveBeenDone();
SendCommandListToWorkerInSingleTransaction(targetNodeName, targetNodePort, tableOwner, SendCommandListToWorkerInSingleTransaction(targetNodeName, targetNodePort, tableOwner,
ddlCommandList); ddlCommandList);

View File

@ -16,7 +16,9 @@
#include "access/skey.h" #include "access/skey.h"
#include "catalog/dependency.h" #include "catalog/dependency.h"
#include "catalog/indexing.h" #include "catalog/indexing.h"
#include "catalog/pg_class.h"
#include "catalog/pg_depend.h" #include "catalog/pg_depend.h"
#include "catalog/pg_type.h"
#include "distributed/metadata/dependency.h" #include "distributed/metadata/dependency.h"
#include "distributed/metadata/distobject.h" #include "distributed/metadata/distobject.h"
#include "utils/fmgroids.h" #include "utils/fmgroids.h"
@ -55,6 +57,7 @@ static void recurse_pg_depend(const ObjectAddress *target,
static bool FollowAllSupportedDependencies(void *context, Form_pg_depend pg_depend); static bool FollowAllSupportedDependencies(void *context, Form_pg_depend pg_depend);
static bool FollowNewSupportedDependencies(void *context, Form_pg_depend pg_depend); static bool FollowNewSupportedDependencies(void *context, Form_pg_depend pg_depend);
static void ApplyAddToDependencyList(void *context, Form_pg_depend pg_depend); static void ApplyAddToDependencyList(void *context, Form_pg_depend pg_depend);
static List * ExpandCitusSupportedTypes(void *context, const ObjectAddress *target);
/* forward declaration of support functions to decide what to follow */ /* forward declaration of support functions to decide what to follow */
static bool SupportedDependencyByCitus(const ObjectAddress *address); static bool SupportedDependencyByCitus(const ObjectAddress *address);
@ -74,7 +77,7 @@ GetDependenciesForObject(const ObjectAddress *target)
InitObjectAddressCollector(&collector); InitObjectAddressCollector(&collector);
recurse_pg_depend(target, recurse_pg_depend(target,
NULL, &ExpandCitusSupportedTypes,
&FollowNewSupportedDependencies, &FollowNewSupportedDependencies,
&ApplyAddToDependencyList, &ApplyAddToDependencyList,
&collector); &collector);
@ -114,7 +117,7 @@ OrderObjectAddressListInDependencyOrder(List *objectAddressList)
} }
recurse_pg_depend(objectAddress, recurse_pg_depend(objectAddress,
NULL, &ExpandCitusSupportedTypes,
&FollowAllSupportedDependencies, &FollowAllSupportedDependencies,
&ApplyAddToDependencyList, &ApplyAddToDependencyList,
&collector); &collector);
@ -313,6 +316,52 @@ SupportedDependencyByCitus(const ObjectAddress *address)
return true; return true;
} }
case OCLASS_TYPE:
{
switch (get_typtype(address->objectId))
{
case TYPTYPE_ENUM:
case TYPTYPE_COMPOSITE:
{
return true;
}
case TYPTYPE_BASE:
{
/*
* array types should be followed but not created, as they get created
* by the original type.
*/
return type_is_array(address->objectId);
}
default:
{
/* type not supported */
return false;
}
}
/*
* should be unreachable, break here is to make sure the function has a path
* without return, instead of falling through to the next block */
break;
}
case OCLASS_CLASS:
{
/*
* composite types have a reference to a relation of composite type, we need
* to follow those to get the dependencies of type fields.
*/
if (get_rel_relkind(address->objectId) == RELKIND_COMPOSITE_TYPE)
{
return true;
}
return false;
}
default: default:
{ {
/* unsupported type */ /* unsupported type */
@ -487,3 +536,78 @@ ApplyAddToDependencyList(void *context, Form_pg_depend pg_depend)
CollectObjectAddress(collector, &address); CollectObjectAddress(collector, &address);
} }
/*
* ExpandCitusSupportedTypes base on supported types by citus we might want to expand
* the list of objects to visit in pg_depend.
*
* An example where we want to expand is for types. Their dependencies are not captured
* with an entry in pg_depend from their object address, but by the object address of the
* relation describing the type.
*/
static List *
ExpandCitusSupportedTypes(void *context, const ObjectAddress *target)
{
List *result = NIL;
switch (target->classId)
{
case TypeRelationId:
{
/*
* types depending on other types are not captured in pg_depend, instead they
* are described with their dependencies by the relation that describes the
* composite type.
*/
if (get_typtype(target->objectId) == TYPTYPE_COMPOSITE)
{
Form_pg_depend dependency = palloc0(sizeof(FormData_pg_depend));
dependency->classid = target->classId;
dependency->objid = target->objectId;
dependency->objsubid = target->objectSubId;
/* add outward edge to the type's relation */
dependency->refclassid = RelationRelationId;
dependency->refobjid = get_typ_typrelid(target->objectId);
dependency->refobjsubid = 0;
dependency->deptype = DEPENDENCY_NORMAL;
result = lappend(result, dependency);
}
/*
* array types don't have a normal dependency on their element type, instead
* their dependency is an internal one. We can't follow interal dependencies
* as that would cause a cyclic dependency on others, instead we expand here
* to follow the dependency on the element type.
*/
if (type_is_array(target->objectId))
{
Form_pg_depend dependency = palloc0(sizeof(FormData_pg_depend));
dependency->classid = target->classId;
dependency->objid = target->objectId;
dependency->objsubid = target->objectSubId;
/* add outward edge to the element type */
dependency->refclassid = TypeRelationId;
dependency->refobjid = get_element_type(target->objectId);
dependency->refobjsubid = 0;
dependency->deptype = DEPENDENCY_NORMAL;
result = lappend(result, dependency);
}
break;
}
default:
{
/* no expansion for unsupported types */
break;
}
}
return result;
}

View File

@ -89,6 +89,11 @@ master_unmark_object_distributed(PG_FUNCTION_ARGS)
bool bool
ObjectExists(const ObjectAddress *address) ObjectExists(const ObjectAddress *address)
{ {
if (address == NULL)
{
return false;
}
if (is_objectclass_supported(address->classId)) if (is_objectclass_supported(address->classId))
{ {
HeapTuple objtup; HeapTuple objtup;

View File

@ -145,6 +145,7 @@ start_metadata_sync_to_node(PG_FUNCTION_ARGS)
* createMetadataSnapshotCommandList in the same transaction that we send * createMetadataSnapshotCommandList in the same transaction that we send
* nodeDeleteCommand and nodeInsertCommand commands below. * nodeDeleteCommand and nodeInsertCommand commands below.
*/ */
EnsureNoModificationsHaveBeenDone();
SendCommandListToWorkerInSingleTransaction(nodeNameString, nodePort, extensionOwner, SendCommandListToWorkerInSingleTransaction(nodeNameString, nodePort, extensionOwner,
recreateMetadataSnapshotCommandList); recreateMetadataSnapshotCommandList);

View File

@ -0,0 +1,35 @@
#include "postgres.h"
#include "nodes/primnodes.h"
#include "nodes/value.h"
#include "distributed/metadata/namespace.h"
/*
* MakeNameListFromRangeVar makes a namelist from a RangeVar. Its behaviour should be the
* exact opposite of postgres' makeRangeVarFromNameList.
*/
List *
MakeNameListFromRangeVar(const RangeVar *rel)
{
if (rel->catalogname != NULL)
{
Assert(rel->schemaname != NULL);
Assert(rel->relname != NULL);
return list_make3(makeString(rel->catalogname),
makeString(rel->schemaname),
makeString(rel->relname));
}
else if (rel->schemaname != NULL)
{
Assert(rel->relname != NULL);
return list_make2(makeString(rel->schemaname),
makeString(rel->relname));
}
else
{
Assert(rel->relname != NULL);
return list_make1(makeString(rel->relname));
}
}

View File

@ -439,7 +439,10 @@ RecordRelationParallelDDLAccessForTask(Task *task)
lastRelationId = currentRelationId; lastRelationId = currentRelationId;
} }
RecordParallelDDLAccess(RelationIdForShard(task->anchorShardId)); if (task->anchorShardId != INVALID_SHARD_ID)
{
RecordParallelDDLAccess(RelationIdForShard(task->anchorShardId));
}
} }

View File

@ -38,11 +38,23 @@
* 2PC. * 2PC.
*/ */
void void
SendCommandToWorker(char *nodeName, int32 nodePort, char *command) SendCommandToWorker(char *nodeName, int32 nodePort, const char *command)
{
const char *nodeUser = CitusExtensionOwnerName();
SendCommandToWorkerAsUser(nodeName, nodePort, nodeUser, command);
}
/*
* SendCommandToWorkerAsUSer sends a command to a particular worker as a particular user
* as part of the 2PC.
*/
void
SendCommandToWorkerAsUser(char *nodeName, int32 nodePort, const char *nodeUser,
const char *command)
{ {
MultiConnection *transactionConnection = NULL; MultiConnection *transactionConnection = NULL;
char *nodeUser = CitusExtensionOwnerName(); uint connectionFlags = 0;
int connectionFlags = 0;
BeginOrContinueCoordinatedTransaction(); BeginOrContinueCoordinatedTransaction();
CoordinatedTransactionUse2PC(); CoordinatedTransactionUse2PC();
@ -87,9 +99,42 @@ SendCommandToFirstWorker(char *command)
* owner to ensure write access to the Citus metadata tables. * owner to ensure write access to the Citus metadata tables.
*/ */
void void
SendCommandToWorkers(TargetWorkerSet targetWorkerSet, char *command) SendCommandToWorkers(TargetWorkerSet targetWorkerSet, const char *command)
{ {
SendCommandToWorkersParams(targetWorkerSet, command, 0, NULL, NULL); SendCommandToWorkersParams(targetWorkerSet, command, CitusExtensionOwnerName(),
0, NULL, NULL);
}
/*
* TargetWorkerSetNodeList returns a list of WorkerNode's that satisfies the
* TargetWorkerSet.
*/
List *
TargetWorkerSetNodeList(TargetWorkerSet targetWorkerSet)
{
List *workerNodeList = ActivePrimaryNodeList();
ListCell *workerNodeCell = NULL;
List *result = NIL;
foreach(workerNodeCell, workerNodeList)
{
WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell);
if (targetWorkerSet == WORKERS_WITH_METADATA &&
!workerNode->hasMetadata)
{
continue;
}
if (targetWorkerSet == OTHER_WORKERS &&
workerNode->groupId == GetLocalGroupId())
{
continue;
}
result = lappend(result, workerNode);
}
return result;
} }
@ -103,7 +148,7 @@ SendCommandToWorkers(TargetWorkerSet targetWorkerSet, char *command)
void void
SendBareCommandListToWorkers(TargetWorkerSet targetWorkerSet, List *commandList) SendBareCommandListToWorkers(TargetWorkerSet targetWorkerSet, List *commandList)
{ {
List *workerNodeList = ActivePrimaryNodeList(); List *workerNodeList = TargetWorkerSetNodeList(targetWorkerSet);
ListCell *workerNodeCell = NULL; ListCell *workerNodeCell = NULL;
char *nodeUser = CitusExtensionOwnerName(); char *nodeUser = CitusExtensionOwnerName();
ListCell *commandCell = NULL; ListCell *commandCell = NULL;
@ -117,18 +162,6 @@ SendBareCommandListToWorkers(TargetWorkerSet targetWorkerSet, List *commandList)
int nodePort = workerNode->workerPort; int nodePort = workerNode->workerPort;
int connectionFlags = FORCE_NEW_CONNECTION; int connectionFlags = FORCE_NEW_CONNECTION;
if (targetWorkerSet == WORKERS_WITH_METADATA &&
!workerNode->hasMetadata)
{
continue;
}
if (targetWorkerSet == OTHER_WORKERS &&
workerNode->groupId == GetLocalGroupId())
{
continue;
}
workerConnection = GetNodeUserDatabaseConnection(connectionFlags, nodeName, workerConnection = GetNodeUserDatabaseConnection(connectionFlags, nodeName,
nodePort, nodeUser, NULL); nodePort, nodeUser, NULL);
@ -145,6 +178,52 @@ SendBareCommandListToWorkers(TargetWorkerSet targetWorkerSet, List *commandList)
} }
/*
* SendBareOptionalCommandListToWorkersAsUser sends a list of commands to a set of target
* workers in serial. Commands are committed immediately: new connections are
* always used and no transaction block is used (hence "bare").
*/
int
SendBareOptionalCommandListToWorkersAsUser(TargetWorkerSet targetWorkerSet,
List *commandList, const char *user)
{
List *workerNodeList = TargetWorkerSetNodeList(targetWorkerSet);
ListCell *workerNodeCell = NULL;
ListCell *commandCell = NULL;
int maxError = RESPONSE_OKAY;
/* run commands serially */
foreach(workerNodeCell, workerNodeList)
{
MultiConnection *workerConnection = NULL;
WorkerNode *workerNode = (WorkerNode *) lfirst(workerNodeCell);
char *nodeName = workerNode->workerName;
int nodePort = workerNode->workerPort;
int connectionFlags = FORCE_NEW_CONNECTION;
workerConnection = GetNodeUserDatabaseConnection(connectionFlags, nodeName,
nodePort, user, NULL);
/* iterate over the commands and execute them in the same connection */
foreach(commandCell, commandList)
{
char *commandString = lfirst(commandCell);
int result = ExecuteOptionalRemoteCommand(workerConnection, commandString,
NULL);
if (result != RESPONSE_OKAY)
{
maxError = Max(maxError, result);
break;
}
}
CloseConnection(workerConnection);
}
return maxError;
}
/* /*
* SendCommandToWorkersParams sends a command to all workers in parallel. * SendCommandToWorkersParams sends a command to all workers in parallel.
* Commands are committed on the workers when the local transaction commits. The * Commands are committed on the workers when the local transaction commits. The
@ -154,15 +233,14 @@ SendBareCommandListToWorkers(TargetWorkerSet targetWorkerSet, List *commandList)
* respectively. * respectively.
*/ */
void void
SendCommandToWorkersParams(TargetWorkerSet targetWorkerSet, char *command, SendCommandToWorkersParams(TargetWorkerSet targetWorkerSet, const char *command,
int parameterCount, const Oid *parameterTypes, const char *user, int parameterCount,
const char *const *parameterValues) const Oid *parameterTypes, const char *const *parameterValues)
{ {
List *connectionList = NIL; List *connectionList = NIL;
ListCell *connectionCell = NULL; ListCell *connectionCell = NULL;
List *workerNodeList = ActivePrimaryNodeList(); List *workerNodeList = TargetWorkerSetNodeList(targetWorkerSet);
ListCell *workerNodeCell = NULL; ListCell *workerNodeCell = NULL;
char *nodeUser = CitusExtensionOwnerName();
BeginOrContinueCoordinatedTransaction(); BeginOrContinueCoordinatedTransaction();
CoordinatedTransactionUse2PC(); CoordinatedTransactionUse2PC();
@ -174,22 +252,10 @@ SendCommandToWorkersParams(TargetWorkerSet targetWorkerSet, char *command,
char *nodeName = workerNode->workerName; char *nodeName = workerNode->workerName;
int nodePort = workerNode->workerPort; int nodePort = workerNode->workerPort;
MultiConnection *connection = NULL; MultiConnection *connection = NULL;
int connectionFlags = 0; int32 connectionFlags = 0;
if (targetWorkerSet == WORKERS_WITH_METADATA &&
!workerNode->hasMetadata)
{
continue;
}
if (targetWorkerSet == OTHER_WORKERS &&
workerNode->groupId == GetLocalGroupId())
{
continue;
}
connection = StartNodeUserDatabaseConnection(connectionFlags, nodeName, nodePort, connection = StartNodeUserDatabaseConnection(connectionFlags, nodeName, nodePort,
nodeUser, NULL); user, NULL);
MarkRemoteTransactionCritical(connection); MarkRemoteTransactionCritical(connection);
@ -197,12 +263,7 @@ SendCommandToWorkersParams(TargetWorkerSet targetWorkerSet, char *command,
} }
/* finish opening connections */ /* finish opening connections */
foreach(connectionCell, connectionList) FinishConnectionListEstablishment(connectionList);
{
MultiConnection *connection = (MultiConnection *) lfirst(connectionCell);
FinishConnectionEstablishment(connection);
}
RemoteTransactionsBeginIfNecessary(connectionList); RemoteTransactionsBeginIfNecessary(connectionList);
@ -238,25 +299,34 @@ SendCommandToWorkersParams(TargetWorkerSet targetWorkerSet, char *command,
/* /*
* SendCommandListToWorkerInSingleTransaction opens connection to the node with the given * EnsureNoModificationsHaveBeenDone reports an error if we have performed any
* nodeName and nodePort. Then, the connection starts a transaction on the remote * modification in the current transaction to prevent opening a connection is such cases.
* node and executes the commands in the transaction. The function raises error if
* any of the queries fails.
*/ */
void void
SendCommandListToWorkerInSingleTransaction(char *nodeName, int32 nodePort, char *nodeUser, EnsureNoModificationsHaveBeenDone()
List *commandList)
{ {
MultiConnection *workerConnection = NULL;
ListCell *commandCell = NULL;
int connectionFlags = FORCE_NEW_CONNECTION;
if (XactModificationLevel > XACT_MODIFICATION_NONE) if (XactModificationLevel > XACT_MODIFICATION_NONE)
{ {
ereport(ERROR, (errcode(ERRCODE_ACTIVE_SQL_TRANSACTION), ereport(ERROR, (errcode(ERRCODE_ACTIVE_SQL_TRANSACTION),
errmsg("cannot open new connections after the first modification " errmsg("cannot open new connections after the first modification "
"command within a transaction"))); "command within a transaction")));
} }
}
/*
* SendCommandListToWorkerInSingleTransaction opens connection to the node with the given
* nodeName and nodePort. Then, the connection starts a transaction on the remote
* node and executes the commands in the transaction. The function raises error if
* any of the queries fails.
*/
void
SendCommandListToWorkerInSingleTransaction(const char *nodeName, int32 nodePort,
const char *nodeUser, List *commandList)
{
MultiConnection *workerConnection = NULL;
ListCell *commandCell = NULL;
int connectionFlags = FORCE_NEW_CONNECTION;
workerConnection = GetNodeUserDatabaseConnection(connectionFlags, nodeName, nodePort, workerConnection = GetNodeUserDatabaseConnection(connectionFlags, nodeName, nodePort,
nodeUser, NULL); nodeUser, NULL);

View File

@ -468,6 +468,7 @@ ActivateNode(char *nodeName, int nodePort)
if (WorkerNodeIsPrimary(workerNode)) if (WorkerNodeIsPrimary(workerNode))
{ {
EnsureNoModificationsHaveBeenDone();
ReplicateAllDependenciesToNode(nodeName, nodePort); ReplicateAllDependenciesToNode(nodeName, nodePort);
ReplicateAllReferenceTablesToNode(nodeName, nodePort); ReplicateAllReferenceTablesToNode(nodeName, nodePort);
} }

View File

@ -316,6 +316,7 @@ ReplicateShardToNode(ShardInterval *shardInterval, char *nodeName, int nodePort)
get_rel_name(shardInterval->relationId), nodeName, get_rel_name(shardInterval->relationId), nodeName,
nodePort))); nodePort)));
EnsureNoModificationsHaveBeenDone();
SendCommandListToWorkerInSingleTransaction(nodeName, nodePort, tableOwner, SendCommandListToWorkerInSingleTransaction(nodeName, nodePort, tableOwner,
ddlCommandList); ddlCommandList);
if (targetPlacement == NULL) if (targetPlacement == NULL)

View File

@ -0,0 +1,180 @@
/*-------------------------------------------------------------------------
*
* worker_create_if_not_exist.c
* TODO rename file and document, was named after old function
*
* Copyright (c) 2019, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include "catalog/dependency.h"
#include "catalog/pg_type.h"
#include "fmgr.h"
#include "nodes/makefuncs.h"
#include "nodes/nodes.h"
#include "parser/parse_type.h"
#include "tcop/dest.h"
#include "tcop/utility.h"
#include "utils/builtins.h"
#include "utils/regproc.h"
#include "distributed/commands.h"
#include "distributed/commands/utility_hook.h"
#include "distributed/deparser.h"
#include "distributed/metadata/distobject.h"
#include "distributed/worker_protocol.h"
static Node * CreateStmtByObjectAddress(const ObjectAddress *address);
static DropStmt * CreateDropStmtBasedOnCreateStmt(Node *createStmt);
PG_FUNCTION_INFO_V1(worker_create_or_replace_object);
/*
* worker_create_or_replace_object(statement text)
*
* function is called, by the coordinator, with a CREATE statement for an object. This
* function implements the CREATE ... IF NOT EXISTS functionality for objects that do not
* have this functionality or where their implementation is not sufficient.
*
* Besides checking if an object of said name exists it tries to compare the object to be
* created with the one in the local catalog. If there is a difference the on in the local
* catalog will be renamed after which the statement can be executed on this worker to
* create the object.
*
* Renaming has two purposes
* - free the identifier for creation
* - non destructive if there is data store that would be destroyed if the object was
* used in a table on this node, eg. types. If the type would be dropped with a cascade
* it would drop any column holding user data for this type.
*/
Datum
worker_create_or_replace_object(PG_FUNCTION_ARGS)
{
text *sqlStatementText = PG_GETARG_TEXT_P(0);
const char *sqlStatement = text_to_cstring(sqlStatementText);
const ObjectAddress *address = NULL;
Node *parseTree = ParseTreeNode(sqlStatement);
/*
* since going to the drop statement might require some resolving we will do a check
* if the type actually exists instead of adding the IF EXISTS keyword to the
* statement.
*/
address = GetObjectAddressFromParseTree(parseTree, true);
if (ObjectExists(address))
{
Node *localCreateStmt = NULL;
const char *localSqlStatement = NULL;
DropStmt *dropStmtParseTree = NULL;
localCreateStmt = CreateStmtByObjectAddress(address);
localSqlStatement = DeparseTreeNode(localCreateStmt);
if (strcmp(sqlStatement, localSqlStatement) == 0)
{
/*
* TODO string compare is a poor mans comparison, but calling equal on the
* parsetree's returns false because there is extra information list character
* position of some sort
*/
/*
* parseTree sent by the coordinator is the same as we would create for our
* object, therefore we can omit the create statement locally and not create
* the object as it already exists.
*
* We let the coordinator know we didn't create the object.
*/
PG_RETURN_BOOL(false);
}
/* TODO don't drop, instead rename as described in documentation */
/*
* there might be dependencies left on the worker on this type, these are not
* managed by citus anyway so it should be ok to drop, thus we cascade to any such
* dependencies
*/
dropStmtParseTree = CreateDropStmtBasedOnCreateStmt(parseTree);
if (dropStmtParseTree != NULL)
{
const char *sqlDropStmt = NULL;
/* force the drop */
dropStmtParseTree->behavior = DROP_CASCADE;
sqlDropStmt = DeparseTreeNode((Node *) dropStmtParseTree);
CitusProcessUtility((Node *) dropStmtParseTree, sqlDropStmt,
PROCESS_UTILITY_TOPLEVEL, NULL, None_Receiver, NULL);
}
}
/* apply create statement locally */
CitusProcessUtility(parseTree, sqlStatement, PROCESS_UTILITY_TOPLEVEL, NULL,
None_Receiver, NULL);
/* type has been created */
PG_RETURN_BOOL(true);
}
/*
* CreateStmtByObjectAddress returns a parsetree that will recreate the object addressed
* by the ObjectAddress provided.
*
* Note: this tree does not contain position information that is normally in a parsetree,
* therefore you cannot equal this tree against parsed statement. Instead it can be
* deparsed to do a string comparison.
*/
static Node *
CreateStmtByObjectAddress(const ObjectAddress *address)
{
switch (getObjectClass(address))
{
case OCLASS_TYPE:
{
return CreateTypeStmtByObjectAddress(address);
}
default:
{
ereport(ERROR, (errmsg("unsupported object to construct a create statment")));
}
}
}
/* TODO will be removed as we will not drop but rename instead */
static DropStmt *
CreateDropStmtBasedOnCreateStmt(Node *createStmt)
{
switch (nodeTag(createStmt))
{
case T_CompositeTypeStmt:
{
return CreateDropStmtBasedOnCompositeTypeStmt(
castNode(CompositeTypeStmt, createStmt));
}
case T_CreateEnumStmt:
{
return CreateDropStmtBasedOnEnumStmt(castNode(CreateEnumStmt, createStmt));
}
default:
{
/*
* should not be reached, indicates the coordinator is sending unsupported
* statements
*/
ereport(ERROR, (errmsg("unsupported statement to transform to a drop stmt"),
errhint("The coordinator send an unsupported command to the "
"worker")));
}
}
}

View File

@ -84,9 +84,14 @@ extern void ErrorIfUnsupportedRenameStmt(RenameStmt *renameStmt);
/* schema.c - forward declarations */ /* schema.c - forward declarations */
extern void ProcessDropSchemaStmt(DropStmt *dropSchemaStatement); extern void ProcessDropSchemaStmt(DropStmt *dropSchemaStatement);
extern List * PlanAlterTableSchemaStmt(AlterObjectSchemaStmt *stmt,
const char *queryString);
extern List * PlanAlterObjectSchemaStmt(AlterObjectSchemaStmt *alterObjectSchemaStmt, extern List * PlanAlterObjectSchemaStmt(AlterObjectSchemaStmt *alterObjectSchemaStmt,
const char *alterObjectSchemaCommand); const char *alterObjectSchemaCommand);
extern void ProcessAlterObjectSchemaStmt(AlterObjectSchemaStmt *stmt,
const char *queryString);
/* sequence.c - forward declarations */ /* sequence.c - forward declarations */
extern void ErrorIfUnsupportedSeqStmt(CreateSeqStmt *createSeqStmt); extern void ErrorIfUnsupportedSeqStmt(CreateSeqStmt *createSeqStmt);
@ -117,6 +122,42 @@ extern void ErrorIfUnsupportedConstraint(Relation relation, char distributionMet
/* truncate.c - forward declarations */ /* truncate.c - forward declarations */
extern void ProcessTruncateStatement(TruncateStmt *truncateStatement); extern void ProcessTruncateStatement(TruncateStmt *truncateStatement);
/* type.c - forward declarations */
extern List * PlanCompositeTypeStmt(CompositeTypeStmt *stmt, const char *queryString);
extern void ProcessCompositeTypeStmt(CompositeTypeStmt *stmt, const char *queryString);
extern List * PlanAlterTypeStmt(AlterTableStmt *stmt, const char *queryString);
extern List * PlanCreateEnumStmt(CreateEnumStmt *createEnumStmt, const char *queryString);
extern void ProcessCreateEnumStmt(CreateEnumStmt *stmt, const char *queryString);
extern List * PlanAlterEnumStmt(AlterEnumStmt *stmt, const char *queryString);
extern void ProcessAlterEnumStmt(AlterEnumStmt *stmt, const char *queryString);
extern List * PlanDropTypeStmt(DropStmt *stmt, const char *queryString);
extern List * PlanRenameTypeStmt(RenameStmt *stmt, const char *queryString);
extern List * PlanRenameTypeAttributeStmt(RenameStmt *stmt, const char *queryString);
extern List * PlanAlterTypeSchemaStmt(AlterObjectSchemaStmt *stmt,
const char *queryString);
extern List * PlanAlterTypeOwnerStmt(AlterOwnerStmt *stmt, const char *queryString);
extern void ProcessAlterTypeSchemaStmt(AlterObjectSchemaStmt *stmt,
const char *queryString);
extern Node * CreateTypeStmtByObjectAddress(const ObjectAddress *address);
extern const ObjectAddress * CompositeTypeStmtObjectAddress(CompositeTypeStmt *stmt,
bool missing_ok);
extern const ObjectAddress * CreateEnumStmtObjectAddress(CreateEnumStmt *stmt,
bool missing_ok);
extern const ObjectAddress * AlterTypeStmtObjectAddress(AlterTableStmt *stmt,
bool missing_ok);
extern const ObjectAddress * AlterEnumStmtObjectAddress(AlterEnumStmt *stmt,
bool missing_ok);
extern const ObjectAddress * RenameTypeStmtObjectAddress(RenameStmt *stmt,
bool missing_ok);
extern const ObjectAddress * AlterTypeSchemaStmtObjectAddress(AlterObjectSchemaStmt *stmt,
bool missing_ok);
extern const ObjectAddress * RenameTypeAttributeStmtObjectAddress(RenameStmt *stmt,
bool missing_ok);
extern const ObjectAddress * AlterTypeOwnerObjectAddress(AlterOwnerStmt *stmt,
bool missing_ok);
extern DropStmt * CreateDropStmtBasedOnCompositeTypeStmt(CompositeTypeStmt *stmt);
extern DropStmt * CreateDropStmtBasedOnEnumStmt(CreateEnumStmt *stmt);
extern List * CreateTypeDDLCommandsIdempotent(const ObjectAddress *typeAddress);
/* vacuum.c - froward declarations */ /* vacuum.c - froward declarations */
extern void ProcessVacuumStmt(VacuumStmt *vacuumStmt, const char *vacuumCommand); extern void ProcessVacuumStmt(VacuumStmt *vacuumStmt, const char *vacuumCommand);

View File

@ -12,10 +12,12 @@
#include "postgres.h" #include "postgres.h"
#include "distributed/version_compat.h"
#include "utils/relcache.h" #include "utils/relcache.h"
#include "tcop/utility.h" #include "tcop/utility.h"
#include "distributed/version_compat.h"
#include "distributed/worker_transaction.h"
typedef enum typedef enum
{ {
PROPSETCMD_INVALID = -1, PROPSETCMD_INVALID = -1,
@ -52,6 +54,7 @@ extern void CitusProcessUtility(Node *node, const char *queryString,
extern void MarkInvalidateForeignKeyGraph(void); extern void MarkInvalidateForeignKeyGraph(void);
extern void InvalidateForeignKeyGraphForDDL(void); extern void InvalidateForeignKeyGraphForDDL(void);
extern List * DDLTaskList(Oid relationId, const char *commandString); extern List * DDLTaskList(Oid relationId, const char *commandString);
extern List * NodeDDLTaskList(TargetWorkerSet targets, List *commands);
extern bool AlterTableInProgress(void); extern bool AlterTableInProgress(void);
#endif /* MULTI_UTILITY_H */ #endif /* MULTI_UTILITY_H */

View File

@ -0,0 +1,55 @@
/*-------------------------------------------------------------------------
*
* deparser.h
* Used when deparsing any ddl parsetree into its sql from.
*
* Copyright (c) 2019, Citus Data, Inc.
*
* $Id$
*
*-------------------------------------------------------------------------
*/
#ifndef CITUS_DEPARSER_H
#define CITUS_DEPARSER_H
#include "postgres.h"
#include "nodes/nodes.h"
#include "nodes/parsenodes.h"
#include "catalog/objectaddress.h"
/* forward declarations for format_collate.c */
/* Control flags for FormatCollateExtended, compatible with format_type_extended */
#define FORMAT_COLLATE_ALLOW_INVALID 0x02 /* allow invalid types */
#define FORMAT_COLLATE_FORCE_QUALIFY 0x04 /* force qualification of collate */
extern char * FormatCollateBE(Oid collate_oid);
extern char * FormatCollateBEQualified(Oid collate_oid);
extern char * FormatCollateExtended(Oid collid, bits16 flags);
extern void QualifyTreeNode(Node *stmt);
extern const char * DeparseTreeNode(Node *stmt);
extern const char * DeparseCompositeTypeStmt(CompositeTypeStmt *stmt);
extern const char * DeparseCreateEnumStmt(CreateEnumStmt *stmt);
extern const char * DeparseDropTypeStmt(DropStmt *stmt);
extern const char * DeparseAlterEnumStmt(AlterEnumStmt *stmt);
extern const char * DeparseAlterTypeStmt(AlterTableStmt *stmt);
extern const char * DeparseRenameTypeStmt(RenameStmt *stmt);
extern const char * DeparseRenameTypeAttributeStmt(RenameStmt *stmt);
extern const char * DeparseAlterTypeSchemaStmt(AlterObjectSchemaStmt *stmt);
extern const char * DeparseAlterTypeOwnerStmt(AlterOwnerStmt *stmt);
extern void QualifyRenameTypeStmt(RenameStmt *stmt);
extern void QualifyRenameTypeAttributeStmt(RenameStmt *stmt);
extern void QualifyAlterEnumStmt(AlterEnumStmt *stmt);
extern void QualifyAlterTypeStmt(AlterTableStmt *stmt);
extern void QualifyCompositeTypeStmt(CompositeTypeStmt *stmt);
extern void QualifyCreateEnumStmt(CreateEnumStmt *stmt);
extern void QualifyAlterTypeSchemaStmt(AlterObjectSchemaStmt *stmt);
extern void QualifyAlterTypeOwnerStmt(AlterOwnerStmt *stmt);
extern const ObjectAddress * GetObjectAddressFromParseTree(Node *parseTree, bool
missing_ok);
#endif /* CITUS_DEPARSER_H */

View File

@ -0,0 +1,20 @@
/*-------------------------------------------------------------------------
*
* namespace.h
* Helper functions for citus to work with postgres namespaces/schemas
*
* Copyright (c) 2019, Citus Data, Inc.
*
*-------------------------------------------------------------------------
*/
#ifndef CITUS_NAMESPACE_H
#define CITUS_NAMESPACE_H
#include "postgres.h"
#include "nodes/primnodes.h"
extern List * MakeNameListFromRangeVar(const RangeVar *rel);
#endif /*CITUS_NAMESPACE_H */

View File

@ -14,6 +14,7 @@
#include "distributed/connection_management.h" #include "distributed/connection_management.h"
/* errors which ExecuteRemoteCommand might return */ /* errors which ExecuteRemoteCommand might return */
#define RESPONSE_OKAY 0
#define QUERY_SEND_FAILED 1 #define QUERY_SEND_FAILED 1
#define RESPONSE_NOT_OKAY 2 #define RESPONSE_NOT_OKAY 2

View File

@ -29,16 +29,26 @@ typedef enum TargetWorkerSet
/* Functions declarations for worker transactions */ /* Functions declarations for worker transactions */
extern List * GetWorkerTransactions(void); extern List * GetWorkerTransactions(void);
extern void SendCommandToWorker(char *nodeName, int32 nodePort, char *command); extern List * TargetWorkerSetNodeList(TargetWorkerSet targetWorkerSet);
extern void SendCommandToWorker(char *nodeName, int32 nodePort, const char *command);
extern void SendCommandToWorkerAsUser(char *nodeName, int32 nodePort,
const char *nodeUser, const char *command);
extern void SendCommandToFirstWorker(char *command); extern void SendCommandToFirstWorker(char *command);
extern void SendCommandToWorkers(TargetWorkerSet targetWorkerSet, char *command); extern void SendCommandToWorkers(TargetWorkerSet targetWorkerSet, const char *command);
extern void SendBareCommandListToWorkers(TargetWorkerSet targetWorkerSet, extern void SendBareCommandListToWorkers(TargetWorkerSet targetWorkerSet,
List *commandList); List *commandList);
extern void SendCommandToWorkersParams(TargetWorkerSet targetWorkerSet, char *command, extern int SendBareOptionalCommandListToWorkersAsUser(TargetWorkerSet targetWorkerSet,
List *commandList,
const char *user);
extern void SendCommandToWorkersParams(TargetWorkerSet targetWorkerSet,
const char *command, const char *user,
int parameterCount, const Oid *parameterTypes, int parameterCount, const Oid *parameterTypes,
const char *const *parameterValues); const char *const *parameterValues);
extern void SendCommandListToWorkerInSingleTransaction(char *nodeName, int32 nodePort, extern void EnsureNoModificationsHaveBeenDone(void);
char *nodeUser, List *commandList); extern void SendCommandListToWorkerInSingleTransaction(const char *nodeName,
int32 nodePort,
const char *nodeUser,
List *commandList);
extern void RemoveWorkerTransaction(char *nodeName, int32 nodePort); extern void RemoveWorkerTransaction(char *nodeName, int32 nodePort);
/* helper functions for worker transactions */ /* helper functions for worker transactions */

View File

@ -57,6 +57,11 @@ check-base: all
$(pg_regress_multi_check) --load-extension=citus \ $(pg_regress_multi_check) --load-extension=citus \
-- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/base_schedule $(EXTRA_TESTS) -- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/base_schedule $(EXTRA_TESTS)
check-base-non-adaptive: all tempinstall-main
$(pg_regress_multi_check) --load-extension=citus \
--server-option=citus.task_executor_type=real-time \
-- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/base_schedule $(EXTRA_TESTS)
# check-minimal only sets up the cluster # check-minimal only sets up the cluster
check-minimal: all check-minimal: all
$(pg_regress_multi_check) --load-extension=citus \ $(pg_regress_multi_check) --load-extension=citus \

View File

@ -0,0 +1,338 @@
SET citus.next_shard_id TO 20010000;
CREATE USER typeuser;
NOTICE: not propagating CREATE ROLE/USER commands to worker nodes
HINT: Connect to worker nodes directly to manually create all necessary users and roles.
SELECT run_command_on_workers($$CREATE USER typeuser;$$);
run_command_on_workers
-----------------------------------
(localhost,57637,t,"CREATE ROLE")
(localhost,57638,t,"CREATE ROLE")
(2 rows)
CREATE SCHEMA type_tests AUTHORIZATION typeuser;
CREATE SCHEMA type_tests2 AUTHORIZATION typeuser; -- to test creation in a specific schema and moving to schema
SET search_path TO type_tests;
SET citus.shard_count TO 4;
-- single statement transactions with a simple type used in a table
CREATE TYPE tc1 AS (a int, b int);
CREATE TABLE t1 (a int PRIMARY KEY, b tc1);
SELECT create_distributed_table('t1','a');
create_distributed_table
--------------------------
(1 row)
INSERT INTO t1 VALUES (1, (2,3)::tc1);
SELECT * FROM t1;
a | b
---+-------
1 | (2,3)
(1 row)
ALTER TYPE tc1 RENAME TO tc1_newname;
INSERT INTO t1 VALUES (3, (4,5)::tc1_newname); -- insert with a cast would fail if the rename didn't propagate
ALTER TYPE tc1_newname SET SCHEMA type_tests2;
INSERT INTO t1 VALUES (6, (7,8)::type_tests2.tc1_newname); -- insert with a cast would fail if the rename didn't propagate
-- single statement transactions with a an enum used in a table
CREATE TYPE te1 AS ENUM ('one', 'two', 'three');
CREATE TABLE t2 (a int PRIMARY KEY, b te1);
SELECT create_distributed_table('t2','a');
create_distributed_table
--------------------------
(1 row)
INSERT INTO t2 VALUES (1, 'two');
SELECT * FROM t2;
a | b
---+-----
1 | two
(1 row)
-- rename enum, subsequent operations on the type would fail if the rename was not propagated
ALTER TYPE te1 RENAME TO te1_newname;
-- add an extra value to the enum and use in table
ALTER TYPE te1_newname ADD VALUE 'four';
UPDATE t2 SET b = 'four';
SELECT * FROM t2;
a | b
---+------
1 | four
(1 row)
-- change the schema of the type and use the new fully qualified name in an insert
ALTER TYPE te1_newname SET SCHEMA type_tests2;
INSERT INTO t2 VALUES (3, 'three'::type_tests2.te1_newname);
-- transaction block with simple type
BEGIN;
CREATE TYPE tc2 AS (a int, b int);
CREATE TABLE t3 (a int PRIMARY KEY, b tc2);
SELECT create_distributed_table('t3','a');
create_distributed_table
--------------------------
(1 row)
INSERT INTO t3 VALUES (4, (5,6)::tc2);
SELECT * FROM t3;
a | b
---+-------
4 | (5,6)
(1 row)
COMMIT;
-- transaction block with simple type
BEGIN;
CREATE TYPE te2 AS ENUM ('yes', 'no');
CREATE TABLE t4 (a int PRIMARY KEY, b te2);
SELECT create_distributed_table('t4','a');
create_distributed_table
--------------------------
(1 row)
INSERT INTO t4 VALUES (1, 'yes');
SELECT * FROM t4;
a | b
---+-----
1 | yes
(1 row)
-- ALTER TYPE ... ADD VALUE does not work in transactions
COMMIT;
-- test some combination of types without ddl propagation, this will prevent the workers
-- from having those types created. They are created just-in-time on table distribution
SET citus.enable_ddl_propagation TO off;
CREATE TYPE tc3 AS (a int, b int);
CREATE TYPE tc4 AS (a int, b tc3[]);
CREATE TYPE tc5 AS (a int, b tc4);
CREATE TYPE te3 AS ENUM ('a','b');
RESET citus.enable_ddl_propagation;
CREATE TABLE t5 (a int PRIMARY KEY, b tc5[], c te3);
SELECT create_distributed_table('t5','a');
create_distributed_table
--------------------------
(1 row)
-- test adding an attribute to a type and a column to a table both for a non-distributed type
SET citus.enable_ddl_propagation TO off;
CREATE TYPE te4 AS ENUM ('c','d');
CREATE TYPE tc6 AS (a int, b int);
CREATE TYPE tc6c AS (a int, b int);
RESET citus.enable_ddl_propagation;
-- types need to be fully qualified because of the search_path which is not supported by ALTER TYPE ... ADD COLUMN
ALTER TABLE t5 ADD COLUMN d type_tests.te4;
ALTER TABLE t5 ADD COLUMN e type_tests.tc6;
ALTER TYPE tc6 ADD ATTRIBUTE c tc6c;
-- last two values are only there if above commands succeeded
INSERT INTO t5 VALUES (1, NULL, 'a', 'd', (1,2,(4,5)::tc6c)::tc6);
-- test renaming an attribute of a distrbuted type and read it by its new name to verify propagation
ALTER TYPE tc6 RENAME ATTRIBUTE b TO d;
SELECT (e::tc6).d FROM t5 ORDER BY 1;
d
---
2
(1 row)
-- change owner of supported types and check ownership on remote server
ALTER TYPE te4 OWNER TO typeuser;
SELECT typname, usename FROM pg_type, pg_user where typname = 'te4' and typowner = usesysid;
typname | usename
---------+----------
te4 | typeuser
(1 row)
SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'te4' and typowner = usesysid;$$);
run_command_on_workers
--------------------------------------
(localhost,57637,t,"(te4,typeuser)")
(localhost,57638,t,"(te4,typeuser)")
(2 rows)
ALTER TYPE tc6 OWNER TO typeuser;
SELECT typname, usename FROM pg_type, pg_user where typname = 'tc6' and typowner = usesysid;
typname | usename
---------+----------
tc6 | typeuser
(1 row)
SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'tc6' and typowner = usesysid;$$);
run_command_on_workers
--------------------------------------
(localhost,57637,t,"(tc6,typeuser)")
(localhost,57638,t,"(tc6,typeuser)")
(2 rows)
-- create a type as a different user
SET ROLE typeuser;
-- create directly on the worker
CREATE TYPE tc7 AS (a int, b int);
CREATE TYPE te5 AS ENUM ('a','b','c');
-- cascade to the worker when table gets created
SET citus.enable_ddl_propagation TO off;
CREATE TYPE tc8 AS (a int, b int);
CREATE TYPE te6 AS ENUM ('a','b','c');
RESET citus.enable_ddl_propagation;
CREATE TABLE t6 (a int, b tc8, c te6);
SELECT create_distributed_table('t6', 'a');
create_distributed_table
--------------------------
(1 row)
RESET ROLE;
-- test ownership of all types
SELECT typname, usename FROM pg_type, pg_user where typname = 'tc7' and typowner = usesysid;
typname | usename
---------+----------
tc7 | typeuser
(1 row)
SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'tc7' and typowner = usesysid;$$);
run_command_on_workers
--------------------------------------
(localhost,57637,t,"(tc7,typeuser)")
(localhost,57638,t,"(tc7,typeuser)")
(2 rows)
SELECT typname, usename FROM pg_type, pg_user where typname = 'te5' and typowner = usesysid;
typname | usename
---------+----------
te5 | typeuser
(1 row)
SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'te5' and typowner = usesysid;$$);
run_command_on_workers
--------------------------------------
(localhost,57637,t,"(te5,typeuser)")
(localhost,57638,t,"(te5,typeuser)")
(2 rows)
SELECT typname, usename FROM pg_type, pg_user where typname = 'tc8' and typowner = usesysid;
typname | usename
---------+----------
tc8 | typeuser
(1 row)
SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'tc8' and typowner = usesysid;$$);
run_command_on_workers
--------------------------------------
(localhost,57637,t,"(tc8,typeuser)")
(localhost,57638,t,"(tc8,typeuser)")
(2 rows)
SELECT typname, usename FROM pg_type, pg_user where typname = 'te6' and typowner = usesysid;
typname | usename
---------+----------
te6 | typeuser
(1 row)
SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'te6' and typowner = usesysid;$$);
run_command_on_workers
--------------------------------------
(localhost,57637,t,"(te6,typeuser)")
(localhost,57638,t,"(te6,typeuser)")
(2 rows)
-- deleting the enum cascade will remove the type from the table and the workers
DROP TYPE te3 CASCADE;
NOTICE: drop cascades to column c of table t5
-- DELETE multiple types at once
DROP TYPE tc3, tc4, tc5 CASCADE;
NOTICE: drop cascades to column b of table t5
-- test if the types are deleted
SELECT typname FROM pg_type, pg_user where typname IN ('te3','tc3','tc4','tc5') and typowner = usesysid ORDER BY typname;
typname
---------
(0 rows)
SELECT run_command_on_workers($$SELECT typname FROM pg_type, pg_user where typname IN ('te3','tc3','tc4','tc5') and typowner = usesysid ORDER BY typname;$$);
run_command_on_workers
------------------------
(localhost,57637,t,"")
(localhost,57638,t,"")
(2 rows)
-- make sure attribute names are quoted correctly, no errors indicates types are propagated correctly
CREATE TYPE tc9 AS ("field-with-dashes" text COLLATE "en_US");
ALTER TYPE tc9 ADD ATTRIBUTE "some-more" int, ADD ATTRIBUTE normal int;
ALTER TYPE tc9 RENAME ATTRIBUTE normal TO "not-so-normal";
-- test alter statements for non-distributed types, if they would be propagated they would
-- error, preventing from changing them
SET citus.enable_ddl_propagation TO off;
CREATE TYPE non_distributed_composite_type AS (a int, b int);
CREATE TYPE non_distributed_enum_type AS ENUM ('a', 'c');
SET citus.enable_ddl_propagation TO on;
ALTER TYPE non_distributed_composite_type ADD ATTRIBUTE c int;
ALTER TYPE non_distributed_composite_type RENAME ATTRIBUTE c TO d;
ALTER TYPE non_distributed_composite_type ALTER ATTRIBUTE d SET DATA TYPE text COLLATE "en_US" CASCADE;
ALTER TYPE non_distributed_composite_type DROP ATTRIBUTE d;
ALTER TYPE non_distributed_composite_type OWNER TO typeuser;
ALTER TYPE non_distributed_composite_type RENAME TO non_distributed_composite_type_renamed;
ALTER TYPE non_distributed_composite_type_renamed RENAME TO non_distributed_composite_type;
ALTER TYPE non_distributed_composite_type SET SCHEMA type_tests2;
ALTER TYPE type_tests2.non_distributed_composite_type SET SCHEMA type_tests;
ALTER TYPE non_distributed_enum_type OWNER TO typeuser;
ALTER TYPE non_distributed_enum_type RENAME TO non_distributed_enum_type_renamed;
ALTER TYPE non_distributed_enum_type_renamed RENAME TO non_distributed_enum_type;
ALTER TYPE non_distributed_enum_type SET SCHEMA type_tests2;
ALTER TYPE type_tests2.non_distributed_enum_type SET SCHEMA type_tests;
ALTER TYPE non_distributed_enum_type ADD VALUE 'b' BEFORE 'c';
ALTER TYPE non_distributed_enum_type ADD VALUE 'd' AFTER 'c';
ALTER TYPE non_distributed_enum_type RENAME VALUE 'd' TO 'something-with-quotes''andstuff';
-- test all forms of alter statements on distributed types
CREATE TYPE distributed_composite_type AS (a int, b int);
CREATE TYPE distributed_enum_type AS ENUM ('a', 'c');
-- enforce distribution of types in every case
CREATE TABLE type_proc (a int, b distributed_composite_type, c distributed_enum_type);
SELECT create_distributed_table('type_proc','a');
create_distributed_table
--------------------------
(1 row)
DROP TABLE type_proc;
ALTER TYPE distributed_composite_type ADD ATTRIBUTE c int;
ALTER TYPE distributed_composite_type RENAME ATTRIBUTE c TO d;
ALTER TYPE distributed_composite_type ALTER ATTRIBUTE d SET DATA TYPE text COLLATE "en_US" CASCADE;
ALTER TYPE distributed_composite_type DROP ATTRIBUTE d;
ALTER TYPE distributed_composite_type OWNER TO typeuser;
ALTER TYPE distributed_composite_type RENAME TO distributed_composite_type_renamed;
ALTER TYPE distributed_composite_type_renamed RENAME TO distributed_composite_type;
ALTER TYPE distributed_composite_type SET SCHEMA type_tests2;
ALTER TYPE type_tests2.distributed_composite_type SET SCHEMA type_tests;
ALTER TYPE distributed_enum_type OWNER TO typeuser;
ALTER TYPE distributed_enum_type RENAME TO distributed_enum_type_renamed;
ALTER TYPE distributed_enum_type_renamed RENAME TO distributed_enum_type;
ALTER TYPE distributed_enum_type SET SCHEMA type_tests2;
ALTER TYPE type_tests2.distributed_enum_type SET SCHEMA type_tests;
ALTER TYPE distributed_enum_type ADD VALUE 'b' BEFORE 'c';
ALTER TYPE distributed_enum_type ADD VALUE 'd' AFTER 'c';
ALTER TYPE distributed_enum_type RENAME VALUE 'd' TO 'something-with-quotes''andstuff';
-- clear objects
SET client_min_messages TO fatal; -- 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
---------------------------------
(localhost,57637,t,"DROP ROLE")
(localhost,57638,t,"DROP ROLE")
(2 rows)

View File

@ -0,0 +1,338 @@
SET citus.next_shard_id TO 20010000;
CREATE USER typeuser;
NOTICE: not propagating CREATE ROLE/USER commands to worker nodes
HINT: Connect to worker nodes directly to manually create all necessary users and roles.
SELECT run_command_on_workers($$CREATE USER typeuser;$$);
run_command_on_workers
-----------------------------------
(localhost,57637,t,"CREATE ROLE")
(localhost,57638,t,"CREATE ROLE")
(2 rows)
CREATE SCHEMA type_tests AUTHORIZATION typeuser;
CREATE SCHEMA type_tests2 AUTHORIZATION typeuser; -- to test creation in a specific schema and moving to schema
SET search_path TO type_tests;
SET citus.shard_count TO 4;
-- single statement transactions with a simple type used in a table
CREATE TYPE tc1 AS (a int, b int);
CREATE TABLE t1 (a int PRIMARY KEY, b tc1);
SELECT create_distributed_table('t1','a');
create_distributed_table
--------------------------
(1 row)
INSERT INTO t1 VALUES (1, (2,3)::tc1);
SELECT * FROM t1;
a | b
---+-------
1 | (2,3)
(1 row)
ALTER TYPE tc1 RENAME TO tc1_newname;
INSERT INTO t1 VALUES (3, (4,5)::tc1_newname); -- insert with a cast would fail if the rename didn't propagate
ALTER TYPE tc1_newname SET SCHEMA type_tests2;
INSERT INTO t1 VALUES (6, (7,8)::type_tests2.tc1_newname); -- insert with a cast would fail if the rename didn't propagate
-- single statement transactions with a an enum used in a table
CREATE TYPE te1 AS ENUM ('one', 'two', 'three');
CREATE TABLE t2 (a int PRIMARY KEY, b te1);
SELECT create_distributed_table('t2','a');
create_distributed_table
--------------------------
(1 row)
INSERT INTO t2 VALUES (1, 'two');
SELECT * FROM t2;
a | b
---+-----
1 | two
(1 row)
-- rename enum, subsequent operations on the type would fail if the rename was not propagated
ALTER TYPE te1 RENAME TO te1_newname;
-- add an extra value to the enum and use in table
ALTER TYPE te1_newname ADD VALUE 'four';
UPDATE t2 SET b = 'four';
SELECT * FROM t2;
a | b
---+------
1 | four
(1 row)
-- change the schema of the type and use the new fully qualified name in an insert
ALTER TYPE te1_newname SET SCHEMA type_tests2;
INSERT INTO t2 VALUES (3, 'three'::type_tests2.te1_newname);
-- transaction block with simple type
BEGIN;
CREATE TYPE tc2 AS (a int, b int);
CREATE TABLE t3 (a int PRIMARY KEY, b tc2);
SELECT create_distributed_table('t3','a');
create_distributed_table
--------------------------
(1 row)
INSERT INTO t3 VALUES (4, (5,6)::tc2);
SELECT * FROM t3;
a | b
---+-------
4 | (5,6)
(1 row)
COMMIT;
-- transaction block with simple type
BEGIN;
CREATE TYPE te2 AS ENUM ('yes', 'no');
CREATE TABLE t4 (a int PRIMARY KEY, b te2);
SELECT create_distributed_table('t4','a');
create_distributed_table
--------------------------
(1 row)
INSERT INTO t4 VALUES (1, 'yes');
SELECT * FROM t4;
a | b
---+-----
1 | yes
(1 row)
-- ALTER TYPE ... ADD VALUE does not work in transactions
COMMIT;
-- test some combination of types without ddl propagation, this will prevent the workers
-- from having those types created. They are created just-in-time on table distribution
SET citus.enable_ddl_propagation TO off;
CREATE TYPE tc3 AS (a int, b int);
CREATE TYPE tc4 AS (a int, b tc3[]);
CREATE TYPE tc5 AS (a int, b tc4);
CREATE TYPE te3 AS ENUM ('a','b');
RESET citus.enable_ddl_propagation;
CREATE TABLE t5 (a int PRIMARY KEY, b tc5[], c te3);
SELECT create_distributed_table('t5','a');
create_distributed_table
--------------------------
(1 row)
-- test adding an attribute to a type and a column to a table both for a non-distributed type
SET citus.enable_ddl_propagation TO off;
CREATE TYPE te4 AS ENUM ('c','d');
CREATE TYPE tc6 AS (a int, b int);
CREATE TYPE tc6c AS (a int, b int);
RESET citus.enable_ddl_propagation;
-- types need to be fully qualified because of the search_path which is not supported by ALTER TYPE ... ADD COLUMN
ALTER TABLE t5 ADD COLUMN d type_tests.te4;
ALTER TABLE t5 ADD COLUMN e type_tests.tc6;
ALTER TYPE tc6 ADD ATTRIBUTE c tc6c;
-- last two values are only there if above commands succeeded
INSERT INTO t5 VALUES (1, NULL, 'a', 'd', (1,2,(4,5)::tc6c)::tc6);
-- test renaming an attribute of a distrbuted type and read it by its new name to verify propagation
ALTER TYPE tc6 RENAME ATTRIBUTE b TO d;
SELECT (e::tc6).d FROM t5 ORDER BY 1;
d
---
2
(1 row)
-- change owner of supported types and check ownership on remote server
ALTER TYPE te4 OWNER TO typeuser;
SELECT typname, usename FROM pg_type, pg_user where typname = 'te4' and typowner = usesysid;
typname | usename
---------+----------
te4 | typeuser
(1 row)
SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'te4' and typowner = usesysid;$$);
run_command_on_workers
--------------------------------------
(localhost,57637,t,"(te4,typeuser)")
(localhost,57638,t,"(te4,typeuser)")
(2 rows)
ALTER TYPE tc6 OWNER TO typeuser;
SELECT typname, usename FROM pg_type, pg_user where typname = 'tc6' and typowner = usesysid;
typname | usename
---------+----------
tc6 | typeuser
(1 row)
SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'tc6' and typowner = usesysid;$$);
run_command_on_workers
--------------------------------------
(localhost,57637,t,"(tc6,typeuser)")
(localhost,57638,t,"(tc6,typeuser)")
(2 rows)
-- create a type as a different user
SET ROLE typeuser;
-- create directly on the worker
CREATE TYPE tc7 AS (a int, b int);
CREATE TYPE te5 AS ENUM ('a','b','c');
-- cascade to the worker when table gets created
SET citus.enable_ddl_propagation TO off;
CREATE TYPE tc8 AS (a int, b int);
CREATE TYPE te6 AS ENUM ('a','b','c');
RESET citus.enable_ddl_propagation;
CREATE TABLE t6 (a int, b tc8, c te6);
SELECT create_distributed_table('t6', 'a');
create_distributed_table
--------------------------
(1 row)
RESET ROLE;
-- test ownership of all types
SELECT typname, usename FROM pg_type, pg_user where typname = 'tc7' and typowner = usesysid;
typname | usename
---------+----------
tc7 | typeuser
(1 row)
SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'tc7' and typowner = usesysid;$$);
run_command_on_workers
--------------------------------------
(localhost,57637,t,"(tc7,typeuser)")
(localhost,57638,t,"(tc7,typeuser)")
(2 rows)
SELECT typname, usename FROM pg_type, pg_user where typname = 'te5' and typowner = usesysid;
typname | usename
---------+----------
te5 | typeuser
(1 row)
SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'te5' and typowner = usesysid;$$);
run_command_on_workers
--------------------------------------
(localhost,57637,t,"(te5,typeuser)")
(localhost,57638,t,"(te5,typeuser)")
(2 rows)
SELECT typname, usename FROM pg_type, pg_user where typname = 'tc8' and typowner = usesysid;
typname | usename
---------+----------
tc8 | typeuser
(1 row)
SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'tc8' and typowner = usesysid;$$);
run_command_on_workers
--------------------------------------
(localhost,57637,t,"(tc8,typeuser)")
(localhost,57638,t,"(tc8,typeuser)")
(2 rows)
SELECT typname, usename FROM pg_type, pg_user where typname = 'te6' and typowner = usesysid;
typname | usename
---------+----------
te6 | typeuser
(1 row)
SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'te6' and typowner = usesysid;$$);
run_command_on_workers
--------------------------------------
(localhost,57637,t,"(te6,typeuser)")
(localhost,57638,t,"(te6,typeuser)")
(2 rows)
-- deleting the enum cascade will remove the type from the table and the workers
DROP TYPE te3 CASCADE;
NOTICE: drop cascades to table t5 column c
-- DELETE multiple types at once
DROP TYPE tc3, tc4, tc5 CASCADE;
NOTICE: drop cascades to table t5 column b
-- test if the types are deleted
SELECT typname FROM pg_type, pg_user where typname IN ('te3','tc3','tc4','tc5') and typowner = usesysid ORDER BY typname;
typname
---------
(0 rows)
SELECT run_command_on_workers($$SELECT typname FROM pg_type, pg_user where typname IN ('te3','tc3','tc4','tc5') and typowner = usesysid ORDER BY typname;$$);
run_command_on_workers
------------------------
(localhost,57637,t,"")
(localhost,57638,t,"")
(2 rows)
-- make sure attribute names are quoted correctly, no errors indicates types are propagated correctly
CREATE TYPE tc9 AS ("field-with-dashes" text COLLATE "en_US");
ALTER TYPE tc9 ADD ATTRIBUTE "some-more" int, ADD ATTRIBUTE normal int;
ALTER TYPE tc9 RENAME ATTRIBUTE normal TO "not-so-normal";
-- test alter statements for non-distributed types, if they would be propagated they would
-- error, preventing from changing them
SET citus.enable_ddl_propagation TO off;
CREATE TYPE non_distributed_composite_type AS (a int, b int);
CREATE TYPE non_distributed_enum_type AS ENUM ('a', 'c');
SET citus.enable_ddl_propagation TO on;
ALTER TYPE non_distributed_composite_type ADD ATTRIBUTE c int;
ALTER TYPE non_distributed_composite_type RENAME ATTRIBUTE c TO d;
ALTER TYPE non_distributed_composite_type ALTER ATTRIBUTE d SET DATA TYPE text COLLATE "en_US" CASCADE;
ALTER TYPE non_distributed_composite_type DROP ATTRIBUTE d;
ALTER TYPE non_distributed_composite_type OWNER TO typeuser;
ALTER TYPE non_distributed_composite_type RENAME TO non_distributed_composite_type_renamed;
ALTER TYPE non_distributed_composite_type_renamed RENAME TO non_distributed_composite_type;
ALTER TYPE non_distributed_composite_type SET SCHEMA type_tests2;
ALTER TYPE type_tests2.non_distributed_composite_type SET SCHEMA type_tests;
ALTER TYPE non_distributed_enum_type OWNER TO typeuser;
ALTER TYPE non_distributed_enum_type RENAME TO non_distributed_enum_type_renamed;
ALTER TYPE non_distributed_enum_type_renamed RENAME TO non_distributed_enum_type;
ALTER TYPE non_distributed_enum_type SET SCHEMA type_tests2;
ALTER TYPE type_tests2.non_distributed_enum_type SET SCHEMA type_tests;
ALTER TYPE non_distributed_enum_type ADD VALUE 'b' BEFORE 'c';
ALTER TYPE non_distributed_enum_type ADD VALUE 'd' AFTER 'c';
ALTER TYPE non_distributed_enum_type RENAME VALUE 'd' TO 'something-with-quotes''andstuff';
-- test all forms of alter statements on distributed types
CREATE TYPE distributed_composite_type AS (a int, b int);
CREATE TYPE distributed_enum_type AS ENUM ('a', 'c');
-- enforce distribution of types in every case
CREATE TABLE type_proc (a int, b distributed_composite_type, c distributed_enum_type);
SELECT create_distributed_table('type_proc','a');
create_distributed_table
--------------------------
(1 row)
DROP TABLE type_proc;
ALTER TYPE distributed_composite_type ADD ATTRIBUTE c int;
ALTER TYPE distributed_composite_type RENAME ATTRIBUTE c TO d;
ALTER TYPE distributed_composite_type ALTER ATTRIBUTE d SET DATA TYPE text COLLATE "en_US" CASCADE;
ALTER TYPE distributed_composite_type DROP ATTRIBUTE d;
ALTER TYPE distributed_composite_type OWNER TO typeuser;
ALTER TYPE distributed_composite_type RENAME TO distributed_composite_type_renamed;
ALTER TYPE distributed_composite_type_renamed RENAME TO distributed_composite_type;
ALTER TYPE distributed_composite_type SET SCHEMA type_tests2;
ALTER TYPE type_tests2.distributed_composite_type SET SCHEMA type_tests;
ALTER TYPE distributed_enum_type OWNER TO typeuser;
ALTER TYPE distributed_enum_type RENAME TO distributed_enum_type_renamed;
ALTER TYPE distributed_enum_type_renamed RENAME TO distributed_enum_type;
ALTER TYPE distributed_enum_type SET SCHEMA type_tests2;
ALTER TYPE type_tests2.distributed_enum_type SET SCHEMA type_tests;
ALTER TYPE distributed_enum_type ADD VALUE 'b' BEFORE 'c';
ALTER TYPE distributed_enum_type ADD VALUE 'd' AFTER 'c';
ALTER TYPE distributed_enum_type RENAME VALUE 'd' TO 'something-with-quotes''andstuff';
-- clear objects
SET client_min_messages TO fatal; -- 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
---------------------------------
(localhost,57637,t,"DROP ROLE")
(localhost,57638,t,"DROP ROLE")
(2 rows)

View File

@ -97,7 +97,7 @@ SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata WHERE schema_name = 'create_distributed_table_non_empty_failure'$$); SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata WHERE schema_name = 'create_distributed_table_non_empty_failure'$$);
run_command_on_workers run_command_on_workers
------------------------ ------------------------
(localhost,9060,t,1) (localhost,9060,t,0)
(localhost,57637,t,1) (localhost,57637,t,1)
(2 rows) (2 rows)
@ -108,6 +108,15 @@ SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS create_distributed_table_n
(localhost,57637,t,"DROP SCHEMA") (localhost,57637,t,"DROP SCHEMA")
(2 rows) (2 rows)
-- this triggers a schema creation which prevents further transactions around dependency propagation
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
CREATE TYPE schema_proc AS (a int);
DROP TYPE schema_proc;
-- kill as soon as the coordinator sends begin -- kill as soon as the coordinator sends begin
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()');
mitmproxy mitmproxy
@ -639,6 +648,15 @@ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables W
(localhost,57637,t,0) (localhost,57637,t,0)
(2 rows) (2 rows)
-- this triggers a schema creation which prevents further transactions around dependency propagation
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
CREATE TYPE schema_proc AS (a int);
DROP TYPE schema_proc;
-- kill as soon as the coordinator sends begin -- kill as soon as the coordinator sends begin
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()');
mitmproxy mitmproxy

View File

@ -97,7 +97,7 @@ SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata WHERE schema_name = 'create_distributed_table_non_empty_failure'$$); SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata WHERE schema_name = 'create_distributed_table_non_empty_failure'$$);
run_command_on_workers run_command_on_workers
------------------------ ------------------------
(localhost,9060,t,1) (localhost,9060,t,0)
(localhost,57637,t,1) (localhost,57637,t,1)
(2 rows) (2 rows)
@ -108,6 +108,15 @@ SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS create_distributed_table_n
(localhost,57637,t,"DROP SCHEMA") (localhost,57637,t,"DROP SCHEMA")
(2 rows) (2 rows)
-- this triggers a schema creation which prevents further transactions around dependency propagation
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
CREATE TYPE schema_proc AS (a int);
DROP TYPE schema_proc;
-- kill as soon as the coordinator sends begin -- kill as soon as the coordinator sends begin
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()');
mitmproxy mitmproxy
@ -635,6 +644,15 @@ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables W
(localhost,57637,t,0) (localhost,57637,t,0)
(2 rows) (2 rows)
-- this triggers a schema creation which prevents further transactions around dependency propagation
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
CREATE TYPE schema_proc AS (a int);
DROP TYPE schema_proc;
-- kill as soon as the coordinator sends begin -- kill as soon as the coordinator sends begin
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()');
mitmproxy mitmproxy

View File

@ -10,6 +10,10 @@ SELECT citus.mitmproxy('conn.allow()');
(1 row) (1 row)
-- this is merely used to get the schema creation propagated. Without there are failures
-- not related to reference tables but schema creation due to dependency creation on workers
CREATE TYPE schema_proc AS (a int);
DROP TYPE schema_proc;
CREATE TABLE ref_table(id int); CREATE TABLE ref_table(id int);
INSERT INTO ref_table VALUES(1),(2),(3); INSERT INTO ref_table VALUES(1),(2),(3);
-- Kill on sending first query to worker node, should error -- Kill on sending first query to worker node, should error
@ -21,10 +25,10 @@ SELECT citus.mitmproxy('conn.onQuery().kill()');
(1 row) (1 row)
SELECT create_reference_table('ref_table'); SELECT create_reference_table('ref_table');
ERROR: server closed the connection unexpectedly WARNING: connection not open
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:9060 CONTEXT: while executing command on localhost:9060
ERROR: connection error: localhost:9060
DETAIL: connection not open
SELECT count(*) FROM pg_dist_shard_placement; SELECT count(*) FROM pg_dist_shard_placement;
count count
------- -------
@ -106,7 +110,7 @@ SELECT citus.mitmproxy('conn.onCommandComplete(command="COPY 3").kill()');
SELECT create_reference_table('ref_table'); SELECT create_reference_table('ref_table');
NOTICE: Copying data from local table... NOTICE: Copying data from local table...
ERROR: failed to COPY to shard 10000004 on localhost:9060 ERROR: failed to COPY to shard 10000005 on localhost:9060
SELECT count(*) FROM pg_dist_shard_placement; SELECT count(*) FROM pg_dist_shard_placement;
count count
------- -------
@ -170,8 +174,8 @@ SELECT create_reference_table('ref_table');
SELECT shardid, nodeport, shardstate FROM pg_dist_shard_placement ORDER BY shardid, nodeport; SELECT shardid, nodeport, shardstate FROM pg_dist_shard_placement ORDER BY shardid, nodeport;
shardid | nodeport | shardstate shardid | nodeport | shardstate
----------+----------+------------ ----------+----------+------------
10000007 | 9060 | 1 10000008 | 9060 | 1
10000007 | 57637 | 1 10000008 | 57637 | 1
(2 rows) (2 rows)
SET client_min_messages TO NOTICE; SET client_min_messages TO NOTICE;
@ -195,10 +199,11 @@ SELECT citus.mitmproxy('conn.onQuery().kill()');
BEGIN; BEGIN;
SELECT create_reference_table('ref_table'); SELECT create_reference_table('ref_table');
ERROR: server closed the connection unexpectedly WARNING: server closed the connection unexpectedly
This probably means the server terminated abnormally This probably means the server terminated abnormally
before or while processing the request. before or while processing the request.
CONTEXT: while executing command on localhost:9060 CONTEXT: while executing command on localhost:9060
ERROR: failure on connection marked as essential: localhost:9060
COMMIT; COMMIT;
-- kill on ROLLBACK, should be rollbacked -- kill on ROLLBACK, should be rollbacked
SELECT citus.mitmproxy('conn.onQuery(query="^ROLLBACK").kill()'); SELECT citus.mitmproxy('conn.onQuery(query="^ROLLBACK").kill()');

View File

@ -10,6 +10,10 @@ SELECT citus.mitmproxy('conn.allow()');
(1 row) (1 row)
-- this is merely used to get the schema creation propagated. Without there are failures
-- not related to reference tables but schema creation due to dependency creation on workers
CREATE TYPE schema_proc AS (a int);
DROP TYPE schema_proc;
CREATE TABLE ref_table(id int); CREATE TABLE ref_table(id int);
INSERT INTO ref_table VALUES(1),(2),(3); INSERT INTO ref_table VALUES(1),(2),(3);
-- Kill on sending first query to worker node, should error -- Kill on sending first query to worker node, should error
@ -21,10 +25,10 @@ SELECT citus.mitmproxy('conn.onQuery().kill()');
(1 row) (1 row)
SELECT create_reference_table('ref_table'); SELECT create_reference_table('ref_table');
ERROR: server closed the connection unexpectedly ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally This probably means the server terminated abnormally
before or while processing the request. before or while processing the request.
CONTEXT: while executing command on localhost:9060
SELECT count(*) FROM pg_dist_shard_placement; SELECT count(*) FROM pg_dist_shard_placement;
count count
------- -------
@ -106,7 +110,7 @@ SELECT citus.mitmproxy('conn.onCommandComplete(command="COPY 3").kill()');
SELECT create_reference_table('ref_table'); SELECT create_reference_table('ref_table');
NOTICE: Copying data from local table... NOTICE: Copying data from local table...
ERROR: failed to COPY to shard 10000004 on localhost:9060 ERROR: failed to COPY to shard 10000005 on localhost:9060
SELECT count(*) FROM pg_dist_shard_placement; SELECT count(*) FROM pg_dist_shard_placement;
count count
------- -------
@ -170,8 +174,8 @@ SELECT create_reference_table('ref_table');
SELECT shardid, nodeport, shardstate FROM pg_dist_shard_placement ORDER BY shardid, nodeport; SELECT shardid, nodeport, shardstate FROM pg_dist_shard_placement ORDER BY shardid, nodeport;
shardid | nodeport | shardstate shardid | nodeport | shardstate
----------+----------+------------ ----------+----------+------------
10000007 | 9060 | 1 10000008 | 9060 | 1
10000007 | 57637 | 1 10000008 | 57637 | 1
(2 rows) (2 rows)
SET client_min_messages TO NOTICE; SET client_min_messages TO NOTICE;
@ -195,10 +199,11 @@ SELECT citus.mitmproxy('conn.onQuery().kill()');
BEGIN; BEGIN;
SELECT create_reference_table('ref_table'); SELECT create_reference_table('ref_table');
ERROR: server closed the connection unexpectedly WARNING: server closed the connection unexpectedly
This probably means the server terminated abnormally This probably means the server terminated abnormally
before or while processing the request. before or while processing the request.
CONTEXT: while executing command on localhost:9060 CONTEXT: while executing command on localhost:9060
ERROR: failure on connection marked as essential: localhost:9060
COMMIT; COMMIT;
-- kill on ROLLBACK, should be rollbacked -- kill on ROLLBACK, should be rollbacked
SELECT citus.mitmproxy('conn.onQuery(query="^ROLLBACK").kill()'); SELECT citus.mitmproxy('conn.onQuery(query="^ROLLBACK").kill()');

View File

@ -77,6 +77,10 @@ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata
(localhost,57637,t,1) (localhost,57637,t,1)
(2 rows) (2 rows)
-- this is merely used to get the schema creation propagated. Without there are failures
-- not related to reference tables but schema creation due to dependency creation on workers
CREATE TYPE schema_proc AS (a int);
DROP TYPE schema_proc;
-- Now, kill the connection while opening transaction on workers. -- Now, kill the connection while opening transaction on workers.
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()');
mitmproxy mitmproxy
@ -372,6 +376,10 @@ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables W
(localhost,57637,t,0) (localhost,57637,t,0)
(2 rows) (2 rows)
-- this is merely used to get the schema creation propagated. Without there are failures
-- not related to reference tables but schema creation due to dependency creation on workers
CREATE TYPE schema_proc AS (a int);
DROP TYPE schema_proc;
-- Now, kill the connection while creating transaction on workers in transaction. -- Now, kill the connection while creating transaction on workers in transaction.
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()');
mitmproxy mitmproxy
@ -522,6 +530,10 @@ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables W
(localhost,57637,t,0) (localhost,57637,t,0)
(2 rows) (2 rows)
-- this is merely used to get the schema creation propagated. Without there are failures
-- not related to reference tables but schema creation due to dependency creation on workers
CREATE TYPE schema_proc AS (a int);
DROP TYPE schema_proc;
-- Now, kill the connection while opening transactions on workers with 1pc. Transaction will be opened due to BEGIN. -- Now, kill the connection while opening transactions on workers with 1pc. Transaction will be opened due to BEGIN.
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()');
mitmproxy mitmproxy

View File

@ -77,6 +77,10 @@ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata
(localhost,57637,t,1) (localhost,57637,t,1)
(2 rows) (2 rows)
-- this is merely used to get the schema creation propagated. Without there are failures
-- not related to reference tables but schema creation due to dependency creation on workers
CREATE TYPE schema_proc AS (a int);
DROP TYPE schema_proc;
-- Now, kill the connection while opening transaction on workers. -- Now, kill the connection while opening transaction on workers.
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()');
mitmproxy mitmproxy
@ -367,6 +371,10 @@ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables W
(localhost,57637,t,0) (localhost,57637,t,0)
(2 rows) (2 rows)
-- this is merely used to get the schema creation propagated. Without there are failures
-- not related to reference tables but schema creation due to dependency creation on workers
CREATE TYPE schema_proc AS (a int);
DROP TYPE schema_proc;
-- Now, kill the connection while creating transaction on workers in transaction. -- Now, kill the connection while creating transaction on workers in transaction.
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()');
mitmproxy mitmproxy
@ -512,6 +520,10 @@ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables W
(localhost,57637,t,0) (localhost,57637,t,0)
(2 rows) (2 rows)
-- this is merely used to get the schema creation propagated. Without there are failures
-- not related to reference tables but schema creation due to dependency creation on workers
CREATE TYPE schema_proc AS (a int);
DROP TYPE schema_proc;
-- Now, kill the connection while opening transactions on workers with 1pc. Transaction will be opened due to BEGIN. -- Now, kill the connection while opening transactions on workers with 1pc. Transaction will be opened due to BEGIN.
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()');
mitmproxy mitmproxy

View File

@ -15,13 +15,6 @@ SET citus.shard_count TO 8;
SET citus.next_shard_id TO 7000000; SET citus.next_shard_id TO 7000000;
SET citus.next_placement_id TO 7000000; SET citus.next_placement_id TO 7000000;
CREATE TYPE foreign_details AS (name text, relid text, refd_relid text); CREATE TYPE foreign_details AS (name text, relid text, refd_relid text);
SELECT run_command_on_workers($$CREATE TYPE foreign_details AS (name text, relid text, refd_relid text)$$);
run_command_on_workers
-----------------------------------
(localhost,57637,t,"CREATE TYPE")
(localhost,57638,t,"CREATE TYPE")
(2 rows)
CREATE VIEW table_fkeys_in_workers AS CREATE VIEW table_fkeys_in_workers AS
SELECT SELECT
(json_populate_record(NULL::foreign_details, (json_populate_record(NULL::foreign_details,
@ -576,13 +569,6 @@ DROP TABLE referencing_table;
DROP TABLE referenced_table; DROP TABLE referenced_table;
-- foreign key as composite key -- foreign key as composite key
CREATE TYPE fkey_reference_table.composite AS (key1 int, key2 int); CREATE TYPE fkey_reference_table.composite AS (key1 int, key2 int);
SELECT run_command_on_workers($$CREATE TYPE fkey_reference_table.composite AS (key1 int, key2 int)$$) ORDER BY 1;
run_command_on_workers
-----------------------------------
(localhost,57637,t,"CREATE TYPE")
(localhost,57638,t,"CREATE TYPE")
(2 rows)
CREATE TABLE referenced_table(test_column composite, PRIMARY KEY(test_column)); CREATE TABLE referenced_table(test_column composite, PRIMARY KEY(test_column));
CREATE TABLE referencing_table(id int, referencing_composite composite); CREATE TABLE referencing_table(id int, referencing_composite composite);
SELECT create_reference_table('referenced_table'); SELECT create_reference_table('referenced_table');

View File

@ -126,13 +126,6 @@ ERROR: invalid input syntax for integer: "PGCOPY"
END; END;
-- try a composite type -- try a composite type
CREATE TYPE intermediate_results.square_type AS (x text, x2 int); CREATE TYPE intermediate_results.square_type AS (x text, x2 int);
SELECT run_command_on_workers('CREATE TYPE intermediate_results.square_type AS (x text, x2 int)');
run_command_on_workers
-----------------------------------
(localhost,57637,t,"CREATE TYPE")
(localhost,57638,t,"CREATE TYPE")
(2 rows)
CREATE TABLE stored_squares (user_id text, square intermediate_results.square_type, metadata jsonb); CREATE TABLE stored_squares (user_id text, square intermediate_results.square_type, metadata jsonb);
INSERT INTO stored_squares VALUES ('jon', '(2,4)'::intermediate_results.square_type, '{"value":2}'); INSERT INTO stored_squares VALUES ('jon', '(2,4)'::intermediate_results.square_type, '{"value":2}');
INSERT INTO stored_squares VALUES ('jon', '(3,9)'::intermediate_results.square_type, '{"value":3}'); INSERT INTO stored_squares VALUES ('jon', '(3,9)'::intermediate_results.square_type, '{"value":3}');

View File

@ -41,16 +41,16 @@ step s3-view-worker:
query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname
SELECT worker_apply_shard_ddl_command (102141, 'public', ' SELECT worker_apply_shard_ddl_command (102145, 'public', '
ALTER TABLE test_table ADD COLUMN x INT; ALTER TABLE test_table ADD COLUMN x INT;
')localhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression ')localhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression
SELECT worker_apply_shard_ddl_command (102140, 'public', ' SELECT worker_apply_shard_ddl_command (102144, 'public', '
ALTER TABLE test_table ADD COLUMN x INT; ALTER TABLE test_table ADD COLUMN x INT;
')localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression ')localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression
SELECT worker_apply_shard_ddl_command (102139, 'public', ' SELECT worker_apply_shard_ddl_command (102143, 'public', '
ALTER TABLE test_table ADD COLUMN x INT; ALTER TABLE test_table ADD COLUMN x INT;
')localhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression ')localhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression
SELECT worker_apply_shard_ddl_command (102138, 'public', ' SELECT worker_apply_shard_ddl_command (102142, 'public', '
ALTER TABLE test_table ADD COLUMN x INT; ALTER TABLE test_table ADD COLUMN x INT;
')localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression ')localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression
step s2-rollback: step s2-rollback:
@ -104,7 +104,7 @@ step s3-view-worker:
query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname
INSERT INTO public.test_table_102144 (column1, column2) VALUES (100, 100)localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression INSERT INTO public.test_table_102148 (column1, column2) VALUES (100, 100)localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression
step s2-rollback: step s2-rollback:
ROLLBACK; ROLLBACK;
@ -159,10 +159,10 @@ step s3-view-worker:
query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname
COPY (SELECT count(*) AS count FROM test_table_102149 test_table WHERE true) TO STDOUTlocalhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression COPY (SELECT count(*) AS count FROM test_table_102153 test_table WHERE true) TO STDOUTlocalhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression
COPY (SELECT count(*) AS count FROM test_table_102148 test_table WHERE true) TO STDOUTlocalhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression COPY (SELECT count(*) AS count FROM test_table_102152 test_table WHERE true) TO STDOUTlocalhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression
COPY (SELECT count(*) AS count FROM test_table_102147 test_table WHERE true) TO STDOUTlocalhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression COPY (SELECT count(*) AS count FROM test_table_102151 test_table WHERE true) TO STDOUTlocalhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression
COPY (SELECT count(*) AS count FROM test_table_102146 test_table WHERE true) TO STDOUTlocalhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression COPY (SELECT count(*) AS count FROM test_table_102150 test_table WHERE true) TO STDOUTlocalhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression
step s2-rollback: step s2-rollback:
ROLLBACK; ROLLBACK;
@ -217,7 +217,7 @@ step s3-view-worker:
query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname
SELECT count(*) AS count FROM public.test_table_102151 test_table WHERE (column1 OPERATOR(pg_catalog.=) 55)localhost 57638 0 idle Client ClientRead postgres regression SELECT count(*) AS count FROM public.test_table_102155 test_table WHERE (column1 OPERATOR(pg_catalog.=) 55)localhost 57638 0 idle Client ClientRead postgres regression
step s2-rollback: step s2-rollback:
ROLLBACK; ROLLBACK;

View File

@ -41,16 +41,16 @@ step s3-view-worker:
query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname
SELECT worker_apply_shard_ddl_command (102141, 'public', ' SELECT worker_apply_shard_ddl_command (102145, 'public', '
ALTER TABLE test_table ADD COLUMN x INT; ALTER TABLE test_table ADD COLUMN x INT;
')localhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression ')localhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression
SELECT worker_apply_shard_ddl_command (102140, 'public', ' SELECT worker_apply_shard_ddl_command (102144, 'public', '
ALTER TABLE test_table ADD COLUMN x INT; ALTER TABLE test_table ADD COLUMN x INT;
')localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression ')localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression
SELECT worker_apply_shard_ddl_command (102139, 'public', ' SELECT worker_apply_shard_ddl_command (102143, 'public', '
ALTER TABLE test_table ADD COLUMN x INT; ALTER TABLE test_table ADD COLUMN x INT;
')localhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression ')localhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression
SELECT worker_apply_shard_ddl_command (102138, 'public', ' SELECT worker_apply_shard_ddl_command (102142, 'public', '
ALTER TABLE test_table ADD COLUMN x INT; ALTER TABLE test_table ADD COLUMN x INT;
')localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression ')localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression
step s2-rollback: step s2-rollback:
@ -104,7 +104,7 @@ step s3-view-worker:
query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname
INSERT INTO public.test_table_102144 (column1, column2) VALUES (100, 100)localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression INSERT INTO public.test_table_102148 (column1, column2) VALUES (100, 100)localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression
step s2-rollback: step s2-rollback:
ROLLBACK; ROLLBACK;
@ -159,10 +159,10 @@ step s3-view-worker:
query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname
SELECT count(*) AS count FROM test_table_102149 test_table WHERE truelocalhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression SELECT count(*) AS count FROM test_table_102153 test_table WHERE truelocalhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression
SELECT count(*) AS count FROM test_table_102148 test_table WHERE truelocalhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression SELECT count(*) AS count FROM test_table_102152 test_table WHERE truelocalhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression
SELECT count(*) AS count FROM test_table_102147 test_table WHERE truelocalhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression SELECT count(*) AS count FROM test_table_102151 test_table WHERE truelocalhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression
SELECT count(*) AS count FROM test_table_102146 test_table WHERE truelocalhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression SELECT count(*) AS count FROM test_table_102150 test_table WHERE truelocalhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression
step s2-rollback: step s2-rollback:
ROLLBACK; ROLLBACK;
@ -217,7 +217,7 @@ step s3-view-worker:
query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname
SELECT count(*) AS count FROM public.test_table_102151 test_table WHERE (column1 OPERATOR(pg_catalog.=) 55)localhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression SELECT count(*) AS count FROM public.test_table_102155 test_table WHERE (column1 OPERATOR(pg_catalog.=) 55)localhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression
step s2-rollback: step s2-rollback:
ROLLBACK; ROLLBACK;

View File

@ -56,7 +56,7 @@ step s3-commit:
COMMIT; COMMIT;
starting permutation: s1-create-table s1-begin s1-insert s1-get-current-transaction-id s2-get-first-worker-active-transactions s1-commit starting permutation: s1-create-table s1-begin s1-insert s1-verify-current-xact-is-on-worker s1-commit
step s1-create-table: step s1-create-table:
-- some tests also use distributed table -- some tests also use distributed table
CREATE TABLE distributed_transaction_id_table(some_value int, other_value int); CREATE TABLE distributed_transaction_id_table(some_value int, other_value int);
@ -72,23 +72,22 @@ step s1-begin:
step s1-insert: step s1-insert:
INSERT INTO distributed_transaction_id_table VALUES (1, 1); INSERT INTO distributed_transaction_id_table VALUES (1, 1);
step s1-get-current-transaction-id: step s1-verify-current-xact-is-on-worker:
SELECT row(initiator_node_identifier, transaction_number) FROM get_current_transaction_id(); SELECT
remote.nodeport,
remote.result = row(xact.initiator_node_identifier, xact.transaction_number)::text AS xact_exists
FROM
get_current_transaction_id() as xact,
run_command_on_workers($$
SELECT row(initiator_node_identifier, transaction_number)
FROM get_all_active_transactions();
$$) as remote
ORDER BY remote.nodeport ASC;
row nodeport xact_exists
(0,229) 57637 t
step s2-get-first-worker-active-transactions: 57638 t
SELECT * FROM run_command_on_workers('SELECT row(initiator_node_identifier, transaction_number)
FROM
get_all_active_transactions();
')
WHERE nodeport = 57637;
;
nodename nodeport success result
localhost 57637 t (0,229)
step s1-commit: step s1-commit:
COMMIT; COMMIT;

View File

@ -29,11 +29,11 @@ step detector-dump-wait-edges:
waiting_transaction_numblocking_transaction_numblocking_transaction_waiting waiting_transaction_numblocking_transaction_numblocking_transaction_waiting
232 231 f 246 245 f
transactionnumberwaitingtransactionnumbers transactionnumberwaitingtransactionnumbers
231 245
232 231 246 245
step s1-abort: step s1-abort:
ABORT; ABORT;
@ -77,14 +77,14 @@ step detector-dump-wait-edges:
waiting_transaction_numblocking_transaction_numblocking_transaction_waiting waiting_transaction_numblocking_transaction_numblocking_transaction_waiting
236 235 f 250 249 f
237 235 f 251 249 f
237 236 t 251 250 t
transactionnumberwaitingtransactionnumbers transactionnumberwaitingtransactionnumbers
235 249
236 235 250 249
237 235,236 251 249,250
step s1-abort: step s1-abort:
ABORT; ABORT;

View File

@ -0,0 +1,98 @@
Parsed test spec with 4 sessions
starting permutation: s1-begin s2-begin s1-update s2-update detector-dump-wait-edges s1-abort s2-abort
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s1-update:
UPDATE distributed_table SET y = 1 WHERE x = 1;
step s2-update:
UPDATE distributed_table SET y = 2 WHERE x = 1;
<waiting ...>
step detector-dump-wait-edges:
SELECT
waiting_transaction_num,
blocking_transaction_num,
blocking_transaction_waiting
FROM
dump_global_wait_edges()
ORDER BY
waiting_transaction_num,
blocking_transaction_num,
blocking_transaction_waiting;
SELECT * FROM get_adjacency_list_wait_graph() ORDER BY 1;
waiting_transaction_numblocking_transaction_numblocking_transaction_waiting
247 246 f
transactionnumberwaitingtransactionnumbers
246
247 246
step s1-abort:
ABORT;
step s2-update: <... completed>
step s2-abort:
ABORT;
starting permutation: s1-begin s2-begin s3-begin s1-update s2-update s3-update detector-dump-wait-edges s1-abort s2-abort s3-abort
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s3-begin:
BEGIN;
step s1-update:
UPDATE distributed_table SET y = 1 WHERE x = 1;
step s2-update:
UPDATE distributed_table SET y = 2 WHERE x = 1;
<waiting ...>
step s3-update:
UPDATE distributed_table SET y = 3 WHERE x = 1;
<waiting ...>
step detector-dump-wait-edges:
SELECT
waiting_transaction_num,
blocking_transaction_num,
blocking_transaction_waiting
FROM
dump_global_wait_edges()
ORDER BY
waiting_transaction_num,
blocking_transaction_num,
blocking_transaction_waiting;
SELECT * FROM get_adjacency_list_wait_graph() ORDER BY 1;
waiting_transaction_numblocking_transaction_numblocking_transaction_waiting
251 250 f
252 250 f
252 251 t
transactionnumberwaitingtransactionnumbers
250
251 250
252 250,251
step s1-abort:
ABORT;
step s2-update: <... completed>
step s2-abort:
ABORT;
step s3-update: <... completed>
step s3-abort:
ABORT;

View File

@ -14,6 +14,10 @@ step s1-print-distributed-objects:
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
SELECT master_remove_node('localhost', 57638); SELECT master_remove_node('localhost', 57638);
?column? ?column?
@ -26,6 +30,13 @@ count
0 0
run_command_on_workers run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
count
0
run_command_on_workers
(localhost,57637,t,0) (localhost,57637,t,0)
(localhost,57638,t,0) (localhost,57638,t,0)
master_remove_node master_remove_node
@ -64,6 +75,10 @@ step s2-print-distributed-objects:
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
pg_identify_object_as_address pg_identify_object_as_address
count count
@ -71,6 +86,13 @@ count
0 0
run_command_on_workers run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
count
0
run_command_on_workers
(localhost,57637,t,0) (localhost,57637,t,0)
(localhost,57638,t,0) (localhost,57638,t,0)
master_remove_node master_remove_node
@ -92,6 +114,10 @@ step s1-print-distributed-objects:
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
SELECT master_remove_node('localhost', 57638); SELECT master_remove_node('localhost', 57638);
?column? ?column?
@ -104,6 +130,13 @@ count
0 0
run_command_on_workers run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
count
0
run_command_on_workers
(localhost,57637,t,0) (localhost,57637,t,0)
(localhost,57638,t,0) (localhost,57638,t,0)
master_remove_node master_remove_node
@ -148,6 +181,10 @@ step s2-print-distributed-objects:
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
pg_identify_object_as_address pg_identify_object_as_address
count count
@ -155,6 +192,13 @@ count
0 0
run_command_on_workers run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
count
0
run_command_on_workers
(localhost,57637,t,0) (localhost,57637,t,0)
(localhost,57638,t,0) (localhost,57638,t,0)
master_remove_node master_remove_node
@ -176,6 +220,10 @@ step s1-print-distributed-objects:
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
SELECT master_remove_node('localhost', 57638); SELECT master_remove_node('localhost', 57638);
?column? ?column?
@ -188,6 +236,13 @@ count
0 0
run_command_on_workers run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
count
0
run_command_on_workers
(localhost,57637,t,0) (localhost,57637,t,0)
(localhost,57638,t,0) (localhost,57638,t,0)
master_remove_node master_remove_node
@ -232,6 +287,10 @@ step s2-print-distributed-objects:
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
pg_identify_object_as_address pg_identify_object_as_address
count count
@ -239,6 +298,13 @@ count
0 0
run_command_on_workers run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
count
0
run_command_on_workers
(localhost,57637,t,0) (localhost,57637,t,0)
(localhost,57638,t,0) (localhost,57638,t,0)
master_remove_node master_remove_node
@ -260,6 +326,10 @@ step s1-print-distributed-objects:
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
SELECT master_remove_node('localhost', 57638); SELECT master_remove_node('localhost', 57638);
?column? ?column?
@ -272,6 +342,13 @@ count
0 0
run_command_on_workers run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
count
0
run_command_on_workers
(localhost,57637,t,0) (localhost,57637,t,0)
(localhost,57638,t,0) (localhost,57638,t,0)
master_remove_node master_remove_node
@ -311,6 +388,10 @@ step s2-print-distributed-objects:
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
pg_identify_object_as_address pg_identify_object_as_address
(schema,{myschema},{}) (schema,{myschema},{})
@ -321,6 +402,13 @@ run_command_on_workers
(localhost,57637,t,1) (localhost,57637,t,1)
(localhost,57638,t,1) (localhost,57638,t,1)
count
0
run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
master_remove_node master_remove_node
@ -340,6 +428,10 @@ step s1-print-distributed-objects:
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
SELECT master_remove_node('localhost', 57638); SELECT master_remove_node('localhost', 57638);
?column? ?column?
@ -352,6 +444,13 @@ count
0 0
run_command_on_workers run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
count
0
run_command_on_workers
(localhost,57637,t,0) (localhost,57637,t,0)
(localhost,57638,t,0) (localhost,57638,t,0)
master_remove_node master_remove_node
@ -397,6 +496,10 @@ step s2-print-distributed-objects:
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
pg_identify_object_as_address pg_identify_object_as_address
(schema,{myschema},{}) (schema,{myschema},{})
@ -407,6 +510,13 @@ run_command_on_workers
(localhost,57637,t,1) (localhost,57637,t,1)
(localhost,57638,t,1) (localhost,57638,t,1)
count
0
run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
master_remove_node master_remove_node
@ -426,6 +536,10 @@ step s1-print-distributed-objects:
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
SELECT master_remove_node('localhost', 57638); SELECT master_remove_node('localhost', 57638);
?column? ?column?
@ -438,6 +552,13 @@ count
0 0
run_command_on_workers run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
count
0
run_command_on_workers
(localhost,57637,t,0) (localhost,57637,t,0)
(localhost,57638,t,0) (localhost,57638,t,0)
master_remove_node master_remove_node
@ -483,6 +604,10 @@ step s2-print-distributed-objects:
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
pg_identify_object_as_address pg_identify_object_as_address
(schema,{myschema},{}) (schema,{myschema},{})
@ -493,6 +618,13 @@ run_command_on_workers
(localhost,57637,t,1) (localhost,57637,t,1)
(localhost,57638,t,1) (localhost,57638,t,1)
count
0
run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
master_remove_node master_remove_node
@ -512,6 +644,10 @@ step s1-print-distributed-objects:
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
SELECT master_remove_node('localhost', 57638); SELECT master_remove_node('localhost', 57638);
?column? ?column?
@ -524,6 +660,13 @@ count
0 0
run_command_on_workers run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
count
0
run_command_on_workers
(localhost,57637,t,0) (localhost,57637,t,0)
(localhost,57638,t,0) (localhost,57638,t,0)
master_remove_node master_remove_node
@ -588,6 +731,10 @@ step s2-print-distributed-objects:
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
pg_identify_object_as_address pg_identify_object_as_address
(schema,{myschema},{}) (schema,{myschema},{})
@ -598,6 +745,13 @@ run_command_on_workers
(localhost,57637,t,1) (localhost,57637,t,1)
(localhost,57638,t,1) (localhost,57638,t,1)
count
0
run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
master_remove_node master_remove_node
@ -617,6 +771,10 @@ step s1-print-distributed-objects:
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
SELECT master_remove_node('localhost', 57638); SELECT master_remove_node('localhost', 57638);
?column? ?column?
@ -629,6 +787,13 @@ count
0 0
run_command_on_workers run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
count
0
run_command_on_workers
(localhost,57637,t,0) (localhost,57637,t,0)
(localhost,57638,t,0) (localhost,57638,t,0)
master_remove_node master_remove_node
@ -712,6 +877,10 @@ step s2-print-distributed-objects:
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
pg_identify_object_as_address pg_identify_object_as_address
(schema,{myschema},{}) (schema,{myschema},{})
@ -722,6 +891,13 @@ run_command_on_workers
(localhost,57637,t,1) (localhost,57637,t,1)
(localhost,57638,t,1) (localhost,57638,t,1)
count
0
run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
master_remove_node master_remove_node
@ -741,6 +917,10 @@ step s1-print-distributed-objects:
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
SELECT master_remove_node('localhost', 57638); SELECT master_remove_node('localhost', 57638);
?column? ?column?
@ -753,6 +933,13 @@ count
0 0
run_command_on_workers run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
count
0
run_command_on_workers
(localhost,57637,t,0) (localhost,57637,t,0)
(localhost,57638,t,0) (localhost,57638,t,0)
master_remove_node master_remove_node
@ -810,6 +997,10 @@ step s2-print-distributed-objects:
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
pg_identify_object_as_address pg_identify_object_as_address
(schema,{myschema},{}) (schema,{myschema},{})
@ -820,6 +1011,13 @@ run_command_on_workers
(localhost,57637,t,1) (localhost,57637,t,1)
(localhost,57638,t,1) (localhost,57638,t,1)
count
0
run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
master_remove_node master_remove_node
@ -839,6 +1037,10 @@ step s1-print-distributed-objects:
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
SELECT master_remove_node('localhost', 57638); SELECT master_remove_node('localhost', 57638);
?column? ?column?
@ -851,6 +1053,13 @@ count
0 0
run_command_on_workers run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
count
0
run_command_on_workers
(localhost,57637,t,0) (localhost,57637,t,0)
(localhost,57638,t,0) (localhost,57638,t,0)
master_remove_node master_remove_node
@ -916,6 +1125,10 @@ step s2-print-distributed-objects:
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
pg_identify_object_as_address pg_identify_object_as_address
(schema,{myschema},{}) (schema,{myschema},{})
@ -925,6 +1138,314 @@ count
1 1
run_command_on_workers run_command_on_workers
(localhost,57637,t,1)
(localhost,57638,t,1)
count
0
run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
master_remove_node
starting permutation: s1-print-distributed-objects s1-begin s1-add-worker s2-public-schema s2-create-type s1-commit s2-print-distributed-objects
?column?
1
step s1-print-distributed-objects:
SELECT 1 FROM master_add_node('localhost', 57638);
-- print an overview of all distributed objects
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
-- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
SELECT master_remove_node('localhost', 57638);
?column?
1
pg_identify_object_as_address
count
0
run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
count
0
run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
master_remove_node
step s1-begin:
BEGIN;
step s1-add-worker:
SELECT 1 FROM master_add_node('localhost', 57638);
?column?
1
step s2-public-schema:
SET search_path TO public;
step s2-create-type:
CREATE TYPE tt1 AS (a int, b int);
<waiting ...>
step s1-commit:
COMMIT;
step s2-create-type: <... completed>
step s2-print-distributed-objects:
-- print an overview of all distributed objects
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
-- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
pg_identify_object_as_address
(type,{public.tt1},{})
count
0
run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
count
1
run_command_on_workers
(localhost,57637,t,1)
(localhost,57638,t,1)
master_remove_node
starting permutation: s1-print-distributed-objects s1-begin s2-public-schema s2-create-type s1-add-worker s1-commit s2-print-distributed-objects
?column?
1
step s1-print-distributed-objects:
SELECT 1 FROM master_add_node('localhost', 57638);
-- print an overview of all distributed objects
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
-- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
SELECT master_remove_node('localhost', 57638);
?column?
1
pg_identify_object_as_address
count
0
run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
count
0
run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
master_remove_node
step s1-begin:
BEGIN;
step s2-public-schema:
SET search_path TO public;
step s2-create-type:
CREATE TYPE tt1 AS (a int, b int);
step s1-add-worker:
SELECT 1 FROM master_add_node('localhost', 57638);
?column?
1
step s1-commit:
COMMIT;
step s2-print-distributed-objects:
-- print an overview of all distributed objects
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
-- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
pg_identify_object_as_address
(type,{public.tt1},{})
count
0
run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
count
1
run_command_on_workers
(localhost,57637,t,1)
(localhost,57638,t,1)
master_remove_node
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
?column?
1
step s1-print-distributed-objects:
SELECT 1 FROM master_add_node('localhost', 57638);
-- print an overview of all distributed objects
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
-- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
SELECT master_remove_node('localhost', 57638);
?column?
1
pg_identify_object_as_address
count
0
run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
count
0
run_command_on_workers
(localhost,57637,t,0)
(localhost,57638,t,0)
master_remove_node
step s1-begin:
BEGIN;
step s2-begin:
BEGIN;
step s2-create-schema:
CREATE SCHEMA myschema;
SET search_path TO myschema;
step s2-create-type:
CREATE TYPE tt1 AS (a int, b int);
step s2-create-table-with-type:
CREATE TABLE t1 (a int, b tt1);
-- session needs to have replication factor set to 1, can't do in setup
SET citus.shard_replication_factor TO 1;
SELECT create_distributed_table('t1', 'a');
create_distributed_table
step s1-add-worker:
SELECT 1 FROM master_add_node('localhost', 57638);
<waiting ...>
step s2-commit:
COMMIT;
step s1-add-worker: <... completed>
?column?
1
step s1-commit:
COMMIT;
step s2-print-distributed-objects:
-- print an overview of all distributed objects
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
-- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
pg_identify_object_as_address
(schema,{myschema},{})
(type,{myschema.tt1},{})
count
1
run_command_on_workers
(localhost,57637,t,1)
(localhost,57638,t,1)
count
1
run_command_on_workers
(localhost,57637,t,1) (localhost,57637,t,1)
(localhost,57638,t,1) (localhost,57638,t,1)
master_remove_node master_remove_node

View File

@ -16,7 +16,7 @@ step s1-commit:
COMMIT; COMMIT;
step s2-insert: <... completed> step s2-insert: <... completed>
error in steps s1-commit s2-insert: ERROR: duplicate key value violates unique constraint "test_locking_a_key_102413" error in steps s1-commit s2-insert: ERROR: duplicate key value violates unique constraint "test_locking_a_key_102417"
step s2-commit: step s2-commit:
COMMIT; COMMIT;

View File

@ -9,23 +9,44 @@ CREATE TYPE test_composite_type AS (
i2 integer i2 integer
); );
-- ... as well as a function to use as its comparator... -- ... as well as a function to use as its comparator...
CREATE FUNCTION equal_test_composite_type_function(test_composite_type, test_composite_type) RETURNS boolean SELECT run_command_on_coordinator_and_workers($cf$
LANGUAGE 'internal' CREATE FUNCTION equal_test_composite_type_function(test_composite_type, test_composite_type) RETURNS boolean
AS 'record_eq' LANGUAGE 'internal'
IMMUTABLE AS 'record_eq'
RETURNS NULL ON NULL INPUT; IMMUTABLE
CREATE FUNCTION cmp_test_composite_type_function(test_composite_type, test_composite_type) RETURNS int RETURNS NULL ON NULL INPUT;
LANGUAGE 'internal' $cf$);
AS 'btrecordcmp' run_command_on_coordinator_and_workers
IMMUTABLE ----------------------------------------
RETURNS NULL ON NULL INPUT;
(1 row)
SELECT run_command_on_coordinator_and_workers($cf$
CREATE FUNCTION cmp_test_composite_type_function(test_composite_type, test_composite_type) RETURNS int
LANGUAGE 'internal'
AS 'btrecordcmp'
IMMUTABLE
RETURNS NULL ON NULL INPUT;
$cf$);
run_command_on_coordinator_and_workers
----------------------------------------
(1 row)
-- ... use that function to create a custom equality operator... -- ... use that function to create a custom equality operator...
CREATE OPERATOR = ( SELECT run_command_on_coordinator_and_workers($co$
LEFTARG = test_composite_type, CREATE OPERATOR = (
RIGHTARG = test_composite_type, LEFTARG = test_composite_type,
PROCEDURE = equal_test_composite_type_function, RIGHTARG = test_composite_type,
HASHES PROCEDURE = equal_test_composite_type_function,
); HASHES
);
$co$);
run_command_on_coordinator_and_workers
----------------------------------------
(1 row)
-- ... and create a custom operator family for hash indexes... -- ... and create a custom operator family for hash indexes...
CREATE OPERATOR FAMILY cats_op_fam USING hash; CREATE OPERATOR FAMILY cats_op_fam USING hash;
-- ... create a test HASH function. Though it is a poor hash function, -- ... create a test HASH function. Though it is a poor hash function,

View File

@ -158,10 +158,12 @@ DEBUG: Plan 13 query after replacing subqueries and CTEs: SELECT f.a, f.b, tabl
(3 rows) (3 rows)
-- Custom Type returning function used in a join -- Custom Type returning function used in a join
RESET client_min_messages;
CREATE TYPE min_and_max AS ( CREATE TYPE min_and_max AS (
minimum INT, minimum INT,
maximum INT maximum INT
); );
SET client_min_messages TO DEBUG1;
CREATE OR REPLACE FUNCTION max_and_min () RETURNS CREATE OR REPLACE FUNCTION max_and_min () RETURNS
min_and_max AS $$ min_and_max AS $$
DECLARE DECLARE

View File

@ -60,7 +60,6 @@ CREATE TYPE order_side_mx AS ENUM ('buy', 'sell');
-- now create required stuff in the worker 1 -- now create required stuff in the worker 1
\c - - - :worker_1_port \c - - - :worker_1_port
-- create schema to test schema support -- create schema to test schema support
CREATE SCHEMA citus_mx_test_schema;
CREATE SCHEMA citus_mx_test_schema_join_1; CREATE SCHEMA citus_mx_test_schema_join_1;
CREATE SCHEMA citus_mx_test_schema_join_2; CREATE SCHEMA citus_mx_test_schema_join_2;
-- create UDFs in worker node -- create UDFs in worker node
@ -98,13 +97,9 @@ CREATE OPERATOR citus_mx_test_schema.=== (
); );
SET search_path TO public; SET search_path TO public;
CREATE COLLATION citus_mx_test_schema.english (LOCALE=:current_locale); CREATE COLLATION citus_mx_test_schema.english (LOCALE=:current_locale);
SET search_path TO public;
CREATE TYPE citus_mx_test_schema.new_composite_type as (key1 text, key2 text);
CREATE TYPE order_side_mx AS ENUM ('buy', 'sell');
-- now create required stuff in the worker 2 -- now create required stuff in the worker 2
\c - - - :worker_2_port \c - - - :worker_2_port
-- create schema to test schema support -- create schema to test schema support
CREATE SCHEMA citus_mx_test_schema;
CREATE SCHEMA citus_mx_test_schema_join_1; CREATE SCHEMA citus_mx_test_schema_join_1;
CREATE SCHEMA citus_mx_test_schema_join_2; CREATE SCHEMA citus_mx_test_schema_join_2;
-- create UDF -- create UDF
@ -142,9 +137,6 @@ CREATE OPERATOR citus_mx_test_schema.=== (
); );
SET search_path TO public; SET search_path TO public;
CREATE COLLATION citus_mx_test_schema.english (LOCALE=:current_locale); CREATE COLLATION citus_mx_test_schema.english (LOCALE=:current_locale);
SET search_path TO public;
CREATE TYPE citus_mx_test_schema.new_composite_type as (key1 text, key2 text);
CREATE TYPE order_side_mx AS ENUM ('buy', 'sell');
-- connect back to the master, and do some more tests -- connect back to the master, and do some more tests
\c - - - :master_port \c - - - :master_port
SET citus.shard_replication_factor TO 1; SET citus.shard_replication_factor TO 1;

View File

@ -51,10 +51,7 @@ CREATE TABLE repartition_udt_other (
-- proceed with type creation as above; thus the OIDs will be different. -- proceed with type creation as above; thus the OIDs will be different.
-- so that the OID is off. -- so that the OID is off.
\c - - - :worker_1_port \c - - - :worker_1_port
CREATE TYPE test_udt AS (i integer, i2 integer);
DROP TYPE test_udt CASCADE;
-- START type creation -- START type creation
CREATE TYPE test_udt AS (i integer, i2 integer);
-- ... as well as a function to use as its comparator... -- ... as well as a function to use as its comparator...
CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean
AS 'select $1.i = $2.i AND $1.i2 = $2.i2;' AS 'select $1.i = $2.i AND $1.i2 = $2.i2;'
@ -90,7 +87,6 @@ FUNCTION 1 test_udt_hash(test_udt);
-- END type creation -- END type creation
\c - - - :worker_2_port \c - - - :worker_2_port
-- START type creation -- START type creation
CREATE TYPE test_udt AS (i integer, i2 integer);
-- ... as well as a function to use as its comparator... -- ... as well as a function to use as its comparator...
CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean
AS 'select $1.i = $2.i AND $1.i2 = $2.i2;' AS 'select $1.i = $2.i AND $1.i2 = $2.i2;'

View File

@ -110,13 +110,13 @@ INSERT INTO pg_dist_shard_placement (shardid, shardstate, shardlength, nodename,
ORDER BY nodename, nodeport ASC ORDER BY nodename, nodeport ASC
LIMIT 1; LIMIT 1;
-- Create composite type partitioned table -- Create composite type partitioned table
RESET client_min_messages; -- avoid debug messages
CREATE TYPE composite_type AS CREATE TYPE composite_type AS
( (
text_column text, text_column text,
double_column decimal, double_column decimal,
varchar_column varchar(50) varchar_column varchar(50)
); );
RESET client_min_messages; -- avoid debug messages about toast index creation
CREATE TABLE composite_partitioned_table CREATE TABLE composite_partitioned_table
( (
composite_column composite_type composite_column composite_type

View File

@ -1254,7 +1254,6 @@ SELECT schemaname, indexrelname FROM pg_stat_all_indexes WHERE indexrelname = 'p
-- cleanup -- cleanup
DROP TABLE prepare_ddl; DROP TABLE prepare_ddl;
DROP SCHEMA otherschema;
RESET search_path; RESET search_path;
-- test prepared COPY -- test prepared COPY
CREATE OR REPLACE FUNCTION copy_in_plpgsql() CREATE OR REPLACE FUNCTION copy_in_plpgsql()
@ -1297,10 +1296,49 @@ SELECT local_copy_in_plpgsql();
(1 row) (1 row)
-- types statements should not crash nor leak schema specifications on to cached statements
CREATE TYPE prepare_ddl_type AS (x int, y int);
SET search_path TO 'otherschema', public;
CREATE OR REPLACE FUNCTION public.type_ddl_plpgsql()
RETURNS void
LANGUAGE plpgsql
AS $function$
DECLARE
BEGIN
ALTER TYPE prepare_ddl_type RENAME TO prepare_ddl_type_backup;
END;
$function$;
SELECT type_ddl_plpgsql();
type_ddl_plpgsql
------------------
(1 row)
-- create same type in new schema, owner of this new type should change
CREATE TYPE prepare_ddl_type AS (x int, y int);
SELECT type_ddl_plpgsql();
type_ddl_plpgsql
------------------
(1 row)
-- find all renamed types to verify the schema name didn't leak, nor a crash happened
SELECT nspname, typname FROM pg_type JOIN pg_namespace ON pg_namespace.oid = pg_type.typnamespace WHERE typname = 'prepare_ddl_type_backup';
nspname | typname
-------------+-------------------------
public | prepare_ddl_type_backup
otherschema | prepare_ddl_type_backup
(2 rows)
DROP TYPE prepare_ddl_type_backup;
RESET search_path;
DROP TYPE prepare_ddl_type_backup;
DROP FUNCTION type_ddl_plpgsql();
DROP FUNCTION ddl_in_plpgsql(); DROP FUNCTION ddl_in_plpgsql();
DROP FUNCTION copy_in_plpgsql(); DROP FUNCTION copy_in_plpgsql();
DROP TABLE prepare_ddl; DROP TABLE prepare_ddl;
DROP TABLE local_ddl; DROP TABLE local_ddl;
DROP SCHEMA otherschema;
-- clean-up functions -- clean-up functions
DROP FUNCTION plpgsql_test_1(); DROP FUNCTION plpgsql_test_1();
DROP FUNCTION plpgsql_test_2(); DROP FUNCTION plpgsql_test_2();

View File

@ -1522,11 +1522,6 @@ TRUNCATE reference_table_test;
-- function etc. -- function etc.
-- first create the type on all nodes -- first create the type on all nodes
CREATE TYPE reference_comp_key as (key text, value text); CREATE TYPE reference_comp_key as (key text, value text);
\c - - - :worker_1_port
CREATE TYPE reference_comp_key as (key text, value text);
\c - - - :worker_2_port
CREATE TYPE reference_comp_key as (key text, value text);
\c - - - :master_port
CREATE TABLE reference_table_composite (id int PRIMARY KEY, data reference_comp_key); CREATE TABLE reference_table_composite (id int PRIMARY KEY, data reference_comp_key);
SELECT create_reference_table('reference_table_composite'); SELECT create_reference_table('reference_table_composite');
create_reference_table create_reference_table

View File

@ -51,10 +51,7 @@ CREATE TABLE repartition_udt_other (
-- proceed with type creation as above; thus the OIDs will be different. -- proceed with type creation as above; thus the OIDs will be different.
-- so that the OID is off. -- so that the OID is off.
\c - - - :worker_1_port \c - - - :worker_1_port
CREATE TYPE test_udt AS (i integer, i2 integer);
DROP TYPE test_udt CASCADE;
-- START type creation -- START type creation
CREATE TYPE test_udt AS (i integer, i2 integer);
-- ... as well as a function to use as its comparator... -- ... as well as a function to use as its comparator...
CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean
AS 'select $1.i = $2.i AND $1.i2 = $2.i2;' AS 'select $1.i = $2.i AND $1.i2 = $2.i2;'
@ -90,7 +87,6 @@ FUNCTION 1 test_udt_hash(test_udt);
-- END type creation -- END type creation
\c - - - :worker_2_port \c - - - :worker_2_port
-- START type creation -- START type creation
CREATE TYPE test_udt AS (i integer, i2 integer);
-- ... as well as a function to use as its comparator... -- ... as well as a function to use as its comparator...
CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean
AS 'select $1.i = $2.i AND $1.i2 = $2.i2;' AS 'select $1.i = $2.i AND $1.i2 = $2.i2;'

View File

@ -394,7 +394,7 @@ ORDER BY logicalrelid;
logicalrelid | partmethod | colocationid | repmodel logicalrelid | partmethod | colocationid | repmodel
-----------------------------------------+------------+--------------+---------- -----------------------------------------+------------+--------------+----------
replicate_reference_table_reference_one | n | 10004 | t replicate_reference_table_reference_one | n | 10004 | t
replicate_reference_table_hash | h | 1360004 | c replicate_reference_table_hash | h | 1360005 | c
(2 rows) (2 rows)
BEGIN; BEGIN;
@ -481,7 +481,6 @@ SELECT create_reference_table('replicate_reference_table_insert');
BEGIN; BEGIN;
INSERT INTO replicate_reference_table_insert VALUES(1); INSERT INTO replicate_reference_table_insert VALUES(1);
SELECT 1 FROM master_add_node('localhost', :worker_2_port); SELECT 1 FROM master_add_node('localhost', :worker_2_port);
NOTICE: Replicating reference table "replicate_reference_table_insert" to the node localhost:57638
ERROR: cannot open new connections after the first modification command within a transaction ERROR: cannot open new connections after the first modification command within a transaction
ROLLBACK; ROLLBACK;
DROP TABLE replicate_reference_table_insert; DROP TABLE replicate_reference_table_insert;
@ -496,7 +495,6 @@ SELECT create_reference_table('replicate_reference_table_copy');
BEGIN; BEGIN;
COPY replicate_reference_table_copy FROM STDIN; COPY replicate_reference_table_copy FROM STDIN;
SELECT 1 FROM master_add_node('localhost', :worker_2_port); SELECT 1 FROM master_add_node('localhost', :worker_2_port);
NOTICE: Replicating reference table "replicate_reference_table_copy" to the node localhost:57638
ERROR: cannot open new connections after the first modification command within a transaction ERROR: cannot open new connections after the first modification command within a transaction
ROLLBACK; ROLLBACK;
DROP TABLE replicate_reference_table_copy; DROP TABLE replicate_reference_table_copy;
@ -511,7 +509,6 @@ SELECT create_reference_table('replicate_reference_table_ddl');
BEGIN; BEGIN;
ALTER TABLE replicate_reference_table_ddl ADD column2 int; ALTER TABLE replicate_reference_table_ddl ADD column2 int;
SELECT 1 FROM master_add_node('localhost', :worker_2_port); SELECT 1 FROM master_add_node('localhost', :worker_2_port);
NOTICE: Replicating reference table "replicate_reference_table_ddl" to the node localhost:57638
ERROR: cannot open new connections after the first modification command within a transaction ERROR: cannot open new connections after the first modification command within a transaction
ROLLBACK; ROLLBACK;
DROP TABLE replicate_reference_table_ddl; DROP TABLE replicate_reference_table_ddl;

View File

@ -536,13 +536,6 @@ SELECT n_comment FROM nation_hash_collation_search_path ORDER BY n_comment COLLA
--test composite types with schema --test composite types with schema
SET search_path TO public; SET search_path TO public;
CREATE TYPE test_schema_support.new_composite_type as (key1 text, key2 text); CREATE TYPE test_schema_support.new_composite_type as (key1 text, key2 text);
-- create type in worker node 1 in schema
\c - - - :worker_1_port
CREATE TYPE test_schema_support.new_composite_type as (key1 text, key2 text);
-- create type in worker node 2 in schema
\c - - - :worker_2_port
CREATE TYPE test_schema_support.new_composite_type as (key1 text, key2 text);
\c - - - :master_port
CREATE TABLE test_schema_support.nation_hash_composite_types( CREATE TABLE test_schema_support.nation_hash_composite_types(
n_nationkey integer not null, n_nationkey integer not null,
n_name char(25) not null, n_name char(25) not null,
@ -1002,13 +995,6 @@ SET search_path TO public;
ALTER TABLE test_schema_support.nation_hash SET SCHEMA public; ALTER TABLE test_schema_support.nation_hash SET SCHEMA public;
WARNING: not propagating ALTER ... SET SCHEMA commands to worker nodes WARNING: not propagating ALTER ... SET SCHEMA commands to worker nodes
HINT: Connect to worker nodes directly to manually change schemas of affected objects. HINT: Connect to worker nodes directly to manually change schemas of affected objects.
-- we will use this function in next test
CREATE FUNCTION run_command_on_coordinator_and_workers(p_sql text)
RETURNS void LANGUAGE plpgsql AS $$
BEGIN
EXECUTE p_sql;
PERFORM run_command_on_workers(p_sql);
END;$$;
-- test schema propagation with user other than current user -- test schema propagation with user other than current user
SELECT run_command_on_coordinator_and_workers('CREATE USER "test-user"'); SELECT run_command_on_coordinator_and_workers('CREATE USER "test-user"');
NOTICE: not propagating CREATE ROLE/USER commands to worker nodes NOTICE: not propagating CREATE ROLE/USER commands to worker nodes

View File

@ -1120,17 +1120,6 @@ LIMIT
-- CASCADE NOTICE messagez -- CASCADE NOTICE messagez
SET client_min_messages TO WARNING; SET client_min_messages TO WARNING;
DROP TABLE users, events; DROP TABLE users, events;
SELECT run_command_on_master_and_workers($f$ DROP TYPE user_composite_type CASCADE;
DROP TYPE user_composite_type CASCADE;
$f$);
run_command_on_master_and_workers
-----------------------------------
(1 row)
-- createed in multi_behavioral_analytics_create_table
DROP FUNCTION run_command_on_master_and_workers(p_sql text);
SET client_min_messages TO DEFAULT; SET client_min_messages TO DEFAULT;
SET citus.subquery_pushdown to OFF; SET citus.subquery_pushdown to OFF;

View File

@ -133,3 +133,10 @@ BEGIN
END LOOP; END LOOP;
RETURN; RETURN;
END; $$ language plpgsql; END; $$ language plpgsql;
-- helper function to quickly run SQL on the whole cluster
CREATE FUNCTION run_command_on_coordinator_and_workers(p_sql text)
RETURNS void LANGUAGE plpgsql AS $$
BEGIN
EXECUTE p_sql;
PERFORM run_command_on_workers(p_sql);
END;$$;

View File

@ -80,9 +80,6 @@ ERROR: could not find any healthy placement for shard 1360006
DROP TABLE upgrade_reference_table_unhealthy; DROP TABLE upgrade_reference_table_unhealthy;
-- test with table containing composite type -- test with table containing composite type
CREATE TYPE upgrade_test_composite_type AS (key1 text, key2 text); CREATE TYPE upgrade_test_composite_type AS (key1 text, key2 text);
\c - - - :worker_1_port
CREATE TYPE upgrade_test_composite_type AS (key1 text, key2 text);
\c - - - :master_port
SET citus.shard_count TO 1; SET citus.shard_count TO 1;
SET citus.shard_replication_factor TO 1; SET citus.shard_replication_factor TO 1;
CREATE TABLE upgrade_reference_table_composite(column1 int, column2 upgrade_test_composite_type); CREATE TABLE upgrade_reference_table_composite(column1 int, column2 upgrade_test_composite_type);
@ -95,8 +92,11 @@ SELECT create_distributed_table('upgrade_reference_table_composite', 'column1');
UPDATE pg_dist_partition SET repmodel='c' WHERE logicalrelid='upgrade_reference_table_composite'::regclass; UPDATE pg_dist_partition SET repmodel='c' WHERE logicalrelid='upgrade_reference_table_composite'::regclass;
SELECT upgrade_to_reference_table('upgrade_reference_table_composite'); SELECT upgrade_to_reference_table('upgrade_reference_table_composite');
NOTICE: Replicating reference table "upgrade_reference_table_composite" to the node localhost:57638 NOTICE: Replicating reference table "upgrade_reference_table_composite" to the node localhost:57638
ERROR: type "public.upgrade_test_composite_type" does not exist upgrade_to_reference_table
CONTEXT: while executing command on localhost:57638 ----------------------------
(1 row)
DROP TABLE upgrade_reference_table_composite; DROP TABLE upgrade_reference_table_composite;
-- test with reference table -- test with reference table
CREATE TABLE upgrade_reference_table_reference(column1 int); CREATE TABLE upgrade_reference_table_reference(column1 int);
@ -240,7 +240,7 @@ WHERE
logicalrelid = 'upgrade_reference_table_one_worker'::regclass; logicalrelid = 'upgrade_reference_table_one_worker'::regclass;
partmethod | partkeyisnull | colocationid | repmodel partmethod | partkeyisnull | colocationid | repmodel
------------+---------------+--------------+---------- ------------+---------------+--------------+----------
h | f | 1360000 | c h | f | 1360001 | c
(1 row) (1 row)
SELECT SELECT
@ -262,7 +262,7 @@ WHERE colocationid IN
WHERE logicalrelid = 'upgrade_reference_table_one_worker'::regclass); WHERE logicalrelid = 'upgrade_reference_table_one_worker'::regclass);
colocationid | shardcount | replicationfactor | distributioncolumntype colocationid | shardcount | replicationfactor | distributioncolumntype
--------------+------------+-------------------+------------------------ --------------+------------+-------------------+------------------------
1360000 | 1 | 1 | 23 1360001 | 1 | 1 | 23
(1 row) (1 row)
SELECT SELECT
@ -354,7 +354,7 @@ WHERE
logicalrelid = 'upgrade_reference_table_one_unhealthy'::regclass; logicalrelid = 'upgrade_reference_table_one_unhealthy'::regclass;
partmethod | partkeyisnull | colocationid | repmodel partmethod | partkeyisnull | colocationid | repmodel
------------+---------------+--------------+---------- ------------+---------------+--------------+----------
h | f | 1360001 | c h | f | 1360002 | c
(1 row) (1 row)
SELECT SELECT
@ -376,7 +376,7 @@ WHERE colocationid IN
WHERE logicalrelid = 'upgrade_reference_table_one_unhealthy'::regclass); WHERE logicalrelid = 'upgrade_reference_table_one_unhealthy'::regclass);
colocationid | shardcount | replicationfactor | distributioncolumntype colocationid | shardcount | replicationfactor | distributioncolumntype
--------------+------------+-------------------+------------------------ --------------+------------+-------------------+------------------------
1360001 | 1 | 2 | 23 1360002 | 1 | 2 | 23
(1 row) (1 row)
SELECT SELECT
@ -468,7 +468,7 @@ WHERE
logicalrelid = 'upgrade_reference_table_both_healthy'::regclass; logicalrelid = 'upgrade_reference_table_both_healthy'::regclass;
partmethod | partkeyisnull | colocationid | repmodel partmethod | partkeyisnull | colocationid | repmodel
------------+---------------+--------------+---------- ------------+---------------+--------------+----------
h | f | 1360002 | c h | f | 1360003 | c
(1 row) (1 row)
SELECT SELECT
@ -490,7 +490,7 @@ WHERE colocationid IN
WHERE logicalrelid = 'upgrade_reference_table_both_healthy'::regclass); WHERE logicalrelid = 'upgrade_reference_table_both_healthy'::regclass);
colocationid | shardcount | replicationfactor | distributioncolumntype colocationid | shardcount | replicationfactor | distributioncolumntype
--------------+------------+-------------------+------------------------ --------------+------------+-------------------+------------------------
1360002 | 1 | 2 | 23 1360003 | 1 | 2 | 23
(1 row) (1 row)
SELECT SELECT
@ -584,7 +584,7 @@ WHERE
logicalrelid = 'upgrade_reference_table_transaction_rollback'::regclass; logicalrelid = 'upgrade_reference_table_transaction_rollback'::regclass;
partmethod | partkeyisnull | colocationid | repmodel partmethod | partkeyisnull | colocationid | repmodel
------------+---------------+--------------+---------- ------------+---------------+--------------+----------
h | f | 1360003 | c h | f | 1360004 | c
(1 row) (1 row)
SELECT SELECT
@ -606,7 +606,7 @@ WHERE colocationid IN
WHERE logicalrelid = 'upgrade_reference_table_transaction_rollback'::regclass); WHERE logicalrelid = 'upgrade_reference_table_transaction_rollback'::regclass);
colocationid | shardcount | replicationfactor | distributioncolumntype colocationid | shardcount | replicationfactor | distributioncolumntype
--------------+------------+-------------------+------------------------ --------------+------------+-------------------+------------------------
1360003 | 1 | 1 | 23 1360004 | 1 | 1 | 23
(1 row) (1 row)
SELECT SELECT
@ -639,7 +639,7 @@ WHERE
logicalrelid = 'upgrade_reference_table_transaction_rollback'::regclass; logicalrelid = 'upgrade_reference_table_transaction_rollback'::regclass;
partmethod | partkeyisnull | colocationid | repmodel partmethod | partkeyisnull | colocationid | repmodel
------------+---------------+--------------+---------- ------------+---------------+--------------+----------
h | f | 1360003 | c h | f | 1360004 | c
(1 row) (1 row)
SELECT SELECT
@ -661,7 +661,7 @@ WHERE colocationid IN
WHERE logicalrelid = 'upgrade_reference_table_transaction_rollback'::regclass); WHERE logicalrelid = 'upgrade_reference_table_transaction_rollback'::regclass);
colocationid | shardcount | replicationfactor | distributioncolumntype colocationid | shardcount | replicationfactor | distributioncolumntype
--------------+------------+-------------------+------------------------ --------------+------------+-------------------+------------------------
1360003 | 1 | 1 | 23 1360004 | 1 | 1 | 23
(1 row) (1 row)
SELECT SELECT
@ -697,7 +697,7 @@ WHERE
logicalrelid = 'upgrade_reference_table_transaction_commit'::regclass; logicalrelid = 'upgrade_reference_table_transaction_commit'::regclass;
partmethod | partkeyisnull | colocationid | repmodel partmethod | partkeyisnull | colocationid | repmodel
------------+---------------+--------------+---------- ------------+---------------+--------------+----------
h | f | 1360003 | c h | f | 1360004 | c
(1 row) (1 row)
SELECT SELECT
@ -719,7 +719,7 @@ WHERE colocationid IN
WHERE logicalrelid = 'upgrade_reference_table_transaction_commit'::regclass); WHERE logicalrelid = 'upgrade_reference_table_transaction_commit'::regclass);
colocationid | shardcount | replicationfactor | distributioncolumntype colocationid | shardcount | replicationfactor | distributioncolumntype
--------------+------------+-------------------+------------------------ --------------+------------+-------------------+------------------------
1360003 | 1 | 1 | 23 1360004 | 1 | 1 | 23
(1 row) (1 row)
SELECT SELECT
@ -823,7 +823,7 @@ WHERE
logicalrelid = 'upgrade_reference_table_mx'::regclass; logicalrelid = 'upgrade_reference_table_mx'::regclass;
partmethod | partkeyisnull | colocationid | repmodel partmethod | partkeyisnull | colocationid | repmodel
------------+---------------+--------------+---------- ------------+---------------+--------------+----------
h | f | 1360004 | s h | f | 1360005 | s
(1 row) (1 row)
SELECT SELECT
@ -845,7 +845,7 @@ WHERE colocationid IN
WHERE logicalrelid = 'upgrade_reference_table_mx'::regclass); WHERE logicalrelid = 'upgrade_reference_table_mx'::regclass);
colocationid | shardcount | replicationfactor | distributioncolumntype colocationid | shardcount | replicationfactor | distributioncolumntype
--------------+------------+-------------------+------------------------ --------------+------------+-------------------+------------------------
1360004 | 1 | 1 | 23 1360005 | 1 | 1 | 23
(1 row) (1 row)
SELECT SELECT
@ -875,7 +875,7 @@ WHERE
logicalrelid = 'upgrade_reference_table_mx'::regclass; logicalrelid = 'upgrade_reference_table_mx'::regclass;
partmethod | partkeyisnull | colocationid | repmodel partmethod | partkeyisnull | colocationid | repmodel
------------+---------------+--------------+---------- ------------+---------------+--------------+----------
h | f | 1360004 | s h | f | 1360005 | s
(1 row) (1 row)
SELECT SELECT
@ -897,7 +897,7 @@ WHERE colocationid IN
WHERE logicalrelid = 'upgrade_reference_table_mx'::regclass); WHERE logicalrelid = 'upgrade_reference_table_mx'::regclass);
colocationid | shardcount | replicationfactor | distributioncolumntype colocationid | shardcount | replicationfactor | distributioncolumntype
--------------+------------+-------------------+------------------------ --------------+------------+-------------------+------------------------
1360004 | 1 | 1 | 23 1360005 | 1 | 1 | 23
(1 row) (1 row)
SELECT SELECT
@ -944,7 +944,7 @@ WHERE
logicalrelid = 'upgrade_reference_table_mx'::regclass; logicalrelid = 'upgrade_reference_table_mx'::regclass;
partmethod | partkeyisnull | colocationid | repmodel partmethod | partkeyisnull | colocationid | repmodel
------------+---------------+--------------+---------- ------------+---------------+--------------+----------
h | f | 1360005 | c h | f | 1360006 | c
(1 row) (1 row)
SELECT SELECT
@ -966,7 +966,7 @@ WHERE colocationid IN
WHERE logicalrelid = 'upgrade_reference_table_mx'::regclass); WHERE logicalrelid = 'upgrade_reference_table_mx'::regclass);
colocationid | shardcount | replicationfactor | distributioncolumntype colocationid | shardcount | replicationfactor | distributioncolumntype
--------------+------------+-------------------+------------------------ --------------+------------+-------------------+------------------------
1360005 | 1 | 2 | 23 1360006 | 1 | 2 | 23
(1 row) (1 row)
SELECT SELECT

View File

@ -7,13 +7,6 @@ SET citus.next_placement_id TO 7000000;
SET citus.replication_model TO streaming; SET citus.replication_model TO streaming;
-- Setup the view so that we can check if the foreign keys are created properly -- Setup the view so that we can check if the foreign keys are created properly
CREATE TYPE foreign_details AS (name text, relid text, refd_relid text); CREATE TYPE foreign_details AS (name text, relid text, refd_relid text);
SELECT run_command_on_workers($$CREATE TYPE foreign_details AS (name text, relid text, refd_relid text)$$);
run_command_on_workers
-----------------------------------
(localhost,57637,t,"CREATE TYPE")
(localhost,57638,t,"CREATE TYPE")
(2 rows)
CREATE VIEW table_fkeys_in_workers AS CREATE VIEW table_fkeys_in_workers AS
SELECT SELECT
(json_populate_record(NULL::foreign_details, (json_populate_record(NULL::foreign_details,

View File

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

View File

@ -28,14 +28,6 @@ SET citus.shard_count TO 8;
SET citus.next_shard_id TO 8000000; SET citus.next_shard_id TO 8000000;
SET citus.next_placement_id TO 8000000; SET citus.next_placement_id TO 8000000;
CREATE TYPE constraint_validity AS (name text, validated bool); CREATE TYPE constraint_validity AS (name text, validated bool);
SELECT run_command_on_workers(
$$CREATE TYPE constraint_validity AS (name text, validated bool)$$);
run_command_on_workers
-----------------------------------
(localhost,57637,t,"CREATE TYPE")
(localhost,57638,t,"CREATE TYPE")
(2 rows)
CREATE VIEW constraint_validations_in_workers AS CREATE VIEW constraint_validations_in_workers AS
SELECT (json_populate_record(NULL :: constraint_validity, SELECT (json_populate_record(NULL :: constraint_validity,
json_array_elements_text((run_command_on_workers($$ json_array_elements_text((run_command_on_workers($$

View File

@ -1,13 +1,6 @@
-- Test the basic CTE functionality and expected error messages -- Test the basic CTE functionality and expected error messages
SET search_path TO 'with_basics'; SET search_path TO 'with_basics';
CREATE TYPE xy AS (x int, y int); CREATE TYPE with_basics.xy AS (x int, y int);
SELECT run_command_on_workers('CREATE TYPE with_basics.xy AS (x int, y int)');
run_command_on_workers
-----------------------------------
(localhost,57637,t,"CREATE TYPE")
(localhost,57638,t,"CREATE TYPE")
(2 rows)
-- CTEs in FROM should work -- CTEs in FROM should work
WITH cte AS ( WITH cte AS (
SELECT user_id, value_2 from users_table WHERE user_id IN (1, 2) ORDER BY 1,2 LIMIT 5 SELECT user_id, value_2 from users_table WHERE user_id IN (1, 2) ORDER BY 1,2 LIMIT 5

View File

@ -81,14 +81,11 @@ CREATE INDEX is_index5 ON users_table(value_2);
CREATE INDEX is_index6 ON events_table(value_2); CREATE INDEX is_index6 ON events_table(value_2);
-- Create composite type to use in subquery pushdown -- Create composite type to use in subquery pushdown
SELECT run_command_on_master_and_workers($f$ CREATE TYPE user_composite_type AS
(
CREATE TYPE user_composite_type AS tenant_id BIGINT,
( user_id BIGINT
tenant_id BIGINT, );
user_id BIGINT
);
$f$);
SELECT run_command_on_master_and_workers($f$ SELECT run_command_on_master_and_workers($f$

View File

@ -464,11 +464,6 @@ CREATE TYPE test_item AS
id INTEGER, id INTEGER,
duration INTEGER duration INTEGER
); );
SELECT * FROM run_command_on_workers($$CREATE TYPE test_item AS
(
id INTEGER,
duration INTEGER
)$$) ORDER BY nodeport;
CREATE TABLE test_count_distinct_array (key int, value int , value_arr test_item[]); CREATE TABLE test_count_distinct_array (key int, value int , value_arr test_item[]);
SELECT create_distributed_table('test_count_distinct_array', 'key'); SELECT create_distributed_table('test_count_distinct_array', 'key');
@ -491,7 +486,6 @@ LIMIT 5;
DROP TABLE test_count_distinct_array; DROP TABLE test_count_distinct_array;
DROP TYPE test_item; DROP TYPE test_item;
SELECT * FROM run_command_on_workers($$DROP TYPE test_item$$) ORDER BY nodeport;
-- other distinct aggregate are not supported -- other distinct aggregate are not supported
SELECT * SELECT *

View File

@ -443,37 +443,6 @@ CREATE TYPE super_number_pack AS (
packed_number2 number_pack packed_number2 number_pack
); );
-- Create same types in worker1
\c - - - :worker_1_port
CREATE TYPE number_pack AS (
number1 integer,
number2 integer
);
CREATE TYPE super_number_pack AS (
packed_number1 number_pack,
packed_number2 number_pack
);
-- Create same types in worker2
\c - - - :worker_2_port
CREATE TYPE number_pack AS (
number1 integer,
number2 integer
);
CREATE TYPE super_number_pack AS (
packed_number1 number_pack,
packed_number2 number_pack
);
-- Connect back to master
\c - - - :master_port
-- Test array of user-defined type with hash distribution -- Test array of user-defined type with hash distribution
CREATE TABLE packed_numbers_hash ( CREATE TABLE packed_numbers_hash (
id integer, id integer,

View File

@ -276,3 +276,8 @@ test: multi_task_string_size
# connection encryption tests # connection encryption tests
# --------- # ---------
test: ssl_by_default test: ssl_by_default
# ---------
# object distribution tests
# ---------
test: distributed_types

View File

@ -121,19 +121,11 @@ CREATE INDEX is_index4 ON events_table(event_type);
CREATE INDEX is_index5 ON users_table(value_2); CREATE INDEX is_index5 ON users_table(value_2);
CREATE INDEX is_index6 ON events_table(value_2); CREATE INDEX is_index6 ON events_table(value_2);
-- Create composite type to use in subquery pushdown -- Create composite type to use in subquery pushdown
SELECT run_command_on_master_and_workers($f$ CREATE TYPE user_composite_type AS
(
CREATE TYPE user_composite_type AS tenant_id BIGINT,
( user_id BIGINT
tenant_id BIGINT, );
user_id BIGINT
);
$f$);
run_command_on_master_and_workers
-----------------------------------
(1 row)
SELECT run_command_on_master_and_workers($f$ SELECT run_command_on_master_and_workers($f$
CREATE FUNCTION cmp_user_composite_type_function(user_composite_type, user_composite_type) RETURNS int CREATE FUNCTION cmp_user_composite_type_function(user_composite_type, user_composite_type) RETURNS int

View File

@ -942,17 +942,6 @@ CREATE TYPE test_item AS
id INTEGER, id INTEGER,
duration INTEGER duration INTEGER
); );
SELECT * FROM run_command_on_workers($$CREATE TYPE test_item AS
(
id INTEGER,
duration INTEGER
)$$) ORDER BY nodeport;
nodename | nodeport | success | result
-----------+----------+---------+-------------
localhost | 57637 | t | CREATE TYPE
localhost | 57638 | t | CREATE TYPE
(2 rows)
CREATE TABLE test_count_distinct_array (key int, value int , value_arr test_item[]); CREATE TABLE test_count_distinct_array (key int, value int , value_arr test_item[]);
SELECT create_distributed_table('test_count_distinct_array', 'key'); SELECT create_distributed_table('test_count_distinct_array', 'key');
create_distributed_table create_distributed_table
@ -984,13 +973,6 @@ LIMIT 5;
DROP TABLE test_count_distinct_array; DROP TABLE test_count_distinct_array;
DROP TYPE test_item; DROP TYPE test_item;
SELECT * FROM run_command_on_workers($$DROP TYPE test_item$$) ORDER BY nodeport;
nodename | nodeport | success | result
-----------+----------+---------+-----------
localhost | 57637 | t | DROP TYPE
localhost | 57638 | t | DROP TYPE
(2 rows)
-- other distinct aggregate are not supported -- other distinct aggregate are not supported
SELECT * SELECT *
FROM ( FROM (

View File

@ -575,28 +575,6 @@ CREATE TYPE super_number_pack AS (
packed_number1 number_pack, packed_number1 number_pack,
packed_number2 number_pack packed_number2 number_pack
); );
-- Create same types in worker1
\c - - - :worker_1_port
CREATE TYPE number_pack AS (
number1 integer,
number2 integer
);
CREATE TYPE super_number_pack AS (
packed_number1 number_pack,
packed_number2 number_pack
);
-- Create same types in worker2
\c - - - :worker_2_port
CREATE TYPE number_pack AS (
number1 integer,
number2 integer
);
CREATE TYPE super_number_pack AS (
packed_number1 number_pack,
packed_number2 number_pack
);
-- Connect back to master
\c - - - :master_port
-- Test array of user-defined type with hash distribution -- Test array of user-defined type with hash distribution
CREATE TABLE packed_numbers_hash ( CREATE TABLE packed_numbers_hash (
id integer, id integer,

View File

@ -69,11 +69,9 @@ my $postgresSrcdir = "";
my $majorversion = ""; my $majorversion = "";
my @extensions = (); my @extensions = ();
my @userPgOptions = (); my @userPgOptions = ();
my %dataTypes = ();
my %fdws = (); my %fdws = ();
my %fdwServers = (); my %fdwServers = ();
my %functions = (); my %functions = ();
my %operators = ();
my $valgrind = 0; my $valgrind = 0;
my $valgrindPath = "valgrind"; my $valgrindPath = "valgrind";
my $valgrindLogFile = "valgrind_test_log.txt"; my $valgrindLogFile = "valgrind_test_log.txt";
@ -386,19 +384,8 @@ for my $option (@userPgOptions)
push(@pgOptions, '-c', $option); push(@pgOptions, '-c', $option);
} }
#define data types as a name->definition
%dataTypes = ('dummy_type', '(i integer)',
'order_side', ' ENUM (\'buy\', \'sell\')',
'test_composite_type', '(i integer, i2 integer)',
'bug_status', ' ENUM (\'new\', \'open\', \'closed\')');
# define functions as signature->definition # define functions as signature->definition
%functions = ('fake_fdw_handler()', 'fdw_handler AS \'citus\' LANGUAGE C STRICT;', %functions = ('fake_fdw_handler()', 'fdw_handler AS \'citus\' LANGUAGE C STRICT;');
'equal_test_composite_type_function(test_composite_type, test_composite_type)',
'boolean AS \'select $1.i = $2.i AND $1.i2 = $2.i2;\' LANGUAGE SQL IMMUTABLE RETURNS NULL ON NULL INPUT;');
%operators = ('=', '(LEFTARG = test_composite_type, RIGHTARG = test_composite_type, PROCEDURE = equal_test_composite_type_function, HASHES)');
#define fdws as name->handler name #define fdws as name->handler name
%fdws = ('fake_fdw', 'fake_fdw_handler'); %fdws = ('fake_fdw', 'fake_fdw_handler');
@ -750,14 +737,6 @@ for my $port (@workerPorts)
or die "Could not create extension on worker"; or die "Could not create extension on worker";
} }
foreach my $dataType (keys %dataTypes)
{
system(catfile($bindir, "psql"),
('-X', '-h', $host, '-p', $port, '-U', $user, "-d", "regression",
'-c', "CREATE TYPE $dataType AS $dataTypes{$dataType};")) == 0
or die "Could not create TYPE $dataType on worker";
}
foreach my $function (keys %functions) foreach my $function (keys %functions)
{ {
system(catfile($bindir, "psql"), system(catfile($bindir, "psql"),
@ -766,14 +745,6 @@ for my $port (@workerPorts)
or die "Could not create FUNCTION $function on worker"; or die "Could not create FUNCTION $function on worker";
} }
foreach my $operator (keys %operators)
{
system(catfile($bindir, "psql"),
('-X', '-h', $host, '-p', $port, '-U', $user, "-d", "regression",
'-c', "CREATE OPERATOR $operator $operators{$operator};")) == 0
or die "Could not create OPERATOR $operator on worker";
}
foreach my $fdw (keys %fdws) foreach my $fdw (keys %fdws)
{ {
system(catfile($bindir, "psql"), system(catfile($bindir, "psql"),

View File

@ -45,9 +45,18 @@ step "s1-insert"
INSERT INTO distributed_transaction_id_table VALUES (1, 1); INSERT INTO distributed_transaction_id_table VALUES (1, 1);
} }
step "s1-get-current-transaction-id" step "s1-verify-current-xact-is-on-worker"
{ {
SELECT row(initiator_node_identifier, transaction_number) FROM get_current_transaction_id(); SELECT
remote.nodeport,
remote.result = row(xact.initiator_node_identifier, xact.transaction_number)::text AS xact_exists
FROM
get_current_transaction_id() as xact,
run_command_on_workers($$
SELECT row(initiator_node_identifier, transaction_number)
FROM get_all_active_transactions();
$$) as remote
ORDER BY remote.nodeport ASC;
} }
step "s1-get-all-transactions" step "s1-get-all-transactions"
@ -121,7 +130,7 @@ permutation "s1-begin" "s1-assign-transaction-id" "s1-get-all-transactions" "s2-
# now show that distributed transaction id on the coordinator # now show that distributed transaction id on the coordinator
# is the same with the one on the worker # is the same with the one on the worker
permutation "s1-create-table" "s1-begin" "s1-insert" "s1-get-current-transaction-id" "s2-get-first-worker-active-transactions" "s1-commit" permutation "s1-create-table" "s1-begin" "s1-insert" "s1-verify-current-xact-is-on-worker" "s1-commit"
# we would initially forget the distributed transaction ID on pg_dist_partition invalidations # we would initially forget the distributed transaction ID on pg_dist_partition invalidations
permutation "s1-begin" "s1-assign-transaction-id" "s1-has-transaction-number" "s2-vacuum" "s1-has-transaction-number" "s1-commit" permutation "s1-begin" "s1-assign-transaction-id" "s1-has-transaction-number" "s2-vacuum" "s1-has-transaction-number" "s1-commit"

View File

@ -19,6 +19,7 @@ teardown
DROP TABLE IF EXISTS t1 CASCADE; DROP TABLE IF EXISTS t1 CASCADE;
DROP TABLE IF EXISTS t2 CASCADE; DROP TABLE IF EXISTS t2 CASCADE;
DROP TABLE IF EXISTS t3 CASCADE; DROP TABLE IF EXISTS t3 CASCADE;
DROP TYPE IF EXISTS tt1 CASCADE;
SELECT master_remove_node(nodename, nodeport) FROM pg_dist_node; SELECT master_remove_node(nodename, nodeport) FROM pg_dist_node;
} }
@ -53,6 +54,10 @@ step "s1-print-distributed-objects"
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
SELECT master_remove_node('localhost', 57638); SELECT master_remove_node('localhost', 57638);
} }
@ -77,6 +82,19 @@ step "s2-create-table"
SELECT create_distributed_table('t1', 'a'); SELECT create_distributed_table('t1', 'a');
} }
step "s2-create-type"
{
CREATE TYPE tt1 AS (a int, b int);
}
step "s2-create-table-with-type"
{
CREATE TABLE t1 (a int, b tt1);
-- session needs to have replication factor set to 1, can't do in setup
SET citus.shard_replication_factor TO 1;
SELECT create_distributed_table('t1', 'a');
}
step "s2-begin" step "s2-begin"
{ {
BEGIN; BEGIN;
@ -97,6 +115,10 @@ step "s2-print-distributed-objects"
-- print if the schema has been created -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
-- print if the type has been created
SELECT count(*) FROM pg_type where typname = 'tt1';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_type where typname = 'tt1';$$);
} }
session "s3" session "s3"
@ -178,3 +200,8 @@ permutation "s1-print-distributed-objects" "s2-create-schema" "s1-begin" "s2-beg
permutation "s1-print-distributed-objects" "s2-create-schema" "s1-begin" "s2-begin" "s3-begin" "s4-begin" "s1-add-worker" "s2-create-table" "s3-use-schema" "s3-create-table" "s4-use-schema" "s4-create-table" "s1-commit" "s2-commit" "s3-commit" "s4-commit" "s2-print-distributed-objects" permutation "s1-print-distributed-objects" "s2-create-schema" "s1-begin" "s2-begin" "s3-begin" "s4-begin" "s1-add-worker" "s2-create-table" "s3-use-schema" "s3-create-table" "s4-use-schema" "s4-create-table" "s1-commit" "s2-commit" "s3-commit" "s4-commit" "s2-print-distributed-objects"
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" 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"
permutation "s1-print-distributed-objects" "s1-begin" "s2-begin" "s4-begin" "s1-add-worker" "s2-create-schema" "s4-create-schema2" "s2-create-table" "s4-create-table" "s1-commit" "s2-commit" "s4-commit" "s2-print-distributed-objects" permutation "s1-print-distributed-objects" "s1-begin" "s2-begin" "s4-begin" "s1-add-worker" "s2-create-schema" "s4-create-schema2" "s2-create-table" "s4-create-table" "s1-commit" "s2-commit" "s4-commit" "s2-print-distributed-objects"
# 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"
permutation "s1-print-distributed-objects" "s1-begin" "s2-public-schema" "s2-create-type" "s1-add-worker" "s1-commit" "s2-print-distributed-objects"
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"

View File

@ -0,0 +1,222 @@
SET citus.next_shard_id TO 20010000;
CREATE USER typeuser;
SELECT run_command_on_workers($$CREATE USER typeuser;$$);
CREATE SCHEMA type_tests AUTHORIZATION typeuser;
CREATE SCHEMA type_tests2 AUTHORIZATION typeuser; -- to test creation in a specific schema and moving to schema
SET search_path TO type_tests;
SET citus.shard_count TO 4;
-- single statement transactions with a simple type used in a table
CREATE TYPE tc1 AS (a int, b int);
CREATE TABLE t1 (a int PRIMARY KEY, b tc1);
SELECT create_distributed_table('t1','a');
INSERT INTO t1 VALUES (1, (2,3)::tc1);
SELECT * FROM t1;
ALTER TYPE tc1 RENAME TO tc1_newname;
INSERT INTO t1 VALUES (3, (4,5)::tc1_newname); -- insert with a cast would fail if the rename didn't propagate
ALTER TYPE tc1_newname SET SCHEMA type_tests2;
INSERT INTO t1 VALUES (6, (7,8)::type_tests2.tc1_newname); -- insert with a cast would fail if the rename didn't propagate
-- single statement transactions with a an enum used in a table
CREATE TYPE te1 AS ENUM ('one', 'two', 'three');
CREATE TABLE t2 (a int PRIMARY KEY, b te1);
SELECT create_distributed_table('t2','a');
INSERT INTO t2 VALUES (1, 'two');
SELECT * FROM t2;
-- rename enum, subsequent operations on the type would fail if the rename was not propagated
ALTER TYPE te1 RENAME TO te1_newname;
-- add an extra value to the enum and use in table
ALTER TYPE te1_newname ADD VALUE 'four';
UPDATE t2 SET b = 'four';
SELECT * FROM t2;
-- change the schema of the type and use the new fully qualified name in an insert
ALTER TYPE te1_newname SET SCHEMA type_tests2;
INSERT INTO t2 VALUES (3, 'three'::type_tests2.te1_newname);
-- transaction block with simple type
BEGIN;
CREATE TYPE tc2 AS (a int, b int);
CREATE TABLE t3 (a int PRIMARY KEY, b tc2);
SELECT create_distributed_table('t3','a');
INSERT INTO t3 VALUES (4, (5,6)::tc2);
SELECT * FROM t3;
COMMIT;
-- transaction block with simple type
BEGIN;
CREATE TYPE te2 AS ENUM ('yes', 'no');
CREATE TABLE t4 (a int PRIMARY KEY, b te2);
SELECT create_distributed_table('t4','a');
INSERT INTO t4 VALUES (1, 'yes');
SELECT * FROM t4;
-- ALTER TYPE ... ADD VALUE does not work in transactions
COMMIT;
-- test some combination of types without ddl propagation, this will prevent the workers
-- from having those types created. They are created just-in-time on table distribution
SET citus.enable_ddl_propagation TO off;
CREATE TYPE tc3 AS (a int, b int);
CREATE TYPE tc4 AS (a int, b tc3[]);
CREATE TYPE tc5 AS (a int, b tc4);
CREATE TYPE te3 AS ENUM ('a','b');
RESET citus.enable_ddl_propagation;
CREATE TABLE t5 (a int PRIMARY KEY, b tc5[], c te3);
SELECT create_distributed_table('t5','a');
-- test adding an attribute to a type and a column to a table both for a non-distributed type
SET citus.enable_ddl_propagation TO off;
CREATE TYPE te4 AS ENUM ('c','d');
CREATE TYPE tc6 AS (a int, b int);
CREATE TYPE tc6c AS (a int, b int);
RESET citus.enable_ddl_propagation;
-- types need to be fully qualified because of the search_path which is not supported by ALTER TYPE ... ADD COLUMN
ALTER TABLE t5 ADD COLUMN d type_tests.te4;
ALTER TABLE t5 ADD COLUMN e type_tests.tc6;
ALTER TYPE tc6 ADD ATTRIBUTE c tc6c;
-- last two values are only there if above commands succeeded
INSERT INTO t5 VALUES (1, NULL, 'a', 'd', (1,2,(4,5)::tc6c)::tc6);
-- test renaming an attribute of a distrbuted type and read it by its new name to verify propagation
ALTER TYPE tc6 RENAME ATTRIBUTE b TO d;
SELECT (e::tc6).d FROM t5 ORDER BY 1;
-- change owner of supported types and check ownership on remote server
ALTER TYPE te4 OWNER TO typeuser;
SELECT typname, usename FROM pg_type, pg_user where typname = 'te4' and typowner = usesysid;
SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'te4' and typowner = usesysid;$$);
ALTER TYPE tc6 OWNER TO typeuser;
SELECT typname, usename FROM pg_type, pg_user where typname = 'tc6' and typowner = usesysid;
SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'tc6' and typowner = usesysid;$$);
-- create a type as a different user
SET ROLE typeuser;
-- create directly on the worker
CREATE TYPE tc7 AS (a int, b int);
CREATE TYPE te5 AS ENUM ('a','b','c');
-- cascade to the worker when table gets created
SET citus.enable_ddl_propagation TO off;
CREATE TYPE tc8 AS (a int, b int);
CREATE TYPE te6 AS ENUM ('a','b','c');
RESET citus.enable_ddl_propagation;
CREATE TABLE t6 (a int, b tc8, c te6);
SELECT create_distributed_table('t6', 'a');
RESET ROLE;
-- test ownership of all types
SELECT typname, usename FROM pg_type, pg_user where typname = 'tc7' and typowner = usesysid;
SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'tc7' and typowner = usesysid;$$);
SELECT typname, usename FROM pg_type, pg_user where typname = 'te5' and typowner = usesysid;
SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'te5' and typowner = usesysid;$$);
SELECT typname, usename FROM pg_type, pg_user where typname = 'tc8' and typowner = usesysid;
SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'tc8' and typowner = usesysid;$$);
SELECT typname, usename FROM pg_type, pg_user where typname = 'te6' and typowner = usesysid;
SELECT run_command_on_workers($$SELECT row(typname, usename) FROM pg_type, pg_user where typname = 'te6' and typowner = usesysid;$$);
-- deleting the enum cascade will remove the type from the table and the workers
DROP TYPE te3 CASCADE;
-- DELETE multiple types at once
DROP TYPE tc3, tc4, tc5 CASCADE;
-- test if the types are deleted
SELECT typname FROM pg_type, pg_user where typname IN ('te3','tc3','tc4','tc5') and typowner = usesysid ORDER BY typname;
SELECT run_command_on_workers($$SELECT typname FROM pg_type, pg_user where typname IN ('te3','tc3','tc4','tc5') and typowner = usesysid ORDER BY typname;$$);
-- make sure attribute names are quoted correctly, no errors indicates types are propagated correctly
CREATE TYPE tc9 AS ("field-with-dashes" text COLLATE "en_US");
ALTER TYPE tc9 ADD ATTRIBUTE "some-more" int, ADD ATTRIBUTE normal int;
ALTER TYPE tc9 RENAME ATTRIBUTE normal TO "not-so-normal";
-- test alter statements for non-distributed types, if they would be propagated they would
-- error, preventing from changing them
SET citus.enable_ddl_propagation TO off;
CREATE TYPE non_distributed_composite_type AS (a int, b int);
CREATE TYPE non_distributed_enum_type AS ENUM ('a', 'c');
SET citus.enable_ddl_propagation TO on;
ALTER TYPE non_distributed_composite_type ADD ATTRIBUTE c int;
ALTER TYPE non_distributed_composite_type RENAME ATTRIBUTE c TO d;
ALTER TYPE non_distributed_composite_type ALTER ATTRIBUTE d SET DATA TYPE text COLLATE "en_US" CASCADE;
ALTER TYPE non_distributed_composite_type DROP ATTRIBUTE d;
ALTER TYPE non_distributed_composite_type OWNER TO typeuser;
ALTER TYPE non_distributed_composite_type RENAME TO non_distributed_composite_type_renamed;
ALTER TYPE non_distributed_composite_type_renamed RENAME TO non_distributed_composite_type;
ALTER TYPE non_distributed_composite_type SET SCHEMA type_tests2;
ALTER TYPE type_tests2.non_distributed_composite_type SET SCHEMA type_tests;
ALTER TYPE non_distributed_enum_type OWNER TO typeuser;
ALTER TYPE non_distributed_enum_type RENAME TO non_distributed_enum_type_renamed;
ALTER TYPE non_distributed_enum_type_renamed RENAME TO non_distributed_enum_type;
ALTER TYPE non_distributed_enum_type SET SCHEMA type_tests2;
ALTER TYPE type_tests2.non_distributed_enum_type SET SCHEMA type_tests;
ALTER TYPE non_distributed_enum_type ADD VALUE 'b' BEFORE 'c';
ALTER TYPE non_distributed_enum_type ADD VALUE 'd' AFTER 'c';
ALTER TYPE non_distributed_enum_type RENAME VALUE 'd' TO 'something-with-quotes''andstuff';
-- test all forms of alter statements on distributed types
CREATE TYPE distributed_composite_type AS (a int, b int);
CREATE TYPE distributed_enum_type AS ENUM ('a', 'c');
-- enforce distribution of types in every case
CREATE TABLE type_proc (a int, b distributed_composite_type, c distributed_enum_type);
SELECT create_distributed_table('type_proc','a');
DROP TABLE type_proc;
ALTER TYPE distributed_composite_type ADD ATTRIBUTE c int;
ALTER TYPE distributed_composite_type RENAME ATTRIBUTE c TO d;
ALTER TYPE distributed_composite_type ALTER ATTRIBUTE d SET DATA TYPE text COLLATE "en_US" CASCADE;
ALTER TYPE distributed_composite_type DROP ATTRIBUTE d;
ALTER TYPE distributed_composite_type OWNER TO typeuser;
ALTER TYPE distributed_composite_type RENAME TO distributed_composite_type_renamed;
ALTER TYPE distributed_composite_type_renamed RENAME TO distributed_composite_type;
ALTER TYPE distributed_composite_type SET SCHEMA type_tests2;
ALTER TYPE type_tests2.distributed_composite_type SET SCHEMA type_tests;
ALTER TYPE distributed_enum_type OWNER TO typeuser;
ALTER TYPE distributed_enum_type RENAME TO distributed_enum_type_renamed;
ALTER TYPE distributed_enum_type_renamed RENAME TO distributed_enum_type;
ALTER TYPE distributed_enum_type SET SCHEMA type_tests2;
ALTER TYPE type_tests2.distributed_enum_type SET SCHEMA type_tests;
ALTER TYPE distributed_enum_type ADD VALUE 'b' BEFORE 'c';
ALTER TYPE distributed_enum_type ADD VALUE 'd' AFTER 'c';
ALTER TYPE distributed_enum_type RENAME VALUE 'd' TO 'something-with-quotes''andstuff';
-- clear objects
SET client_min_messages TO fatal; -- 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

@ -46,6 +46,11 @@ SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata WHERE schema_name = 'create_distributed_table_non_empty_failure'$$); SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata WHERE schema_name = 'create_distributed_table_non_empty_failure'$$);
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS create_distributed_table_non_empty_failure$$); SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS create_distributed_table_non_empty_failure$$);
-- this triggers a schema creation which prevents further transactions around dependency propagation
SELECT citus.mitmproxy('conn.allow()');
CREATE TYPE schema_proc AS (a int);
DROP TYPE schema_proc;
-- kill as soon as the coordinator sends begin -- kill as soon as the coordinator sends begin
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()');
SELECT create_distributed_table('test_table', 'id'); SELECT create_distributed_table('test_table', 'id');
@ -225,6 +230,11 @@ SELECT citus.mitmproxy('conn.allow()');
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass; SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'create_distributed_table_non_empty_failure' and table_name LIKE 'test_table%'$$); SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'create_distributed_table_non_empty_failure' and table_name LIKE 'test_table%'$$);
-- this triggers a schema creation which prevents further transactions around dependency propagation
SELECT citus.mitmproxy('conn.allow()');
CREATE TYPE schema_proc AS (a int);
DROP TYPE schema_proc;
-- kill as soon as the coordinator sends begin -- kill as soon as the coordinator sends begin
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()');
SELECT create_distributed_table('test_table', 'id'); SELECT create_distributed_table('test_table', 'id');

View File

@ -9,6 +9,11 @@ SET citus.next_shard_id TO 10000000;
SELECT citus.mitmproxy('conn.allow()'); SELECT citus.mitmproxy('conn.allow()');
-- this is merely used to get the schema creation propagated. Without there are failures
-- not related to reference tables but schema creation due to dependency creation on workers
CREATE TYPE schema_proc AS (a int);
DROP TYPE schema_proc;
CREATE TABLE ref_table(id int); CREATE TABLE ref_table(id int);
INSERT INTO ref_table VALUES(1),(2),(3); INSERT INTO ref_table VALUES(1),(2),(3);

View File

@ -30,6 +30,11 @@ SELECT citus.mitmproxy('conn.allow()');
SELECT count(*) FROM pg_dist_shard; SELECT count(*) FROM pg_dist_shard;
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata WHERE schema_name = 'failure_create_table'$$); SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata WHERE schema_name = 'failure_create_table'$$);
-- this is merely used to get the schema creation propagated. Without there are failures
-- not related to reference tables but schema creation due to dependency creation on workers
CREATE TYPE schema_proc AS (a int);
DROP TYPE schema_proc;
-- Now, kill the connection while opening transaction on workers. -- Now, kill the connection while opening transaction on workers.
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()');
SELECT create_distributed_table('test_table','id'); SELECT create_distributed_table('test_table','id');
@ -123,6 +128,11 @@ SELECT citus.mitmproxy('conn.allow()');
SELECT count(*) FROM pg_dist_shard; SELECT count(*) FROM pg_dist_shard;
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'failure_create_table' and table_name LIKE 'test_table%' ORDER BY 1$$); SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'failure_create_table' and table_name LIKE 'test_table%' ORDER BY 1$$);
-- this is merely used to get the schema creation propagated. Without there are failures
-- not related to reference tables but schema creation due to dependency creation on workers
CREATE TYPE schema_proc AS (a int);
DROP TYPE schema_proc;
-- Now, kill the connection while creating transaction on workers in transaction. -- Now, kill the connection while creating transaction on workers in transaction.
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()');
@ -179,6 +189,11 @@ SELECT citus.mitmproxy('conn.allow()');
SELECT count(*) FROM pg_dist_shard; SELECT count(*) FROM pg_dist_shard;
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'failure_create_table' and table_name LIKE 'test_table%' ORDER BY 1$$); SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'failure_create_table' and table_name LIKE 'test_table%' ORDER BY 1$$);
-- this is merely used to get the schema creation propagated. Without there are failures
-- not related to reference tables but schema creation due to dependency creation on workers
CREATE TYPE schema_proc AS (a int);
DROP TYPE schema_proc;
-- Now, kill the connection while opening transactions on workers with 1pc. Transaction will be opened due to BEGIN. -- Now, kill the connection while opening transactions on workers with 1pc. Transaction will be opened due to BEGIN.
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()');

View File

@ -12,7 +12,6 @@ SET citus.next_shard_id TO 7000000;
SET citus.next_placement_id TO 7000000; SET citus.next_placement_id TO 7000000;
CREATE TYPE foreign_details AS (name text, relid text, refd_relid text); CREATE TYPE foreign_details AS (name text, relid text, refd_relid text);
SELECT run_command_on_workers($$CREATE TYPE foreign_details AS (name text, relid text, refd_relid text)$$);
CREATE VIEW table_fkeys_in_workers AS CREATE VIEW table_fkeys_in_workers AS
SELECT SELECT
@ -273,7 +272,6 @@ DROP TABLE referenced_table;
-- foreign key as composite key -- foreign key as composite key
CREATE TYPE fkey_reference_table.composite AS (key1 int, key2 int); CREATE TYPE fkey_reference_table.composite AS (key1 int, key2 int);
SELECT run_command_on_workers($$CREATE TYPE fkey_reference_table.composite AS (key1 int, key2 int)$$) ORDER BY 1;
CREATE TABLE referenced_table(test_column composite, PRIMARY KEY(test_column)); CREATE TABLE referenced_table(test_column composite, PRIMARY KEY(test_column));
CREATE TABLE referencing_table(id int, referencing_composite composite); CREATE TABLE referencing_table(id int, referencing_composite composite);

View File

@ -80,7 +80,6 @@ END;
-- try a composite type -- try a composite type
CREATE TYPE intermediate_results.square_type AS (x text, x2 int); CREATE TYPE intermediate_results.square_type AS (x text, x2 int);
SELECT run_command_on_workers('CREATE TYPE intermediate_results.square_type AS (x text, x2 int)');
CREATE TABLE stored_squares (user_id text, square intermediate_results.square_type, metadata jsonb); CREATE TABLE stored_squares (user_id text, square intermediate_results.square_type, metadata jsonb);
INSERT INTO stored_squares VALUES ('jon', '(2,4)'::intermediate_results.square_type, '{"value":2}'); INSERT INTO stored_squares VALUES ('jon', '(2,4)'::intermediate_results.square_type, '{"value":2}');

View File

@ -6,7 +6,6 @@
SET citus.next_shard_id TO 530000; SET citus.next_shard_id TO 530000;
-- create a custom type... -- create a custom type...
CREATE TYPE test_composite_type AS ( CREATE TYPE test_composite_type AS (
i integer, i integer,
@ -14,25 +13,31 @@ CREATE TYPE test_composite_type AS (
); );
-- ... as well as a function to use as its comparator... -- ... as well as a function to use as its comparator...
CREATE FUNCTION equal_test_composite_type_function(test_composite_type, test_composite_type) RETURNS boolean SELECT run_command_on_coordinator_and_workers($cf$
LANGUAGE 'internal' CREATE FUNCTION equal_test_composite_type_function(test_composite_type, test_composite_type) RETURNS boolean
AS 'record_eq' LANGUAGE 'internal'
IMMUTABLE AS 'record_eq'
RETURNS NULL ON NULL INPUT; IMMUTABLE
RETURNS NULL ON NULL INPUT;
$cf$);
CREATE FUNCTION cmp_test_composite_type_function(test_composite_type, test_composite_type) RETURNS int SELECT run_command_on_coordinator_and_workers($cf$
LANGUAGE 'internal' CREATE FUNCTION cmp_test_composite_type_function(test_composite_type, test_composite_type) RETURNS int
AS 'btrecordcmp' LANGUAGE 'internal'
IMMUTABLE AS 'btrecordcmp'
RETURNS NULL ON NULL INPUT; IMMUTABLE
RETURNS NULL ON NULL INPUT;
$cf$);
-- ... use that function to create a custom equality operator... -- ... use that function to create a custom equality operator...
CREATE OPERATOR = ( SELECT run_command_on_coordinator_and_workers($co$
LEFTARG = test_composite_type, CREATE OPERATOR = (
RIGHTARG = test_composite_type, LEFTARG = test_composite_type,
PROCEDURE = equal_test_composite_type_function, RIGHTARG = test_composite_type,
HASHES PROCEDURE = equal_test_composite_type_function,
); HASHES
);
$co$);
-- ... and create a custom operator family for hash indexes... -- ... and create a custom operator family for hash indexes...
CREATE OPERATOR FAMILY cats_op_fam USING hash; CREATE OPERATOR FAMILY cats_op_fam USING hash;

View File

@ -93,10 +93,12 @@ SELECT * FROM ROWS FROM (next_k_integers(5), next_k_integers(10)) AS f(a, b),
-- Custom Type returning function used in a join -- Custom Type returning function used in a join
RESET client_min_messages;
CREATE TYPE min_and_max AS ( CREATE TYPE min_and_max AS (
minimum INT, minimum INT,
maximum INT maximum INT
); );
SET client_min_messages TO DEBUG1;
CREATE OR REPLACE FUNCTION max_and_min () RETURNS CREATE OR REPLACE FUNCTION max_and_min () RETURNS
min_and_max AS $$ min_and_max AS $$

View File

@ -61,7 +61,6 @@ CREATE TYPE order_side_mx AS ENUM ('buy', 'sell');
\c - - - :worker_1_port \c - - - :worker_1_port
-- create schema to test schema support -- create schema to test schema support
CREATE SCHEMA citus_mx_test_schema;
CREATE SCHEMA citus_mx_test_schema_join_1; CREATE SCHEMA citus_mx_test_schema_join_1;
CREATE SCHEMA citus_mx_test_schema_join_2; CREATE SCHEMA citus_mx_test_schema_join_2;
@ -105,15 +104,10 @@ CREATE OPERATOR citus_mx_test_schema.=== (
SET search_path TO public; SET search_path TO public;
CREATE COLLATION citus_mx_test_schema.english (LOCALE=:current_locale); CREATE COLLATION citus_mx_test_schema.english (LOCALE=:current_locale);
SET search_path TO public;
CREATE TYPE citus_mx_test_schema.new_composite_type as (key1 text, key2 text);
CREATE TYPE order_side_mx AS ENUM ('buy', 'sell');
-- now create required stuff in the worker 2 -- now create required stuff in the worker 2
\c - - - :worker_2_port \c - - - :worker_2_port
-- create schema to test schema support -- create schema to test schema support
CREATE SCHEMA citus_mx_test_schema;
CREATE SCHEMA citus_mx_test_schema_join_1; CREATE SCHEMA citus_mx_test_schema_join_1;
CREATE SCHEMA citus_mx_test_schema_join_2; CREATE SCHEMA citus_mx_test_schema_join_2;
@ -159,10 +153,6 @@ CREATE OPERATOR citus_mx_test_schema.=== (
SET search_path TO public; SET search_path TO public;
CREATE COLLATION citus_mx_test_schema.english (LOCALE=:current_locale); CREATE COLLATION citus_mx_test_schema.english (LOCALE=:current_locale);
SET search_path TO public;
CREATE TYPE citus_mx_test_schema.new_composite_type as (key1 text, key2 text);
CREATE TYPE order_side_mx AS ENUM ('buy', 'sell');
-- connect back to the master, and do some more tests -- connect back to the master, and do some more tests
\c - - - :master_port \c - - - :master_port

View File

@ -67,13 +67,7 @@ CREATE TABLE repartition_udt_other (
\c - - - :worker_1_port \c - - - :worker_1_port
CREATE TYPE test_udt AS (i integer, i2 integer);
DROP TYPE test_udt CASCADE;
-- START type creation -- START type creation
CREATE TYPE test_udt AS (i integer, i2 integer);
-- ... as well as a function to use as its comparator... -- ... as well as a function to use as its comparator...
CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean
AS 'select $1.i = $2.i AND $1.i2 = $2.i2;' AS 'select $1.i = $2.i AND $1.i2 = $2.i2;'
@ -118,9 +112,6 @@ FUNCTION 1 test_udt_hash(test_udt);
\c - - - :worker_2_port \c - - - :worker_2_port
-- START type creation -- START type creation
CREATE TYPE test_udt AS (i integer, i2 integer);
-- ... as well as a function to use as its comparator... -- ... as well as a function to use as its comparator...
CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean
AS 'select $1.i = $2.i AND $1.i2 = $2.i2;' AS 'select $1.i = $2.i AND $1.i2 = $2.i2;'

View File

@ -92,6 +92,7 @@ INSERT INTO pg_dist_shard_placement (shardid, shardstate, shardlength, nodename,
LIMIT 1; LIMIT 1;
-- Create composite type partitioned table -- Create composite type partitioned table
RESET client_min_messages; -- avoid debug messages
CREATE TYPE composite_type AS CREATE TYPE composite_type AS
( (
@ -100,7 +101,6 @@ CREATE TYPE composite_type AS
varchar_column varchar(50) varchar_column varchar(50)
); );
RESET client_min_messages; -- avoid debug messages about toast index creation
CREATE TABLE composite_partitioned_table CREATE TABLE composite_partitioned_table
( (
composite_column composite_type composite_column composite_type

View File

@ -582,8 +582,6 @@ SELECT schemaname, indexrelname FROM pg_stat_all_indexes WHERE indexrelname = 'p
-- cleanup -- cleanup
DROP TABLE prepare_ddl; DROP TABLE prepare_ddl;
DROP SCHEMA otherschema;
RESET search_path; RESET search_path;
-- test prepared COPY -- test prepared COPY
@ -612,10 +610,39 @@ $BODY$ LANGUAGE plpgsql;
SELECT local_copy_in_plpgsql(); SELECT local_copy_in_plpgsql();
SELECT local_copy_in_plpgsql(); SELECT local_copy_in_plpgsql();
-- types statements should not crash nor leak schema specifications on to cached statements
CREATE TYPE prepare_ddl_type AS (x int, y int);
SET search_path TO 'otherschema', public;
CREATE OR REPLACE FUNCTION public.type_ddl_plpgsql()
RETURNS void
LANGUAGE plpgsql
AS $function$
DECLARE
BEGIN
ALTER TYPE prepare_ddl_type RENAME TO prepare_ddl_type_backup;
END;
$function$;
SELECT type_ddl_plpgsql();
-- create same type in new schema, owner of this new type should change
CREATE TYPE prepare_ddl_type AS (x int, y int);
SELECT type_ddl_plpgsql();
-- find all renamed types to verify the schema name didn't leak, nor a crash happened
SELECT nspname, typname FROM pg_type JOIN pg_namespace ON pg_namespace.oid = pg_type.typnamespace WHERE typname = 'prepare_ddl_type_backup';
DROP TYPE prepare_ddl_type_backup;
RESET search_path;
DROP TYPE prepare_ddl_type_backup;
DROP FUNCTION type_ddl_plpgsql();
DROP FUNCTION ddl_in_plpgsql(); DROP FUNCTION ddl_in_plpgsql();
DROP FUNCTION copy_in_plpgsql(); DROP FUNCTION copy_in_plpgsql();
DROP TABLE prepare_ddl; DROP TABLE prepare_ddl;
DROP TABLE local_ddl; DROP TABLE local_ddl;
DROP SCHEMA otherschema;
-- clean-up functions -- clean-up functions
DROP FUNCTION plpgsql_test_1(); DROP FUNCTION plpgsql_test_1();

View File

@ -954,12 +954,7 @@ TRUNCATE reference_table_test;
-- first create the type on all nodes -- first create the type on all nodes
CREATE TYPE reference_comp_key as (key text, value text); CREATE TYPE reference_comp_key as (key text, value text);
\c - - - :worker_1_port
CREATE TYPE reference_comp_key as (key text, value text);
\c - - - :worker_2_port
CREATE TYPE reference_comp_key as (key text, value text);
\c - - - :master_port
CREATE TABLE reference_table_composite (id int PRIMARY KEY, data reference_comp_key); CREATE TABLE reference_table_composite (id int PRIMARY KEY, data reference_comp_key);
SELECT create_reference_table('reference_table_composite'); SELECT create_reference_table('reference_table_composite');

View File

@ -67,13 +67,7 @@ CREATE TABLE repartition_udt_other (
\c - - - :worker_1_port \c - - - :worker_1_port
CREATE TYPE test_udt AS (i integer, i2 integer);
DROP TYPE test_udt CASCADE;
-- START type creation -- START type creation
CREATE TYPE test_udt AS (i integer, i2 integer);
-- ... as well as a function to use as its comparator... -- ... as well as a function to use as its comparator...
CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean
AS 'select $1.i = $2.i AND $1.i2 = $2.i2;' AS 'select $1.i = $2.i AND $1.i2 = $2.i2;'
@ -118,9 +112,6 @@ FUNCTION 1 test_udt_hash(test_udt);
\c - - - :worker_2_port \c - - - :worker_2_port
-- START type creation -- START type creation
CREATE TYPE test_udt AS (i integer, i2 integer);
-- ... as well as a function to use as its comparator... -- ... as well as a function to use as its comparator...
CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean
AS 'select $1.i = $2.i AND $1.i2 = $2.i2;' AS 'select $1.i = $2.i AND $1.i2 = $2.i2;'

View File

@ -381,15 +381,6 @@ SELECT n_comment FROM nation_hash_collation_search_path ORDER BY n_comment COLLA
SET search_path TO public; SET search_path TO public;
CREATE TYPE test_schema_support.new_composite_type as (key1 text, key2 text); CREATE TYPE test_schema_support.new_composite_type as (key1 text, key2 text);
-- create type in worker node 1 in schema
\c - - - :worker_1_port
CREATE TYPE test_schema_support.new_composite_type as (key1 text, key2 text);
-- create type in worker node 2 in schema
\c - - - :worker_2_port
CREATE TYPE test_schema_support.new_composite_type as (key1 text, key2 text);
\c - - - :master_port
CREATE TABLE test_schema_support.nation_hash_composite_types( CREATE TABLE test_schema_support.nation_hash_composite_types(
n_nationkey integer not null, n_nationkey integer not null,
n_name char(25) not null, n_name char(25) not null,
@ -742,14 +733,6 @@ SET citus.task_executor_type TO "real-time";
SET search_path TO public; SET search_path TO public;
ALTER TABLE test_schema_support.nation_hash SET SCHEMA public; ALTER TABLE test_schema_support.nation_hash SET SCHEMA public;
-- we will use this function in next test
CREATE FUNCTION run_command_on_coordinator_and_workers(p_sql text)
RETURNS void LANGUAGE plpgsql AS $$
BEGIN
EXECUTE p_sql;
PERFORM run_command_on_workers(p_sql);
END;$$;
-- test schema propagation with user other than current user -- test schema propagation with user other than current user
SELECT run_command_on_coordinator_and_workers('CREATE USER "test-user"'); SELECT run_command_on_coordinator_and_workers('CREATE USER "test-user"');
SELECT run_command_on_coordinator_and_workers('GRANT ALL ON DATABASE postgres to "test-user"'); SELECT run_command_on_coordinator_and_workers('GRANT ALL ON DATABASE postgres to "test-user"');

View File

@ -840,14 +840,7 @@ LIMIT
SET client_min_messages TO WARNING; SET client_min_messages TO WARNING;
DROP TABLE users, events; DROP TABLE users, events;
SELECT run_command_on_master_and_workers($f$ DROP TYPE user_composite_type CASCADE;
DROP TYPE user_composite_type CASCADE;
$f$);
-- createed in multi_behavioral_analytics_create_table
DROP FUNCTION run_command_on_master_and_workers(p_sql text);
SET client_min_messages TO DEFAULT; SET client_min_messages TO DEFAULT;

View File

@ -132,3 +132,11 @@ BEGIN
END LOOP; END LOOP;
RETURN; RETURN;
END; $$ language plpgsql; END; $$ language plpgsql;
-- helper function to quickly run SQL on the whole cluster
CREATE FUNCTION run_command_on_coordinator_and_workers(p_sql text)
RETURNS void LANGUAGE plpgsql AS $$
BEGIN
EXECUTE p_sql;
PERFORM run_command_on_workers(p_sql);
END;$$;

View File

@ -55,10 +55,6 @@ DROP TABLE upgrade_reference_table_unhealthy;
-- test with table containing composite type -- test with table containing composite type
CREATE TYPE upgrade_test_composite_type AS (key1 text, key2 text); CREATE TYPE upgrade_test_composite_type AS (key1 text, key2 text);
\c - - - :worker_1_port
CREATE TYPE upgrade_test_composite_type AS (key1 text, key2 text);
\c - - - :master_port
SET citus.shard_count TO 1; SET citus.shard_count TO 1;
SET citus.shard_replication_factor TO 1; SET citus.shard_replication_factor TO 1;
CREATE TABLE upgrade_reference_table_composite(column1 int, column2 upgrade_test_composite_type); CREATE TABLE upgrade_reference_table_composite(column1 int, column2 upgrade_test_composite_type);

View File

@ -8,7 +8,6 @@ SET citus.replication_model TO streaming;
-- Setup the view so that we can check if the foreign keys are created properly -- Setup the view so that we can check if the foreign keys are created properly
CREATE TYPE foreign_details AS (name text, relid text, refd_relid text); CREATE TYPE foreign_details AS (name text, relid text, refd_relid text);
SELECT run_command_on_workers($$CREATE TYPE foreign_details AS (name text, relid text, refd_relid text)$$);
CREATE VIEW table_fkeys_in_workers AS CREATE VIEW table_fkeys_in_workers AS
SELECT SELECT

View File

@ -2,12 +2,11 @@
-- test recursive planning functionality on prepared statements -- test recursive planning functionality on prepared statements
-- =================================================================== -- ===================================================================
CREATE SCHEMA subquery_prepared_statements; CREATE SCHEMA subquery_prepared_statements;
SELECT run_command_on_workers('CREATE SCHEMA subquery_prepared_statements;');
SET search_path TO subquery_prepared_statements, public; SET search_path TO subquery_prepared_statements, public;
CREATE TYPE xy AS (x int, y int); CREATE TYPE subquery_prepared_statements.xy AS (x int, y int);
SELECT run_command_on_workers('CREATE SCHEMA subquery_prepared_statements');
SELECT run_command_on_workers('CREATE TYPE subquery_prepared_statements.xy AS (x int, y int)');
SET client_min_messages TO DEBUG1; SET client_min_messages TO DEBUG1;

View File

@ -34,8 +34,6 @@ SET citus.next_shard_id TO 8000000;
SET citus.next_placement_id TO 8000000; SET citus.next_placement_id TO 8000000;
CREATE TYPE constraint_validity AS (name text, validated bool); CREATE TYPE constraint_validity AS (name text, validated bool);
SELECT run_command_on_workers(
$$CREATE TYPE constraint_validity AS (name text, validated bool)$$);
CREATE VIEW constraint_validations_in_workers AS CREATE VIEW constraint_validations_in_workers AS
SELECT (json_populate_record(NULL :: constraint_validity, SELECT (json_populate_record(NULL :: constraint_validity,

View File

@ -1,7 +1,6 @@
-- Test the basic CTE functionality and expected error messages -- Test the basic CTE functionality and expected error messages
SET search_path TO 'with_basics'; SET search_path TO 'with_basics';
CREATE TYPE xy AS (x int, y int); CREATE TYPE with_basics.xy AS (x int, y int);
SELECT run_command_on_workers('CREATE TYPE with_basics.xy AS (x int, y int)');
-- CTEs in FROM should work -- CTEs in FROM should work
WITH cte AS ( WITH cte AS (