Merge pull request #5663 from citusdata/schema-propagation

Propagate schema operations
pull/5700/head
Ahmet Gedemenli 2022-02-10 17:30:35 +03:00 committed by GitHub
commit defc2d991f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
73 changed files with 449 additions and 873 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -5,6 +5,13 @@ SET citus.next_placement_id TO 1500000;
-- supress notice messages to make sure that the tests
-- do not diverge with enterprise
SET client_min_messages TO WARNING;
SELECT run_command_on_workers($$CREATE ROLE metadata_sync_helper_role WITH LOGIN;$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"CREATE ROLE")
(localhost,57638,t,"CREATE ROLE")
(2 rows)
CREATE ROLE metadata_sync_helper_role WITH LOGIN;
GRANT ALL ON SCHEMA metadata_sync_helpers TO metadata_sync_helper_role;
RESET client_min_messages;
@ -40,26 +47,10 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT citus_internal_add_partition_metadata ('test'::regclass, 'h', 'col_1', 0, 's');
ERROR: This is an internal Citus function can only be used in a distributed transaction
ROLLBACK;
-- connect back as super user, and then connect to the worker
-- with the superuser to make sure we can ingest metadata with
-- a regular user under the certain conditions
\c - postgres -
-- we don't need the table/schema anymore
SET client_min_messages TO ERROR;
DROP SCHEMA metadata_sync_helpers CASCADE;
DROP ROLE metadata_sync_helper_role;
\c - - - :worker_1_port
CREATE SCHEMA metadata_sync_helpers;
SET search_path TO metadata_sync_helpers;
CREATE TABLE test(col_1 int, col_2 int);
-- supress notice messages to make sure that the tests
-- do not diverge with enterprise
SET client_min_messages TO WARNING;
SET citus.enable_ddl_propagation TO OFF;
CREATE ROLE metadata_sync_helper_role WITH LOGIN;
GRANT ALL ON SCHEMA metadata_sync_helpers TO metadata_sync_helper_role;
RESET client_min_messages;
RESET citus.enable_ddl_propagation;
-- connect back with the regular user
\c - metadata_sync_helper_role - :worker_1_port
SET search_path TO metadata_sync_helpers;
@ -596,7 +587,7 @@ ROLLBACK;
-- Now let's check valid pg_dist_object updates
-- check with non-existing object type
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
assign_distributed_transaction_id
---------------------------------------------------------------------
@ -640,7 +631,7 @@ ERROR: colocationId must be a positive number
ROLLBACK;
-- check with non-existing object
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
assign_distributed_transaction_id
---------------------------------------------------------------------
@ -656,7 +647,7 @@ ROLLBACK;
-- since citus_internal_add_object_metadata is strict function returns NULL
-- if any parameter is NULL
BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
SELECT assign_distributed_transaction_id(0, 8, '2021-07-09 15:41:55.542377+02');
assign_distributed_transaction_id
---------------------------------------------------------------------
@ -1364,4 +1355,7 @@ SET client_min_messages TO ERROR;
SET citus.enable_ddl_propagation TO OFF;
DROP OWNED BY metadata_sync_helper_role;
DROP ROLE metadata_sync_helper_role;
\c - - - :master_port
-- cleanup
SET client_min_messages TO ERROR;
DROP SCHEMA metadata_sync_helpers CASCADE;

View File

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

View File

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

View File

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

View File

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

View File

@ -63,9 +63,6 @@ CREATE TABLE citus_local_table(a int);
SELECT citus_add_local_table_to_metadata('citus_local_table');
ERROR: operation is not allowed on this node
HINT: Connect to the coordinator and run it again.
-- create schema to test schema support
CREATE SCHEMA citus_mx_test_schema_join_1;
CREATE SCHEMA citus_mx_test_schema_join_2;
-- create UDFs in worker node
CREATE OR REPLACE FUNCTION simpleTestFunction(theValue integer)
RETURNS text AS
@ -101,9 +98,6 @@ CREATE OPERATOR citus_mx_test_schema.=== (
);
-- now create required stuff in the worker 2
\c - - - :worker_2_port
-- create schema to test schema support
CREATE SCHEMA citus_mx_test_schema_join_1;
CREATE SCHEMA citus_mx_test_schema_join_2;
-- create UDF
CREATE OR REPLACE FUNCTION simpleTestFunction(theValue integer)
RETURNS text AS

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -32,4 +32,3 @@ SELECT run_command_on_workers($$SELECT string_agg(enumlabel, ',' ORDER BY enumso
-- clear objects
SET client_min_messages TO error; -- suppress cascading objects dropping
DROP SCHEMA xact_enum_type CASCADE;
SELECT run_command_on_workers($$DROP SCHEMA xact_enum_type CASCADE;$$);

View File

@ -53,7 +53,6 @@ WHERE n.nspname IN ('drop_partitioned_table', 'schema1')
ORDER BY 1, 2;
\c - - - :worker_1_port
CREATE SCHEMA drop_partitioned_table;
SET search_path = drop_partitioned_table;
CREATE VIEW tables_info AS
SELECT n.nspname as "Schema",
@ -240,5 +239,4 @@ DROP TABLE parent, child1;
ROLLBACK;
DROP SCHEMA drop_partitioned_table CASCADE;
SELECT run_command_on_workers('DROP SCHEMA IF EXISTS drop_partitioned_table CASCADE');
SET search_path TO public;

View File

@ -124,5 +124,3 @@ ORDER BY placementid;
RESET SEARCH_PATH;
DROP SCHEMA add_remove_node CASCADE;
SELECT * FROM run_command_on_workers('DROP SCHEMA IF EXISTS add_remove_node CASCADE')
ORDER BY nodeport;

View File

@ -6,7 +6,9 @@
-- because if the shards are created via the executor
-- cancellations are processed, otherwise they are not
SET citus.enable_ddl_propagation TO OFF;
CREATE SCHEMA create_distributed_table_non_empty_failure;
SET citus.enable_ddl_propagation TO ON;
SET search_path TO 'create_distributed_table_non_empty_failure';
SET citus.next_shard_id TO 11000000;
@ -44,7 +46,6 @@ SELECT citus.mitmproxy('conn.onQuery(query="^CREATE SCHEMA").cancel(' || pg_bac
SELECT create_distributed_table('test_table', 'id');
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.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$$);
-- this triggers a schema creation which prevents further transactions around dependency propagation
SELECT citus.mitmproxy('conn.allow()');

View File

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

View File

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

View File

@ -247,6 +247,7 @@ BEGIN;
ROLLBACK;
BEGIN;
SET LOCAL citus.multi_shard_modify_mode TO 'sequential';
CREATE SCHEMA another_schema_fkeys_between_local_ref;
CREATE TABLE another_schema_fkeys_between_local_ref.local_table_6 (col_1 INT PRIMARY KEY);
@ -276,6 +277,7 @@ BEGIN;
ROLLBACK;
BEGIN;
SET LOCAL citus.multi_shard_modify_mode TO 'sequential';
CREATE TABLE local_table_6 (col_1 INT PRIMARY KEY);
-- first convert local tables to citus local tables in graph
ALTER TABLE local_table_2 ADD CONSTRAINT fkey_11 FOREIGN KEY (col_1) REFERENCES reference_table_1(col_1) ON DELETE CASCADE;

View File

@ -7,7 +7,9 @@ CREATE SCHEMA dist_schema;
CREATE TABLE dist_schema.dist_table (id int);
CREATE SCHEMA another_dist_schema;
CREATE TABLE another_dist_schema.dist_table (id int);
SET citus.enable_ddl_propagation TO off;
CREATE SCHEMA non_dist_schema;
SET citus.enable_ddl_propagation TO on;
-- create roles on all nodes
SELECT run_command_on_coordinator_and_workers('CREATE USER role_1');
@ -89,9 +91,9 @@ ALTER ROLE role_1 NOSUPERUSER;
SET citus.enable_alter_role_propagation TO OFF;
DROP TABLE dist_schema.dist_table, another_dist_schema.dist_table;
SELECT run_command_on_coordinator_and_workers('DROP SCHEMA dist_schema');
SELECT run_command_on_coordinator_and_workers('DROP SCHEMA another_dist_schema');
SELECT run_command_on_coordinator_and_workers('DROP SCHEMA non_dist_schema');
DROP SCHEMA dist_schema;
DROP SCHEMA another_dist_schema;
DROP SCHEMA non_dist_schema;
-- test if the grantors are propagated correctly
-- first remove one of the worker nodes
@ -157,7 +159,7 @@ SELECT nspname, nspacl FROM pg_namespace WHERE nspname = 'grantor_schema' ORDER
\c - - - :master_port
DROP TABLE grantor_schema.grantor_table;
SELECT run_command_on_coordinator_and_workers('DROP SCHEMA grantor_schema CASCADE');
DROP SCHEMA grantor_schema CASCADE;
-- test distributing the schema with another user
CREATE SCHEMA dist_schema;
@ -175,7 +177,7 @@ SELECT nspname, nspacl FROM pg_namespace WHERE nspname = 'dist_schema' ORDER BY
\c - - - :master_port
DROP TABLE dist_schema.dist_table;
SELECT run_command_on_coordinator_and_workers('DROP SCHEMA dist_schema CASCADE');
DROP SCHEMA dist_schema CASCADE;
-- test grants on public schema
-- first remove one of the worker nodes

View File

@ -6,6 +6,7 @@ SET citus.next_placement_id TO 1500000;
-- supress notice messages to make sure that the tests
-- do not diverge with enterprise
SET client_min_messages TO WARNING;
SELECT run_command_on_workers($$CREATE ROLE metadata_sync_helper_role WITH LOGIN;$$);
CREATE ROLE metadata_sync_helper_role WITH LOGIN;
GRANT ALL ON SCHEMA metadata_sync_helpers TO metadata_sync_helper_role;
RESET client_min_messages;
@ -31,33 +32,13 @@ BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;
SET application_name to 'citus_internal gpid=10000000001';
SELECT citus_internal_add_partition_metadata ('test'::regclass, 'h', 'col_1', 0, 's');
ROLLBACK;
-- connect back as super user, and then connect to the worker
-- with the superuser to make sure we can ingest metadata with
-- a regular user under the certain conditions
\c - postgres -
-- we don't need the table/schema anymore
SET client_min_messages TO ERROR;
DROP SCHEMA metadata_sync_helpers CASCADE;
DROP ROLE metadata_sync_helper_role;
\c - - - :worker_1_port
CREATE SCHEMA metadata_sync_helpers;
SET search_path TO metadata_sync_helpers;
CREATE TABLE test(col_1 int, col_2 int);
-- supress notice messages to make sure that the tests
-- do not diverge with enterprise
SET client_min_messages TO WARNING;
SET citus.enable_ddl_propagation TO OFF;
CREATE ROLE metadata_sync_helper_role WITH LOGIN;
GRANT ALL ON SCHEMA metadata_sync_helpers TO metadata_sync_helper_role;
RESET client_min_messages;
RESET citus.enable_ddl_propagation;
-- connect back with the regular user
\c - metadata_sync_helper_role - :worker_1_port
SET search_path TO metadata_sync_helpers;
@ -875,4 +856,8 @@ SET client_min_messages TO ERROR;
SET citus.enable_ddl_propagation TO OFF;
DROP OWNED BY metadata_sync_helper_role;
DROP ROLE metadata_sync_helper_role;
\c - - - :master_port
-- cleanup
SET client_min_messages TO ERROR;
DROP SCHEMA metadata_sync_helpers CASCADE;

View File

@ -258,14 +258,11 @@ SELECT deparse_and_run_on_workers($cmd$
DROP FUNCTION IF EXISTS missing_func_without_args;
$cmd$);
set citus.enable_ddl_propagation to on;
-- create schema with weird names
CREATE SCHEMA "CiTuS.TeeN";
CREATE SCHEMA "CiTUS.TEEN2";
SELECT run_command_on_workers($$
CREATE SCHEMA IF NOT EXISTS "CiTuS.TeeN";
CREATE SCHEMA IF NOT EXISTS "CiTUS.TEEN2";
$$);
set citus.enable_ddl_propagation to off;
-- create table with weird names
CREATE FUNCTION "CiTuS.TeeN"."TeeNFunCT10N.1!?!"() RETURNS TEXT
@ -387,10 +384,4 @@ DROP SCHEMA "CiTuS.TeeN" CASCADE;
DROP SCHEMA "CiTUS.TEEN2" CASCADE;
DROP SCHEMA function_tests CASCADE;
SELECT run_command_on_workers($$
DROP SCHEMA "CiTuS.TeeN" CASCADE;
DROP SCHEMA "CiTUS.TEEN2" CASCADE;
DROP SCHEMA function_tests CASCADE;
$$);
DROP ROLE function_role;

View File

@ -342,4 +342,3 @@ ALTER TABLE parent_table DROP CONSTRAINT unique_cst CASCADE;
DROP SCHEMA fix_idx_names CASCADE;
SELECT citus_remove_node('localhost', :master_port);
SELECT run_command_on_workers($$ DROP SCHEMA IF EXISTS fix_idx_names CASCADE $$);

View File

@ -161,6 +161,7 @@ ROLLBACK;
-- Test schemas
BEGIN;
SET LOCAL citus.multi_shard_modify_mode TO 'sequential';
CREATE SCHEMA fkey_intermediate_schema_1;
CREATE SCHEMA fkey_intermediate_schema_2;
SET search_path TO fkey_graph, fkey_intermediate_schema_1, fkey_intermediate_schema_2;
@ -192,6 +193,7 @@ BEGIN;
ROLLBACK;
BEGIN;
SET LOCAL citus.multi_shard_modify_mode TO 'sequential';
CREATE SCHEMA fkey_intermediate_schema_1;
CREATE SCHEMA fkey_intermediate_schema_2;
SET search_path TO fkey_graph, fkey_intermediate_schema_1, fkey_intermediate_schema_2;

View File

@ -54,12 +54,6 @@ GRANT SELECT ON TABLE test_1420000 TO read_access;
GRANT ALL ON TABLE test_1420002 TO full_access;
GRANT SELECT ON TABLE test_1420002 TO read_access;
CREATE SCHEMA full_access_user_schema;
REVOKE ALL ON SCHEMA full_access_user_schema FROM PUBLIC;
GRANT USAGE ON SCHEMA full_access_user_schema TO full_access;
GRANT ALL ON SCHEMA full_access_user_schema TO full_access;
GRANT USAGE ON SCHEMA full_access_user_schema TO usage_access;
\c - - - :worker_2_port
CREATE USER full_access;
CREATE USER usage_access;
@ -75,12 +69,6 @@ GRANT SELECT ON TABLE test_1420001 TO read_access;
GRANT ALL ON TABLE test_1420003 TO full_access;
GRANT SELECT ON TABLE test_1420003 TO read_access;
CREATE SCHEMA full_access_user_schema;
REVOKE ALL ON SCHEMA full_access_user_schema FROM PUBLIC;
GRANT USAGE ON SCHEMA full_access_user_schema TO full_access;
GRANT ALL ON SCHEMA full_access_user_schema TO full_access;
GRANT USAGE ON SCHEMA full_access_user_schema TO usage_access;
\c - - - :master_port
SET citus.shard_replication_factor TO 1;

View File

@ -63,10 +63,6 @@ CREATE TYPE order_side_mx AS ENUM ('buy', 'sell');
CREATE TABLE citus_local_table(a int);
SELECT citus_add_local_table_to_metadata('citus_local_table');
-- create schema to test schema support
CREATE SCHEMA citus_mx_test_schema_join_1;
CREATE SCHEMA citus_mx_test_schema_join_2;
-- create UDFs in worker node
CREATE OR REPLACE FUNCTION simpleTestFunction(theValue integer)
RETURNS text AS
@ -107,11 +103,6 @@ CREATE OPERATOR citus_mx_test_schema.=== (
-- now create required stuff in the worker 2
\c - - - :worker_2_port
-- create schema to test schema support
CREATE SCHEMA citus_mx_test_schema_join_1;
CREATE SCHEMA citus_mx_test_schema_join_2;
-- create UDF
CREATE OR REPLACE FUNCTION simpleTestFunction(theValue integer)
RETURNS text AS

View File

@ -173,7 +173,6 @@ SELECT * FROM seg_test;
\c - - - :master_port
CREATE SCHEMA ext_owned_tables;
SELECT run_command_on_workers($$CREATE SCHEMA ext_owned_tables;$$);
SET search_path TO ext_owned_tables;

View File

@ -295,7 +295,9 @@ ALTER TABLE "CiTuS.TeAeN"."TeeNTabLE.1!?!" ADD COLUMN new_col INT;
SET search_path TO not_existing_schema;
ALTER TABLE "CiTuS.TeAeN"."TeeNTabLE.1!?!" DROP COLUMN new_col;
SET client_min_messages TO ERROR;
DROP SCHEMA mx_ddl_schema_1, mx_ddl_schema_2, "CiTuS.TeAeN" CASCADE;
RESET client_min_messages;
-- test if ALTER TABLE SET SCHEMA sets the original table in the worker
SET search_path TO public;
@ -307,7 +309,8 @@ CREATE SCHEMA mx_new_schema;
SELECT objid::oid::regnamespace as "Distributed Schemas"
FROM citus.pg_dist_object
WHERE objid::oid::regnamespace IN ('mx_old_schema', 'mx_new_schema');
WHERE objid::oid::regnamespace IN ('mx_old_schema', 'mx_new_schema')
ORDER BY "Distributed Schemas";
\c - - - :worker_1_port
SELECT table_schema AS "Table's Schema" FROM information_schema.tables WHERE table_name='table_set_schema';
SELECT table_schema AS "Shards' Schema"

View File

@ -1026,10 +1026,10 @@ ROLLBACK;
-- Clean up the created schema
SET client_min_messages TO WARNING;
DROP SCHEMA run_test_schema CASCADE;
DROP SCHEMA test_schema_support_join_1 CASCADE;
DROP SCHEMA test_schema_support_join_2 CASCADE;
DROP SCHEMA "Citus'Teen123" CASCADE;
DROP SCHEMA "CiTUS.TEEN2" CASCADE;
DROP SCHEMA bar CASCADE;
DROP SCHEMA test_schema_support CASCADE;
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object
WHERE classid=2615 and objid IN (select oid from pg_namespace where nspname='run_test_schema');
DROP SCHEMA run_test_schema, test_schema_support_join_1, test_schema_support_join_2, "Citus'Teen123", "CiTUS.TEEN2", bar, test_schema_support CASCADE;
-- verify that the dropped schema is removed from worker's pg_dist_object
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object
WHERE classid=2615 and objid IN (select oid from pg_namespace where nspname='run_test_schema');

View File

@ -187,34 +187,6 @@ SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
DROP SEQUENCE sequence_default_0.sequence_3 CASCADE;
DROP SCHEMA sequence_default_0;
-- DROP SCHEMA problem: expected since we don't propagate DROP SCHEMA
CREATE TABLE seq_test_5 (x int, y int);
SELECT create_distributed_table('seq_test_5','x');
CREATE SCHEMA sequence_default_1;
CREATE SEQUENCE sequence_default_1.seq_5;
ALTER TABLE seq_test_5 ADD COLUMN a bigint DEFAULT nextval('sequence_default_1.seq_5');
DROP SCHEMA sequence_default_1 CASCADE;
-- sequence is gone from coordinator
INSERT INTO seq_test_5 VALUES (1, 2) RETURNING *;
-- but is still present on worker
\c - - - :worker_1_port
INSERT INTO sequence_default.seq_test_5 VALUES (1, 2) RETURNING *;
\c - - - :master_port
SET citus.shard_replication_factor TO 1;
SET search_path = sequence_default, public;
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
-- apply workaround
SELECT run_command_on_workers('DROP SCHEMA sequence_default_1 CASCADE');
-- now the sequence is gone from the worker as well
\c - - - :worker_1_port
INSERT INTO sequence_default.seq_test_5 VALUES (1, 2) RETURNING *;
\c - - - :master_port
SET citus.shard_replication_factor TO 1;
SET search_path = sequence_default, public;
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
-- check some more complex cases
CREATE SEQUENCE seq_6;
CREATE TABLE seq_test_6 (x int, t timestamptz DEFAULT now(), s int DEFAULT nextval('seq_6'), m int) PARTITION BY RANGE (t);
@ -287,8 +259,7 @@ ALTER TABLE sequence_default_8.seq_8 SET SCHEMA sequence_default;
SET citus.shard_replication_factor TO 1;
SET search_path = sequence_default, public;
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
DROP SCHEMA sequence_default_8;
SELECT run_command_on_workers('DROP SCHEMA IF EXISTS sequence_default_8 CASCADE');
DROP SCHEMA sequence_default_8 CASCADE;
-- cannot use more than one sequence in a column default
@ -442,10 +413,6 @@ SELECT nextval('seq_14');
CREATE SCHEMA test_schema_for_sequence_default_propagation;
CREATE SEQUENCE test_schema_for_sequence_default_propagation.seq_10;
-- Both should return 0 rows
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation.seq_10'::regclass);
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation'::regnamespace);
-- Create distributed table with default column to propagate dependencies
CREATE TABLE test_seq_dist(a int, x BIGINT DEFAULT nextval('test_schema_for_sequence_default_propagation.seq_10'));
SELECT create_distributed_table('test_seq_dist', 'a');
@ -487,6 +454,5 @@ DROP TABLE test_seq_dist;
DROP TABLE sequence_default.seq_test_7_par;
SET client_min_messages TO error; -- suppress cascading objects dropping
DROP SCHEMA sequence_default CASCADE;
SELECT run_command_on_workers('DROP SCHEMA IF EXISTS sequence_default CASCADE');
SELECT master_remove_node('localhost', :master_port);
SET search_path TO public;

View File

@ -100,18 +100,6 @@ SELECT lock_shard_resources(8, ARRAY[2980000]);
SET client_min_messages TO ERROR;
DROP SCHEMA "Mx Super User" CASCADE;
\c - postgres - :worker_1_port;
SET client_min_messages TO ERROR;
SET citus.enable_ddl_propagation TO OFF;
CREATE SCHEMA "Mx Regular User";
GRANT ALL ON SCHEMA "Mx Regular User" TO regular_mx_user;
\c - postgres - :worker_2_port;
SET client_min_messages TO ERROR;
SET citus.enable_ddl_propagation TO OFF;
CREATE SCHEMA "Mx Regular User";
GRANT ALL ON SCHEMA "Mx Regular User" TO regular_mx_user;
-- now connect with that user
\c - regular_mx_user - :master_port
SET search_path TO "Mx Regular User";

View File

@ -179,7 +179,7 @@ SELECT create_distributed_function('metadata_dist_test_proc(integer, integer)',
SELECT distribution_argument_index FROM citus.pg_dist_object WHERE objid = 'metadata_dist_test_proc'::regproc;
SELECT * FROM run_command_on_workers($$ SELECT distribution_argument_index FROM citus.pg_dist_object WHERE objid = 'local_schema.metadata_dist_test_proc'::regproc;$$) ORDER BY 1,2;
-- Show that dropping schema doesn't affect the worker node
-- Show that the schema is dropped on worker node as well
DROP SCHEMA local_schema CASCADE;
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{local_schema}%';

View File

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

View File

@ -2,7 +2,6 @@
-- test recursive planning functionality on 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;