mirror of https://github.com/citusdata/citus.git
Move pg_dist_object to pg_catalog (#5765)
DESCRIPTION: Move pg_dist_object to pg_catalog Historically `pg_dist_object` had been created in the `citus` schema as an experiment to understand if we could move our catalog tables to a branded schema. We quickly realised that this interfered with the UX on our managed services and other environments, where users connected via a user with the name of `citus`. By default postgres put the username on the search_path. To be able to read the catalog in the `citus` schema we would need to grant access permissions to the schema. This caused newly created objects like tables etc, to default to this schema for creation. This failed due to the write permissions to that schema. With this change we move the `pg_dist_object` catalog table to the `pg_catalog` schema, where our other schema's are also located. This makes the catalog table visible and readable by any user, like our other catalog tables, for debugging purposes. Note: due to the change of schema, we had to disable 1 test that was running into a discrepancy between the schema and binary. Secondly, we needed to make the lookup functions for the `pg_dist_object` relation and their indexes less strict on the fallback of the naming due to an other test that, due to an unfortunate cache invalidation, needed to lookup the relation again. This makes that we won't default to _only_ resolving from `pg_catalog` outside of upgrades.pull/5762/head
parent
12d4486567
commit
3801576dfb
|
@ -209,8 +209,9 @@ MarkObjectDistributedLocally(const ObjectAddress *distAddress)
|
|||
ObjectIdGetDatum(distAddress->objectId),
|
||||
Int32GetDatum(distAddress->objectSubId)
|
||||
};
|
||||
char *insertQuery = "INSERT INTO citus.pg_dist_object (classid, objid, objsubid) "
|
||||
"VALUES ($1, $2, $3) ON CONFLICT DO NOTHING";
|
||||
char *insertQuery =
|
||||
"INSERT INTO pg_catalog.pg_dist_object (classid, objid, objsubid) "
|
||||
"VALUES ($1, $2, $3) ON CONFLICT DO NOTHING";
|
||||
int spiStatus = ExecuteCommandAsSuperuser(insertQuery, paramCount, paramTypes,
|
||||
paramValues);
|
||||
if (spiStatus < 0)
|
||||
|
@ -324,7 +325,7 @@ UnmarkObjectDistributed(const ObjectAddress *address)
|
|||
Int32GetDatum(address->objectSubId)
|
||||
};
|
||||
|
||||
char *deleteQuery = "DELETE FROM citus.pg_dist_object WHERE classid = $1 AND "
|
||||
char *deleteQuery = "DELETE FROM pg_catalog.pg_dist_object WHERE classid = $1 AND "
|
||||
"objid = $2 AND objsubid = $3";
|
||||
|
||||
int spiStatus = ExecuteCommandAsSuperuser(deleteQuery, paramCount, paramTypes,
|
||||
|
|
|
@ -247,8 +247,13 @@ static void GetPartitionTypeInputInfo(char *partitionKeyString, char partitionMe
|
|||
Oid *intervalTypeId, int32 *intervalTypeMod);
|
||||
static void CachedNamespaceLookup(const char *nspname, Oid *cachedOid);
|
||||
static void CachedRelationLookup(const char *relationName, Oid *cachedOid);
|
||||
static void CachedRelationLookupExtended(const char *relationName, Oid *cachedOid,
|
||||
bool missing_ok);
|
||||
static void CachedRelationNamespaceLookup(const char *relationName, Oid relnamespace,
|
||||
Oid *cachedOid);
|
||||
static void CachedRelationNamespaceLookupExtended(const char *relationName,
|
||||
Oid renamespace, Oid *cachedOid,
|
||||
bool missing_ok);
|
||||
static ShardPlacement * ResolveGroupShardPlacement(
|
||||
GroupShardPlacement *groupShardPlacement, CitusTableCacheEntry *tableEntry,
|
||||
int shardIndex);
|
||||
|
@ -2321,8 +2326,37 @@ CitusCatalogNamespaceId(void)
|
|||
Oid
|
||||
DistObjectRelationId(void)
|
||||
{
|
||||
CachedRelationNamespaceLookup("pg_dist_object", CitusCatalogNamespaceId(),
|
||||
&MetadataCache.distObjectRelationId);
|
||||
/*
|
||||
* In older versions pg_dist_object was living in the `citus` namespace, With Citus 11
|
||||
* this has been moved to pg_dist_catalog.
|
||||
*
|
||||
* During upgrades it could therefore be that we simply need to look in the old
|
||||
* catalog. Since we expect to find it most of the time in the pg_catalog schema from
|
||||
* now on we will start there.
|
||||
*
|
||||
* even after the table has been moved, the oid's stay the same, so we don't have to
|
||||
* invalidate the cache after a move
|
||||
*
|
||||
* Note: during testing we also up/downgrade the extension, and sometimes interact
|
||||
* with the database when the schema and the binary are not in sync. Hance we always
|
||||
* allow the catalog to be missing on our first lookup. The error message might
|
||||
* therefore become misleading as it will complain about citus.pg_dist_object not
|
||||
* being found when called too early.
|
||||
*/
|
||||
CachedRelationLookupExtended("pg_dist_object",
|
||||
&MetadataCache.distObjectRelationId,
|
||||
true);
|
||||
if (!OidIsValid(MetadataCache.distObjectRelationId))
|
||||
{
|
||||
/*
|
||||
* We can only ever reach here while we are creating/altering our extension before
|
||||
* the table is moved to pg_catalog.
|
||||
*/
|
||||
CachedRelationNamespaceLookupExtended("pg_dist_object",
|
||||
CitusCatalogNamespaceId(),
|
||||
&MetadataCache.distObjectRelationId,
|
||||
false);
|
||||
}
|
||||
|
||||
return MetadataCache.distObjectRelationId;
|
||||
}
|
||||
|
@ -2332,9 +2366,38 @@ DistObjectRelationId(void)
|
|||
Oid
|
||||
DistObjectPrimaryKeyIndexId(void)
|
||||
{
|
||||
CachedRelationNamespaceLookup("pg_dist_object_pkey",
|
||||
CitusCatalogNamespaceId(),
|
||||
&MetadataCache.distObjectPrimaryKeyIndexId);
|
||||
/*
|
||||
* In older versions pg_dist_object was living in the `citus` namespace, With Citus 11
|
||||
* this has been moved to pg_dist_catalog.
|
||||
*
|
||||
* During upgrades it could therefore be that we simply need to look in the old
|
||||
* catalog. Since we expect to find it most of the time in the pg_catalog schema from
|
||||
* now on we will start there.
|
||||
*
|
||||
* even after the table has been moved, the oid's stay the same, so we don't have to
|
||||
* invalidate the cache after a move
|
||||
*
|
||||
* Note: during testing we also up/downgrade the extension, and sometimes interact
|
||||
* with the database when the schema and the binary are not in sync. Hance we always
|
||||
* allow the catalog to be missing on our first lookup. The error message might
|
||||
* therefore become misleading as it will complain about citus.pg_dist_object not
|
||||
* being found when called too early.
|
||||
*/
|
||||
CachedRelationLookupExtended("pg_dist_object_pkey",
|
||||
&MetadataCache.distObjectPrimaryKeyIndexId,
|
||||
true);
|
||||
|
||||
if (!OidIsValid(MetadataCache.distObjectPrimaryKeyIndexId))
|
||||
{
|
||||
/*
|
||||
* We can only ever reach here while we are creating/altering our extension before
|
||||
* the table is moved to pg_catalog.
|
||||
*/
|
||||
CachedRelationNamespaceLookupExtended("pg_dist_object_pkey",
|
||||
CitusCatalogNamespaceId(),
|
||||
&MetadataCache.distObjectPrimaryKeyIndexId,
|
||||
false);
|
||||
}
|
||||
|
||||
return MetadataCache.distObjectPrimaryKeyIndexId;
|
||||
}
|
||||
|
@ -4591,9 +4654,30 @@ CachedRelationLookup(const char *relationName, Oid *cachedOid)
|
|||
}
|
||||
|
||||
|
||||
/*
|
||||
* CachedRelationLookupExtended performs a cached lookup for the relation
|
||||
* relationName, with the result cached in *cachedOid. Will _not_ throw an error when
|
||||
* missing_ok is set to true.
|
||||
*/
|
||||
static void
|
||||
CachedRelationLookupExtended(const char *relationName, Oid *cachedOid, bool missing_ok)
|
||||
{
|
||||
CachedRelationNamespaceLookupExtended(relationName, PG_CATALOG_NAMESPACE, cachedOid,
|
||||
missing_ok);
|
||||
}
|
||||
|
||||
|
||||
static void
|
||||
CachedRelationNamespaceLookup(const char *relationName, Oid relnamespace,
|
||||
Oid *cachedOid)
|
||||
{
|
||||
CachedRelationNamespaceLookupExtended(relationName, relnamespace, cachedOid, false);
|
||||
}
|
||||
|
||||
|
||||
static void
|
||||
CachedRelationNamespaceLookupExtended(const char *relationName, Oid relnamespace,
|
||||
Oid *cachedOid, bool missing_ok)
|
||||
{
|
||||
/* force callbacks to be registered, so we always get notified upon changes */
|
||||
InitializeCaches();
|
||||
|
@ -4602,7 +4686,7 @@ CachedRelationNamespaceLookup(const char *relationName, Oid relnamespace,
|
|||
{
|
||||
*cachedOid = get_relname_relid(relationName, relnamespace);
|
||||
|
||||
if (*cachedOid == InvalidOid)
|
||||
if (*cachedOid == InvalidOid && !missing_ok)
|
||||
{
|
||||
ereport(ERROR, (errmsg(
|
||||
"cache lookup failed for %s, called too early?",
|
||||
|
|
|
@ -79,3 +79,8 @@ END;
|
|||
$$;
|
||||
|
||||
#include "udfs/citus_finalize_upgrade_to_citus11/11.0-1.sql"
|
||||
|
||||
ALTER TABLE citus.pg_dist_object SET SCHEMA pg_catalog;
|
||||
GRANT SELECT ON pg_catalog.pg_dist_object TO public;
|
||||
#include "udfs/citus_prepare_pg_upgrade/11.0-1.sql"
|
||||
#include "udfs/citus_finish_pg_upgrade/11.0-1.sql"
|
||||
|
|
|
@ -1,4 +1,6 @@
|
|||
-- citus--11.0-1--10.2-4
|
||||
REVOKE SELECT ON pg_catalog.pg_dist_object FROM public;
|
||||
ALTER TABLE pg_catalog.pg_dist_object SET SCHEMA citus;
|
||||
|
||||
DROP FUNCTION pg_catalog.create_distributed_function(regprocedure, text, text, bool);
|
||||
|
||||
|
|
|
@ -0,0 +1,151 @@
|
|||
CREATE OR REPLACE FUNCTION pg_catalog.citus_finish_pg_upgrade()
|
||||
RETURNS void
|
||||
LANGUAGE plpgsql
|
||||
SET search_path = pg_catalog
|
||||
AS $cppu$
|
||||
DECLARE
|
||||
table_name regclass;
|
||||
command text;
|
||||
trigger_name text;
|
||||
BEGIN
|
||||
|
||||
|
||||
IF substring(current_Setting('server_version'), '\d+')::int >= 14 THEN
|
||||
EXECUTE $cmd$
|
||||
-- disable propagation to prevent EnsureCoordinator errors
|
||||
-- the aggregate created here does not depend on Citus extension (yet)
|
||||
-- since we add the dependency with the next command
|
||||
SET citus.enable_ddl_propagation TO OFF;
|
||||
CREATE AGGREGATE array_cat_agg(anycompatiblearray) (SFUNC = array_cat, STYPE = anycompatiblearray);
|
||||
COMMENT ON AGGREGATE array_cat_agg(anycompatiblearray)
|
||||
IS 'concatenate input arrays into a single array';
|
||||
RESET citus.enable_ddl_propagation;
|
||||
$cmd$;
|
||||
ELSE
|
||||
EXECUTE $cmd$
|
||||
SET citus.enable_ddl_propagation TO OFF;
|
||||
CREATE AGGREGATE array_cat_agg(anyarray) (SFUNC = array_cat, STYPE = anyarray);
|
||||
COMMENT ON AGGREGATE array_cat_agg(anyarray)
|
||||
IS 'concatenate input arrays into a single array';
|
||||
RESET citus.enable_ddl_propagation;
|
||||
$cmd$;
|
||||
END IF;
|
||||
|
||||
--
|
||||
-- Citus creates the array_cat_agg but because of a compatibility
|
||||
-- issue between pg13-pg14, we drop and create it during upgrade.
|
||||
-- And as Citus creates it, there needs to be a dependency to the
|
||||
-- Citus extension, so we create that dependency here.
|
||||
-- We are not using:
|
||||
-- ALTER EXENSION citus DROP/CREATE AGGREGATE array_cat_agg
|
||||
-- because we don't have an easy way to check if the aggregate
|
||||
-- exists with anyarray type or anycompatiblearray type.
|
||||
|
||||
INSERT INTO pg_depend
|
||||
SELECT
|
||||
'pg_proc'::regclass::oid as classid,
|
||||
(SELECT oid FROM pg_proc WHERE proname = 'array_cat_agg') as objid,
|
||||
0 as objsubid,
|
||||
'pg_extension'::regclass::oid as refclassid,
|
||||
(select oid from pg_extension where extname = 'citus') as refobjid,
|
||||
0 as refobjsubid ,
|
||||
'e' as deptype;
|
||||
|
||||
--
|
||||
-- restore citus catalog tables
|
||||
--
|
||||
INSERT INTO pg_catalog.pg_dist_partition SELECT * FROM public.pg_dist_partition;
|
||||
INSERT INTO pg_catalog.pg_dist_shard SELECT * FROM public.pg_dist_shard;
|
||||
INSERT INTO pg_catalog.pg_dist_placement SELECT * FROM public.pg_dist_placement;
|
||||
INSERT INTO pg_catalog.pg_dist_node_metadata SELECT * FROM public.pg_dist_node_metadata;
|
||||
INSERT INTO pg_catalog.pg_dist_node SELECT * FROM public.pg_dist_node;
|
||||
INSERT INTO pg_catalog.pg_dist_local_group SELECT * FROM public.pg_dist_local_group;
|
||||
INSERT INTO pg_catalog.pg_dist_transaction SELECT * FROM public.pg_dist_transaction;
|
||||
INSERT INTO pg_catalog.pg_dist_colocation SELECT * FROM public.pg_dist_colocation;
|
||||
-- enterprise catalog tables
|
||||
INSERT INTO pg_catalog.pg_dist_authinfo SELECT * FROM public.pg_dist_authinfo;
|
||||
INSERT INTO pg_catalog.pg_dist_poolinfo SELECT * FROM public.pg_dist_poolinfo;
|
||||
|
||||
INSERT INTO pg_catalog.pg_dist_rebalance_strategy SELECT
|
||||
name,
|
||||
default_strategy,
|
||||
shard_cost_function::regprocedure::regproc,
|
||||
node_capacity_function::regprocedure::regproc,
|
||||
shard_allowed_on_node_function::regprocedure::regproc,
|
||||
default_threshold,
|
||||
minimum_threshold,
|
||||
improvement_threshold
|
||||
FROM public.pg_dist_rebalance_strategy;
|
||||
|
||||
--
|
||||
-- drop backup tables
|
||||
--
|
||||
DROP TABLE public.pg_dist_authinfo;
|
||||
DROP TABLE public.pg_dist_colocation;
|
||||
DROP TABLE public.pg_dist_local_group;
|
||||
DROP TABLE public.pg_dist_node;
|
||||
DROP TABLE public.pg_dist_node_metadata;
|
||||
DROP TABLE public.pg_dist_partition;
|
||||
DROP TABLE public.pg_dist_placement;
|
||||
DROP TABLE public.pg_dist_poolinfo;
|
||||
DROP TABLE public.pg_dist_shard;
|
||||
DROP TABLE public.pg_dist_transaction;
|
||||
DROP TABLE public.pg_dist_rebalance_strategy;
|
||||
|
||||
--
|
||||
-- reset sequences
|
||||
--
|
||||
PERFORM setval('pg_catalog.pg_dist_shardid_seq', (SELECT MAX(shardid)+1 AS max_shard_id FROM pg_dist_shard), false);
|
||||
PERFORM setval('pg_catalog.pg_dist_placement_placementid_seq', (SELECT MAX(placementid)+1 AS max_placement_id FROM pg_dist_placement), false);
|
||||
PERFORM setval('pg_catalog.pg_dist_groupid_seq', (SELECT MAX(groupid)+1 AS max_group_id FROM pg_dist_node), false);
|
||||
PERFORM setval('pg_catalog.pg_dist_node_nodeid_seq', (SELECT MAX(nodeid)+1 AS max_node_id FROM pg_dist_node), false);
|
||||
PERFORM setval('pg_catalog.pg_dist_colocationid_seq', (SELECT MAX(colocationid)+1 AS max_colocation_id FROM pg_dist_colocation), false);
|
||||
|
||||
--
|
||||
-- register triggers
|
||||
--
|
||||
FOR table_name IN SELECT logicalrelid FROM pg_catalog.pg_dist_partition
|
||||
LOOP
|
||||
trigger_name := 'truncate_trigger_' || table_name::oid;
|
||||
command := 'create trigger ' || trigger_name || ' after truncate on ' || table_name || ' execute procedure pg_catalog.citus_truncate_trigger()';
|
||||
EXECUTE command;
|
||||
command := 'update pg_trigger set tgisinternal = true where tgname = ' || quote_literal(trigger_name);
|
||||
EXECUTE command;
|
||||
END LOOP;
|
||||
|
||||
--
|
||||
-- set dependencies
|
||||
--
|
||||
INSERT INTO pg_depend
|
||||
SELECT
|
||||
'pg_class'::regclass::oid as classid,
|
||||
p.logicalrelid::regclass::oid as objid,
|
||||
0 as objsubid,
|
||||
'pg_extension'::regclass::oid as refclassid,
|
||||
(select oid from pg_extension where extname = 'citus') as refobjid,
|
||||
0 as refobjsubid ,
|
||||
'n' as deptype
|
||||
FROM pg_catalog.pg_dist_partition p;
|
||||
|
||||
-- set dependencies for columnar table access method
|
||||
PERFORM citus_internal.columnar_ensure_am_depends_catalog();
|
||||
|
||||
-- restore pg_dist_object from the stable identifiers
|
||||
TRUNCATE pg_catalog.pg_dist_object;
|
||||
INSERT INTO pg_catalog.pg_dist_object (classid, objid, objsubid, distribution_argument_index, colocationid)
|
||||
SELECT
|
||||
address.classid,
|
||||
address.objid,
|
||||
address.objsubid,
|
||||
naming.distribution_argument_index,
|
||||
naming.colocationid
|
||||
FROM
|
||||
public.pg_dist_object naming,
|
||||
pg_catalog.pg_get_object_address(naming.type, naming.object_names, naming.object_args) address;
|
||||
|
||||
DROP TABLE public.pg_dist_object;
|
||||
END;
|
||||
$cppu$;
|
||||
|
||||
COMMENT ON FUNCTION pg_catalog.citus_finish_pg_upgrade()
|
||||
IS 'perform tasks to restore citus settings from a location that has been prepared before pg_upgrade';
|
|
@ -131,8 +131,8 @@ BEGIN
|
|||
PERFORM citus_internal.columnar_ensure_am_depends_catalog();
|
||||
|
||||
-- restore pg_dist_object from the stable identifiers
|
||||
TRUNCATE citus.pg_dist_object;
|
||||
INSERT INTO citus.pg_dist_object (classid, objid, objsubid, distribution_argument_index, colocationid)
|
||||
TRUNCATE pg_catalog.pg_dist_object;
|
||||
INSERT INTO pg_catalog.pg_dist_object (classid, objid, objsubid, distribution_argument_index, colocationid)
|
||||
SELECT
|
||||
address.classid,
|
||||
address.objid,
|
||||
|
|
|
@ -0,0 +1,74 @@
|
|||
CREATE OR REPLACE FUNCTION pg_catalog.citus_prepare_pg_upgrade()
|
||||
RETURNS void
|
||||
LANGUAGE plpgsql
|
||||
SET search_path = pg_catalog
|
||||
AS $cppu$
|
||||
BEGIN
|
||||
|
||||
DELETE FROM pg_depend WHERE
|
||||
objid IN (SELECT oid FROM pg_proc WHERE proname = 'array_cat_agg') AND
|
||||
refobjid IN (select oid from pg_extension where extname = 'citus');
|
||||
--
|
||||
-- We are dropping the aggregates because postgres 14 changed
|
||||
-- array_cat type from anyarray to anycompatiblearray. When
|
||||
-- upgrading to pg14, specifically when running pg_restore on
|
||||
-- array_cat_agg we would get an error. So we drop the aggregate
|
||||
-- and create the right one on citus_finish_pg_upgrade.
|
||||
|
||||
DROP AGGREGATE IF EXISTS array_cat_agg(anyarray);
|
||||
DROP AGGREGATE IF EXISTS array_cat_agg(anycompatiblearray);
|
||||
--
|
||||
-- Drop existing backup tables
|
||||
--
|
||||
DROP TABLE IF EXISTS public.pg_dist_partition;
|
||||
DROP TABLE IF EXISTS public.pg_dist_shard;
|
||||
DROP TABLE IF EXISTS public.pg_dist_placement;
|
||||
DROP TABLE IF EXISTS public.pg_dist_node_metadata;
|
||||
DROP TABLE IF EXISTS public.pg_dist_node;
|
||||
DROP TABLE IF EXISTS public.pg_dist_local_group;
|
||||
DROP TABLE IF EXISTS public.pg_dist_transaction;
|
||||
DROP TABLE IF EXISTS public.pg_dist_colocation;
|
||||
DROP TABLE IF EXISTS public.pg_dist_authinfo;
|
||||
DROP TABLE IF EXISTS public.pg_dist_poolinfo;
|
||||
DROP TABLE IF EXISTS public.pg_dist_rebalance_strategy;
|
||||
DROP TABLE IF EXISTS public.pg_dist_object;
|
||||
|
||||
--
|
||||
-- backup citus catalog tables
|
||||
--
|
||||
CREATE TABLE public.pg_dist_partition AS SELECT * FROM pg_catalog.pg_dist_partition;
|
||||
CREATE TABLE public.pg_dist_shard AS SELECT * FROM pg_catalog.pg_dist_shard;
|
||||
CREATE TABLE public.pg_dist_placement AS SELECT * FROM pg_catalog.pg_dist_placement;
|
||||
CREATE TABLE public.pg_dist_node_metadata AS SELECT * FROM pg_catalog.pg_dist_node_metadata;
|
||||
CREATE TABLE public.pg_dist_node AS SELECT * FROM pg_catalog.pg_dist_node;
|
||||
CREATE TABLE public.pg_dist_local_group AS SELECT * FROM pg_catalog.pg_dist_local_group;
|
||||
CREATE TABLE public.pg_dist_transaction AS SELECT * FROM pg_catalog.pg_dist_transaction;
|
||||
CREATE TABLE public.pg_dist_colocation AS SELECT * FROM pg_catalog.pg_dist_colocation;
|
||||
-- enterprise catalog tables
|
||||
CREATE TABLE public.pg_dist_authinfo AS SELECT * FROM pg_catalog.pg_dist_authinfo;
|
||||
CREATE TABLE public.pg_dist_poolinfo AS SELECT * FROM pg_catalog.pg_dist_poolinfo;
|
||||
CREATE TABLE public.pg_dist_rebalance_strategy AS SELECT
|
||||
name,
|
||||
default_strategy,
|
||||
shard_cost_function::regprocedure::text,
|
||||
node_capacity_function::regprocedure::text,
|
||||
shard_allowed_on_node_function::regprocedure::text,
|
||||
default_threshold,
|
||||
minimum_threshold,
|
||||
improvement_threshold
|
||||
FROM pg_catalog.pg_dist_rebalance_strategy;
|
||||
|
||||
-- store upgrade stable identifiers on pg_dist_object catalog
|
||||
CREATE TABLE public.pg_dist_object AS SELECT
|
||||
address.type,
|
||||
address.object_names,
|
||||
address.object_args,
|
||||
objects.distribution_argument_index,
|
||||
objects.colocationid
|
||||
FROM pg_catalog.pg_dist_object objects,
|
||||
pg_catalog.pg_identify_object_as_address(objects.classid, objects.objid, objects.objsubid) address;
|
||||
END;
|
||||
$cppu$;
|
||||
|
||||
COMMENT ON FUNCTION pg_catalog.citus_prepare_pg_upgrade()
|
||||
IS 'perform tasks to copy citus settings to a location that could later be restored after pg_upgrade is done';
|
|
@ -65,7 +65,7 @@ BEGIN
|
|||
address.object_args,
|
||||
objects.distribution_argument_index,
|
||||
objects.colocationid
|
||||
FROM citus.pg_dist_object objects,
|
||||
FROM pg_catalog.pg_dist_object objects,
|
||||
pg_catalog.pg_identify_object_as_address(objects.classid, objects.objid, objects.objsubid) address;
|
||||
END;
|
||||
$cppu$;
|
||||
|
|
|
@ -86,7 +86,7 @@ extern void SyncDeleteColocationGroupToNodes(uint32 colocationId);
|
|||
#define DELETE_ALL_NODES "DELETE FROM pg_dist_node"
|
||||
#define DELETE_ALL_PLACEMENTS "DELETE FROM pg_dist_placement"
|
||||
#define DELETE_ALL_SHARDS "DELETE FROM pg_dist_shard"
|
||||
#define DELETE_ALL_DISTRIBUTED_OBJECTS "DELETE FROM citus.pg_dist_object"
|
||||
#define DELETE_ALL_DISTRIBUTED_OBJECTS "DELETE FROM pg_catalog.pg_dist_object"
|
||||
#define DELETE_ALL_PARTITIONS "DELETE FROM pg_dist_partition"
|
||||
#define DELETE_ALL_COLOCATION "DELETE FROM pg_catalog.pg_dist_colocation"
|
||||
#define REMOVE_ALL_SHELL_TABLES_COMMAND \
|
||||
|
|
|
@ -291,7 +291,7 @@ create aggregate binstragg(text, text)(
|
|||
stype=text
|
||||
);
|
||||
-- verify that the aggregate is added into pg_dist_object, on each worker
|
||||
SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.binstragg'::regproc;$$);
|
||||
SELECT run_command_on_workers($$SELECT count(*) from pg_catalog.pg_dist_object where objid = 'aggregate_support.binstragg'::regproc;$$);
|
||||
run_command_on_workers
|
||||
---------------------------------------------------------------------
|
||||
(localhost,57637,t,1)
|
||||
|
@ -1157,7 +1157,7 @@ SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid
|
|||
(2 rows)
|
||||
|
||||
-- verify that the aggregate is added into pg_dist_object, on each worker
|
||||
SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$);
|
||||
SELECT run_command_on_workers($$SELECT count(*) from pg_catalog.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$);
|
||||
run_command_on_workers
|
||||
---------------------------------------------------------------------
|
||||
(localhost,57637,t,1)
|
||||
|
@ -1168,7 +1168,7 @@ RESET citus.create_object_propagation;
|
|||
-- drop and test outside of tx block
|
||||
drop aggregate dependent_agg (float8);
|
||||
-- verify that the aggregate is removed from pg_dist_object, on each worker
|
||||
SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$);
|
||||
SELECT run_command_on_workers($$SELECT count(*) from pg_catalog.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$);
|
||||
run_command_on_workers
|
||||
---------------------------------------------------------------------
|
||||
(localhost,57637,f,"ERROR: function ""aggregate_support.dependent_agg"" does not exist")
|
||||
|
|
|
@ -77,7 +77,7 @@ SELECT create_distributed_table('t4','a');
|
|||
DROP TABLE t4; -- as long as the table is using the type some operations are hard to force
|
||||
COMMIT;
|
||||
-- verify the type is distributed
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = 'disabled_object_propagation.tt3'::regtype::oid;
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = 'disabled_object_propagation.tt3'::regtype::oid;
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
|
|
@ -195,7 +195,7 @@ SELECT create_distributed_function('"eq_mi''xed_param_names"(macaddr, macaddr)')
|
|||
|
||||
(1 row)
|
||||
|
||||
SELECT distribution_argument_index is NULL, colocationid is NULL from citus.pg_dist_object
|
||||
SELECT distribution_argument_index is NULL, colocationid is NULL from pg_catalog.pg_dist_object
|
||||
WHERE objid = 'eq_mi''xed_param_names(macaddr, macaddr)'::regprocedure;
|
||||
?column? | ?column?
|
||||
---------------------------------------------------------------------
|
||||
|
@ -652,7 +652,7 @@ SELECT create_distributed_function('eq_with_param_names(macaddr, macaddr)', '$1'
|
|||
|
||||
-- show that the colocationIds are the same
|
||||
SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated
|
||||
FROM pg_dist_partition, citus.pg_dist_object as objects
|
||||
FROM pg_dist_partition, pg_catalog.pg_dist_object as objects
|
||||
WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND
|
||||
objects.objid = 'eq_with_param_names(macaddr, macaddr)'::regprocedure;
|
||||
table_and_function_colocated
|
||||
|
@ -669,7 +669,7 @@ SELECT create_distributed_function('eq_with_param_names(macaddr, macaddr)', 'val
|
|||
(1 row)
|
||||
|
||||
SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated
|
||||
FROM pg_dist_partition, citus.pg_dist_object as objects
|
||||
FROM pg_dist_partition, pg_catalog.pg_dist_object as objects
|
||||
WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND
|
||||
objects.objid = 'eq_with_param_names(macaddr, macaddr)'::regprocedure;
|
||||
table_and_function_colocated
|
||||
|
@ -704,7 +704,7 @@ SELECT create_distributed_function('eq8(macaddr8, macaddr8)', '$1', colocate_wit
|
|||
(1 row)
|
||||
|
||||
SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated
|
||||
FROM pg_dist_partition, citus.pg_dist_object as objects
|
||||
FROM pg_dist_partition, pg_catalog.pg_dist_object as objects
|
||||
WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND
|
||||
objects.objid = 'eq8(macaddr8, macaddr8)'::regprocedure;
|
||||
table_and_function_colocated
|
||||
|
@ -719,7 +719,7 @@ SELECT create_distributed_function('add_text(text, text)', '$1', colocate_with:=
|
|||
(1 row)
|
||||
|
||||
SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated
|
||||
FROM pg_dist_partition, citus.pg_dist_object as objects
|
||||
FROM pg_dist_partition, pg_catalog.pg_dist_object as objects
|
||||
WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND
|
||||
objects.objid = 'add_text(text, text)'::regprocedure;
|
||||
table_and_function_colocated
|
||||
|
|
|
@ -14,39 +14,39 @@ BEGIN
|
|||
END;
|
||||
$$;
|
||||
-- Check all dependent objects and function depends on all nodes
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(schema,{function_propagation_schema},{})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(type,{function_propagation_schema.function_prop_type},{})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{function_propagation_schema,func_1}",{function_propagation_schema.function_prop_type})
|
||||
(1 row)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (schema,{function_propagation_schema},{})
|
||||
localhost | 57638 | t | (schema,{function_propagation_schema},{})
|
||||
(2 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (type,{function_propagation_schema.function_prop_type},{})
|
||||
localhost | 57638 | t | (type,{function_propagation_schema.function_prop_type},{})
|
||||
(2 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (function,"{function_propagation_schema,func_1}",{function_propagation_schema.function_prop_type})
|
||||
|
@ -64,26 +64,26 @@ BEGIN
|
|||
return 1;
|
||||
END;
|
||||
$$;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(type,{function_propagation_schema.function_prop_type_2},{})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{function_propagation_schema,func_2}",{integer})
|
||||
(1 row)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (type,{function_propagation_schema.function_prop_type_2},{})
|
||||
localhost | 57638 | t | (type,{function_propagation_schema.function_prop_type_2},{})
|
||||
(2 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (function,"{function_propagation_schema,func_2}",{integer})
|
||||
|
@ -105,18 +105,18 @@ BEGIN
|
|||
return 1;
|
||||
END;
|
||||
$$;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_3'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_3'::regtype::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{function_propagation_schema,func_3}",{integer})
|
||||
(1 row)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (function,"{function_propagation_schema,func_3}",{integer})
|
||||
|
@ -163,13 +163,13 @@ BEGIN
|
|||
return 1;
|
||||
END;
|
||||
$$;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{function_propagation_schema,func_6}",{function_propagation_schema.function_prop_table})
|
||||
(1 row)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (function,"{function_propagation_schema,func_6}",{function_propagation_schema.function_prop_table})
|
||||
|
@ -210,12 +210,12 @@ BEGIN;
|
|||
END;
|
||||
$$;
|
||||
-- Within transaction functions are not distributed
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -230,26 +230,26 @@ BEGIN
|
|||
return 1;
|
||||
END;
|
||||
$$;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(type,{function_propagation_schema.type_in_transaction},{})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{function_propagation_schema,func_in_transaction}",{function_propagation_schema.type_in_transaction})
|
||||
(1 row)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (type,{function_propagation_schema.type_in_transaction},{})
|
||||
localhost | 57638 | t | (type,{function_propagation_schema.type_in_transaction},{})
|
||||
(2 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction}",{function_propagation_schema.type_in_transaction})
|
||||
|
@ -265,18 +265,18 @@ $$
|
|||
SELECT max(id) FROM table_in_sql_body
|
||||
$$;
|
||||
-- Show that only function has propagated, since the table is not resolved as dependency
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regclass::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regclass::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{function_propagation_schema,max_of_table}",{})
|
||||
(1 row)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (function,"{function_propagation_schema,max_of_table}",{})
|
||||
|
@ -328,7 +328,7 @@ BEGIN;
|
|||
END;
|
||||
$$;
|
||||
-- Function shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -341,7 +341,7 @@ BEGIN;
|
|||
(1 row)
|
||||
|
||||
-- Function should be marked as distributed after distributing the table that depends on it
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{function_propagation_schema,func_in_transaction_def}",{})
|
||||
|
@ -349,7 +349,7 @@ BEGIN;
|
|||
|
||||
COMMIT;
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_def}",{})
|
||||
|
@ -375,12 +375,12 @@ BEGIN;
|
|||
END;
|
||||
$$;
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -393,13 +393,13 @@ BEGIN;
|
|||
(1 row)
|
||||
|
||||
-- Functions should be marked as distribued after distributing the table that depends on it
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{function_propagation_schema,func_in_transaction_1}",{})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{function_propagation_schema,func_in_transaction_2}",{})
|
||||
|
@ -407,14 +407,14 @@ BEGIN;
|
|||
|
||||
COMMIT;
|
||||
-- Functions should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_1}",{})
|
||||
localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_1}",{})
|
||||
(2 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_2}",{})
|
||||
|
@ -456,14 +456,14 @@ BEGIN;
|
|||
END;
|
||||
$$;
|
||||
-- Function shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
ALTER TABLE table_to_prop_func_4 ADD COLUMN col_1 int default function_propagation_schema.func_in_transaction_4();
|
||||
-- Function should be marked as distributed after adding the column
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{function_propagation_schema,func_in_transaction_4}",{})
|
||||
|
@ -471,7 +471,7 @@ BEGIN;
|
|||
|
||||
COMMIT;
|
||||
-- Functions should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_4}",{})
|
||||
|
@ -519,12 +519,12 @@ BEGIN;
|
|||
END;
|
||||
$$;
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -537,13 +537,13 @@ BEGIN;
|
|||
(1 row)
|
||||
|
||||
-- Functions should be marked as distributed after adding the column
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{function_propagation_schema,func_in_transaction_5}",{})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{function_propagation_schema,func_in_transaction_6}",{})
|
||||
|
@ -551,14 +551,14 @@ BEGIN;
|
|||
|
||||
COMMIT;
|
||||
-- Functions should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_5}",{})
|
||||
localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_5}",{})
|
||||
(2 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_6}",{})
|
||||
|
@ -576,7 +576,7 @@ BEGIN;
|
|||
END;
|
||||
$$;
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -589,7 +589,7 @@ BEGIN;
|
|||
(1 row)
|
||||
|
||||
-- Function should be marked as distributed after adding the column
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{function_propagation_schema,func_in_transaction_7}",{integer})
|
||||
|
@ -597,7 +597,7 @@ BEGIN;
|
|||
|
||||
COMMIT;
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_7}",{integer})
|
||||
|
@ -623,12 +623,12 @@ BEGIN;
|
|||
END;
|
||||
$$;
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -641,13 +641,13 @@ BEGIN;
|
|||
(1 row)
|
||||
|
||||
-- Function should be marked as distributed after adding the column
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{function_propagation_schema,func_in_transaction_8}",{integer})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{function_propagation_schema,func_in_transaction_9}",{integer})
|
||||
|
@ -655,14 +655,14 @@ BEGIN;
|
|||
|
||||
COMMIT;
|
||||
-- Functions should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_8}",{integer})
|
||||
localhost | 57638 | t | (function,"{function_propagation_schema,func_in_transaction_8}",{integer})
|
||||
(2 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_9}",{integer})
|
||||
|
@ -687,14 +687,14 @@ BEGIN;
|
|||
END;
|
||||
$$;
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
ALTER TABLE table_to_prop_func_8 ADD CONSTRAINT col1_check CHECK (function_propagation_schema.func_in_transaction_10(col_1));
|
||||
-- Function should be marked as distributed after adding the constraint
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{function_propagation_schema,func_in_transaction_10}",{integer})
|
||||
|
@ -702,7 +702,7 @@ BEGIN;
|
|||
|
||||
COMMIT;
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_10}",{integer})
|
||||
|
@ -737,7 +737,7 @@ BEGIN;
|
|||
END;
|
||||
$$;
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_generated_always'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_generated_always'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -753,7 +753,7 @@ BEGIN;
|
|||
(1 row)
|
||||
|
||||
-- Show that function is distributed after distributing the table
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_generated_always'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_generated_always'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{function_propagation_schema,non_sense_func_for_generated_always}",{})
|
||||
|
@ -771,7 +771,7 @@ BEGIN;
|
|||
END;
|
||||
$$;
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -786,7 +786,7 @@ BEGIN;
|
|||
(1 row)
|
||||
|
||||
-- Functions should be distributed after distributing the table
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{function_propagation_schema,func_for_rule}",{})
|
||||
|
@ -794,7 +794,7 @@ BEGIN;
|
|||
|
||||
COMMIT;
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (function,"{function_propagation_schema,func_for_rule}",{})
|
||||
|
@ -812,7 +812,7 @@ BEGIN;
|
|||
END;
|
||||
$$;
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -825,7 +825,7 @@ BEGIN;
|
|||
(1 row)
|
||||
|
||||
-- Show that function is distributed after distributing the table
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{function_propagation_schema,non_sense_func_for_partitioning}",{integer})
|
||||
|
@ -833,7 +833,7 @@ BEGIN;
|
|||
|
||||
COMMIT;
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (function,"{function_propagation_schema,non_sense_func_for_partitioning}",{integer})
|
||||
|
@ -851,7 +851,7 @@ BEGIN;
|
|||
END;
|
||||
$$;
|
||||
-- Function shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_for_local_table'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_for_local_table'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -871,7 +871,7 @@ NOTICE: localhost:xxxxx is the coordinator and already contains metadata, skipp
|
|||
(1 row)
|
||||
|
||||
-- Function should be marked as distributed after distributing the table that depends on it
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_for_local_table'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_for_local_table'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{function_propagation_schema,func_in_transaction_for_local_table}",{})
|
||||
|
@ -889,7 +889,7 @@ BEGIN;
|
|||
END;
|
||||
$$;
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -902,7 +902,7 @@ BEGIN;
|
|||
(1 row)
|
||||
|
||||
-- Function should be marked as distributed after distributing the table that depends on it
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{function_propagation_schema,exclude_bool_func}",{})
|
||||
|
@ -910,7 +910,7 @@ BEGIN;
|
|||
|
||||
COMMIT;
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (function,"{function_propagation_schema,exclude_bool_func}",{})
|
||||
|
@ -928,7 +928,7 @@ BEGIN;
|
|||
END;
|
||||
$$;
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -942,7 +942,7 @@ BEGIN;
|
|||
(1 row)
|
||||
|
||||
-- Function should be marked as distributed after distributing the table that depends on it
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{function_propagation_schema,func_for_index_predicate}",{integer})
|
||||
|
@ -950,7 +950,7 @@ BEGIN;
|
|||
|
||||
COMMIT;
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (function,"{function_propagation_schema,func_for_index_predicate}",{integer})
|
||||
|
@ -983,7 +983,7 @@ BEGIN;
|
|||
(1 row)
|
||||
|
||||
-- Function should be marked as distributed after distributing the table that depends on it
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_func_dep_1'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_for_func_dep_1'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{function_propagation_schema,func_for_func_dep_1}",{})
|
||||
|
@ -991,7 +991,7 @@ BEGIN;
|
|||
|
||||
COMMIT;
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_func_dep_1'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_for_func_dep_1'::regproc::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (function,"{function_propagation_schema,func_for_func_dep_1}",{})
|
||||
|
@ -1013,7 +1013,7 @@ BEGIN;
|
|||
SELECT func_in_transaction_def_with_seq(val);
|
||||
$$;
|
||||
-- Function shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -1027,7 +1027,7 @@ BEGIN;
|
|||
(1 row)
|
||||
|
||||
-- Function should be marked as distributed after distributing the table that depends on it
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{function_propagation_schema,func_in_transaction_def_with_seq}",{bigint})
|
||||
|
@ -1035,7 +1035,7 @@ BEGIN;
|
|||
|
||||
COMMIT;
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (function,"{function_propagation_schema,func_in_transaction_def_with_seq}",{bigint})
|
||||
|
@ -1089,7 +1089,7 @@ SELECT create_reference_table('tbl_to_colocate_ref');
|
|||
|
||||
CREATE FUNCTION func_to_colocate (a int) returns int as $$select 1;$$ language sql;
|
||||
-- see the empty pg_dist_object entries
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM citus.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
distribution_argument_index | colocationid | force_delegation
|
||||
---------------------------------------------------------------------
|
||||
| |
|
||||
|
@ -1103,7 +1103,7 @@ SELECT create_distributed_function('func_to_colocate(int)', colocate_with:='tbl_
|
|||
(1 row)
|
||||
|
||||
-- see the pg_dist_object entry
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM citus.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
distribution_argument_index | colocationid | force_delegation
|
||||
---------------------------------------------------------------------
|
||||
| 10003 |
|
||||
|
@ -1117,7 +1117,7 @@ SELECT create_distributed_function('func_to_colocate(int)');
|
|||
(1 row)
|
||||
|
||||
-- show that the pg_dist_object fields are gone
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM citus.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
distribution_argument_index | colocationid | force_delegation
|
||||
---------------------------------------------------------------------
|
||||
| |
|
||||
|
@ -1131,7 +1131,7 @@ SELECT create_distributed_function('func_to_colocate(int)','$1','tbl_to_colocate
|
|||
(1 row)
|
||||
|
||||
-- see the pg_dist_object entry
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM citus.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
distribution_argument_index | colocationid | force_delegation
|
||||
---------------------------------------------------------------------
|
||||
0 | 10005 |
|
||||
|
@ -1140,7 +1140,7 @@ SELECT distribution_argument_index, colocationid, force_delegation FROM citus.pg
|
|||
-- try create or replace the same func
|
||||
CREATE OR REPLACE FUNCTION func_to_colocate (a int) returns int as $$select 1;$$ language sql;
|
||||
-- verify the pg_dist_object entry is the same
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM citus.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
distribution_argument_index | colocationid | force_delegation
|
||||
---------------------------------------------------------------------
|
||||
0 | 10005 |
|
||||
|
@ -1154,7 +1154,7 @@ SELECT create_distributed_function('func_to_colocate(int)');
|
|||
(1 row)
|
||||
|
||||
-- show that the pg_dist_object fields are gone
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM citus.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
distribution_argument_index | colocationid | force_delegation
|
||||
---------------------------------------------------------------------
|
||||
| |
|
||||
|
@ -1168,7 +1168,7 @@ SELECT create_distributed_function('func_to_colocate(int)','$1','tbl_to_colocate
|
|||
(1 row)
|
||||
|
||||
-- show pg_dist_object fields
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM citus.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
distribution_argument_index | colocationid | force_delegation
|
||||
---------------------------------------------------------------------
|
||||
0 | 10005 | t
|
||||
|
@ -1182,7 +1182,7 @@ SELECT create_distributed_function('func_to_colocate(int)');
|
|||
(1 row)
|
||||
|
||||
-- show that the pg_dist_object fields are gone
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM citus.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
distribution_argument_index | colocationid | force_delegation
|
||||
---------------------------------------------------------------------
|
||||
| |
|
||||
|
|
|
@ -9,7 +9,7 @@ starting permutation: s1-print-distributed-objects s1-begin s1-add-worker s2-pub
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -102,7 +102,7 @@ create_distributed_table
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -180,7 +180,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-begin s1-add-work
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -279,7 +279,7 @@ step s2-commit:
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -357,7 +357,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-begin s2-public-s
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -456,7 +456,7 @@ step s1-commit:
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -534,7 +534,7 @@ starting permutation: s1-print-distributed-objects s1-begin s1-add-worker s2-cre
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -628,7 +628,7 @@ create_distributed_table
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -707,7 +707,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-begin s1-add-work
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -807,7 +807,7 @@ step s2-commit:
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -886,7 +886,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-begin s2-create-s
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -986,7 +986,7 @@ step s1-commit:
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -1065,7 +1065,7 @@ starting permutation: s1-print-distributed-objects s2-create-schema s1-begin s2-
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -1165,7 +1165,7 @@ step s2-commit:
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -1244,7 +1244,7 @@ starting permutation: s1-print-distributed-objects s1-add-worker s2-create-schem
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -1358,7 +1358,7 @@ step s3-commit:
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -1438,7 +1438,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-begin s3-begin s1
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -1560,7 +1560,7 @@ step s2-commit:
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -1641,7 +1641,7 @@ starting permutation: s1-print-distributed-objects s1-begin s1-add-worker s2-pub
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -1726,7 +1726,7 @@ step s1-commit:
|
|||
step s2-create-type: <... completed>
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -1804,7 +1804,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-public-schema s2-
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -1888,7 +1888,7 @@ step s1-commit:
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -1966,7 +1966,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-begin s2-create-s
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -2069,7 +2069,7 @@ step s1-commit:
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -2149,7 +2149,7 @@ starting permutation: s1-print-distributed-objects s1-begin s1-add-worker s2-pub
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -2254,7 +2254,7 @@ wait_until_metadata_sync
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -2332,7 +2332,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-public-schema s2-
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -2444,7 +2444,7 @@ wait_until_metadata_sync
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -2522,7 +2522,7 @@ starting permutation: s1-print-distributed-objects s2-begin s2-create-schema s2-
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -2635,7 +2635,7 @@ wait_until_metadata_sync
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
|
|
@ -20,7 +20,7 @@ step s1-commit:
|
|||
|
||||
step s2-create-extension-version-11: <... completed>
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
@ -81,7 +81,7 @@ step s1-commit:
|
|||
|
||||
step s2-alter-extension-update-to-version-12: <... completed>
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
@ -150,7 +150,7 @@ step s1-commit:
|
|||
|
||||
step s2-drop-extension: <... completed>
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
@ -206,7 +206,7 @@ step s1-commit:
|
|||
|
||||
step s2-create-extension-with-schema1: <... completed>
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
@ -267,7 +267,7 @@ step s1-commit:
|
|||
|
||||
step s2-drop-extension: <... completed>
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
@ -338,7 +338,7 @@ step s1-commit:
|
|||
|
||||
step s2-alter-extension-set-schema3: <... completed>
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
@ -406,7 +406,7 @@ step s1-commit:
|
|||
|
||||
step s2-create-extension-with-schema1: <... completed>
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
@ -481,7 +481,7 @@ step s2-commit:
|
|||
COMMIT;
|
||||
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
@ -564,7 +564,7 @@ step s1-add-node-1: <... completed>
|
|||
(1 row)
|
||||
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
@ -633,7 +633,7 @@ step s1-remove-node-1: <... completed>
|
|||
(1 row)
|
||||
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
@ -688,7 +688,7 @@ step s2-commit:
|
|||
COMMIT;
|
||||
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
@ -763,7 +763,7 @@ step s1-remove-node-1: <... completed>
|
|||
(1 row)
|
||||
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
@ -830,7 +830,7 @@ step s2-commit:
|
|||
COMMIT;
|
||||
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
@ -909,7 +909,7 @@ step s1-add-node-1: <... completed>
|
|||
(1 row)
|
||||
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
|
|
@ -110,8 +110,8 @@ SELECT count(*) FROM history;
|
|||
-- test we can replicate MX tables
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
-- metadata sync will succeed even if we have rep > 1 tables
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles_single_shard'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles_single_shard'::regclass::oid, 0);
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
start_metadata_sync_to_node
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -242,8 +242,8 @@ SELECT master_remove_node('localhost', :worker_2_port);
|
|||
(1 row)
|
||||
|
||||
-- Removing public schema from pg_dist_object because it breaks the next tests
|
||||
DELETE FROM citus.pg_dist_object WHERE objid = 'public'::regnamespace::oid;
|
||||
DELETE FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'plpgsql');
|
||||
DELETE FROM pg_catalog.pg_dist_object WHERE objid = 'public'::regnamespace::oid;
|
||||
DELETE FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'plpgsql');
|
||||
-- try to manipulate node metadata via non-super user
|
||||
SET ROLE non_super_user;
|
||||
SELECT 1 FROM master_add_inactive_node('localhost', :worker_2_port + 1);
|
||||
|
|
|
@ -1048,11 +1048,11 @@ SELECT update_distributed_table_colocation('table1_group_none', colocate_with =>
|
|||
(1 row)
|
||||
|
||||
-- activate nodes to get rid of inconsistencies in pg_dist tables
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table1_group1'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table2_group1'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table3_group2'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table4_group2'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table5_groupX'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table1_group1'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table2_group1'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table3_group2'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table4_group2'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table5_groupX'::regclass::oid, 0);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -492,30 +492,17 @@ SELECT * FROM multi_extension.print_extension_changes();
|
|||
|
||||
-- Test downgrade to 9.4-1 from 9.5-1
|
||||
ALTER EXTENSION citus UPDATE TO '9.5-1';
|
||||
BEGIN;
|
||||
SET citus.enable_metadata_sync TO on;
|
||||
SELECT master_add_node('localhost', :master_port, groupId=>0);
|
||||
NOTICE: localhost:xxxxx is the coordinator and already contains metadata, skipping syncing the metadata
|
||||
master_add_node
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
CREATE TABLE citus_local_table (a int);
|
||||
SELECT create_citus_local_table('citus_local_table');
|
||||
NOTICE: create_citus_local_table is deprecated in favour of citus_add_local_table_to_metadata
|
||||
create_citus_local_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
RESET citus.enable_metadata_sync;
|
||||
-- downgrade from 9.5-1 to 9.4-1 should fail as we have a citus local table
|
||||
ALTER EXTENSION citus UPDATE TO '9.4-1';
|
||||
ERROR: citus local tables are introduced in Citus 9.5
|
||||
HINT: To downgrade Citus to an older version, you should first convert each citus local table to a postgres table by executing SELECT undistribute_table("%s")
|
||||
CONTEXT: PL/pgSQL function inline_code_block line XX at RAISE
|
||||
ROLLBACK;
|
||||
-- TODO: This test should be moved to a valid downgrade testing suite where the downgrade is done, both on the schema and the binaries. Later changes in Citus made a C vs Schema discrepancy error here
|
||||
-- BEGIN;
|
||||
-- SET citus.enable_metadata_sync TO on;
|
||||
-- SELECT master_add_node('localhost', :master_port, groupId=>0);
|
||||
-- CREATE TABLE citus_local_table (a int);
|
||||
-- SELECT create_citus_local_table('citus_local_table');
|
||||
-- RESET citus.enable_metadata_sync;
|
||||
--
|
||||
-- -- downgrade from 9.5-1 to 9.4-1 should fail as we have a citus local table
|
||||
-- ALTER EXTENSION citus UPDATE TO '9.4-1';
|
||||
-- ROLLBACK;
|
||||
-- now we can downgrade as there is no citus local table
|
||||
ALTER EXTENSION citus UPDATE TO '9.4-1';
|
||||
-- Should be empty result since upgrade+downgrade should be a no-op
|
||||
|
@ -1009,6 +996,7 @@ SELECT * FROM multi_extension.print_extension_changes();
|
|||
function master_apply_delete_command(text) integer |
|
||||
function master_get_table_metadata(text) record |
|
||||
function worker_partition_query_result(text,text,integer,citus.distribution_type,text[],text[],boolean) SETOF record |
|
||||
table citus.pg_dist_object |
|
||||
view citus_worker_stat_activity |
|
||||
| function citus_backend_gpid() bigint
|
||||
| function citus_calculate_gpid(integer,integer) bigint
|
||||
|
@ -1033,8 +1021,9 @@ SELECT * FROM multi_extension.print_extension_changes();
|
|||
| function worker_drop_sequence_dependency(text) void
|
||||
| function worker_drop_shell_table(text) void
|
||||
| function worker_partition_query_result(text,text,integer,citus.distribution_type,text[],text[],boolean,boolean,boolean) SETOF record
|
||||
| table pg_dist_object
|
||||
| view citus_stat_activity
|
||||
(33 rows)
|
||||
(35 rows)
|
||||
|
||||
DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff;
|
||||
-- show running version
|
||||
|
|
|
@ -9,7 +9,7 @@ FROM pg_attribute
|
|||
WHERE atthasmissing AND attrelid NOT IN ('pg_dist_node'::regclass,
|
||||
'pg_dist_rebalance_strategy'::regclass,
|
||||
'pg_dist_partition'::regclass,
|
||||
'citus.pg_dist_object'::regclass)
|
||||
'pg_dist_object'::regclass)
|
||||
ORDER BY attrelid, attname;
|
||||
attrelid | attname | atthasmissing | attmissingval
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -62,8 +62,8 @@ SELECT unnest(activate_node_snapshot()) order by 1;
|
|||
unnest
|
||||
---------------------------------------------------------------------
|
||||
CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION postgres
|
||||
DELETE FROM citus.pg_dist_object
|
||||
DELETE FROM pg_catalog.pg_dist_colocation
|
||||
DELETE FROM pg_catalog.pg_dist_object
|
||||
DELETE FROM pg_dist_node
|
||||
DELETE FROM pg_dist_partition
|
||||
DELETE FROM pg_dist_placement
|
||||
|
@ -131,8 +131,8 @@ SELECT unnest(activate_node_snapshot()) order by 1;
|
|||
ALTER TABLE public.mx_test_table OWNER TO postgres
|
||||
CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION postgres
|
||||
CREATE TABLE public.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('public.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass))
|
||||
DELETE FROM citus.pg_dist_object
|
||||
DELETE FROM pg_catalog.pg_dist_colocation
|
||||
DELETE FROM pg_catalog.pg_dist_object
|
||||
DELETE FROM pg_dist_node
|
||||
DELETE FROM pg_dist_partition
|
||||
DELETE FROM pg_dist_placement
|
||||
|
@ -180,8 +180,8 @@ SELECT unnest(activate_node_snapshot()) order by 1;
|
|||
CREATE INDEX mx_index ON public.mx_test_table USING btree (col_2)
|
||||
CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION postgres
|
||||
CREATE TABLE public.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('public.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass))
|
||||
DELETE FROM citus.pg_dist_object
|
||||
DELETE FROM pg_catalog.pg_dist_colocation
|
||||
DELETE FROM pg_catalog.pg_dist_object
|
||||
DELETE FROM pg_dist_node
|
||||
DELETE FROM pg_dist_partition
|
||||
DELETE FROM pg_dist_placement
|
||||
|
@ -231,8 +231,8 @@ SELECT unnest(activate_node_snapshot()) order by 1;
|
|||
CREATE SCHEMA IF NOT EXISTS mx_testing_schema AUTHORIZATION postgres
|
||||
CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION postgres
|
||||
CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass))
|
||||
DELETE FROM citus.pg_dist_object
|
||||
DELETE FROM pg_catalog.pg_dist_colocation
|
||||
DELETE FROM pg_catalog.pg_dist_object
|
||||
DELETE FROM pg_dist_node
|
||||
DELETE FROM pg_dist_partition
|
||||
DELETE FROM pg_dist_placement
|
||||
|
@ -288,8 +288,8 @@ SELECT unnest(activate_node_snapshot()) order by 1;
|
|||
CREATE SCHEMA IF NOT EXISTS mx_testing_schema AUTHORIZATION postgres
|
||||
CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION postgres
|
||||
CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass))
|
||||
DELETE FROM citus.pg_dist_object
|
||||
DELETE FROM pg_catalog.pg_dist_colocation
|
||||
DELETE FROM pg_catalog.pg_dist_object
|
||||
DELETE FROM pg_dist_node
|
||||
DELETE FROM pg_dist_partition
|
||||
DELETE FROM pg_dist_placement
|
||||
|
@ -338,8 +338,8 @@ SELECT unnest(activate_node_snapshot()) order by 1;
|
|||
CREATE SCHEMA IF NOT EXISTS mx_testing_schema AUTHORIZATION postgres
|
||||
CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION postgres
|
||||
CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass))
|
||||
DELETE FROM citus.pg_dist_object
|
||||
DELETE FROM pg_catalog.pg_dist_colocation
|
||||
DELETE FROM pg_catalog.pg_dist_object
|
||||
DELETE FROM pg_dist_node
|
||||
DELETE FROM pg_dist_partition
|
||||
DELETE FROM pg_dist_placement
|
||||
|
@ -1379,10 +1379,10 @@ DROP TABLE mx_table_with_small_sequence, mx_table_with_sequence;
|
|||
-- owner
|
||||
CREATE TABLE pg_dist_placement_temp AS SELECT * FROM pg_dist_placement;
|
||||
CREATE TABLE pg_dist_partition_temp AS SELECT * FROM pg_dist_partition;
|
||||
CREATE TABLE pg_dist_object_temp AS SELECT * FROM citus.pg_dist_object;
|
||||
CREATE TABLE pg_dist_object_temp AS SELECT * FROM pg_catalog.pg_dist_object;
|
||||
DELETE FROM pg_dist_placement;
|
||||
DELETE FROM pg_dist_partition;
|
||||
DELETE FROM citus.pg_dist_object;
|
||||
DELETE FROM pg_catalog.pg_dist_object;
|
||||
SELECT groupid AS old_worker_2_group FROM pg_dist_node WHERE nodeport = :worker_2_port \gset
|
||||
SELECT master_remove_node('localhost', :worker_2_port);
|
||||
master_remove_node
|
||||
|
@ -1459,7 +1459,7 @@ DROP TABLE mx_table;
|
|||
\c - postgres - :master_port
|
||||
INSERT INTO pg_dist_placement SELECT * FROM pg_dist_placement_temp;
|
||||
INSERT INTO pg_dist_partition SELECT * FROM pg_dist_partition_temp;
|
||||
INSERT INTO citus.pg_dist_object SELECT * FROM pg_dist_object_temp ON CONFLICT ON CONSTRAINT pg_dist_object_pkey DO NOTHING;
|
||||
INSERT INTO pg_catalog.pg_dist_object SELECT * FROM pg_dist_object_temp ON CONFLICT ON CONSTRAINT pg_dist_object_pkey DO NOTHING;
|
||||
DROP TABLE pg_dist_placement_temp;
|
||||
DROP TABLE pg_dist_partition_temp;
|
||||
DROP TABLE pg_dist_object_temp;
|
||||
|
@ -1875,8 +1875,8 @@ SELECT unnest(activate_node_snapshot()) order by 1;
|
|||
CREATE TABLE public.dist_table_1 (a integer)
|
||||
CREATE TABLE public.mx_ref (col_1 integer, col_2 text)
|
||||
CREATE TABLE public.test_table (id integer DEFAULT worker_nextval('public.mx_test_sequence_0'::regclass), id2 integer DEFAULT worker_nextval('public.mx_test_sequence_1'::regclass))
|
||||
DELETE FROM citus.pg_dist_object
|
||||
DELETE FROM pg_catalog.pg_dist_colocation
|
||||
DELETE FROM pg_catalog.pg_dist_object
|
||||
DELETE FROM pg_dist_node
|
||||
DELETE FROM pg_dist_partition
|
||||
DELETE FROM pg_dist_placement
|
||||
|
|
|
@ -191,7 +191,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg
|
|||
test_proc_colocation_0 | 1410002
|
||||
(1 row)
|
||||
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
proname | colocationid
|
||||
---------------------------------------------------------------------
|
||||
proc_0 | 1410002
|
||||
|
@ -226,7 +226,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg
|
|||
test_proc_colocation_0 | 1410003
|
||||
(1 row)
|
||||
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
proname | colocationid
|
||||
---------------------------------------------------------------------
|
||||
proc_0 | 1410003
|
||||
|
@ -272,7 +272,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg
|
|||
test_proc_colocation_0 | 1410004
|
||||
(1 row)
|
||||
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
proname | colocationid
|
||||
---------------------------------------------------------------------
|
||||
proc_0 | 1410004
|
||||
|
@ -305,7 +305,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg
|
|||
test_proc_colocation_0 | 1410003
|
||||
(1 row)
|
||||
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
proname | colocationid
|
||||
---------------------------------------------------------------------
|
||||
proc_0 | 1410003
|
||||
|
@ -359,7 +359,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg
|
|||
test_proc_colocation_0 | 1410005
|
||||
(1 row)
|
||||
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
proname | colocationid
|
||||
---------------------------------------------------------------------
|
||||
proc_0 | 1410005
|
||||
|
@ -389,7 +389,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg
|
|||
test_proc_colocation_0 | 1410005
|
||||
(1 row)
|
||||
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname;
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname;
|
||||
proname | colocationid
|
||||
---------------------------------------------------------------------
|
||||
proc_0 | 1410005
|
||||
|
@ -440,7 +440,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg
|
|||
test_proc_colocation_0 | 1410003
|
||||
(1 row)
|
||||
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname;
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname;
|
||||
proname | colocationid
|
||||
---------------------------------------------------------------------
|
||||
proc_0 | 1410003
|
||||
|
@ -465,7 +465,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg
|
|||
test_proc_colocation_0 | 1410006
|
||||
(1 row)
|
||||
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname;
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname;
|
||||
proname | colocationid
|
||||
---------------------------------------------------------------------
|
||||
proc_0 | 1410003
|
||||
|
|
|
@ -459,7 +459,7 @@ 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
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid::oid::regnamespace IN ('mx_old_schema', 'mx_new_schema')
|
||||
ORDER BY "Distributed Schemas";
|
||||
Distributed Schemas
|
||||
|
@ -491,7 +491,7 @@ HINT: Connect to the coordinator and run it again.
|
|||
\c - - - :master_port
|
||||
ALTER TABLE mx_old_schema.table_set_schema SET SCHEMA mx_new_schema;
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid::oid::regnamespace IN ('mx_old_schema', 'mx_new_schema');
|
||||
Distributed Schemas
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -884,7 +884,7 @@ SELECT create_distributed_table('old_schema.table_set_schema', 'id');
|
|||
|
||||
CREATE SCHEMA new_schema;
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema');
|
||||
Distributed Schemas
|
||||
---------------------------------------------------------------------
|
||||
|
@ -906,7 +906,7 @@ SELECT table_schema AS "Shards' Schema"
|
|||
\c - - - :master_port
|
||||
ALTER TABLE old_schema.table_set_schema SET SCHEMA new_schema;
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema');
|
||||
Distributed Schemas
|
||||
---------------------------------------------------------------------
|
||||
|
@ -944,7 +944,7 @@ SELECT create_distributed_table('table_set_schema', 'id');
|
|||
|
||||
CREATE SCHEMA new_schema;
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid='new_schema'::regnamespace::oid;
|
||||
Distributed Schemas
|
||||
---------------------------------------------------------------------
|
||||
|
@ -965,7 +965,7 @@ SELECT table_schema AS "Shards' Schema"
|
|||
\c - - - :master_port
|
||||
ALTER TABLE table_set_schema SET SCHEMA new_schema;
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid='new_schema'::regnamespace::oid;
|
||||
Distributed Schemas
|
||||
---------------------------------------------------------------------
|
||||
|
@ -1010,7 +1010,7 @@ SELECT create_distributed_table('table_set_schema', 'id');
|
|||
|
||||
CREATE SCHEMA new_schema;
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema');
|
||||
Distributed Schemas
|
||||
---------------------------------------------------------------------
|
||||
|
@ -1034,7 +1034,7 @@ SELECT table_schema AS "Shards' Schema", COUNT(*) AS "Counts"
|
|||
SET search_path TO old_schema;
|
||||
ALTER TABLE table_set_schema SET SCHEMA new_schema;
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema');
|
||||
Distributed Schemas
|
||||
---------------------------------------------------------------------
|
||||
|
@ -1392,7 +1392,7 @@ BEGIN;
|
|||
ROLLBACK;
|
||||
-- Clean up the created schema
|
||||
SET client_min_messages TO WARNING;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object
|
||||
WHERE classid=2615 and objid IN (select oid from pg_namespace where nspname='run_test_schema');
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
|
@ -1401,7 +1401,7 @@ SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dis
|
|||
|
||||
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
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object
|
||||
WHERE classid=2615 and objid IN (select oid from pg_namespace where nspname='run_test_schema');
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -852,13 +852,13 @@ SELECT create_distributed_table('test_seq_dist', 'a');
|
|||
(1 row)
|
||||
|
||||
-- Both sequence and dependency schema should be distributed
|
||||
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 pg_catalog.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation.seq_10'::regclass);
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(sequence,"{test_schema_for_sequence_default_propagation,seq_10}",{})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation'::regnamespace);
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation'::regnamespace);
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(schema,{test_schema_for_sequence_default_propagation},{})
|
||||
|
|
|
@ -83,7 +83,7 @@ END;$$;
|
|||
CREATE OR REPLACE FUNCTION colocate_proc_with_table(procname text, tablerelid regclass, argument_index int)
|
||||
RETURNS void LANGUAGE plpgsql AS $$
|
||||
BEGIN
|
||||
update citus.pg_dist_object
|
||||
update pg_catalog.pg_dist_object
|
||||
set distribution_argument_index = argument_index, colocationid = pg_dist_partition.colocationid
|
||||
from pg_proc, pg_dist_partition
|
||||
where proname = procname and oid = objid and pg_dist_partition.logicalrelid = tablerelid;
|
||||
|
|
|
@ -12,27 +12,27 @@ CREATE TYPE mood AS ENUM ('sad', 'ok', 'happy');
|
|||
CREATE SCHEMA test_sequence_schema;
|
||||
CREATE SEQUENCE test_sequence_schema.test_sequence;
|
||||
-- show that none of the objects above are marked as distributed
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema'::regnamespace::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema'::regnamespace::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.mood'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.mood'::regtype::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -97,105 +97,105 @@ SELECT create_distributed_function('plpgsql_dist_function(text)');
|
|||
-- show that schema, types, function and sequence has marked as distributed
|
||||
-- on the coordinator node
|
||||
RESET ROLE;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema'::regnamespace::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema'::regnamespace::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(schema,{local_schema},{})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'test_sequence_schema'::regnamespace::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'test_sequence_schema'::regnamespace::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(schema,{test_sequence_schema},{})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.mood'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.mood'::regtype::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(type,{local_schema.mood},{})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(type,{local_schema.test_type},{})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(sequence,"{test_sequence_schema,test_sequence}",{})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.dist_table_e_seq'::regclass::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.dist_table_e_seq'::regclass::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(sequence,"{local_schema,dist_table_e_seq}",{})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{local_schema,test_function}",{integer})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.plpgsql_dist_function'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.plpgsql_dist_function'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{local_schema,plpgsql_dist_function}",{pg_catalog.text})
|
||||
(1 row)
|
||||
|
||||
-- show those objects marked as distributed on metadata worker node as well
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema'::regnamespace::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema'::regnamespace::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (schema,{local_schema},{})
|
||||
localhost | 57638 | t | (schema,{local_schema},{})
|
||||
(2 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'test_sequence_schema'::regnamespace::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'test_sequence_schema'::regnamespace::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (schema,{test_sequence_schema},{})
|
||||
localhost | 57638 | t | (schema,{test_sequence_schema},{})
|
||||
(2 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.mood'::regtype::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.mood'::regtype::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (type,{local_schema.mood},{})
|
||||
localhost | 57638 | t | (type,{local_schema.mood},{})
|
||||
(2 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (type,{local_schema.test_type},{})
|
||||
localhost | 57638 | t | (type,{local_schema.test_type},{})
|
||||
(2 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (sequence,"{test_sequence_schema,test_sequence}",{})
|
||||
localhost | 57638 | t | (sequence,"{test_sequence_schema,test_sequence}",{})
|
||||
(2 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.dist_table_e_seq'::regclass::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.dist_table_e_seq'::regclass::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (sequence,"{local_schema,dist_table_e_seq}",{})
|
||||
localhost | 57638 | t | (sequence,"{local_schema,dist_table_e_seq}",{})
|
||||
(2 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (function,"{local_schema,test_function}",{integer})
|
||||
localhost | 57638 | t | (function,"{local_schema,test_function}",{integer})
|
||||
(2 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.plpgsql_dist_function'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.plpgsql_dist_function'::regproc::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (function,"{local_schema,plpgsql_dist_function}",{pg_catalog.text})
|
||||
|
@ -203,13 +203,13 @@ SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(clas
|
|||
(2 rows)
|
||||
|
||||
-- Show that extension plpgsql is also marked as distributed as a dependency of plpgsl_dist_function
|
||||
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 '%{plpgsql}%';
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{plpgsql}%';
|
||||
obj_identifier
|
||||
---------------------------------------------------------------------
|
||||
(extension,{plpgsql},{})
|
||||
(1 row)
|
||||
|
||||
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 '%{plpgsql}%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{plpgsql}%';$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (extension,{plpgsql},{})
|
||||
|
@ -278,12 +278,12 @@ SET citus.enable_ddl_propagation TO ON;
|
|||
DROP TABLE dist_table CASCADE;
|
||||
RESET ROLE;
|
||||
SET search_path TO local_schema;
|
||||
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 '%dist_table_e_seq%';
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%dist_table_e_seq%';
|
||||
obj_identifier
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
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 '%dist_table_e_seq%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%dist_table_e_seq%';$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t |
|
||||
|
@ -293,13 +293,13 @@ SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_
|
|||
-- Show that altering the function's schema marks the schema distributed
|
||||
CREATE SCHEMA schema_to_prop_with_function;
|
||||
ALTER FUNCTION test_function SET SCHEMA schema_to_prop_with_function;
|
||||
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 '%{schema_to_prop_with_function}%';
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_function}%';
|
||||
obj_identifier
|
||||
---------------------------------------------------------------------
|
||||
(schema,{schema_to_prop_with_function},{})
|
||||
(1 row)
|
||||
|
||||
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 '%{schema_to_prop_with_function}%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_function}%';$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (schema,{schema_to_prop_with_function},{})
|
||||
|
@ -309,12 +309,12 @@ SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_
|
|||
-- Show that dropping the function removes the metadata from pg_dist_object
|
||||
-- on both coordinator and metadata worker node
|
||||
DROP FUNCTION schema_to_prop_with_function.test_function;
|
||||
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 '%test_function%';
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_function%';
|
||||
obj_identifier
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
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 '%test_function%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_function%';$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t |
|
||||
|
@ -324,13 +324,13 @@ SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_
|
|||
-- Show that altering the type's schema marks the schema distributed
|
||||
CREATE SCHEMA schema_to_prop_with_type;
|
||||
ALTER TYPE test_type SET SCHEMA schema_to_prop_with_type;
|
||||
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 '%{schema_to_prop_with_type}%';
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_type}%';
|
||||
obj_identifier
|
||||
---------------------------------------------------------------------
|
||||
(schema,{schema_to_prop_with_type},{})
|
||||
(1 row)
|
||||
|
||||
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 '%{schema_to_prop_with_type}%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_type}%';$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (schema,{schema_to_prop_with_type},{})
|
||||
|
@ -341,12 +341,12 @@ SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_
|
|||
-- on both coordinator and metadata worker node
|
||||
DROP TYPE mood CASCADE;
|
||||
DROP TYPE schema_to_prop_with_type.test_type 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 '%test_type%' or obj_identifier::text like '%mood%';
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_type%' or obj_identifier::text like '%mood%';
|
||||
obj_identifier
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
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 '%test_type%' or obj_identifier::text like '%mood%'$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_type%' or obj_identifier::text like '%mood%'$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t |
|
||||
|
@ -379,13 +379,13 @@ SELECT create_distributed_function('metadata_dist_test_proc(integer, integer)',
|
|||
|
||||
(1 row)
|
||||
|
||||
SELECT distribution_argument_index FROM citus.pg_dist_object WHERE objid = 'metadata_dist_test_proc'::regproc;
|
||||
SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE objid = 'metadata_dist_test_proc'::regproc;
|
||||
distribution_argument_index
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
(1 row)
|
||||
|
||||
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;
|
||||
SELECT * FROM run_command_on_workers($$SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE objid = 'local_schema.metadata_dist_test_proc'::regproc;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | 0
|
||||
|
@ -399,13 +399,13 @@ SELECT create_distributed_function('metadata_dist_test_proc(integer, integer)',
|
|||
|
||||
(1 row)
|
||||
|
||||
SELECT distribution_argument_index FROM citus.pg_dist_object WHERE objid = 'metadata_dist_test_proc'::regproc;
|
||||
SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE objid = 'metadata_dist_test_proc'::regproc;
|
||||
distribution_argument_index
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
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;
|
||||
SELECT * FROM run_command_on_workers($$ SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE objid = 'local_schema.metadata_dist_test_proc'::regproc;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | 1
|
||||
|
@ -418,12 +418,12 @@ NOTICE: drop cascades to 3 other objects
|
|||
DETAIL: drop cascades to function plpgsql_dist_function(text)
|
||||
drop cascades to table metadata_dist_test_table
|
||||
drop cascades to function metadata_dist_test_proc(integer,integer)
|
||||
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}%';
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{local_schema}%';
|
||||
obj_identifier
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
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;
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{local_schema}%';$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t |
|
||||
|
@ -433,26 +433,26 @@ SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_
|
|||
-- Show that extension and dependent sequence also created and marked as distributed
|
||||
CREATE SCHEMA extension_schema;
|
||||
CREATE EXTENSION ltree WITH SCHEMA extension_schema;
|
||||
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 '%{extension_schema}%';
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{extension_schema}%';
|
||||
obj_identifier
|
||||
---------------------------------------------------------------------
|
||||
(schema,{extension_schema},{})
|
||||
(1 row)
|
||||
|
||||
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 '%{extension_schema}%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{extension_schema}%';$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (schema,{extension_schema},{})
|
||||
localhost | 57638 | t | (schema,{extension_schema},{})
|
||||
(2 rows)
|
||||
|
||||
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 '%{ltree}%';
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{ltree}%';
|
||||
obj_identifier
|
||||
---------------------------------------------------------------------
|
||||
(extension,{ltree},{})
|
||||
(1 row)
|
||||
|
||||
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 '%{ltree}%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{ltree}%';$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (extension,{ltree},{})
|
||||
|
@ -467,13 +467,13 @@ SELECT create_distributed_table('extension_schema.table_to_check_object', 'id');
|
|||
|
||||
(1 row)
|
||||
|
||||
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 '%table_to_check_object%';
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';
|
||||
obj_identifier
|
||||
---------------------------------------------------------------------
|
||||
(table,"{extension_schema,table_to_check_object}",{})
|
||||
(1 row)
|
||||
|
||||
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 '%table_to_check_object%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (table,"{extension_schema,table_to_check_object}",{})
|
||||
|
@ -481,12 +481,12 @@ SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_
|
|||
(2 rows)
|
||||
|
||||
DROP TABLE extension_schema.table_to_check_object;
|
||||
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 '%table_to_check_object%';
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';
|
||||
obj_identifier
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
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 '%table_to_check_object%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t |
|
||||
|
|
|
@ -6,13 +6,13 @@ SET client_min_messages TO WARNING;
|
|||
-- the extension is on contrib, so should be avaliable for the regression tests
|
||||
CREATE EXTENSION seg;
|
||||
-- make sure that both the schema and the extension is distributed
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -34,7 +34,7 @@ SELECT count(*) from pg_dist_partition where logicalrelid='extension''test.test_
|
|||
|
||||
CREATE TYPE two_segs AS (seg_1 seg, seg_2 seg);
|
||||
-- verify that the type that depends on the extension is also marked as distributed
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_type WHERE typname = 'two_segs' AND typnamespace = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test'));
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_type WHERE typname = 'two_segs' AND typnamespace = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test'));
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -67,7 +67,7 @@ BEGIN;
|
|||
|
||||
COMMIT;
|
||||
-- make sure that the extension is distributed even if we run create extension in a transaction block
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -122,7 +122,7 @@ ALTER EXTENSION isn SET SCHEMA public;
|
|||
-- switch back to public schema as we set extension's schema to public
|
||||
SET search_path TO public;
|
||||
-- make sure that the extension is distributed
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -282,7 +282,7 @@ BEGIN;
|
|||
ROLLBACK;
|
||||
-- at the end of the transaction block, we did not create isn extension in coordinator or worker nodes as we rollback'ed
|
||||
-- make sure that the extension is not distributed
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -319,7 +319,7 @@ SELECT count(*) FROM pg_extension WHERE extname IN ('pg_buffercache', 'isn');
|
|||
|
||||
-- drop extension should just work
|
||||
DROP EXTENSION seg CASCADE;
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -333,7 +333,7 @@ SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname
|
|||
(2 rows)
|
||||
|
||||
-- make sure that the extension is not avaliable anymore as a distributed object
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn'));
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn'));
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -439,9 +439,9 @@ BEGIN;
|
|||
COMMIT;
|
||||
-- Check the pg_dist_object
|
||||
SELECT pg_proc.proname as DistributedFunction
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
distributedfunction
|
||||
---------------------------------------------------------------------
|
||||
|
@ -450,9 +450,9 @@ classid = 'pg_proc'::regclass;
|
|||
|
||||
SELECT run_command_on_workers($$
|
||||
SELECT count(*)
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
$$);
|
||||
run_command_on_workers
|
||||
|
@ -468,7 +468,7 @@ SELECT 1 from master_add_node('localhost', :worker_2_port);
|
|||
(1 row)
|
||||
|
||||
-- make sure that both extensions are created on both nodes
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid IN (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn'));
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid IN (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn'));
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
2
|
||||
|
@ -484,9 +484,9 @@ SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname
|
|||
-- Check the pg_dist_object on the both nodes
|
||||
SELECT run_command_on_workers($$
|
||||
SELECT count(*)
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
$$);
|
||||
run_command_on_workers
|
||||
|
@ -498,9 +498,9 @@ $$);
|
|||
DROP EXTENSION seg CASCADE;
|
||||
-- Recheck the pg_dist_object
|
||||
SELECT pg_proc.proname as DistributedFunction
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
distributedfunction
|
||||
---------------------------------------------------------------------
|
||||
|
@ -508,9 +508,9 @@ classid = 'pg_proc'::regclass;
|
|||
|
||||
SELECT run_command_on_workers($$
|
||||
SELECT count(*)
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
$$);
|
||||
run_command_on_workers
|
||||
|
@ -524,7 +524,7 @@ SET citus.enable_ddl_propagation TO false;
|
|||
CREATE EXTENSION seg;
|
||||
SET citus.enable_ddl_propagation TO true;
|
||||
-- Check the extension in pg_dist_object
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND
|
||||
objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -533,9 +533,9 @@ objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
|||
|
||||
SELECT run_command_on_workers($$
|
||||
SELECT count(*)
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
$$);
|
||||
run_command_on_workers
|
||||
|
@ -551,7 +551,7 @@ SELECT create_distributed_function('seg_in(cstring)', '$1', 'test_extension_func
|
|||
(1 row)
|
||||
|
||||
-- Recheck the extension in pg_dist_object
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND
|
||||
objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -559,9 +559,9 @@ objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
|||
(1 row)
|
||||
|
||||
SELECT pg_proc.proname as DistributedFunction
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
distributedfunction
|
||||
---------------------------------------------------------------------
|
||||
|
@ -570,9 +570,9 @@ classid = 'pg_proc'::regclass;
|
|||
|
||||
SELECT run_command_on_workers($$
|
||||
SELECT count(*)
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
$$);
|
||||
run_command_on_workers
|
||||
|
@ -600,7 +600,7 @@ SELECT create_distributed_function('cube(float8[], float8[])', '$1', 'test_exten
|
|||
|
||||
(1 row)
|
||||
|
||||
SELECT distribution_argument_index FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND
|
||||
SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND
|
||||
objid = (SELECT oid FROM pg_proc WHERE prosrc = 'cube_a_f8_f8');
|
||||
distribution_argument_index
|
||||
---------------------------------------------------------------------
|
||||
|
@ -613,7 +613,7 @@ SELECT create_distributed_function('cube(float8[], float8[])', '$2', 'test_exten
|
|||
|
||||
(1 row)
|
||||
|
||||
SELECT distribution_argument_index FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND
|
||||
SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND
|
||||
objid = (SELECT oid FROM pg_proc WHERE prosrc = 'cube_a_f8_f8');
|
||||
distribution_argument_index
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -6,13 +6,13 @@ SET client_min_messages TO WARNING;
|
|||
-- the extension is on contrib, so should be avaliable for the regression tests
|
||||
CREATE EXTENSION seg;
|
||||
-- make sure that both the schema and the extension is distributed
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -34,7 +34,7 @@ SELECT count(*) from pg_dist_partition where logicalrelid='extension''test.test_
|
|||
|
||||
CREATE TYPE two_segs AS (seg_1 seg, seg_2 seg);
|
||||
-- verify that the type that depends on the extension is also marked as distributed
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_type WHERE typname = 'two_segs' AND typnamespace = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test'));
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_type WHERE typname = 'two_segs' AND typnamespace = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test'));
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -67,7 +67,7 @@ BEGIN;
|
|||
|
||||
COMMIT;
|
||||
-- make sure that the extension is distributed even if we run create extension in a transaction block
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -122,7 +122,7 @@ ALTER EXTENSION isn SET SCHEMA public;
|
|||
-- switch back to public schema as we set extension's schema to public
|
||||
SET search_path TO public;
|
||||
-- make sure that the extension is distributed
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -281,7 +281,7 @@ BEGIN;
|
|||
ROLLBACK;
|
||||
-- at the end of the transaction block, we did not create isn extension in coordinator or worker nodes as we rollback'ed
|
||||
-- make sure that the extension is not distributed
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -318,7 +318,7 @@ SELECT count(*) FROM pg_extension WHERE extname IN ('pg_buffercache', 'isn');
|
|||
|
||||
-- drop extension should just work
|
||||
DROP EXTENSION seg CASCADE;
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -332,7 +332,7 @@ SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname
|
|||
(2 rows)
|
||||
|
||||
-- make sure that the extension is not avaliable anymore as a distributed object
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn'));
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn'));
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -438,9 +438,9 @@ BEGIN;
|
|||
COMMIT;
|
||||
-- Check the pg_dist_object
|
||||
SELECT pg_proc.proname as DistributedFunction
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
distributedfunction
|
||||
---------------------------------------------------------------------
|
||||
|
@ -449,9 +449,9 @@ classid = 'pg_proc'::regclass;
|
|||
|
||||
SELECT run_command_on_workers($$
|
||||
SELECT count(*)
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
$$);
|
||||
run_command_on_workers
|
||||
|
@ -467,7 +467,7 @@ SELECT 1 from master_add_node('localhost', :worker_2_port);
|
|||
(1 row)
|
||||
|
||||
-- make sure that both extensions are created on both nodes
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid IN (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn'));
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid IN (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn'));
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
2
|
||||
|
@ -483,9 +483,9 @@ SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname
|
|||
-- Check the pg_dist_object on the both nodes
|
||||
SELECT run_command_on_workers($$
|
||||
SELECT count(*)
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
$$);
|
||||
run_command_on_workers
|
||||
|
@ -497,9 +497,9 @@ $$);
|
|||
DROP EXTENSION seg CASCADE;
|
||||
-- Recheck the pg_dist_object
|
||||
SELECT pg_proc.proname as DistributedFunction
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
distributedfunction
|
||||
---------------------------------------------------------------------
|
||||
|
@ -507,9 +507,9 @@ classid = 'pg_proc'::regclass;
|
|||
|
||||
SELECT run_command_on_workers($$
|
||||
SELECT count(*)
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
$$);
|
||||
run_command_on_workers
|
||||
|
@ -523,7 +523,7 @@ SET citus.enable_ddl_propagation TO false;
|
|||
CREATE EXTENSION seg;
|
||||
SET citus.enable_ddl_propagation TO true;
|
||||
-- Check the extension in pg_dist_object
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND
|
||||
objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -532,9 +532,9 @@ objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
|||
|
||||
SELECT run_command_on_workers($$
|
||||
SELECT count(*)
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
$$);
|
||||
run_command_on_workers
|
||||
|
@ -550,7 +550,7 @@ SELECT create_distributed_function('seg_in(cstring)', '$1', 'test_extension_func
|
|||
(1 row)
|
||||
|
||||
-- Recheck the extension in pg_dist_object
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND
|
||||
objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -558,9 +558,9 @@ objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
|||
(1 row)
|
||||
|
||||
SELECT pg_proc.proname as DistributedFunction
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
distributedfunction
|
||||
---------------------------------------------------------------------
|
||||
|
@ -569,9 +569,9 @@ classid = 'pg_proc'::regclass;
|
|||
|
||||
SELECT run_command_on_workers($$
|
||||
SELECT count(*)
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
$$);
|
||||
run_command_on_workers
|
||||
|
@ -599,7 +599,7 @@ SELECT create_distributed_function('cube(float8[], float8[])', '$1', 'test_exten
|
|||
|
||||
(1 row)
|
||||
|
||||
SELECT distribution_argument_index FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND
|
||||
SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND
|
||||
objid = (SELECT oid FROM pg_proc WHERE prosrc = 'cube_a_f8_f8');
|
||||
distribution_argument_index
|
||||
---------------------------------------------------------------------
|
||||
|
@ -612,7 +612,7 @@ SELECT create_distributed_function('cube(float8[], float8[])', '$2', 'test_exten
|
|||
|
||||
(1 row)
|
||||
|
||||
SELECT distribution_argument_index FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND
|
||||
SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND
|
||||
objid = (SELECT oid FROM pg_proc WHERE prosrc = 'cube_a_f8_f8');
|
||||
distribution_argument_index
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -45,7 +45,7 @@ SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid
|
|||
(1 row)
|
||||
|
||||
-- verify that the aggregate is added top pg_dist_object on the new node
|
||||
SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'propagate_foreign_server.array_agg'::regproc;$$);
|
||||
SELECT run_command_on_workers($$SELECT count(*) from pg_catalog.pg_dist_object where objid = 'propagate_foreign_server.array_agg'::regproc;$$);
|
||||
run_command_on_workers
|
||||
---------------------------------------------------------------------
|
||||
(localhost,57638,t,1)
|
||||
|
|
|
@ -123,7 +123,7 @@ from
|
|||
(select
|
||||
pg_catalog.websearch_to_tsquery(
|
||||
cast(pg_catalog.regconfigin(cast(cast(null as cstring) as cstring)) as regconfig),
|
||||
cast((select type from citus.pg_dist_object limit 1 offset 1) as text)
|
||||
cast((select type from pg_catalog.pg_dist_object limit 1 offset 1) as text)
|
||||
) as c0,
|
||||
sample_0.org_id as c1,
|
||||
sample_0.id as c2,
|
||||
|
|
|
@ -419,7 +419,7 @@ SELECT * FROM run_command_on_workers($$ SELECT 'text_search.concurrent_index_con
|
|||
|
||||
-- verify the objid is correctly committed locally due to the somewhat convoluted commit and new transaction starting when creating an index concurrently
|
||||
SELECT pg_catalog.pg_identify_object_as_address(classid, objid, objsubid)
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE classid = 3602 AND objid = 'text_search.concurrent_index_config'::regconfig::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -247,7 +247,6 @@ ORDER BY 1;
|
|||
sequence pg_dist_node_nodeid_seq
|
||||
sequence pg_dist_placement_placementid_seq
|
||||
sequence pg_dist_shardid_seq
|
||||
table citus.pg_dist_object
|
||||
table columnar.chunk
|
||||
table columnar.chunk_group
|
||||
table columnar.options
|
||||
|
@ -257,6 +256,7 @@ ORDER BY 1;
|
|||
table pg_dist_local_group
|
||||
table pg_dist_node
|
||||
table pg_dist_node_metadata
|
||||
table pg_dist_object
|
||||
table pg_dist_partition
|
||||
table pg_dist_placement
|
||||
table pg_dist_poolinfo
|
||||
|
|
|
@ -36,7 +36,7 @@ drop cascades to table upgrade_basic.t_range
|
|||
-- "seg" extension
|
||||
-- will not be marked as distributed
|
||||
-- see underlying objects
|
||||
SELECT i.* FROM citus.pg_dist_object, pg_identify_object_as_address(classid, objid, objsubid) i ORDER BY 1, 2, 3;
|
||||
SELECT i.* FROM pg_catalog.pg_dist_object, pg_identify_object_as_address(classid, objid, objsubid) i ORDER BY 1, 2, 3;
|
||||
type | object_names | object_args
|
||||
---------------------------------------------------------------------
|
||||
collation | {post_11_upgrade,german_phonebook_unpropagated} | {}
|
||||
|
|
|
@ -36,7 +36,7 @@ drop cascades to table upgrade_basic.t_range
|
|||
-- "seg" extension
|
||||
-- will not be marked as distributed
|
||||
-- see underlying objects
|
||||
SELECT i.* FROM citus.pg_dist_object, pg_identify_object_as_address(classid, objid, objsubid) i ORDER BY 1, 2, 3;
|
||||
SELECT i.* FROM pg_catalog.pg_dist_object, pg_identify_object_as_address(classid, objid, objsubid) i ORDER BY 1, 2, 3;
|
||||
type | object_names | object_args
|
||||
---------------------------------------------------------------------
|
||||
database | {postgres} | {}
|
||||
|
|
|
@ -10,7 +10,7 @@ NOTICE: Preparing to sync the metadata to all nodes
|
|||
(1 row)
|
||||
|
||||
-- tables are objects with Citus 11+
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object WHERE objid IN ('post_11_upgrade'::regnamespace, 'post_11_upgrade.part_table'::regclass, 'post_11_upgrade.sensors'::regclass, 'post_11_upgrade.func_in_transaction_def'::regproc, 'post_11_upgrade.partial_index_test_config'::regconfig, 'post_11_upgrade.my_type'::regtype) ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object WHERE objid IN ('post_11_upgrade'::regnamespace, 'post_11_upgrade.part_table'::regclass, 'post_11_upgrade.sensors'::regclass, 'post_11_upgrade.func_in_transaction_def'::regproc, 'post_11_upgrade.partial_index_test_config'::regconfig, 'post_11_upgrade.my_type'::regtype) ORDER BY 1;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{post_11_upgrade,func_in_transaction_def}",{})
|
||||
|
@ -22,7 +22,7 @@ SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dis
|
|||
(6 rows)
|
||||
|
||||
-- on all nodes
|
||||
SELECT run_command_on_workers($$SELECT array_agg(pg_identify_object_as_address(classid, objid, objsubid)) FROM citus.pg_dist_object WHERE objid IN ('post_11_upgrade'::regnamespace, 'post_11_upgrade.part_table'::regclass, 'post_11_upgrade.sensors'::regclass, 'post_11_upgrade.func_in_transaction_def'::regproc, 'post_11_upgrade.partial_index_test_config'::regconfig, 'post_11_upgrade.my_type'::regtype) ORDER BY 1;$$) ORDER BY 1;
|
||||
SELECT run_command_on_workers($$SELECT array_agg(pg_identify_object_as_address(classid, objid, objsubid)) FROM pg_catalog.pg_dist_object WHERE objid IN ('post_11_upgrade'::regnamespace, 'post_11_upgrade.part_table'::regclass, 'post_11_upgrade.sensors'::regclass, 'post_11_upgrade.func_in_transaction_def'::regproc, 'post_11_upgrade.partial_index_test_config'::regconfig, 'post_11_upgrade.my_type'::regtype) ORDER BY 1;$$) ORDER BY 1;
|
||||
run_command_on_workers
|
||||
---------------------------------------------------------------------
|
||||
(localhost,57636,t,"{""(type,{post_11_upgrade.my_type},{})"",""(function,\\""{post_11_upgrade,func_in_transaction_def}\\"",{})"",""(table,\\""{post_11_upgrade,part_table}\\"",{})"",""(table,\\""{post_11_upgrade,sensors}\\"",{})"",""(schema,{post_11_upgrade},{})"",""(\\""text search configuration\\"",\\""{post_11_upgrade,partial_index_test_config}\\"",{})""}")
|
||||
|
|
|
@ -651,8 +651,8 @@ CREATE SEQUENCE test_schema_for_sequence_propagation.seq_10;
|
|||
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);
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation'::regnamespace);
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation.seq_10'::regclass);
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation'::regnamespace);
|
||||
|
||||
DROP SCHEMA test_schema_for_sequence_propagation CASCADE;
|
||||
DROP TABLE table_without_sequence;
|
||||
|
|
|
@ -500,13 +500,13 @@ SELECT shardid, nodename, nodeport
|
|||
-- before adding the node, add pg_dist_object entry for tables created with
|
||||
-- master_create_distributed_table as we don't have the entry for them.
|
||||
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'objects'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'customer_with_special_\\_character'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, '1_customer'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'packed_numbers_hash'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'super_packed_numbers_hash'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table_to_distribute'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'second_dustbunnies'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'objects'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'customer_with_special_\\_character'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, '1_customer'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'packed_numbers_hash'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'super_packed_numbers_hash'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table_to_distribute'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'second_dustbunnies'::regclass::oid, 0);
|
||||
|
||||
SELECT 1 FROM master_activate_node('localhost', :worker_1_port);
|
||||
RESET client_min_messages;
|
||||
|
|
|
@ -1275,13 +1275,13 @@ CREATE SCHEMA test_schema_for_sequence_propagation;
|
|||
CREATE SEQUENCE test_schema_for_sequence_propagation.seq_10;
|
||||
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);
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation.seq_10'::regclass);
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------
|
||||
(sequence,"{test_schema_for_sequence_propagation,seq_10}",{})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation'::regnamespace);
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation'::regnamespace);
|
||||
pg_identify_object_as_address
|
||||
----------------------------------------------------
|
||||
(schema,{test_schema_for_sequence_propagation},{})
|
||||
|
|
|
@ -627,13 +627,13 @@ SELECT shardid, nodename, nodeport
|
|||
-- add the node back
|
||||
-- before adding the node, add pg_dist_object entry for tables created with
|
||||
-- master_create_distributed_table as we don't have the entry for them.
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'objects'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'customer_with_special_\\_character'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, '1_customer'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'packed_numbers_hash'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'super_packed_numbers_hash'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table_to_distribute'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'second_dustbunnies'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'objects'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'customer_with_special_\\_character'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, '1_customer'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'packed_numbers_hash'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'super_packed_numbers_hash'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table_to_distribute'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'second_dustbunnies'::regclass::oid, 0);
|
||||
SELECT 1 FROM master_activate_node('localhost', :worker_1_port);
|
||||
NOTICE: Replicating postgres objects to node localhost:57637
|
||||
DETAIL: There are 115 objects to replicate, depending on your environment this might take a while
|
||||
|
|
|
@ -51,7 +51,7 @@ step "s1-print-distributed-objects"
|
|||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
|
@ -123,7 +123,7 @@ step "s2-commit"
|
|||
step "s2-print-distributed-objects"
|
||||
{
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
|
|
|
@ -41,7 +41,7 @@ step "s1-create-extension-with-schema2"
|
|||
|
||||
step "s1-print"
|
||||
{
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
|
|
@ -161,7 +161,7 @@ create aggregate binstragg(text, text)(
|
|||
stype=text
|
||||
);
|
||||
-- verify that the aggregate is added into pg_dist_object, on each worker
|
||||
SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.binstragg'::regproc;$$);
|
||||
SELECT run_command_on_workers($$SELECT count(*) from pg_catalog.pg_dist_object where objid = 'aggregate_support.binstragg'::regproc;$$);
|
||||
|
||||
SELECT run_command_on_workers($$select count(*) from pg_aggregate where aggfnoid::text like '%binstragg%';$$);
|
||||
|
||||
|
@ -588,14 +588,14 @@ COMMIT;
|
|||
SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$);
|
||||
|
||||
-- verify that the aggregate is added into pg_dist_object, on each worker
|
||||
SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$);
|
||||
SELECT run_command_on_workers($$SELECT count(*) from pg_catalog.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$);
|
||||
|
||||
RESET citus.create_object_propagation;
|
||||
|
||||
-- drop and test outside of tx block
|
||||
drop aggregate dependent_agg (float8);
|
||||
-- verify that the aggregate is removed from pg_dist_object, on each worker
|
||||
SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$);
|
||||
SELECT run_command_on_workers($$SELECT count(*) from pg_catalog.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$);
|
||||
create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc);
|
||||
--verify
|
||||
SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$);
|
||||
|
|
|
@ -45,7 +45,7 @@ DROP TABLE t4; -- as long as the table is using the type some operations are har
|
|||
COMMIT;
|
||||
|
||||
-- verify the type is distributed
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = 'disabled_object_propagation.tt3'::regtype::oid;
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = 'disabled_object_propagation.tt3'::regtype::oid;
|
||||
|
||||
ALTER TYPE tt3 ADD ATTRIBUTE c int, DROP ATTRIBUTE b, ALTER ATTRIBUTE a SET DATA TYPE text COLLATE "POSIX";
|
||||
ALTER TYPE tt3 OWNER TO typeowner_for_disabled_object_propagation_guc;
|
||||
|
|
|
@ -180,7 +180,7 @@ SELECT create_distributed_table('streaming_table','id');
|
|||
-- if not paremeters are supplied, we'd see that function doesn't have
|
||||
-- distribution_argument_index and colocationid
|
||||
SELECT create_distributed_function('"eq_mi''xed_param_names"(macaddr, macaddr)');
|
||||
SELECT distribution_argument_index is NULL, colocationid is NULL from citus.pg_dist_object
|
||||
SELECT distribution_argument_index is NULL, colocationid is NULL from pg_catalog.pg_dist_object
|
||||
WHERE objid = 'eq_mi''xed_param_names(macaddr, macaddr)'::regprocedure;
|
||||
|
||||
-- also show that we can use the function
|
||||
|
@ -380,7 +380,7 @@ SELECT create_distributed_function('eq_with_param_names(macaddr, macaddr)', '$1'
|
|||
|
||||
-- show that the colocationIds are the same
|
||||
SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated
|
||||
FROM pg_dist_partition, citus.pg_dist_object as objects
|
||||
FROM pg_dist_partition, pg_catalog.pg_dist_object as objects
|
||||
WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND
|
||||
objects.objid = 'eq_with_param_names(macaddr, macaddr)'::regprocedure;
|
||||
|
||||
|
@ -388,7 +388,7 @@ WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass
|
|||
-- group preserved, because we're using the default shard creation settings
|
||||
SELECT create_distributed_function('eq_with_param_names(macaddr, macaddr)', 'val1');
|
||||
SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated
|
||||
FROM pg_dist_partition, citus.pg_dist_object as objects
|
||||
FROM pg_dist_partition, pg_catalog.pg_dist_object as objects
|
||||
WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND
|
||||
objects.objid = 'eq_with_param_names(macaddr, macaddr)'::regprocedure;
|
||||
|
||||
|
@ -405,13 +405,13 @@ SELECT create_distributed_function('eq_with_param_names(macaddr, macaddr)', colo
|
|||
-- to coerce the values
|
||||
SELECT create_distributed_function('eq8(macaddr8, macaddr8)', '$1', colocate_with:='replicated_table_func_test_4');
|
||||
SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated
|
||||
FROM pg_dist_partition, citus.pg_dist_object as objects
|
||||
FROM pg_dist_partition, pg_catalog.pg_dist_object as objects
|
||||
WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND
|
||||
objects.objid = 'eq8(macaddr8, macaddr8)'::regprocedure;
|
||||
|
||||
SELECT create_distributed_function('add_text(text, text)', '$1', colocate_with:='replicated_table_func_test_4');
|
||||
SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated
|
||||
FROM pg_dist_partition, citus.pg_dist_object as objects
|
||||
FROM pg_dist_partition, pg_catalog.pg_dist_object as objects
|
||||
WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND
|
||||
objects.objid = 'add_text(text, text)'::regprocedure;
|
||||
|
||||
|
|
|
@ -18,12 +18,12 @@ END;
|
|||
$$;
|
||||
|
||||
-- Check all dependent objects and function depends on all nodes
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
SET citus.enable_metadata_sync TO OFF;
|
||||
CREATE TYPE function_prop_type_2 AS (a int, b int);
|
||||
|
@ -38,10 +38,10 @@ BEGIN
|
|||
END;
|
||||
$$;
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
-- Have a separate check for type created in transaction
|
||||
BEGIN;
|
||||
|
@ -60,9 +60,9 @@ BEGIN
|
|||
END;
|
||||
$$;
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_3'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_3'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
-- Check table
|
||||
CREATE TABLE function_prop_table(a int, b int);
|
||||
|
@ -97,8 +97,8 @@ BEGIN
|
|||
END;
|
||||
$$;
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
-- Views are not supported
|
||||
CREATE VIEW function_prop_view AS SELECT * FROM function_prop_table;
|
||||
|
@ -133,8 +133,8 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Within transaction functions are not distributed
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Show that recreating it outside transaction distributes the function and dependencies
|
||||
|
@ -147,10 +147,10 @@ BEGIN
|
|||
END;
|
||||
$$;
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
-- Test for SQL function with unsupported object in function body
|
||||
CREATE TABLE table_in_sql_body(id int);
|
||||
|
@ -163,9 +163,9 @@ $$
|
|||
$$;
|
||||
|
||||
-- Show that only function has propagated, since the table is not resolved as dependency
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regclass::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regclass::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
-- Check extension owned table
|
||||
CREATE TABLE extension_owned_table(a int);
|
||||
|
@ -202,17 +202,17 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Function shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid;
|
||||
|
||||
CREATE TABLE table_to_prop_func(id int, col_1 int default func_in_transaction_def());
|
||||
SELECT create_distributed_table('table_to_prop_func','id');
|
||||
|
||||
-- Function should be marked as distributed after distributing the table that depends on it
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
|
||||
-- Multiple functions as a default column
|
||||
|
@ -236,20 +236,20 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid;
|
||||
|
||||
CREATE TABLE table_to_prop_func_2(id int, col_1 int default func_in_transaction_1() + func_in_transaction_2());
|
||||
SELECT create_distributed_table('table_to_prop_func_2','id');
|
||||
|
||||
-- Functions should be marked as distribued after distributing the table that depends on it
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Functions should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
|
||||
-- If function has dependency on non-distributed table it should error out
|
||||
|
@ -287,16 +287,16 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Function shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid;
|
||||
|
||||
ALTER TABLE table_to_prop_func_4 ADD COLUMN col_1 int default function_propagation_schema.func_in_transaction_4();
|
||||
|
||||
-- Function should be marked as distributed after adding the column
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Functions should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
|
||||
-- Adding a column with default function depending on non-distributable table should fail
|
||||
|
@ -342,20 +342,20 @@ BEGIN;
|
|||
|
||||
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid;
|
||||
|
||||
CREATE TABLE table_to_prop_func_5(id int, col_1 int default func_in_transaction_5(), col_2 int default func_in_transaction_6());
|
||||
SELECT create_distributed_table('table_to_prop_func_5', 'id');
|
||||
|
||||
-- Functions should be marked as distributed after adding the column
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Functions should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
-- Adding a constraint with function check should propagate the function
|
||||
BEGIN;
|
||||
|
@ -369,17 +369,17 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid;
|
||||
|
||||
CREATE TABLE table_to_prop_func_6(id int, col_1 int check (function_propagation_schema.func_in_transaction_7(col_1)));
|
||||
SELECT create_distributed_table('table_to_prop_func_6', 'id');
|
||||
|
||||
-- Function should be marked as distributed after adding the column
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
|
||||
-- Adding a constraint with multiple functions check should propagate the function
|
||||
|
@ -403,20 +403,20 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid;
|
||||
|
||||
CREATE TABLE table_to_prop_func_7(id int, col_1 int check (function_propagation_schema.func_in_transaction_8(col_1) and function_propagation_schema.func_in_transaction_9(col_1)));
|
||||
SELECT create_distributed_table('table_to_prop_func_7', 'id');
|
||||
|
||||
-- Function should be marked as distributed after adding the column
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Functions should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
|
||||
-- Adding a column with constraint should propagate the function
|
||||
|
@ -434,16 +434,16 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid;
|
||||
|
||||
ALTER TABLE table_to_prop_func_8 ADD CONSTRAINT col1_check CHECK (function_propagation_schema.func_in_transaction_10(col_1));
|
||||
|
||||
-- Function should be marked as distributed after adding the constraint
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
|
||||
-- If constraint depends on a non-distributed table it should error out
|
||||
|
@ -479,7 +479,7 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_generated_always'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_generated_always'::regproc::oid;
|
||||
|
||||
CREATE TABLE people (
|
||||
id int,
|
||||
|
@ -489,7 +489,7 @@ BEGIN;
|
|||
SELECT create_distributed_table('people', 'id');
|
||||
|
||||
-- Show that function is distributed after distributing the table
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_generated_always'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_generated_always'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
|
||||
|
@ -505,7 +505,7 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid;
|
||||
|
||||
CREATE TABLE table_1_for_rule(id int, col_1 int);
|
||||
CREATE TABLE table_2_for_rule(id int, col_1 int);
|
||||
|
@ -515,11 +515,11 @@ BEGIN;
|
|||
SELECT create_distributed_table('table_1_for_rule','id');
|
||||
|
||||
-- Functions should be distributed after distributing the table
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
|
||||
-- Show that functions as partitioning functions are supported
|
||||
|
@ -535,18 +535,18 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid;
|
||||
|
||||
CREATE TABLE partitioned_table_to_test_func_prop(id INT, a INT) PARTITION BY RANGE (non_sense_func_for_partitioning(id));
|
||||
|
||||
SELECT create_distributed_table('partitioned_table_to_test_func_prop', 'id');
|
||||
|
||||
-- Show that function is distributed after distributing the table
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
|
||||
-- Test function dependency on citus local table
|
||||
|
@ -561,14 +561,14 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Function shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_for_local_table'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_for_local_table'::regproc::oid;
|
||||
|
||||
CREATE TABLE citus_local_table_to_test_func(l1 int DEFAULT func_in_transaction_for_local_table());
|
||||
SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0);
|
||||
SELECT citus_add_local_table_to_metadata('citus_local_table_to_test_func');
|
||||
|
||||
-- Function should be marked as distributed after distributing the table that depends on it
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_for_local_table'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_for_local_table'::regproc::oid;
|
||||
ROLLBACK;
|
||||
|
||||
-- Show that having a function dependency on exlude also works
|
||||
|
@ -583,17 +583,17 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid;
|
||||
|
||||
CREATE TABLE exclusion_func_prop_table (id int, EXCLUDE USING btree (id WITH =) WHERE (exclude_bool_func()));
|
||||
SELECT create_distributed_table('exclusion_func_prop_table', 'id');
|
||||
|
||||
-- Function should be marked as distributed after distributing the table that depends on it
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
|
||||
-- Show that having a function dependency for index also works
|
||||
|
@ -608,7 +608,7 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid;
|
||||
|
||||
CREATE TABLE table_to_check_func_index_dep (id int, col_2 int);
|
||||
CREATE INDEX on table_to_check_func_index_dep(col_2) WHERE (func_for_index_predicate(col_2));
|
||||
|
@ -616,11 +616,11 @@ BEGIN;
|
|||
SELECT create_distributed_table('table_to_check_func_index_dep', 'id');
|
||||
|
||||
-- Function should be marked as distributed after distributing the table that depends on it
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
|
||||
-- Test function to function dependency
|
||||
|
@ -648,11 +648,11 @@ BEGIN;
|
|||
SELECT create_distributed_table('func_dep_table', 'a');
|
||||
|
||||
-- Function should be marked as distributed after distributing the table that depends on it
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_func_dep_1'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_for_func_dep_1'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_func_dep_1'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_for_func_dep_1'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
|
||||
-- Test function with SQL language and sequence dependency
|
||||
|
@ -672,7 +672,7 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Function shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid;
|
||||
|
||||
CREATE SEQUENCE myseq;
|
||||
CREATE TABLE table_to_prop_seq_func(id int, col_1 bigint default func_in_transaction_def_with_func(func_in_transaction_def_with_seq(nextval('myseq'))));
|
||||
|
@ -680,11 +680,11 @@ BEGIN;
|
|||
SELECT create_distributed_table('table_to_prop_seq_func','id');
|
||||
|
||||
-- Function should be marked as distributed after distributing the table that depends on it
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
|
||||
-- Show that having a dependency on another dist table work out tx
|
||||
|
@ -719,42 +719,42 @@ SELECT create_reference_table('tbl_to_colocate_ref');
|
|||
|
||||
CREATE FUNCTION func_to_colocate (a int) returns int as $$select 1;$$ language sql;
|
||||
-- see the empty pg_dist_object entries
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM citus.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
|
||||
-- colocate the function with ref table
|
||||
SELECT create_distributed_function('func_to_colocate(int)', colocate_with:='tbl_to_colocate_ref');
|
||||
-- see the pg_dist_object entry
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM citus.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
|
||||
-- convert to non-delegated
|
||||
SELECT create_distributed_function('func_to_colocate(int)');
|
||||
-- show that the pg_dist_object fields are gone
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM citus.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
|
||||
-- colocate the function with distributed table
|
||||
SELECT create_distributed_function('func_to_colocate(int)','$1','tbl_to_colocate');
|
||||
-- see the pg_dist_object entry
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM citus.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
|
||||
-- try create or replace the same func
|
||||
CREATE OR REPLACE FUNCTION func_to_colocate (a int) returns int as $$select 1;$$ language sql;
|
||||
-- verify the pg_dist_object entry is the same
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM citus.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
|
||||
-- convert to non-delegated
|
||||
SELECT create_distributed_function('func_to_colocate(int)');
|
||||
-- show that the pg_dist_object fields are gone
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM citus.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
|
||||
-- force delegate
|
||||
SELECT create_distributed_function('func_to_colocate(int)','$1','tbl_to_colocate', true);
|
||||
-- show pg_dist_object fields
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM citus.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
|
||||
-- convert to non-delegated
|
||||
SELECT create_distributed_function('func_to_colocate(int)');
|
||||
-- show that the pg_dist_object fields are gone
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM citus.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
SELECT distribution_argument_index, colocationid, force_delegation FROM pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
|
||||
|
||||
RESET search_path;
|
||||
SET client_min_messages TO WARNING;
|
||||
|
|
|
@ -93,8 +93,8 @@ SELECT count(*) FROM history;
|
|||
SET citus.shard_replication_factor TO 1;
|
||||
|
||||
-- metadata sync will succeed even if we have rep > 1 tables
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles_single_shard'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles_single_shard'::regclass::oid, 0);
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
|
||||
CREATE TABLE mx_table(a int);
|
||||
|
|
|
@ -108,8 +108,8 @@ SELECT run_command_on_workers('GRANT ALL ON SCHEMA citus TO node_metadata_user')
|
|||
SELECT master_remove_node('localhost', :worker_2_port);
|
||||
|
||||
-- Removing public schema from pg_dist_object because it breaks the next tests
|
||||
DELETE FROM citus.pg_dist_object WHERE objid = 'public'::regnamespace::oid;
|
||||
DELETE FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'plpgsql');
|
||||
DELETE FROM pg_catalog.pg_dist_object WHERE objid = 'public'::regnamespace::oid;
|
||||
DELETE FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'plpgsql');
|
||||
|
||||
-- try to manipulate node metadata via non-super user
|
||||
SET ROLE non_super_user;
|
||||
|
|
|
@ -437,11 +437,11 @@ SELECT update_distributed_table_colocation('table1_group_none', colocate_with =>
|
|||
SELECT update_distributed_table_colocation('table1_group_none', colocate_with => 'table3_groupE');
|
||||
|
||||
-- activate nodes to get rid of inconsistencies in pg_dist tables
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table1_group1'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table2_group1'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table3_group2'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table4_group2'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table5_groupX'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table1_group1'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table2_group1'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table3_group2'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table4_group2'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table5_groupX'::regclass::oid, 0);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_2_port);
|
||||
|
||||
|
|
|
@ -213,16 +213,17 @@ SELECT * FROM multi_extension.print_extension_changes();
|
|||
-- Test downgrade to 9.4-1 from 9.5-1
|
||||
ALTER EXTENSION citus UPDATE TO '9.5-1';
|
||||
|
||||
BEGIN;
|
||||
SET citus.enable_metadata_sync TO on;
|
||||
SELECT master_add_node('localhost', :master_port, groupId=>0);
|
||||
CREATE TABLE citus_local_table (a int);
|
||||
SELECT create_citus_local_table('citus_local_table');
|
||||
RESET citus.enable_metadata_sync;
|
||||
|
||||
-- downgrade from 9.5-1 to 9.4-1 should fail as we have a citus local table
|
||||
ALTER EXTENSION citus UPDATE TO '9.4-1';
|
||||
ROLLBACK;
|
||||
-- TODO: This test should be moved to a valid downgrade testing suite where the downgrade is done, both on the schema and the binaries. Later changes in Citus made a C vs Schema discrepancy error here
|
||||
-- BEGIN;
|
||||
-- SET citus.enable_metadata_sync TO on;
|
||||
-- SELECT master_add_node('localhost', :master_port, groupId=>0);
|
||||
-- CREATE TABLE citus_local_table (a int);
|
||||
-- SELECT create_citus_local_table('citus_local_table');
|
||||
-- RESET citus.enable_metadata_sync;
|
||||
--
|
||||
-- -- downgrade from 9.5-1 to 9.4-1 should fail as we have a citus local table
|
||||
-- ALTER EXTENSION citus UPDATE TO '9.4-1';
|
||||
-- ROLLBACK;
|
||||
|
||||
-- now we can downgrade as there is no citus local table
|
||||
ALTER EXTENSION citus UPDATE TO '9.4-1';
|
||||
|
|
|
@ -10,5 +10,5 @@ FROM pg_attribute
|
|||
WHERE atthasmissing AND attrelid NOT IN ('pg_dist_node'::regclass,
|
||||
'pg_dist_rebalance_strategy'::regclass,
|
||||
'pg_dist_partition'::regclass,
|
||||
'citus.pg_dist_object'::regclass)
|
||||
'pg_dist_object'::regclass)
|
||||
ORDER BY attrelid, attname;
|
||||
|
|
|
@ -544,10 +544,10 @@ DROP TABLE mx_table_with_small_sequence, mx_table_with_sequence;
|
|||
-- owner
|
||||
CREATE TABLE pg_dist_placement_temp AS SELECT * FROM pg_dist_placement;
|
||||
CREATE TABLE pg_dist_partition_temp AS SELECT * FROM pg_dist_partition;
|
||||
CREATE TABLE pg_dist_object_temp AS SELECT * FROM citus.pg_dist_object;
|
||||
CREATE TABLE pg_dist_object_temp AS SELECT * FROM pg_catalog.pg_dist_object;
|
||||
DELETE FROM pg_dist_placement;
|
||||
DELETE FROM pg_dist_partition;
|
||||
DELETE FROM citus.pg_dist_object;
|
||||
DELETE FROM pg_catalog.pg_dist_object;
|
||||
SELECT groupid AS old_worker_2_group FROM pg_dist_node WHERE nodeport = :worker_2_port \gset
|
||||
SELECT master_remove_node('localhost', :worker_2_port);
|
||||
|
||||
|
@ -586,7 +586,7 @@ DROP TABLE mx_table;
|
|||
\c - postgres - :master_port
|
||||
INSERT INTO pg_dist_placement SELECT * FROM pg_dist_placement_temp;
|
||||
INSERT INTO pg_dist_partition SELECT * FROM pg_dist_partition_temp;
|
||||
INSERT INTO citus.pg_dist_object SELECT * FROM pg_dist_object_temp ON CONFLICT ON CONSTRAINT pg_dist_object_pkey DO NOTHING;
|
||||
INSERT INTO pg_catalog.pg_dist_object SELECT * FROM pg_dist_object_temp ON CONFLICT ON CONSTRAINT pg_dist_object_pkey DO NOTHING;
|
||||
DROP TABLE pg_dist_placement_temp;
|
||||
DROP TABLE pg_dist_partition_temp;
|
||||
DROP TABLE pg_dist_object_temp;
|
||||
|
|
|
@ -67,7 +67,7 @@ END;$$;
|
|||
SELECT create_distributed_function('proc_0(float8)', 'dist_key', 'test_proc_colocation_0' );
|
||||
|
||||
SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('test_proc_colocation_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
|
||||
SET client_min_messages TO DEBUG1;
|
||||
CALL proc_0(1.0);
|
||||
|
@ -81,7 +81,7 @@ CALL proc_0(1.0);
|
|||
RESET client_min_messages;
|
||||
|
||||
SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('test_proc_colocation_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
|
||||
-- colocatewith is not null && list_length(colocatedTableList) = 1
|
||||
SELECT alter_distributed_table('test_proc_colocation_0', shard_count:= 4);
|
||||
|
@ -94,7 +94,7 @@ CALL proc_0(1.0);
|
|||
RESET client_min_messages;
|
||||
|
||||
SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('test_proc_colocation_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
|
||||
-- shardCount is not null && cascade_to_colocated is true
|
||||
SELECT alter_distributed_table('test_proc_colocation_0', shard_count:= 8, cascade_to_colocated := true);
|
||||
|
@ -104,7 +104,7 @@ CALL proc_0(1.0);
|
|||
RESET client_min_messages;
|
||||
|
||||
SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('test_proc_colocation_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
|
||||
-- colocatewith is not null && cascade_to_colocated is true
|
||||
SELECT alter_distributed_table('test_proc_colocation_0', shard_count:= 4, cascade_to_colocated := true);
|
||||
|
@ -117,7 +117,7 @@ CALL proc_0(1.0);
|
|||
RESET client_min_messages;
|
||||
|
||||
SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('test_proc_colocation_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
|
||||
-- try a case with more than one procedure
|
||||
CREATE OR REPLACE procedure proc_1(dist_key float8)
|
||||
|
@ -134,7 +134,7 @@ END;$$;
|
|||
SELECT create_distributed_function('proc_1(float8)', 'dist_key', 'test_proc_colocation_0' );
|
||||
|
||||
SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('test_proc_colocation_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname;
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname;
|
||||
|
||||
SET client_min_messages TO DEBUG1;
|
||||
CALL proc_0(1.0);
|
||||
|
@ -149,14 +149,14 @@ CALL proc_1(2.0);
|
|||
RESET client_min_messages;
|
||||
|
||||
SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('test_proc_colocation_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname;
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname;
|
||||
|
||||
-- case which shouldn't preserve colocation for now
|
||||
-- shardCount is not null && cascade_to_colocated is false
|
||||
SELECT alter_distributed_table('test_proc_colocation_0', shard_count:= 18, cascade_to_colocated := false);
|
||||
|
||||
SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('test_proc_colocation_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname;
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname;
|
||||
|
||||
SET client_min_messages TO WARNING;
|
||||
DROP SCHEMA mx_alter_distributed_table CASCADE;
|
||||
|
|
|
@ -308,7 +308,7 @@ 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
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid::oid::regnamespace IN ('mx_old_schema', 'mx_new_schema')
|
||||
ORDER BY "Distributed Schemas";
|
||||
\c - - - :worker_1_port
|
||||
|
@ -325,7 +325,7 @@ ALTER SCHEMA mx_old_schema RENAME TO temp_mx_old_schema;
|
|||
ALTER TABLE mx_old_schema.table_set_schema SET SCHEMA mx_new_schema;
|
||||
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid::oid::regnamespace IN ('mx_old_schema', 'mx_new_schema');
|
||||
\c - - - :worker_1_port
|
||||
SELECT table_schema AS "Table's Schema" FROM information_schema.tables WHERE table_name='table_set_schema';
|
||||
|
|
|
@ -672,7 +672,7 @@ SELECT create_distributed_table('old_schema.table_set_schema', 'id');
|
|||
CREATE SCHEMA new_schema;
|
||||
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema');
|
||||
\c - - - :worker_1_port
|
||||
SELECT table_schema AS "Shards' Schema"
|
||||
|
@ -685,7 +685,7 @@ SELECT table_schema AS "Shards' Schema"
|
|||
ALTER TABLE old_schema.table_set_schema SET SCHEMA new_schema;
|
||||
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema');
|
||||
\c - - - :worker_1_port
|
||||
SELECT table_schema AS "Shards' Schema"
|
||||
|
@ -706,7 +706,7 @@ SELECT create_distributed_table('table_set_schema', 'id');
|
|||
CREATE SCHEMA new_schema;
|
||||
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid='new_schema'::regnamespace::oid;
|
||||
\c - - - :worker_1_port
|
||||
SELECT table_schema AS "Shards' Schema"
|
||||
|
@ -719,7 +719,7 @@ SELECT table_schema AS "Shards' Schema"
|
|||
ALTER TABLE table_set_schema SET SCHEMA new_schema;
|
||||
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid='new_schema'::regnamespace::oid;
|
||||
\c - - - :worker_1_port
|
||||
SELECT table_schema AS "Shards' Schema"
|
||||
|
@ -743,7 +743,7 @@ SELECT create_distributed_table('table_set_schema', 'id');
|
|||
CREATE SCHEMA new_schema;
|
||||
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema');
|
||||
\c - - - :worker_1_port
|
||||
SELECT table_schema AS "Shards' Schema", COUNT(*) AS "Counts"
|
||||
|
@ -757,7 +757,7 @@ SET search_path TO old_schema;
|
|||
ALTER TABLE table_set_schema SET SCHEMA new_schema;
|
||||
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema');
|
||||
\c - - - :worker_1_port
|
||||
SELECT table_schema AS "Shards' Schema", COUNT(*) AS "Counts"
|
||||
|
@ -969,9 +969,9 @@ ROLLBACK;
|
|||
-- Clean up the created schema
|
||||
SET client_min_messages TO WARNING;
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.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
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object
|
||||
WHERE classid=2615 and objid IN (select oid from pg_namespace where nspname='run_test_schema');
|
||||
|
|
|
@ -418,8 +418,8 @@ CREATE TABLE test_seq_dist(a int, x BIGINT DEFAULT nextval('test_schema_for_sequ
|
|||
SELECT create_distributed_table('test_seq_dist', 'a');
|
||||
|
||||
-- Both sequence and dependency schema should be distributed
|
||||
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);
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.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 pg_catalog.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation'::regnamespace);
|
||||
|
||||
-- Show that sequence can stay on the worker node if the transaction is
|
||||
-- rollbacked after distributing the table
|
||||
|
|
|
@ -91,7 +91,7 @@ END;$$;
|
|||
CREATE OR REPLACE FUNCTION colocate_proc_with_table(procname text, tablerelid regclass, argument_index int)
|
||||
RETURNS void LANGUAGE plpgsql AS $$
|
||||
BEGIN
|
||||
update citus.pg_dist_object
|
||||
update pg_catalog.pg_dist_object
|
||||
set distribution_argument_index = argument_index, colocationid = pg_dist_partition.colocationid
|
||||
from pg_proc, pg_dist_partition
|
||||
where proname = procname and oid = objid and pg_dist_partition.logicalrelid = tablerelid;
|
||||
|
|
|
@ -17,11 +17,11 @@ CREATE SCHEMA test_sequence_schema;
|
|||
CREATE SEQUENCE test_sequence_schema.test_sequence;
|
||||
|
||||
-- show that none of the objects above are marked as distributed
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema'::regnamespace::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.mood'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema'::regnamespace::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.mood'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid;
|
||||
|
||||
SET client_min_messages TO ERROR;
|
||||
CREATE USER non_super_user_test_user;
|
||||
|
@ -70,28 +70,28 @@ SELECT create_distributed_function('plpgsql_dist_function(text)');
|
|||
-- show that schema, types, function and sequence has marked as distributed
|
||||
-- on the coordinator node
|
||||
RESET ROLE;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema'::regnamespace::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'test_sequence_schema'::regnamespace::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.mood'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.dist_table_e_seq'::regclass::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.plpgsql_dist_function'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema'::regnamespace::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'test_sequence_schema'::regnamespace::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.mood'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.dist_table_e_seq'::regclass::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.plpgsql_dist_function'::regproc::oid;
|
||||
|
||||
-- show those objects marked as distributed on metadata worker node as well
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema'::regnamespace::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'test_sequence_schema'::regnamespace::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.mood'::regtype::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.dist_table_e_seq'::regclass::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.plpgsql_dist_function'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema'::regnamespace::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'test_sequence_schema'::regnamespace::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.mood'::regtype::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.dist_table_e_seq'::regclass::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.plpgsql_dist_function'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
-- Show that extension plpgsql is also marked as distributed as a dependency of plpgsl_dist_function
|
||||
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 '%{plpgsql}%';
|
||||
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 '%{plpgsql}%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{plpgsql}%';
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{plpgsql}%';$$) ORDER BY 1,2;
|
||||
|
||||
-- show that schema is owned by the superuser
|
||||
SELECT rolname FROM pg_roles JOIN pg_namespace ON(pg_namespace.nspowner = pg_roles.oid) WHERE nspname = 'local_schema';
|
||||
|
@ -122,36 +122,36 @@ DROP TABLE dist_table CASCADE;
|
|||
|
||||
RESET ROLE;
|
||||
SET search_path TO local_schema;
|
||||
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 '%dist_table_e_seq%';
|
||||
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 '%dist_table_e_seq%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%dist_table_e_seq%';
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%dist_table_e_seq%';$$) ORDER BY 1,2;
|
||||
|
||||
-- Show that altering the function's schema marks the schema distributed
|
||||
CREATE SCHEMA schema_to_prop_with_function;
|
||||
ALTER FUNCTION test_function SET SCHEMA schema_to_prop_with_function;
|
||||
|
||||
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 '%{schema_to_prop_with_function}%';
|
||||
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 '%{schema_to_prop_with_function}%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_function}%';
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_function}%';$$) ORDER BY 1,2;
|
||||
|
||||
-- Show that dropping the function removes the metadata from pg_dist_object
|
||||
-- on both coordinator and metadata worker node
|
||||
DROP FUNCTION schema_to_prop_with_function.test_function;
|
||||
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 '%test_function%';
|
||||
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 '%test_function%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_function%';
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_function%';$$) ORDER BY 1,2;
|
||||
|
||||
-- Show that altering the type's schema marks the schema distributed
|
||||
CREATE SCHEMA schema_to_prop_with_type;
|
||||
ALTER TYPE test_type SET SCHEMA schema_to_prop_with_type;
|
||||
|
||||
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 '%{schema_to_prop_with_type}%';
|
||||
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 '%{schema_to_prop_with_type}%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_type}%';
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_type}%';$$) ORDER BY 1,2;
|
||||
|
||||
-- Show that dropping type removes the metadata from pg_dist_object
|
||||
-- on both coordinator and metadata worker node
|
||||
DROP TYPE mood CASCADE;
|
||||
DROP TYPE schema_to_prop_with_type.test_type 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 '%test_type%' or obj_identifier::text like '%mood%';
|
||||
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 '%test_type%' or obj_identifier::text like '%mood%'$$) ORDER BY 1,2;
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_type%' or obj_identifier::text like '%mood%';
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_type%' or obj_identifier::text like '%mood%'$$) ORDER BY 1,2;
|
||||
|
||||
-- Show that distributed function related metadata are also propagated
|
||||
set citus.shard_replication_factor to 1;
|
||||
|
@ -171,41 +171,41 @@ END;$$;
|
|||
|
||||
-- create a distributed function and show its distribution_argument_index
|
||||
SELECT create_distributed_function('metadata_dist_test_proc(integer, integer)', 'dist_key', 'metadata_dist_test_table');
|
||||
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;
|
||||
SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE objid = 'metadata_dist_test_proc'::regproc;
|
||||
SELECT * FROM run_command_on_workers($$SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE objid = 'local_schema.metadata_dist_test_proc'::regproc;$$) ORDER BY 1,2;
|
||||
|
||||
-- re-distribute and show that now the distribution_argument_index is updated on both the coordinator and workers
|
||||
SELECT create_distributed_function('metadata_dist_test_proc(integer, integer)', 'dist_key_2', 'metadata_dist_test_table');
|
||||
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;
|
||||
SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE objid = 'metadata_dist_test_proc'::regproc;
|
||||
SELECT * FROM run_command_on_workers($$ SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE objid = 'local_schema.metadata_dist_test_proc'::regproc;$$) ORDER BY 1,2;
|
||||
|
||||
-- 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}%';
|
||||
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;
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{local_schema}%';
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{local_schema}%';$$) ORDER BY 1,2;
|
||||
|
||||
-- Show that extension and dependent sequence also created and marked as distributed
|
||||
CREATE SCHEMA extension_schema;
|
||||
CREATE EXTENSION ltree WITH SCHEMA extension_schema;
|
||||
|
||||
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 '%{extension_schema}%';
|
||||
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 '%{extension_schema}%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{extension_schema}%';
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{extension_schema}%';$$) ORDER BY 1,2;
|
||||
|
||||
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 '%{ltree}%';
|
||||
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 '%{ltree}%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{ltree}%';
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{ltree}%';$$) ORDER BY 1,2;
|
||||
|
||||
-- Show that dropping a distributed table drops the pg_dist_object entry on worker
|
||||
CREATE TABLE extension_schema.table_to_check_object(id int);
|
||||
SELECT create_distributed_table('extension_schema.table_to_check_object', 'id');
|
||||
|
||||
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 '%table_to_check_object%';
|
||||
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 '%table_to_check_object%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';$$) ORDER BY 1,2;
|
||||
|
||||
DROP TABLE extension_schema.table_to_check_object;
|
||||
|
||||
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 '%table_to_check_object%';
|
||||
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 '%table_to_check_object%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';$$) ORDER BY 1,2;
|
||||
|
||||
-- Revert the settings for following tests
|
||||
RESET citus.enable_ddl_propagation;
|
||||
|
|
|
@ -10,8 +10,8 @@ SET client_min_messages TO WARNING;
|
|||
CREATE EXTENSION seg;
|
||||
|
||||
-- make sure that both the schema and the extension is distributed
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test');
|
||||
|
||||
CREATE TABLE test_table (key int, value seg);
|
||||
SELECT create_distributed_table('test_table', 'key');
|
||||
|
@ -22,7 +22,7 @@ SELECT count(*) from pg_dist_partition where logicalrelid='extension''test.test_
|
|||
CREATE TYPE two_segs AS (seg_1 seg, seg_2 seg);
|
||||
|
||||
-- verify that the type that depends on the extension is also marked as distributed
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_type WHERE typname = 'two_segs' AND typnamespace = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test'));
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_type WHERE typname = 'two_segs' AND typnamespace = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test'));
|
||||
|
||||
-- now try to run CREATE EXTENSION within a transction block, all should work fine
|
||||
BEGIN;
|
||||
|
@ -40,7 +40,7 @@ BEGIN;
|
|||
COMMIT;
|
||||
|
||||
-- make sure that the extension is distributed even if we run create extension in a transaction block
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname = 'isn'$$);
|
||||
|
||||
|
||||
|
@ -72,7 +72,7 @@ ALTER EXTENSION isn SET SCHEMA public;
|
|||
SET search_path TO public;
|
||||
|
||||
-- make sure that the extension is distributed
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
|
||||
-- show that the ALTER EXTENSION command is propagated
|
||||
SELECT run_command_on_workers($$SELECT nspname from pg_namespace where oid=(SELECT extnamespace FROM pg_extension WHERE extname = 'isn')$$);
|
||||
|
@ -156,7 +156,7 @@ ROLLBACK;
|
|||
|
||||
-- at the end of the transaction block, we did not create isn extension in coordinator or worker nodes as we rollback'ed
|
||||
-- make sure that the extension is not distributed
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
|
||||
-- and the extension does not exist on workers
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname = 'isn'$$);
|
||||
|
@ -184,11 +184,11 @@ SELECT count(*) FROM pg_extension WHERE extname IN ('pg_buffercache', 'isn');
|
|||
-- drop extension should just work
|
||||
DROP EXTENSION seg CASCADE;
|
||||
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname = 'seg'$$);
|
||||
|
||||
-- make sure that the extension is not avaliable anymore as a distributed object
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn'));
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn'));
|
||||
|
||||
CREATE SCHEMA "extension'test";
|
||||
SET search_path TO "extension'test";
|
||||
|
@ -257,16 +257,16 @@ COMMIT;
|
|||
|
||||
-- Check the pg_dist_object
|
||||
SELECT pg_proc.proname as DistributedFunction
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
|
||||
SELECT run_command_on_workers($$
|
||||
SELECT count(*)
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
$$);
|
||||
|
||||
|
@ -274,15 +274,15 @@ $$);
|
|||
SELECT 1 from master_add_node('localhost', :worker_2_port);
|
||||
|
||||
-- make sure that both extensions are created on both nodes
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid IN (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn'));
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid IN (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn'));
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname IN ('seg', 'isn')$$);
|
||||
|
||||
-- Check the pg_dist_object on the both nodes
|
||||
SELECT run_command_on_workers($$
|
||||
SELECT count(*)
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
$$);
|
||||
|
||||
|
@ -290,16 +290,16 @@ DROP EXTENSION seg CASCADE;
|
|||
|
||||
-- Recheck the pg_dist_object
|
||||
SELECT pg_proc.proname as DistributedFunction
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
|
||||
SELECT run_command_on_workers($$
|
||||
SELECT count(*)
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
$$);
|
||||
|
||||
|
@ -309,33 +309,33 @@ CREATE EXTENSION seg;
|
|||
SET citus.enable_ddl_propagation TO true;
|
||||
|
||||
-- Check the extension in pg_dist_object
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND
|
||||
objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
SELECT run_command_on_workers($$
|
||||
SELECT count(*)
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
$$);
|
||||
|
||||
SELECT create_distributed_function('seg_in(cstring)', '$1', 'test_extension_function');
|
||||
|
||||
-- Recheck the extension in pg_dist_object
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND
|
||||
objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
|
||||
SELECT pg_proc.proname as DistributedFunction
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
|
||||
SELECT run_command_on_workers($$
|
||||
SELECT count(*)
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
$$);
|
||||
DROP EXTENSION seg;
|
||||
|
@ -350,11 +350,11 @@ SELECT create_distributed_table('test_extension_function', 'col1', colocate_with
|
|||
CREATE EXTENSION cube;
|
||||
|
||||
SELECT create_distributed_function('cube(float8[], float8[])', '$1', 'test_extension_function');
|
||||
SELECT distribution_argument_index FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND
|
||||
SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND
|
||||
objid = (SELECT oid FROM pg_proc WHERE prosrc = 'cube_a_f8_f8');
|
||||
|
||||
SELECT create_distributed_function('cube(float8[], float8[])', '$2', 'test_extension_function');
|
||||
SELECT distribution_argument_index FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND
|
||||
SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND
|
||||
objid = (SELECT oid FROM pg_proc WHERE prosrc = 'cube_a_f8_f8');
|
||||
ROLLBACK;
|
||||
|
||||
|
|
|
@ -35,7 +35,7 @@ SELECT 1 FROM citus_add_node('localhost', :master_port, groupId=>0);
|
|||
SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%propagate_foreign_server.array_agg%';$$);
|
||||
|
||||
-- verify that the aggregate is added top pg_dist_object on the new node
|
||||
SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'propagate_foreign_server.array_agg'::regproc;$$);
|
||||
SELECT run_command_on_workers($$SELECT count(*) from pg_catalog.pg_dist_object where objid = 'propagate_foreign_server.array_agg'::regproc;$$);
|
||||
|
||||
SELECT citus_add_local_table_to_metadata('foreign_table');
|
||||
ALTER TABLE foreign_table OWNER TO pg_monitor;
|
||||
|
|
|
@ -108,7 +108,7 @@ from
|
|||
(select
|
||||
pg_catalog.websearch_to_tsquery(
|
||||
cast(pg_catalog.regconfigin(cast(cast(null as cstring) as cstring)) as regconfig),
|
||||
cast((select type from citus.pg_dist_object limit 1 offset 1) as text)
|
||||
cast((select type from pg_catalog.pg_dist_object limit 1 offset 1) as text)
|
||||
) as c0,
|
||||
sample_0.org_id as c1,
|
||||
sample_0.id as c2,
|
||||
|
|
|
@ -221,7 +221,7 @@ SELECT * FROM run_command_on_workers($$ SELECT 'text_search.concurrent_index_con
|
|||
|
||||
-- verify the objid is correctly committed locally due to the somewhat convoluted commit and new transaction starting when creating an index concurrently
|
||||
SELECT pg_catalog.pg_identify_object_as_address(classid, objid, objsubid)
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE classid = 3602 AND objid = 'text_search.concurrent_index_config'::regconfig::oid;
|
||||
|
||||
-- verify old text search configurations get renamed if they are not the same as the newly propagated configuration.
|
||||
|
|
|
@ -26,4 +26,4 @@ DROP SCHEMA upgrade_basic CASCADE;
|
|||
-- will not be marked as distributed
|
||||
|
||||
-- see underlying objects
|
||||
SELECT i.* FROM citus.pg_dist_object, pg_identify_object_as_address(classid, objid, objsubid) i ORDER BY 1, 2, 3;
|
||||
SELECT i.* FROM pg_catalog.pg_dist_object, pg_identify_object_as_address(classid, objid, objsubid) i ORDER BY 1, 2, 3;
|
||||
|
|
|
@ -5,10 +5,10 @@ UPDATE pg_dist_node_metadata SET metadata=jsonb_set(metadata, '{partitioned_citu
|
|||
SELECT citus_finalize_upgrade_to_citus11(enforce_version_check:=false);
|
||||
|
||||
-- tables are objects with Citus 11+
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object WHERE objid IN ('post_11_upgrade'::regnamespace, 'post_11_upgrade.part_table'::regclass, 'post_11_upgrade.sensors'::regclass, 'post_11_upgrade.func_in_transaction_def'::regproc, 'post_11_upgrade.partial_index_test_config'::regconfig, 'post_11_upgrade.my_type'::regtype) ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object WHERE objid IN ('post_11_upgrade'::regnamespace, 'post_11_upgrade.part_table'::regclass, 'post_11_upgrade.sensors'::regclass, 'post_11_upgrade.func_in_transaction_def'::regproc, 'post_11_upgrade.partial_index_test_config'::regconfig, 'post_11_upgrade.my_type'::regtype) ORDER BY 1;
|
||||
|
||||
-- on all nodes
|
||||
SELECT run_command_on_workers($$SELECT array_agg(pg_identify_object_as_address(classid, objid, objsubid)) FROM citus.pg_dist_object WHERE objid IN ('post_11_upgrade'::regnamespace, 'post_11_upgrade.part_table'::regclass, 'post_11_upgrade.sensors'::regclass, 'post_11_upgrade.func_in_transaction_def'::regproc, 'post_11_upgrade.partial_index_test_config'::regconfig, 'post_11_upgrade.my_type'::regtype) ORDER BY 1;$$) ORDER BY 1;
|
||||
SELECT run_command_on_workers($$SELECT array_agg(pg_identify_object_as_address(classid, objid, objsubid)) FROM pg_catalog.pg_dist_object WHERE objid IN ('post_11_upgrade'::regnamespace, 'post_11_upgrade.part_table'::regclass, 'post_11_upgrade.sensors'::regclass, 'post_11_upgrade.func_in_transaction_def'::regproc, 'post_11_upgrade.partial_index_test_config'::regconfig, 'post_11_upgrade.my_type'::regtype) ORDER BY 1;$$) ORDER BY 1;
|
||||
|
||||
-- Create the necessary test utility function
|
||||
CREATE OR REPLACE FUNCTION activate_node_snapshot()
|
||||
|
|
Loading…
Reference in New Issue