Merge branch 'master' into velioglu/cyclic_dep

velioglu/cyclic_dep_with_tho
Burak Velioglu 2022-03-07 15:04:48 +03:00
commit 9b583fdb0d
No known key found for this signature in database
GPG Key ID: F6827E620F6549C6
109 changed files with 2063 additions and 2146 deletions

View File

@ -11,11 +11,13 @@
#include "postgres.h" #include "postgres.h"
#include "miscadmin.h" #include "miscadmin.h"
#include "distributed/commands/utility_hook.h" #include "distributed/commands/utility_hook.h"
#include "distributed/commands.h" #include "distributed/commands.h"
#include "distributed/metadata_utility.h" #include "distributed/metadata_utility.h"
#include "distributed/coordinator_protocol.h" #include "distributed/coordinator_protocol.h"
#include "distributed/metadata_sync.h" #include "distributed/metadata_sync.h"
#include "distributed/multi_partitioning_utils.h"
#include "distributed/worker_transaction.h" #include "distributed/worker_transaction.h"
#include "utils/builtins.h" #include "utils/builtins.h"
#include "utils/lsyscache.h" #include "utils/lsyscache.h"
@ -123,6 +125,10 @@ master_remove_distributed_table_metadata_from_workers(PG_FUNCTION_ARGS)
* The function is a no-op for non-distributed tables and clusters that don't * The function is a no-op for non-distributed tables and clusters that don't
* have any workers with metadata. Also, the function errors out if called * have any workers with metadata. Also, the function errors out if called
* from a worker node. * from a worker node.
*
* This function assumed that it is called via a trigger. But we cannot do the
* typical CALLED_AS_TRIGGER check because this is called via another trigger,
* which CALLED_AS_TRIGGER does not cover.
*/ */
static void static void
MasterRemoveDistributedTableMetadataFromWorkers(Oid relationId, char *schemaName, MasterRemoveDistributedTableMetadataFromWorkers(Oid relationId, char *schemaName,
@ -146,6 +152,16 @@ MasterRemoveDistributedTableMetadataFromWorkers(Oid relationId, char *schemaName
return; return;
} }
if (PartitionTable(relationId))
{
/*
* MasterRemoveDistributedTableMetadataFromWorkers is only called from drop trigger.
* When parent is dropped in a drop trigger, we remove all the corresponding
* partitions via the parent, mostly for performance reasons.
*/
return;
}
/* drop the distributed table metadata on the workers */ /* drop the distributed table metadata on the workers */
char *deleteDistributionCommand = DistributionDeleteCommand(schemaName, tableName); char *deleteDistributionCommand = DistributionDeleteCommand(schemaName, tableName);
SendCommandToWorkersWithMetadata(deleteDistributionCommand); SendCommandToWorkersWithMetadata(deleteDistributionCommand);

View File

@ -69,6 +69,10 @@
(strncmp(arg, prefix, strlen(prefix)) == 0) (strncmp(arg, prefix, strlen(prefix)) == 0)
/* forward declaration for helper functions*/ /* forward declaration for helper functions*/
static bool RecreateSameNonColocatedFunction(ObjectAddress functionAddress,
char *distributionArgumentName,
bool colocateWithTableNameDefault,
bool *forceDelegationAddress);
static void ErrorIfAnyNodeDoesNotHaveMetadata(void); static void ErrorIfAnyNodeDoesNotHaveMetadata(void);
static char * GetAggregateDDLCommand(const RegProcedure funcOid, bool useCreateOrReplace); static char * GetAggregateDDLCommand(const RegProcedure funcOid, bool useCreateOrReplace);
static char * GetFunctionAlterOwnerCommand(const RegProcedure funcOid); static char * GetFunctionAlterOwnerCommand(const RegProcedure funcOid);
@ -128,6 +132,7 @@ create_distributed_function(PG_FUNCTION_ARGS)
char *distributionArgumentName = NULL; char *distributionArgumentName = NULL;
char *colocateWithTableName = NULL; char *colocateWithTableName = NULL;
bool colocateWithTableNameDefault = false;
bool *forceDelegationAddress = NULL; bool *forceDelegationAddress = NULL;
bool forceDelegation = false; bool forceDelegation = false;
ObjectAddress extensionAddress = { 0 }; ObjectAddress extensionAddress = { 0 };
@ -167,8 +172,13 @@ create_distributed_function(PG_FUNCTION_ARGS)
colocateWithText = PG_GETARG_TEXT_P(2); colocateWithText = PG_GETARG_TEXT_P(2);
colocateWithTableName = text_to_cstring(colocateWithText); colocateWithTableName = text_to_cstring(colocateWithText);
if (pg_strncasecmp(colocateWithTableName, "default", NAMEDATALEN) == 0)
{
colocateWithTableNameDefault = true;
}
/* check if the colocation belongs to a reference table */ /* check if the colocation belongs to a reference table */
if (pg_strncasecmp(colocateWithTableName, "default", NAMEDATALEN) != 0) if (!colocateWithTableNameDefault)
{ {
Oid colocationRelationId = ResolveRelationId(colocateWithText, false); Oid colocationRelationId = ResolveRelationId(colocateWithText, false);
colocatedWithReferenceTable = IsCitusTableType(colocationRelationId, colocatedWithReferenceTable = IsCitusTableType(colocationRelationId,
@ -192,6 +202,20 @@ create_distributed_function(PG_FUNCTION_ARGS)
ObjectAddressSet(functionAddress, ProcedureRelationId, funcOid); ObjectAddressSet(functionAddress, ProcedureRelationId, funcOid);
if (RecreateSameNonColocatedFunction(functionAddress,
distributionArgumentName,
colocateWithTableNameDefault,
forceDelegationAddress))
{
char *schemaName = get_namespace_name(get_func_namespace(funcOid));
char *functionName = get_func_name(funcOid);
char *qualifiedName = quote_qualified_identifier(schemaName, functionName);
ereport(NOTICE, (errmsg("procedure %s is already distributed", qualifiedName),
errdetail("Citus distributes procedures with CREATE "
"[PROCEDURE|FUNCTION|AGGREGATE] commands")));
PG_RETURN_VOID();
}
/* /*
* If the function is owned by an extension, only update the * If the function is owned by an extension, only update the
* pg_dist_object, and not propagate the CREATE FUNCTION. Function * pg_dist_object, and not propagate the CREATE FUNCTION. Function
@ -259,6 +283,55 @@ create_distributed_function(PG_FUNCTION_ARGS)
} }
/*
* RecreateSameNonColocatedFunction returns true if the given parameters of
* create_distributed_function will not change anything on the given function.
* Returns false otherwise.
*/
static bool
RecreateSameNonColocatedFunction(ObjectAddress functionAddress,
char *distributionArgumentName,
bool colocateWithTableNameDefault,
bool *forceDelegationAddress)
{
DistObjectCacheEntry *cacheEntry =
LookupDistObjectCacheEntry(ProcedureRelationId,
functionAddress.objectId,
InvalidOid);
if (cacheEntry == NULL || !cacheEntry->isValid || !cacheEntry->isDistributed)
{
return false;
}
/*
* If the colocationId, forceDelegation and distributionArgIndex fields of a
* pg_dist_object entry of a distributed function are all set to zero, it means
* that function is either automatically distributed by ddl propagation, without
* calling create_distributed_function. Or, it could be distributed via
* create_distributed_function, but with no parameters.
*
* For these cases, calling create_distributed_function for that function,
* without parameters would be idempotent. Hence we can simply early return here,
* by providing a notice message to the user.
*/
/* are pg_dist_object fields set to zero? */
bool functionDistributedWithoutParams =
cacheEntry->colocationId == 0 &&
cacheEntry->forceDelegation == 0 &&
cacheEntry->distributionArgIndex == 0;
/* called create_distributed_function without parameters? */
bool distributingAgainWithNoParams =
distributionArgumentName == NULL &&
colocateWithTableNameDefault &&
forceDelegationAddress == NULL;
return functionDistributedWithoutParams && distributingAgainWithNoParams;
}
/* /*
* ErrorIfAnyNodeDoesNotHaveMetadata throws error if any * ErrorIfAnyNodeDoesNotHaveMetadata throws error if any
* of the worker nodes does not have the metadata. * of the worker nodes does not have the metadata.

View File

@ -220,8 +220,9 @@ MarkObjectDistributedLocally(const ObjectAddress *distAddress)
ObjectIdGetDatum(distAddress->objectId), ObjectIdGetDatum(distAddress->objectId),
Int32GetDatum(distAddress->objectSubId) Int32GetDatum(distAddress->objectSubId)
}; };
char *insertQuery = "INSERT INTO citus.pg_dist_object (classid, objid, objsubid) " char *insertQuery =
"VALUES ($1, $2, $3) ON CONFLICT DO NOTHING"; "INSERT INTO pg_catalog.pg_dist_object (classid, objid, objsubid) "
"VALUES ($1, $2, $3) ON CONFLICT DO NOTHING";
int spiStatus = ExecuteCommandAsSuperuser(insertQuery, paramCount, paramTypes, int spiStatus = ExecuteCommandAsSuperuser(insertQuery, paramCount, paramTypes,
paramValues); paramValues);
if (spiStatus < 0) if (spiStatus < 0)
@ -335,7 +336,7 @@ UnmarkObjectDistributed(const ObjectAddress *address)
Int32GetDatum(address->objectSubId) 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"; "objid = $2 AND objsubid = $3";
int spiStatus = ExecuteCommandAsSuperuser(deleteQuery, paramCount, paramTypes, int spiStatus = ExecuteCommandAsSuperuser(deleteQuery, paramCount, paramTypes,

View File

@ -247,8 +247,13 @@ static void GetPartitionTypeInputInfo(char *partitionKeyString, char partitionMe
Oid *intervalTypeId, int32 *intervalTypeMod); Oid *intervalTypeId, int32 *intervalTypeMod);
static void CachedNamespaceLookup(const char *nspname, Oid *cachedOid); static void CachedNamespaceLookup(const char *nspname, Oid *cachedOid);
static void CachedRelationLookup(const char *relationName, 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, static void CachedRelationNamespaceLookup(const char *relationName, Oid relnamespace,
Oid *cachedOid); Oid *cachedOid);
static void CachedRelationNamespaceLookupExtended(const char *relationName,
Oid renamespace, Oid *cachedOid,
bool missing_ok);
static ShardPlacement * ResolveGroupShardPlacement( static ShardPlacement * ResolveGroupShardPlacement(
GroupShardPlacement *groupShardPlacement, CitusTableCacheEntry *tableEntry, GroupShardPlacement *groupShardPlacement, CitusTableCacheEntry *tableEntry,
int shardIndex); int shardIndex);
@ -2321,8 +2326,37 @@ CitusCatalogNamespaceId(void)
Oid Oid
DistObjectRelationId(void) 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; return MetadataCache.distObjectRelationId;
} }
@ -2332,9 +2366,38 @@ DistObjectRelationId(void)
Oid Oid
DistObjectPrimaryKeyIndexId(void) DistObjectPrimaryKeyIndexId(void)
{ {
CachedRelationNamespaceLookup("pg_dist_object_pkey", /*
CitusCatalogNamespaceId(), * In older versions pg_dist_object was living in the `citus` namespace, With Citus 11
&MetadataCache.distObjectPrimaryKeyIndexId); * 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; 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 static void
CachedRelationNamespaceLookup(const char *relationName, Oid relnamespace, CachedRelationNamespaceLookup(const char *relationName, Oid relnamespace,
Oid *cachedOid) 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 */ /* force callbacks to be registered, so we always get notified upon changes */
InitializeCaches(); InitializeCaches();
@ -4602,7 +4686,7 @@ CachedRelationNamespaceLookup(const char *relationName, Oid relnamespace,
{ {
*cachedOid = get_relname_relid(relationName, relnamespace); *cachedOid = get_relname_relid(relationName, relnamespace);
if (*cachedOid == InvalidOid) if (*cachedOid == InvalidOid && !missing_ok)
{ {
ereport(ERROR, (errmsg( ereport(ERROR, (errmsg(
"cache lookup failed for %s, called too early?", "cache lookup failed for %s, called too early?",

View File

@ -2226,6 +2226,19 @@ CitusAuthHook(Port *port, int status)
"regular client connections", "regular client connections",
MaxClientConnections))); MaxClientConnections)));
} }
/*
* Right after this, before we assign global pid, this backend
* might get blocked by a DDL as that happens during parsing.
*
* That's why, lets mark the backend as an external backend
* which is likely to execute a distributed command.
*
* We do this so that this backend gets the chance to show
* up in citus_lock_waits.
*/
InitializeBackendData();
SetBackendDataDistributedCommandOriginator(true);
} }
/* let other authentication hooks to kick in first */ /* let other authentication hooks to kick in first */

View File

@ -22,26 +22,29 @@
#include "udfs/citus_internal_local_blocked_processes/11.0-1.sql" #include "udfs/citus_internal_local_blocked_processes/11.0-1.sql"
#include "udfs/citus_internal_global_blocked_processes/11.0-1.sql" #include "udfs/citus_internal_global_blocked_processes/11.0-1.sql"
#include "udfs/citus_worker_stat_activity/11.0-1.sql" #include "udfs/run_command_on_all_nodes/11.0-1.sql"
#include "udfs/citus_stat_activity/11.0-1.sql"
#include "udfs/worker_create_or_replace_object/11.0-1.sql" #include "udfs/worker_create_or_replace_object/11.0-1.sql"
#include "udfs/citus_isolation_test_session_is_blocked/11.0-1.sql" #include "udfs/citus_isolation_test_session_is_blocked/11.0-1.sql"
#include "udfs/citus_blocking_pids/11.0-1.sql" #include "udfs/citus_blocking_pids/11.0-1.sql"
#include "udfs/citus_calculate_gpid/11.0-1.sql" #include "udfs/citus_calculate_gpid/11.0-1.sql"
#include "udfs/citus_backend_gpid/11.0-1.sql" #include "udfs/citus_backend_gpid/11.0-1.sql"
CREATE VIEW citus.citus_worker_stat_activity AS DROP FUNCTION IF EXISTS pg_catalog.citus_dist_stat_activity() CASCADE;
SELECT * FROM pg_catalog.citus_worker_stat_activity(); DROP FUNCTION IF EXISTS pg_catalog.citus_worker_stat_activity() CASCADE;
ALTER VIEW citus.citus_worker_stat_activity SET SCHEMA pg_catalog;
GRANT SELECT ON pg_catalog.citus_worker_stat_activity TO PUBLIC;
#include "udfs/citus_dist_stat_activity/11.0-1.sql" #include "udfs/citus_dist_stat_activity/11.0-1.sql"
CREATE VIEW citus.citus_dist_stat_activity AS -- a very simple helper function defined for citus_lock_waits
SELECT * FROM pg_catalog.citus_dist_stat_activity(); CREATE OR REPLACE FUNCTION get_nodeid_for_groupid(groupIdInput int) RETURNS int AS $$
ALTER VIEW citus.citus_dist_stat_activity SET SCHEMA pg_catalog; DECLARE
GRANT SELECT ON pg_catalog.citus_dist_stat_activity TO PUBLIC; returnNodeNodeId int := 0;
begin
SELECT nodeId into returnNodeNodeId FROM pg_dist_node WHERE groupid = groupIdInput and nodecluster = current_setting('citus.cluster_name');
RETURN returnNodeNodeId;
end
$$ LANGUAGE plpgsql;
-- we have to recreate this view because recreated citus_dist_stat_activity that this view depends
#include "udfs/citus_lock_waits/11.0-1.sql" #include "udfs/citus_lock_waits/11.0-1.sql"
#include "udfs/pg_cancel_backend/11.0-1.sql" #include "udfs/pg_cancel_backend/11.0-1.sql"
@ -87,5 +90,7 @@ $$;
#include "udfs/citus_finalize_upgrade_to_citus11/11.0-1.sql" #include "udfs/citus_finalize_upgrade_to_citus11/11.0-1.sql"
#include "udfs/run_command_on_all_nodes/11.0-1.sql" ALTER TABLE citus.pg_dist_object SET SCHEMA pg_catalog;
#include "udfs/citus_stat_activity/11.0-1.sql" 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"

View File

@ -1,4 +1,6 @@
-- citus--11.0-1--10.2-4 -- 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); DROP FUNCTION pg_catalog.create_distributed_function(regprocedure, text, text, bool);
@ -124,8 +126,8 @@ DROP VIEW pg_catalog.citus_lock_waits;
DROP FUNCTION citus_internal_local_blocked_processes; DROP FUNCTION citus_internal_local_blocked_processes;
DROP FUNCTION citus_internal_global_blocked_processes; DROP FUNCTION citus_internal_global_blocked_processes;
DROP VIEW pg_catalog.citus_dist_stat_activity; DROP VIEW IF EXISTS pg_catalog.citus_dist_stat_activity;
DROP FUNCTION pg_catalog.citus_dist_stat_activity; DROP FUNCTION IF EXISTS pg_catalog.citus_dist_stat_activity;
CREATE OR REPLACE FUNCTION pg_catalog.citus_dist_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int, CREATE OR REPLACE FUNCTION pg_catalog.citus_dist_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int,
OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name, OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name,
@ -151,8 +153,8 @@ ALTER VIEW citus.citus_dist_stat_activity SET SCHEMA pg_catalog;
GRANT SELECT ON pg_catalog.citus_dist_stat_activity TO PUBLIC; GRANT SELECT ON pg_catalog.citus_dist_stat_activity TO PUBLIC;
SET search_path = 'pg_catalog'; SET search_path = 'pg_catalog';
DROP VIEW citus_worker_stat_activity; DROP VIEW IF EXISTS citus_worker_stat_activity;
DROP FUNCTION citus_worker_stat_activity; DROP FUNCTION IF EXISTS citus_worker_stat_activity;
CREATE OR REPLACE FUNCTION citus_worker_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int, CREATE OR REPLACE FUNCTION citus_worker_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int,
OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name, OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name,
@ -355,6 +357,7 @@ GRANT SELECT ON pg_catalog.citus_lock_waits TO PUBLIC;
DROP FUNCTION pg_catalog.citus_finalize_upgrade_to_citus11(bool); DROP FUNCTION pg_catalog.citus_finalize_upgrade_to_citus11(bool);
DROP FUNCTION pg_catalog.citus_calculate_gpid(integer,integer); DROP FUNCTION pg_catalog.citus_calculate_gpid(integer,integer);
DROP FUNCTION pg_catalog.citus_backend_gpid(); DROP FUNCTION pg_catalog.citus_backend_gpid();
DROP FUNCTION get_nodeid_for_groupid(integer);
RESET search_path; RESET search_path;

View File

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

View File

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

View File

@ -12,22 +12,15 @@ BEGIN
IF substring(current_Setting('server_version'), '\d+')::int >= 14 THEN IF substring(current_Setting('server_version'), '\d+')::int >= 14 THEN
EXECUTE $cmd$ 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); CREATE AGGREGATE array_cat_agg(anycompatiblearray) (SFUNC = array_cat, STYPE = anycompatiblearray);
COMMENT ON AGGREGATE array_cat_agg(anycompatiblearray) COMMENT ON AGGREGATE array_cat_agg(anycompatiblearray)
IS 'concatenate input arrays into a single array'; IS 'concatenate input arrays into a single array';
RESET citus.enable_ddl_propagation;
$cmd$; $cmd$;
ELSE ELSE
EXECUTE $cmd$ EXECUTE $cmd$
SET citus.enable_ddl_propagation TO OFF;
CREATE AGGREGATE array_cat_agg(anyarray) (SFUNC = array_cat, STYPE = anyarray); CREATE AGGREGATE array_cat_agg(anyarray) (SFUNC = array_cat, STYPE = anyarray);
COMMENT ON AGGREGATE array_cat_agg(anyarray) COMMENT ON AGGREGATE array_cat_agg(anyarray)
IS 'concatenate input arrays into a single array'; IS 'concatenate input arrays into a single array';
RESET citus.enable_ddl_propagation;
$cmd$; $cmd$;
END IF; END IF;

View File

@ -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';

View File

@ -131,8 +131,8 @@ BEGIN
PERFORM citus_internal.columnar_ensure_am_depends_catalog(); PERFORM citus_internal.columnar_ensure_am_depends_catalog();
-- restore pg_dist_object from the stable identifiers -- restore pg_dist_object from the stable identifiers
TRUNCATE citus.pg_dist_object; TRUNCATE pg_catalog.pg_dist_object;
INSERT INTO citus.pg_dist_object (classid, objid, objsubid, distribution_argument_index, colocationid) INSERT INTO pg_catalog.pg_dist_object (classid, objid, objsubid, distribution_argument_index, colocationid)
SELECT SELECT
address.classid, address.classid,
address.objid, address.objid,

View File

@ -2,43 +2,44 @@ SET search_path = 'pg_catalog';
CREATE VIEW citus.citus_lock_waits AS CREATE VIEW citus.citus_lock_waits AS
WITH WITH
citus_dist_stat_activity AS unique_global_wait_edges_with_calculated_gpids AS (
( SELECT
SELECT * FROM citus_dist_stat_activity -- if global_pid is NULL, it is most likely that a backend is blocked on a DDL
-- also for legacy reasons citus_internal_global_blocked_processes() returns groupId, we replace that with nodeIds
case WHEN waiting_global_pid !=0 THEN waiting_global_pid ELSE citus_calculate_gpid(get_nodeid_for_groupid(waiting_node_id), waiting_pid) END waiting_global_pid,
case WHEN blocking_global_pid !=0 THEN blocking_global_pid ELSE citus_calculate_gpid(get_nodeid_for_groupid(blocking_node_id), blocking_pid) END blocking_global_pid,
-- citus_internal_global_blocked_processes returns groupId, we replace it here with actual
-- nodeId to be consisten with the other views
get_nodeid_for_groupid(blocking_node_id) as blocking_node_id,
get_nodeid_for_groupid(waiting_node_id) as waiting_node_id,
blocking_transaction_waiting
FROM citus_internal_global_blocked_processes()
), ),
unique_global_wait_edges AS unique_global_wait_edges AS
( (
SELECT DISTINCT ON(waiting_global_pid, blocking_global_pid) * FROM citus_internal_global_blocked_processes() SELECT DISTINCT ON(waiting_global_pid, blocking_global_pid) * FROM unique_global_wait_edges_with_calculated_gpids
), ),
citus_dist_stat_activity_with_node_id AS citus_dist_stat_activity_with_calculated_gpids AS
( (
SELECT -- if global_pid is NULL, it is most likely that a backend is blocked on a DDL
citus_dist_stat_activity.*, (CASE citus_dist_stat_activity.distributed_query_host_name WHEN 'coordinator_host' THEN 0 ELSE pg_dist_node.nodeid END) as initiator_node_id SELECT CASE WHEN global_pid != 0 THEN global_pid ELSE citus_calculate_gpid(nodeid, pid) END global_pid, nodeid, pid, query FROM citus_dist_stat_activity
FROM
citus_dist_stat_activity LEFT JOIN pg_dist_node
ON
citus_dist_stat_activity.distributed_query_host_name = pg_dist_node.nodename AND
citus_dist_stat_activity.distributed_query_host_port = pg_dist_node.nodeport
) )
SELECT SELECT
waiting.global_pid as waiting_gpid, waiting.global_pid as waiting_gpid,
blocking.global_pid as blocking_gpid, blocking.global_pid as blocking_gpid,
waiting.pid AS waiting_pid, waiting.query AS blocked_statement,
blocking.pid AS blocking_pid, blocking.query AS current_statement_in_blocking_process,
waiting.query AS blocked_statement, waiting.nodeid AS waiting_nodeid,
blocking.query AS current_statement_in_blocking_process, blocking.nodeid AS blocking_nodeid
waiting.initiator_node_id AS waiting_node_id,
blocking.initiator_node_id AS blocking_node_id,
waiting.distributed_query_host_name AS waiting_node_name,
blocking.distributed_query_host_name AS blocking_node_name,
waiting.distributed_query_host_port AS waiting_node_port,
blocking.distributed_query_host_port AS blocking_node_port
FROM FROM
unique_global_wait_edges unique_global_wait_edges
JOIN JOIN
citus_dist_stat_activity_with_node_id waiting ON (unique_global_wait_edges.waiting_global_pid = waiting.global_pid) citus_dist_stat_activity_with_calculated_gpids waiting ON (unique_global_wait_edges.waiting_global_pid = waiting.global_pid)
JOIN JOIN
citus_dist_stat_activity_with_node_id blocking ON (unique_global_wait_edges.blocking_global_pid = blocking.global_pid); citus_dist_stat_activity_with_calculated_gpids blocking ON (unique_global_wait_edges.blocking_global_pid = blocking.global_pid);
ALTER VIEW citus.citus_lock_waits SET SCHEMA pg_catalog; ALTER VIEW citus.citus_lock_waits SET SCHEMA pg_catalog;
GRANT SELECT ON pg_catalog.citus_lock_waits TO PUBLIC; GRANT SELECT ON pg_catalog.citus_lock_waits TO PUBLIC;

View File

@ -2,43 +2,44 @@ SET search_path = 'pg_catalog';
CREATE VIEW citus.citus_lock_waits AS CREATE VIEW citus.citus_lock_waits AS
WITH WITH
citus_dist_stat_activity AS unique_global_wait_edges_with_calculated_gpids AS (
( SELECT
SELECT * FROM citus_dist_stat_activity -- if global_pid is NULL, it is most likely that a backend is blocked on a DDL
-- also for legacy reasons citus_internal_global_blocked_processes() returns groupId, we replace that with nodeIds
case WHEN waiting_global_pid !=0 THEN waiting_global_pid ELSE citus_calculate_gpid(get_nodeid_for_groupid(waiting_node_id), waiting_pid) END waiting_global_pid,
case WHEN blocking_global_pid !=0 THEN blocking_global_pid ELSE citus_calculate_gpid(get_nodeid_for_groupid(blocking_node_id), blocking_pid) END blocking_global_pid,
-- citus_internal_global_blocked_processes returns groupId, we replace it here with actual
-- nodeId to be consisten with the other views
get_nodeid_for_groupid(blocking_node_id) as blocking_node_id,
get_nodeid_for_groupid(waiting_node_id) as waiting_node_id,
blocking_transaction_waiting
FROM citus_internal_global_blocked_processes()
), ),
unique_global_wait_edges AS unique_global_wait_edges AS
( (
SELECT DISTINCT ON(waiting_global_pid, blocking_global_pid) * FROM citus_internal_global_blocked_processes() SELECT DISTINCT ON(waiting_global_pid, blocking_global_pid) * FROM unique_global_wait_edges_with_calculated_gpids
), ),
citus_dist_stat_activity_with_node_id AS citus_dist_stat_activity_with_calculated_gpids AS
( (
SELECT -- if global_pid is NULL, it is most likely that a backend is blocked on a DDL
citus_dist_stat_activity.*, (CASE citus_dist_stat_activity.distributed_query_host_name WHEN 'coordinator_host' THEN 0 ELSE pg_dist_node.nodeid END) as initiator_node_id SELECT CASE WHEN global_pid != 0 THEN global_pid ELSE citus_calculate_gpid(nodeid, pid) END global_pid, nodeid, pid, query FROM citus_dist_stat_activity
FROM
citus_dist_stat_activity LEFT JOIN pg_dist_node
ON
citus_dist_stat_activity.distributed_query_host_name = pg_dist_node.nodename AND
citus_dist_stat_activity.distributed_query_host_port = pg_dist_node.nodeport
) )
SELECT SELECT
waiting.global_pid as waiting_gpid, waiting.global_pid as waiting_gpid,
blocking.global_pid as blocking_gpid, blocking.global_pid as blocking_gpid,
waiting.pid AS waiting_pid, waiting.query AS blocked_statement,
blocking.pid AS blocking_pid, blocking.query AS current_statement_in_blocking_process,
waiting.query AS blocked_statement, waiting.nodeid AS waiting_nodeid,
blocking.query AS current_statement_in_blocking_process, blocking.nodeid AS blocking_nodeid
waiting.initiator_node_id AS waiting_node_id,
blocking.initiator_node_id AS blocking_node_id,
waiting.distributed_query_host_name AS waiting_node_name,
blocking.distributed_query_host_name AS blocking_node_name,
waiting.distributed_query_host_port AS waiting_node_port,
blocking.distributed_query_host_port AS blocking_node_port
FROM FROM
unique_global_wait_edges unique_global_wait_edges
JOIN JOIN
citus_dist_stat_activity_with_node_id waiting ON (unique_global_wait_edges.waiting_global_pid = waiting.global_pid) citus_dist_stat_activity_with_calculated_gpids waiting ON (unique_global_wait_edges.waiting_global_pid = waiting.global_pid)
JOIN JOIN
citus_dist_stat_activity_with_node_id blocking ON (unique_global_wait_edges.blocking_global_pid = blocking.global_pid); citus_dist_stat_activity_with_calculated_gpids blocking ON (unique_global_wait_edges.blocking_global_pid = blocking.global_pid);
ALTER VIEW citus.citus_lock_waits SET SCHEMA pg_catalog; ALTER VIEW citus.citus_lock_waits SET SCHEMA pg_catalog;
GRANT SELECT ON pg_catalog.citus_lock_waits TO PUBLIC; GRANT SELECT ON pg_catalog.citus_lock_waits TO PUBLIC;

View File

@ -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';

View File

@ -65,7 +65,7 @@ BEGIN
address.object_args, address.object_args,
objects.distribution_argument_index, objects.distribution_argument_index,
objects.colocationid 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; pg_catalog.pg_identify_object_as_address(objects.classid, objects.objid, objects.objsubid) address;
END; END;
$cppu$; $cppu$;

View File

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

View File

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

View File

@ -188,7 +188,7 @@ run_commands_on_session_level_connection_to_node(PG_FUNCTION_ARGS)
/* /*
* override_backend_data_command_originator is a wrapper around * override_backend_data_command_originator is a wrapper around
* OverrideBackendDataDistributedCommandOriginator(). * SetBackendDataDistributedCommandOriginator().
*/ */
Datum Datum
override_backend_data_command_originator(PG_FUNCTION_ARGS) override_backend_data_command_originator(PG_FUNCTION_ARGS)
@ -197,7 +197,7 @@ override_backend_data_command_originator(PG_FUNCTION_ARGS)
bool distributedCommandOriginator = PG_GETARG_BOOL(0); bool distributedCommandOriginator = PG_GETARG_BOOL(0);
OverrideBackendDataDistributedCommandOriginator(distributedCommandOriginator); SetBackendDataDistributedCommandOriginator(distributedCommandOriginator);
PG_RETURN_VOID(); PG_RETURN_VOID();
} }

View File

@ -721,6 +721,7 @@ UnSetGlobalPID(void)
MyBackendData->globalPID = 0; MyBackendData->globalPID = 0;
MyBackendData->databaseId = 0; MyBackendData->databaseId = 0;
MyBackendData->userId = 0; MyBackendData->userId = 0;
MyBackendData->distributedCommandOriginator = false;
SpinLockRelease(&MyBackendData->mutex); SpinLockRelease(&MyBackendData->mutex);
} }
@ -850,12 +851,16 @@ AssignGlobalPID(void)
/* /*
* OverrideBackendDataDistributedCommandOriginator should only be used for isolation testing. * SetBackendDataDistributedCommandOriginator is used to set the distributedCommandOriginator
* See how it is used in the relevant functions. * field on MyBackendData.
*/ */
void void
OverrideBackendDataDistributedCommandOriginator(bool distributedCommandOriginator) SetBackendDataDistributedCommandOriginator(bool distributedCommandOriginator)
{ {
if (!MyBackendData)
{
return;
}
SpinLockAcquire(&MyBackendData->mutex); SpinLockAcquire(&MyBackendData->mutex);
MyBackendData->distributedCommandOriginator = MyBackendData->distributedCommandOriginator =
distributedCommandOriginator; distributedCommandOriginator;

File diff suppressed because it is too large Load Diff

View File

@ -29,6 +29,7 @@
#include "distributed/coordinator_protocol.h" #include "distributed/coordinator_protocol.h"
#include "distributed/metadata_cache.h" #include "distributed/metadata_cache.h"
#include "distributed/metadata/distobject.h" #include "distributed/metadata/distobject.h"
#include "distributed/multi_partitioning_utils.h"
#include "foreign/foreign.h" #include "foreign/foreign.h"
#include "utils/builtins.h" #include "utils/builtins.h"
#include "utils/fmgroids.h" #include "utils/fmgroids.h"
@ -37,12 +38,13 @@ PG_FUNCTION_INFO_V1(worker_drop_distributed_table);
PG_FUNCTION_INFO_V1(worker_drop_shell_table); PG_FUNCTION_INFO_V1(worker_drop_shell_table);
PG_FUNCTION_INFO_V1(worker_drop_sequence_dependency); PG_FUNCTION_INFO_V1(worker_drop_sequence_dependency);
static void WorkerDropDistributedTable(Oid relationId);
#if PG_VERSION_NUM < PG_VERSION_13 #if PG_VERSION_NUM < PG_VERSION_13
static long deleteDependencyRecordsForSpecific(Oid classId, Oid objectId, char deptype, static long deleteDependencyRecordsForSpecific(Oid classId, Oid objectId, char deptype,
Oid refclassId, Oid refobjectId); Oid refclassId, Oid refobjectId);
#endif #endif
/* /*
* worker_drop_distributed_table drops the distributed table with the given oid, * worker_drop_distributed_table drops the distributed table with the given oid,
* then, removes the associated rows from pg_dist_partition, pg_dist_shard and * then, removes the associated rows from pg_dist_partition, pg_dist_shard and
@ -64,8 +66,6 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS)
text *relationName = PG_GETARG_TEXT_P(0); text *relationName = PG_GETARG_TEXT_P(0);
Oid relationId = ResolveRelationId(relationName, true); Oid relationId = ResolveRelationId(relationName, true);
ObjectAddress distributedTableObject = { InvalidOid, InvalidOid, 0 };
if (!OidIsValid(relationId)) if (!OidIsValid(relationId))
{ {
ereport(NOTICE, (errmsg("relation %s does not exist, skipping", ereport(NOTICE, (errmsg("relation %s does not exist, skipping",
@ -75,8 +75,45 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS)
EnsureTableOwner(relationId); EnsureTableOwner(relationId);
List *shardList = LoadShardList(relationId); if (PartitionedTable(relationId))
{
/*
* When "DROP SCHEMA .. CASCADE" happens, we rely on Postgres' drop trigger
* to send the individual DROP TABLE commands for tables.
*
* In case of partitioned tables, we have no control on the order of DROP
* commands that is sent to the extension. We can try to sort while processing
* on the coordinator, but we prefer to handle it in a more flexible manner.
*
* That's why, whenever we see a partitioned table, we drop all the corresponding
* partitions first. Otherwise, WorkerDropDistributedTable() would already drop
* the shell tables of the partitions (e.g., due to performDeletion(..CASCADE),
* and further WorkerDropDistributedTable() on the partitions would become no-op.
*
* If, say one partition has already been dropped earlier, that should also be fine
* because we read the existing partitions.
*/
List *partitionList = PartitionList(relationId);
Oid partitionOid = InvalidOid;
foreach_oid(partitionOid, partitionList)
{
WorkerDropDistributedTable(partitionOid);
}
}
WorkerDropDistributedTable(relationId);
PG_RETURN_VOID();
}
/*
* WorkerDropDistributedTable is a helper function for worker_drop_distributed_table, see
* tha function for the details.
*/
static void
WorkerDropDistributedTable(Oid relationId)
{
/* first check the relation type */ /* first check the relation type */
Relation distributedRelation = relation_open(relationId, AccessShareLock); Relation distributedRelation = relation_open(relationId, AccessShareLock);
@ -86,9 +123,7 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS)
relation_close(distributedRelation, AccessShareLock); relation_close(distributedRelation, AccessShareLock);
/* prepare distributedTableObject for dropping the table */ /* prepare distributedTableObject for dropping the table */
distributedTableObject.classId = RelationRelationId; ObjectAddress distributedTableObject = { RelationRelationId, relationId, 0 };
distributedTableObject.objectId = relationId;
distributedTableObject.objectSubId = 0;
/* Drop dependent sequences from pg_dist_object */ /* Drop dependent sequences from pg_dist_object */
#if PG_VERSION_NUM >= PG_VERSION_13 #if PG_VERSION_NUM >= PG_VERSION_13
@ -121,6 +156,7 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS)
} }
/* iterate over shardList to delete the corresponding rows */ /* iterate over shardList to delete the corresponding rows */
List *shardList = LoadShardList(relationId);
uint64 *shardIdPointer = NULL; uint64 *shardIdPointer = NULL;
foreach_ptr(shardIdPointer, shardList) foreach_ptr(shardIdPointer, shardList)
{ {
@ -140,8 +176,6 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS)
/* delete the row from pg_dist_partition */ /* delete the row from pg_dist_partition */
DeletePartitionRow(relationId); DeletePartitionRow(relationId);
PG_RETURN_VOID();
} }

View File

@ -55,8 +55,8 @@ extern void UnSetGlobalPID(void);
extern void AssignDistributedTransactionId(void); extern void AssignDistributedTransactionId(void);
extern void AssignGlobalPID(void); extern void AssignGlobalPID(void);
extern uint64 GetGlobalPID(void); extern uint64 GetGlobalPID(void);
extern void OverrideBackendDataDistributedCommandOriginator(bool extern void SetBackendDataDistributedCommandOriginator(bool
distributedCommandOriginator); distributedCommandOriginator);
extern uint64 ExtractGlobalPID(char *applicationName); extern uint64 ExtractGlobalPID(char *applicationName);
extern int ExtractNodeIdFromGlobalPID(uint64 globalPID, bool missingOk); extern int ExtractNodeIdFromGlobalPID(uint64 globalPID, bool missingOk);
extern int ExtractProcessIdFromGlobalPID(uint64 globalPID); extern int ExtractProcessIdFromGlobalPID(uint64 globalPID);

View File

@ -86,7 +86,7 @@ extern void SyncDeleteColocationGroupToNodes(uint32 colocationId);
#define DELETE_ALL_NODES "DELETE FROM pg_dist_node" #define DELETE_ALL_NODES "DELETE FROM pg_dist_node"
#define DELETE_ALL_PLACEMENTS "DELETE FROM pg_dist_placement" #define DELETE_ALL_PLACEMENTS "DELETE FROM pg_dist_placement"
#define DELETE_ALL_SHARDS "DELETE FROM pg_dist_shard" #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_PARTITIONS "DELETE FROM pg_dist_partition"
#define DELETE_ALL_COLOCATION "DELETE FROM pg_catalog.pg_dist_colocation" #define DELETE_ALL_COLOCATION "DELETE FROM pg_catalog.pg_dist_colocation"
#define REMOVE_ALL_SHELL_TABLES_COMMAND \ #define REMOVE_ALL_SHELL_TABLES_COMMAND \

View File

@ -40,12 +40,16 @@ create aggregate sum2_strict (int) (
combinefunc = sum2_sfunc_strict combinefunc = sum2_sfunc_strict
); );
select create_distributed_function('sum2(int)'); select create_distributed_function('sum2(int)');
NOTICE: procedure aggregate_support.sum2 is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
select create_distributed_function('sum2_strict(int)'); select create_distributed_function('sum2_strict(int)');
NOTICE: procedure aggregate_support.sum2_strict is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -96,12 +100,16 @@ create aggregate psum_strict(int, int)(
initcond=0 initcond=0
); );
select create_distributed_function('psum(int,int)'); select create_distributed_function('psum(int,int)');
NOTICE: procedure aggregate_support.psum is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
select create_distributed_function('psum_strict(int,int)'); select create_distributed_function('psum_strict(int,int)');
NOTICE: procedure aggregate_support.psum_strict is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -283,7 +291,7 @@ create aggregate binstragg(text, text)(
stype=text stype=text
); );
-- verify that the aggregate is added into pg_dist_object, on each worker -- 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 run_command_on_workers
--------------------------------------------------------------------- ---------------------------------------------------------------------
(localhost,57637,t,1) (localhost,57637,t,1)
@ -298,6 +306,8 @@ SELECT run_command_on_workers($$select count(*) from pg_aggregate where aggfnoid
(2 rows) (2 rows)
select create_distributed_function('binstragg(text,text)'); select create_distributed_function('binstragg(text,text)');
NOTICE: procedure aggregate_support.binstragg is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -527,12 +537,16 @@ CREATE AGGREGATE last (
combinefunc = last_agg combinefunc = last_agg
); );
SELECT create_distributed_function('first(anyelement)'); SELECT create_distributed_function('first(anyelement)');
NOTICE: procedure aggregate_support.first is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
SELECT create_distributed_function('last(anyelement)'); SELECT create_distributed_function('last(anyelement)');
NOTICE: procedure aggregate_support.last is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -590,6 +604,8 @@ SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid
(2 rows) (2 rows)
select create_distributed_function('sumstring(text)'); select create_distributed_function('sumstring(text)');
NOTICE: procedure aggregate_support.sumstring is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -614,6 +630,8 @@ create aggregate array_collect_sort(el int) (
initcond = '{}' initcond = '{}'
); );
select create_distributed_function('array_collect_sort(int)'); select create_distributed_function('array_collect_sort(int)');
NOTICE: procedure aggregate_support.array_collect_sort is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -1139,7 +1157,7 @@ SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid
(2 rows) (2 rows)
-- verify that the aggregate is added into pg_dist_object, on each worker -- 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 run_command_on_workers
--------------------------------------------------------------------- ---------------------------------------------------------------------
(localhost,57637,t,1) (localhost,57637,t,1)
@ -1150,7 +1168,7 @@ RESET citus.create_object_propagation;
-- drop and test outside of tx block -- drop and test outside of tx block
drop aggregate dependent_agg (float8); drop aggregate dependent_agg (float8);
-- verify that the aggregate is removed from pg_dist_object, on each worker -- 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 run_command_on_workers
--------------------------------------------------------------------- ---------------------------------------------------------------------
(localhost,57637,f,"ERROR: function ""aggregate_support.dependent_agg"" does not exist") (localhost,57637,f,"ERROR: function ""aggregate_support.dependent_agg"" does not exist")

View File

@ -11,6 +11,8 @@ BEGIN
RETURN localGroupId; RETURN localGroupId;
END; $$ language plpgsql VOLATILE; END; $$ language plpgsql VOLATILE;
SELECT create_distributed_function('get_local_node_id_volatile()'); SELECT create_distributed_function('get_local_node_id_volatile()');
NOTICE: procedure coordinator_evaluation.get_local_node_id_volatile is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -24,6 +26,8 @@ BEGIN
RETURN localGroupId; RETURN localGroupId;
END; $$ language plpgsql VOLATILE; END; $$ language plpgsql VOLATILE;
SELECT create_distributed_function('get_local_node_id_volatile_sum_with_param(int)'); SELECT create_distributed_function('get_local_node_id_volatile_sum_with_param(int)');
NOTICE: procedure coordinator_evaluation.get_local_node_id_volatile_sum_with_param is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -21,6 +21,8 @@ BEGIN
RETURN localGroupId; RETURN localGroupId;
END; $$ language plpgsql STABLE; END; $$ language plpgsql STABLE;
SELECT create_distributed_function('get_local_node_id_stable()'); SELECT create_distributed_function('get_local_node_id_stable()');
NOTICE: procedure coordinator_evaluation_combinations_modify.get_local_node_id_stable is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -21,6 +21,8 @@ BEGIN
RETURN localGroupId; RETURN localGroupId;
END; $$ language plpgsql VOLATILE; END; $$ language plpgsql VOLATILE;
SELECT create_distributed_function('get_local_node_id_volatile()'); SELECT create_distributed_function('get_local_node_id_volatile()');
NOTICE: procedure coordinator_evaluation_combinations.get_local_node_id_volatile is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -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 DROP TABLE t4; -- as long as the table is using the type some operations are hard to force
COMMIT; COMMIT;
-- verify the type is distributed -- 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1

View File

@ -33,6 +33,8 @@ BEGIN
END; END;
$$; $$;
SELECT create_distributed_function('notice(text)'); SELECT create_distributed_function('notice(text)');
NOTICE: procedure function_tests.notice is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -193,7 +195,7 @@ SELECT create_distributed_function('"eq_mi''xed_param_names"(macaddr, macaddr)')
(1 row) (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; WHERE objid = 'eq_mi''xed_param_names(macaddr, macaddr)'::regprocedure;
?column? | ?column? ?column? | ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -486,6 +488,8 @@ AS 'select $1 = $2;'
IMMUTABLE IMMUTABLE
RETURNS NULL ON NULL INPUT; RETURNS NULL ON NULL INPUT;
select create_distributed_function('eq(macaddr,macaddr)'); select create_distributed_function('eq(macaddr,macaddr)');
NOTICE: procedure function_tests.eq is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -648,7 +652,7 @@ SELECT create_distributed_function('eq_with_param_names(macaddr, macaddr)', '$1'
-- show that the colocationIds are the same -- show that the colocationIds are the same
SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated 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 WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND
objects.objid = 'eq_with_param_names(macaddr, macaddr)'::regprocedure; objects.objid = 'eq_with_param_names(macaddr, macaddr)'::regprocedure;
table_and_function_colocated table_and_function_colocated
@ -665,7 +669,7 @@ SELECT create_distributed_function('eq_with_param_names(macaddr, macaddr)', 'val
(1 row) (1 row)
SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated 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 WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND
objects.objid = 'eq_with_param_names(macaddr, macaddr)'::regprocedure; objects.objid = 'eq_with_param_names(macaddr, macaddr)'::regprocedure;
table_and_function_colocated table_and_function_colocated
@ -700,7 +704,7 @@ SELECT create_distributed_function('eq8(macaddr8, macaddr8)', '$1', colocate_wit
(1 row) (1 row)
SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated 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 WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND
objects.objid = 'eq8(macaddr8, macaddr8)'::regprocedure; objects.objid = 'eq8(macaddr8, macaddr8)'::regprocedure;
table_and_function_colocated table_and_function_colocated
@ -715,7 +719,7 @@ SELECT create_distributed_function('add_text(text, text)', '$1', colocate_with:=
(1 row) (1 row)
SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated 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 WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND
objects.objid = 'add_text(text, text)'::regprocedure; objects.objid = 'add_text(text, text)'::regprocedure;
table_and_function_colocated table_and_function_colocated
@ -797,6 +801,8 @@ BEGIN
END; END;
$$; $$;
SELECT create_distributed_function('func_with_return_table(int)'); SELECT create_distributed_function('func_with_return_table(int)');
NOTICE: procedure function_tests.func_with_return_table is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -824,6 +830,8 @@ CREATE OR REPLACE FUNCTION func_with_out_param(a int, out b int)
RETURNS int RETURNS int
LANGUAGE sql AS $$ select 1; $$; LANGUAGE sql AS $$ select 1; $$;
SELECT create_distributed_function('func_with_out_param(int)'); SELECT create_distributed_function('func_with_out_param(int)');
NOTICE: procedure function_tests.func_with_out_param is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -869,6 +877,8 @@ SELECT create_distributed_function('func_with_inout_param(int)');
ERROR: function "func_with_inout_param(int)" does not exist ERROR: function "func_with_inout_param(int)" does not exist
-- this should work -- this should work
SELECT create_distributed_function('func_with_inout_param(int,int)'); SELECT create_distributed_function('func_with_inout_param(int,int)');
NOTICE: procedure function_tests.func_with_inout_param is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -893,6 +903,8 @@ CREATE OR REPLACE FUNCTION func_with_variadic_param(a int, variadic b int[])
LANGUAGE sql AS $$ select 1; $$; LANGUAGE sql AS $$ select 1; $$;
-- this should work -- this should work
SELECT create_distributed_function('func_with_variadic_param(int,int[])'); SELECT create_distributed_function('func_with_variadic_param(int,int[])');
NOTICE: procedure function_tests.func_with_variadic_param is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -923,6 +935,8 @@ $BODY$
LANGUAGE plpgsql VOLATILE LANGUAGE plpgsql VOLATILE
COST 100; COST 100;
SELECT create_distributed_function('func_returning_setof_int(date,interval)'); SELECT create_distributed_function('func_returning_setof_int(date,interval)');
NOTICE: procedure function_tests.func_returning_setof_int is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -961,6 +975,8 @@ $BODY$
LANGUAGE plpgsql VOLATILE LANGUAGE plpgsql VOLATILE
COST 100; COST 100;
SELECT create_distributed_function('func_returning_setof_int_with_variadic_param(date,int[])'); SELECT create_distributed_function('func_returning_setof_int_with_variadic_param(date,int[])');
NOTICE: procedure function_tests.func_returning_setof_int_with_variadic_param is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -997,6 +1013,8 @@ SELECT create_distributed_function('proc_with_variadic_param(date)');
ERROR: function "proc_with_variadic_param(date)" does not exist ERROR: function "proc_with_variadic_param(date)" does not exist
-- this should work -- this should work
SELECT create_distributed_function('proc_with_variadic_param(date,int[])'); SELECT create_distributed_function('proc_with_variadic_param(date,int[])');
NOTICE: procedure function_tests.proc_with_variadic_param is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -1028,6 +1046,8 @@ SELECT create_distributed_function('proc_with_inout_param(date)');
ERROR: function "proc_with_inout_param(date)" does not exist ERROR: function "proc_with_inout_param(date)" does not exist
-- this should work -- this should work
SELECT create_distributed_function('proc_with_inout_param(date,int)'); SELECT create_distributed_function('proc_with_inout_param(date,int)');
NOTICE: procedure function_tests.proc_with_inout_param is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -26,6 +26,8 @@ CREATE AGGREGATE existing_agg(int) (
STYPE = int STYPE = int
); );
SELECT create_distributed_function('existing_agg(int)'); SELECT create_distributed_function('existing_agg(int)');
NOTICE: procedure proc_conflict.existing_agg is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -87,6 +89,8 @@ CREATE AGGREGATE existing_agg(int) (
STYPE = int STYPE = int
); );
SELECT create_distributed_function('existing_agg(int)'); SELECT create_distributed_function('existing_agg(int)');
NOTICE: procedure proc_conflict.existing_agg is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -395,3 +395,117 @@ NOTICE: issuing ROLLBACK
DROP SCHEMA drop_partitioned_table CASCADE; DROP SCHEMA drop_partitioned_table CASCADE;
NOTICE: drop cascades to 3 other objects NOTICE: drop cascades to 3 other objects
SET search_path TO public; SET search_path TO public;
-- dropping the schema should drop the metadata on the workers
CREATE SCHEMA partitioning_schema;
SET search_path TO partitioning_schema;
CREATE TABLE part_table (
col timestamp
) PARTITION BY RANGE (col);
CREATE TABLE part_table_1
PARTITION OF part_table
FOR VALUES FROM ('2010-01-01') TO ('2015-01-01');
SELECT create_distributed_table('part_table', 'col');
create_distributed_table
---------------------------------------------------------------------
(1 row)
-- show we have pg_dist_partition entries on the workers
SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_partition where exists(select * from pg_class where pg_class.oid=pg_dist_partition.logicalrelid AND relname ILIKE '%part_table%');$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,2)
(localhost,57638,t,2)
(2 rows)
-- show we have pg_dist_object entries on the workers
SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_object as obj where classid = 1259 AND exists(select * from pg_class where pg_class.oid=obj.objid AND relname ILIKE '%part_table%');$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,2)
(localhost,57638,t,2)
(2 rows)
DROP SCHEMA partitioning_schema CASCADE;
NOTICE: drop cascades to table part_table
-- show we don't have pg_dist_partition entries on the workers after dropping the schema
SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_partition where exists(select * from pg_class where pg_class.oid=pg_dist_partition.logicalrelid AND relname ILIKE '%part_table%');$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,0)
(localhost,57638,t,0)
(2 rows)
-- show we don't have pg_dist_object entries on the workers after dropping the schema
SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_object as obj where classid = 1259 AND exists(select * from pg_class where pg_class.oid=obj.objid AND relname ILIKE '%part_table%');$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,0)
(localhost,57638,t,0)
(2 rows)
-- dropping the parent should drop the metadata on the workers
CREATE SCHEMA partitioning_schema;
SET search_path TO partitioning_schema;
CREATE TABLE part_table (
col timestamp
) PARTITION BY RANGE (col);
CREATE TABLE part_table_1
PARTITION OF part_table
FOR VALUES FROM ('2010-01-01') TO ('2015-01-01');
SELECT create_distributed_table('part_table', 'col');
create_distributed_table
---------------------------------------------------------------------
(1 row)
DROP TABLE part_table;
-- show we don't have pg_dist_partition entries on the workers after dropping the parent
SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_partition where exists(select * from pg_class where pg_class.oid=pg_dist_partition.logicalrelid AND relname ILIKE '%part_table%');$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,0)
(localhost,57638,t,0)
(2 rows)
-- show we don't have pg_dist_object entries on the workers after dropping the parent
SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_object as obj where classid = 1259 AND exists(select * from pg_class where pg_class.oid=obj.objid AND relname ILIKE '%part_table%');$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,0)
(localhost,57638,t,0)
(2 rows)
SET search_path TO partitioning_schema;
CREATE TABLE part_table (
col timestamp
) PARTITION BY RANGE (col);
CREATE TABLE part_table_1
PARTITION OF part_table
FOR VALUES FROM ('2010-01-01') TO ('2015-01-01');
SELECT create_distributed_table('part_table', 'col');
create_distributed_table
---------------------------------------------------------------------
(1 row)
DROP TABLE part_table_1;
-- show we have pg_dist_partition entries for the parent on the workers after dropping the partition
SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_partition where exists(select * from pg_class where pg_class.oid=pg_dist_partition.logicalrelid AND relname ILIKE '%part_table%');$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,1)
(localhost,57638,t,1)
(2 rows)
-- show we have pg_dist_object entries for the parent on the workers after dropping the partition
SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_object as obj where classid = 1259 AND exists(select * from pg_class where pg_class.oid=obj.objid AND relname ILIKE '%part_table%');$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,1)
(localhost,57638,t,1)
(2 rows)
-- clean-up
DROP SCHEMA partitioning_schema CASCADE;
NOTICE: drop cascades to table part_table

View File

@ -307,8 +307,8 @@ $$ LANGUAGE plpgsql;
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function('func_calls_forcepush_func()'); SELECT create_distributed_function('func_calls_forcepush_func()');
DEBUG: switching to sequential query execution mode NOTICE: procedure forcepushdown_schema.func_calls_forcepush_func is already distributed
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------

File diff suppressed because it is too large Load Diff

View File

@ -1182,6 +1182,7 @@ END;
$$ $$
LANGUAGE plpgsql STABLE; LANGUAGE plpgsql STABLE;
SELECT create_distributed_function('dist_func(int, int)'); SELECT create_distributed_function('dist_func(int, int)');
NOTICE: procedure insert_select_repartition.dist_func is already distributed
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -32,32 +32,32 @@ pg_sleep
(1 row) (1 row)
step s2-view-dist: step s2-view-dist:
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' and query not ILIKE '%BEGIN%' and query NOT ILIKE '%pg_catalog.pg_isolation_test_session_is_blocked%' ORDER BY query DESC; SELECT query, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%BEGIN%'), ('%pg_catalog.pg_isolation_test_session_is_blocked%')) AND backend_type = 'client backend' ORDER BY query DESC;
query |query_hostname |query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname query |state |wait_event_type|wait_event|usename |datname
--------------------------------------------------------------------- ---------------------------------------------------------------------
ALTER TABLE test_table ADD COLUMN x INT; ALTER TABLE test_table ADD COLUMN x INT;
|coordinator_host| 57636|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression |idle in transaction|Client |ClientRead|postgres|regression
(1 row) (1 row)
step s3-view-worker: step s3-view-worker:
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND backend_type = 'client backend' ORDER BY query DESC; SELECT query, state, wait_event_type, wait_event, usename, datname FROM citus_stat_activity WHERE query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%csa_from_one_node%')) AND is_worker_query = true AND backend_type = 'client backend' ORDER BY query DESC;
query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname query |state |wait_event_type|wait_event|usename |datname
--------------------------------------------------------------------- ---------------------------------------------------------------------
SELECT worker_apply_shard_ddl_command (1300004, 'public', ' SELECT worker_apply_shard_ddl_command (1300004, 'public', '
ALTER TABLE test_table ADD COLUMN x INT; ALTER TABLE test_table ADD COLUMN x INT;
')|localhost | 57638|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression ')|idle in transaction|Client |ClientRead|postgres|regression
SELECT worker_apply_shard_ddl_command (1300003, 'public', ' SELECT worker_apply_shard_ddl_command (1300003, 'public', '
ALTER TABLE test_table ADD COLUMN x INT; ALTER TABLE test_table ADD COLUMN x INT;
')|localhost | 57637|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression ')|idle in transaction|Client |ClientRead|postgres|regression
SELECT worker_apply_shard_ddl_command (1300002, 'public', ' SELECT worker_apply_shard_ddl_command (1300002, 'public', '
ALTER TABLE test_table ADD COLUMN x INT; ALTER TABLE test_table ADD COLUMN x INT;
')|localhost | 57638|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression ')|idle in transaction|Client |ClientRead|postgres|regression
SELECT worker_apply_shard_ddl_command (1300001, 'public', ' SELECT worker_apply_shard_ddl_command (1300001, 'public', '
ALTER TABLE test_table ADD COLUMN x INT; ALTER TABLE test_table ADD COLUMN x INT;
')|localhost | 57637|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression ')|idle in transaction|Client |ClientRead|postgres|regression
(4 rows) (4 rows)
step s2-rollback: step s2-rollback:
@ -102,21 +102,21 @@ pg_sleep
(1 row) (1 row)
step s2-view-dist: step s2-view-dist:
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' and query not ILIKE '%BEGIN%' and query NOT ILIKE '%pg_catalog.pg_isolation_test_session_is_blocked%' ORDER BY query DESC; SELECT query, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%BEGIN%'), ('%pg_catalog.pg_isolation_test_session_is_blocked%')) AND backend_type = 'client backend' ORDER BY query DESC;
query |query_hostname |query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname query |state |wait_event_type|wait_event|usename |datname
--------------------------------------------------------------------- ---------------------------------------------------------------------
INSERT INTO test_table VALUES (100, 100); INSERT INTO test_table VALUES (100, 100);
|coordinator_host| 57636|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression |idle in transaction|Client |ClientRead|postgres|regression
(1 row) (1 row)
step s3-view-worker: step s3-view-worker:
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND backend_type = 'client backend' ORDER BY query DESC; SELECT query, state, wait_event_type, wait_event, usename, datname FROM citus_stat_activity WHERE query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%csa_from_one_node%')) AND is_worker_query = true AND backend_type = 'client backend' ORDER BY query DESC;
query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname query |state |wait_event_type|wait_event|usename |datname
--------------------------------------------------------------------- ---------------------------------------------------------------------
INSERT INTO public.test_table_1300008 (column1, column2) VALUES (100, 100)|localhost | 57637|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression INSERT INTO public.test_table_1300008 (column1, column2) VALUES (100, 100)|idle in transaction|Client |ClientRead|postgres|regression
(1 row) (1 row)
step s2-rollback: step s2-rollback:
@ -166,24 +166,24 @@ pg_sleep
(1 row) (1 row)
step s2-view-dist: step s2-view-dist:
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' and query not ILIKE '%BEGIN%' and query NOT ILIKE '%pg_catalog.pg_isolation_test_session_is_blocked%' ORDER BY query DESC; SELECT query, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%BEGIN%'), ('%pg_catalog.pg_isolation_test_session_is_blocked%')) AND backend_type = 'client backend' ORDER BY query DESC;
query |query_hostname |query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname query |state |wait_event_type|wait_event|usename |datname
--------------------------------------------------------------------- ---------------------------------------------------------------------
SELECT count(*) FROM test_table; SELECT count(*) FROM test_table;
|coordinator_host| 57636|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression |idle in transaction|Client |ClientRead|postgres|regression
(1 row) (1 row)
step s3-view-worker: step s3-view-worker:
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND backend_type = 'client backend' ORDER BY query DESC; SELECT query, state, wait_event_type, wait_event, usename, datname FROM citus_stat_activity WHERE query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%csa_from_one_node%')) AND is_worker_query = true AND backend_type = 'client backend' ORDER BY query DESC;
query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname query |state |wait_event_type|wait_event|usename |datname
--------------------------------------------------------------------- ---------------------------------------------------------------------
SELECT count(*) AS count FROM public.test_table_1300014 test_table WHERE true|localhost | 57638|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression SELECT count(*) AS count FROM public.test_table_1300014 test_table WHERE true|idle in transaction|Client |ClientRead|postgres|regression
SELECT count(*) AS count FROM public.test_table_1300013 test_table WHERE true|localhost | 57637|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression SELECT count(*) AS count FROM public.test_table_1300013 test_table WHERE true|idle in transaction|Client |ClientRead|postgres|regression
SELECT count(*) AS count FROM public.test_table_1300012 test_table WHERE true|localhost | 57638|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression SELECT count(*) AS count FROM public.test_table_1300012 test_table WHERE true|idle in transaction|Client |ClientRead|postgres|regression
SELECT count(*) AS count FROM public.test_table_1300011 test_table WHERE true|localhost | 57637|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression SELECT count(*) AS count FROM public.test_table_1300011 test_table WHERE true|idle in transaction|Client |ClientRead|postgres|regression
(4 rows) (4 rows)
step s2-rollback: step s2-rollback:
@ -233,21 +233,21 @@ pg_sleep
(1 row) (1 row)
step s2-view-dist: step s2-view-dist:
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' and query not ILIKE '%BEGIN%' and query NOT ILIKE '%pg_catalog.pg_isolation_test_session_is_blocked%' ORDER BY query DESC; SELECT query, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%BEGIN%'), ('%pg_catalog.pg_isolation_test_session_is_blocked%')) AND backend_type = 'client backend' ORDER BY query DESC;
query |query_hostname |query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname query |state |wait_event_type|wait_event|usename |datname
--------------------------------------------------------------------- ---------------------------------------------------------------------
SELECT count(*) FROM test_table WHERE column1 = 55; SELECT count(*) FROM test_table WHERE column1 = 55;
|coordinator_host| 57636|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression |idle in transaction|Client |ClientRead|postgres|regression
(1 row) (1 row)
step s3-view-worker: step s3-view-worker:
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND backend_type = 'client backend' ORDER BY query DESC; SELECT query, state, wait_event_type, wait_event, usename, datname FROM citus_stat_activity WHERE query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%csa_from_one_node%')) AND is_worker_query = true AND backend_type = 'client backend' ORDER BY query DESC;
query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname query |state |wait_event_type|wait_event|usename |datname
--------------------------------------------------------------------- ---------------------------------------------------------------------
SELECT count(*) AS count FROM public.test_table_1300017 test_table WHERE (column1 OPERATOR(pg_catalog.=) 55)|localhost | 57638|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression SELECT count(*) AS count FROM public.test_table_1300017 test_table WHERE (column1 OPERATOR(pg_catalog.=) 55)|idle in transaction|Client |ClientRead|postgres|regression
(1 row) (1 row)
step s2-rollback: step s2-rollback:

View File

@ -9,7 +9,7 @@ starting permutation: s1-print-distributed-objects s1-begin s1-add-worker s2-pub
step s1-print-distributed-objects: step s1-print-distributed-objects:
SELECT 1 FROM master_add_node('localhost', 57638); SELECT 1 FROM master_add_node('localhost', 57638);
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -102,7 +102,7 @@ create_distributed_table
step s2-print-distributed-objects: step s2-print-distributed-objects:
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -180,7 +180,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-begin s1-add-work
step s1-print-distributed-objects: step s1-print-distributed-objects:
SELECT 1 FROM master_add_node('localhost', 57638); SELECT 1 FROM master_add_node('localhost', 57638);
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -279,7 +279,7 @@ step s2-commit:
step s2-print-distributed-objects: step s2-print-distributed-objects:
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -357,7 +357,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-begin s2-public-s
step s1-print-distributed-objects: step s1-print-distributed-objects:
SELECT 1 FROM master_add_node('localhost', 57638); SELECT 1 FROM master_add_node('localhost', 57638);
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -456,7 +456,7 @@ step s1-commit:
step s2-print-distributed-objects: step s2-print-distributed-objects:
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -534,7 +534,7 @@ starting permutation: s1-print-distributed-objects s1-begin s1-add-worker s2-cre
step s1-print-distributed-objects: step s1-print-distributed-objects:
SELECT 1 FROM master_add_node('localhost', 57638); SELECT 1 FROM master_add_node('localhost', 57638);
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -628,7 +628,7 @@ create_distributed_table
step s2-print-distributed-objects: step s2-print-distributed-objects:
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -707,7 +707,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-begin s1-add-work
step s1-print-distributed-objects: step s1-print-distributed-objects:
SELECT 1 FROM master_add_node('localhost', 57638); SELECT 1 FROM master_add_node('localhost', 57638);
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -807,7 +807,7 @@ step s2-commit:
step s2-print-distributed-objects: step s2-print-distributed-objects:
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -886,7 +886,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-begin s2-create-s
step s1-print-distributed-objects: step s1-print-distributed-objects:
SELECT 1 FROM master_add_node('localhost', 57638); SELECT 1 FROM master_add_node('localhost', 57638);
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -986,7 +986,7 @@ step s1-commit:
step s2-print-distributed-objects: step s2-print-distributed-objects:
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -1065,7 +1065,7 @@ starting permutation: s1-print-distributed-objects s2-create-schema s1-begin s2-
step s1-print-distributed-objects: step s1-print-distributed-objects:
SELECT 1 FROM master_add_node('localhost', 57638); SELECT 1 FROM master_add_node('localhost', 57638);
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -1165,7 +1165,7 @@ step s2-commit:
step s2-print-distributed-objects: step s2-print-distributed-objects:
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -1244,7 +1244,7 @@ starting permutation: s1-print-distributed-objects s1-add-worker s2-create-schem
step s1-print-distributed-objects: step s1-print-distributed-objects:
SELECT 1 FROM master_add_node('localhost', 57638); SELECT 1 FROM master_add_node('localhost', 57638);
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -1358,7 +1358,7 @@ step s3-commit:
step s2-print-distributed-objects: step s2-print-distributed-objects:
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -1438,7 +1438,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-begin s3-begin s1
step s1-print-distributed-objects: step s1-print-distributed-objects:
SELECT 1 FROM master_add_node('localhost', 57638); SELECT 1 FROM master_add_node('localhost', 57638);
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -1560,7 +1560,7 @@ step s2-commit:
step s2-print-distributed-objects: step s2-print-distributed-objects:
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -1641,7 +1641,7 @@ starting permutation: s1-print-distributed-objects s1-begin s1-add-worker s2-pub
step s1-print-distributed-objects: step s1-print-distributed-objects:
SELECT 1 FROM master_add_node('localhost', 57638); SELECT 1 FROM master_add_node('localhost', 57638);
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -1726,7 +1726,7 @@ step s1-commit:
step s2-create-type: <... completed> step s2-create-type: <... completed>
step s2-print-distributed-objects: step s2-print-distributed-objects:
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -1804,7 +1804,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-public-schema s2-
step s1-print-distributed-objects: step s1-print-distributed-objects:
SELECT 1 FROM master_add_node('localhost', 57638); SELECT 1 FROM master_add_node('localhost', 57638);
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -1888,7 +1888,7 @@ step s1-commit:
step s2-print-distributed-objects: step s2-print-distributed-objects:
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -1966,7 +1966,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-begin s2-create-s
step s1-print-distributed-objects: step s1-print-distributed-objects:
SELECT 1 FROM master_add_node('localhost', 57638); SELECT 1 FROM master_add_node('localhost', 57638);
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -2069,7 +2069,7 @@ step s1-commit:
step s2-print-distributed-objects: step s2-print-distributed-objects:
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -2149,7 +2149,7 @@ starting permutation: s1-print-distributed-objects s1-begin s1-add-worker s2-pub
step s1-print-distributed-objects: step s1-print-distributed-objects:
SELECT 1 FROM master_add_node('localhost', 57638); SELECT 1 FROM master_add_node('localhost', 57638);
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -2254,7 +2254,7 @@ wait_until_metadata_sync
step s2-print-distributed-objects: step s2-print-distributed-objects:
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -2332,7 +2332,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-public-schema s2-
step s1-print-distributed-objects: step s1-print-distributed-objects:
SELECT 1 FROM master_add_node('localhost', 57638); SELECT 1 FROM master_add_node('localhost', 57638);
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -2444,7 +2444,7 @@ wait_until_metadata_sync
step s2-print-distributed-objects: step s2-print-distributed-objects:
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -2522,7 +2522,7 @@ starting permutation: s1-print-distributed-objects s2-begin s2-create-schema s2-
step s1-print-distributed-objects: step s1-print-distributed-objects:
SELECT 1 FROM master_add_node('localhost', 57638); SELECT 1 FROM master_add_node('localhost', 57638);
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
@ -2635,7 +2635,7 @@ wait_until_metadata_sync
step s2-print-distributed-objects: step s2-print-distributed-objects:
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$); SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);

View File

@ -20,7 +20,7 @@ step s1-commit:
step s2-create-extension-version-11: <... completed> step s2-create-extension-version-11: <... completed>
step s1-print: 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 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 extname from pg_extension where extname='seg'$$);
SELECT run_command_on_workers($$select extversion 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 s2-alter-extension-update-to-version-12: <... completed>
step s1-print: 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 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 extname from pg_extension where extname='seg'$$);
SELECT run_command_on_workers($$select extversion 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 s2-drop-extension: <... completed>
step s1-print: 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 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 extname from pg_extension where extname='seg'$$);
SELECT run_command_on_workers($$select extversion 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 s2-create-extension-with-schema1: <... completed>
step s1-print: 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 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 extname from pg_extension where extname='seg'$$);
SELECT run_command_on_workers($$select extversion 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 s2-drop-extension: <... completed>
step s1-print: 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 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 extname from pg_extension where extname='seg'$$);
SELECT run_command_on_workers($$select extversion 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 s2-alter-extension-set-schema3: <... completed>
step s1-print: 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 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 extname from pg_extension where extname='seg'$$);
SELECT run_command_on_workers($$select extversion 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 s2-create-extension-with-schema1: <... completed>
step s1-print: 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 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 extname from pg_extension where extname='seg'$$);
SELECT run_command_on_workers($$select extversion 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; COMMIT;
step s1-print: 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 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 extname from pg_extension where extname='seg'$$);
SELECT run_command_on_workers($$select extversion 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) (1 row)
step s1-print: 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 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 extname from pg_extension where extname='seg'$$);
SELECT run_command_on_workers($$select extversion 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) (1 row)
step s1-print: 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 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 extname from pg_extension where extname='seg'$$);
SELECT run_command_on_workers($$select extversion 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; COMMIT;
step s1-print: 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 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 extname from pg_extension where extname='seg'$$);
SELECT run_command_on_workers($$select extversion 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) (1 row)
step s1-print: 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 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 extname from pg_extension where extname='seg'$$);
SELECT run_command_on_workers($$select extversion 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; COMMIT;
step s1-print: 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 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 extname from pg_extension where extname='seg'$$);
SELECT run_command_on_workers($$select extversion 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) (1 row)
step s1-print: 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 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 extname from pg_extension where extname='seg'$$);
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$); SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);

View File

@ -1,4 +1,4 @@
Parsed test spec with 4 sessions Parsed test spec with 8 sessions
starting permutation: s1-begin s1-update-ref-table-from-coordinator s2-start-session-level-connection s2-begin-on-worker s2-update-ref-table s3-select-distributed-waiting-queries s1-commit s2-commit-worker s2-stop-connection starting permutation: s1-begin s1-update-ref-table-from-coordinator s2-start-session-level-connection s2-begin-on-worker s2-update-ref-table s3-select-distributed-waiting-queries s1-commit s2-commit-worker s2-stop-connection
step s1-begin: step s1-begin:
@ -27,13 +27,13 @@ step s2-update-ref-table:
SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1');
<waiting ...> <waiting ...>
step s3-select-distributed-waiting-queries: step s3-select-distributed-waiting-queries:
SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%';
blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port blocked_statement |current_statement_in_blocking_process
--------------------------------------------------------------------- ---------------------------------------------------------------------
UPDATE ref_table SET value_1 = 12 WHERE user_id = 1| UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|
UPDATE ref_table SET value_1 = 15; UPDATE ref_table SET value_1 = 15;
|coordinator_host |coordinator_host | 57636| 57636
(1 row) (1 row)
step s1-commit: step s1-commit:
@ -61,7 +61,7 @@ stop_session_level_connection_to_node
(1 row) (1 row)
restore_isolation_tester_func citus_remove_node
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
@ -112,11 +112,11 @@ step s2-update-ref-table:
SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1');
<waiting ...> <waiting ...>
step s3-select-distributed-waiting-queries: step s3-select-distributed-waiting-queries:
SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%';
blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port blocked_statement |current_statement_in_blocking_process
--------------------------------------------------------------------- ---------------------------------------------------------------------
UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|coordinator_host |coordinator_host | 57636| 57636 UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|UPDATE ref_table SET value_1 = 12 WHERE user_id = 1
(1 row) (1 row)
step s1-commit-worker: step s1-commit-worker:
@ -157,7 +157,7 @@ stop_session_level_connection_to_node
(1 row) (1 row)
restore_isolation_tester_func citus_remove_node
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
@ -208,11 +208,11 @@ step s2-update-dist-table:
SELECT run_commands_on_session_level_connection_to_node('UPDATE tt1 SET value_1 = 5'); SELECT run_commands_on_session_level_connection_to_node('UPDATE tt1 SET value_1 = 5');
<waiting ...> <waiting ...>
step s3-select-distributed-waiting-queries: step s3-select-distributed-waiting-queries:
SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%';
blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port blocked_statement |current_statement_in_blocking_process
--------------------------------------------------------------------- ---------------------------------------------------------------------
UPDATE tt1 SET value_1 = 5|UPDATE tt1 SET value_1 = 4 |coordinator_host |coordinator_host | 57636| 57636 UPDATE tt1 SET value_1 = 5|UPDATE tt1 SET value_1 = 4
(1 row) (1 row)
step s1-commit-worker: step s1-commit-worker:
@ -253,7 +253,7 @@ stop_session_level_connection_to_node
(1 row) (1 row)
restore_isolation_tester_func citus_remove_node
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
@ -304,11 +304,11 @@ step s2-update-ref-table:
SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1');
<waiting ...> <waiting ...>
step s3-select-distributed-waiting-queries: step s3-select-distributed-waiting-queries:
SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%';
blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port blocked_statement |current_statement_in_blocking_process
--------------------------------------------------------------------- ---------------------------------------------------------------------
UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|DELETE FROM ref_table WHERE user_id = 1|coordinator_host |coordinator_host | 57636| 57636 UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|DELETE FROM ref_table WHERE user_id = 1
(1 row) (1 row)
step s1-commit-worker: step s1-commit-worker:
@ -349,7 +349,7 @@ stop_session_level_connection_to_node
(1 row) (1 row)
restore_isolation_tester_func citus_remove_node
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
@ -400,11 +400,11 @@ step s2-update-ref-table:
SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1');
<waiting ...> <waiting ...>
step s3-select-distributed-waiting-queries: step s3-select-distributed-waiting-queries:
SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%';
blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port blocked_statement |current_statement_in_blocking_process
--------------------------------------------------------------------- ---------------------------------------------------------------------
UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|INSERT INTO ref_table VALUES(8,81),(9,91)|coordinator_host |coordinator_host | 57636| 57636 UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|INSERT INTO ref_table VALUES(8,81),(9,91)
(1 row) (1 row)
step s1-commit-worker: step s1-commit-worker:
@ -445,7 +445,7 @@ stop_session_level_connection_to_node
(1 row) (1 row)
restore_isolation_tester_func citus_remove_node
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
@ -501,9 +501,9 @@ run_commands_on_session_level_connection_to_node
(1 row) (1 row)
step s3-select-distributed-waiting-queries: step s3-select-distributed-waiting-queries:
SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%';
blocked_statement|current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port blocked_statement|current_statement_in_blocking_process
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (0 rows)
@ -539,7 +539,7 @@ stop_session_level_connection_to_node
(1 row) (1 row)
restore_isolation_tester_func citus_remove_node
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
@ -590,11 +590,11 @@ step s2-update-ref-table:
SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1');
<waiting ...> <waiting ...>
step s3-select-distributed-waiting-queries: step s3-select-distributed-waiting-queries:
SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%';
blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port blocked_statement |current_statement_in_blocking_process
--------------------------------------------------------------------- ---------------------------------------------------------------------
UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|COPY ref_table FROM PROGRAM 'echo 10, 101 && echo 11, 111' WITH CSV|coordinator_host |coordinator_host | 57636| 57636 UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|COPY ref_table FROM PROGRAM 'echo 10, 101 && echo 11, 111' WITH CSV
(1 row) (1 row)
step s1-commit-worker: step s1-commit-worker:
@ -635,7 +635,7 @@ stop_session_level_connection_to_node
(1 row) (1 row)
restore_isolation_tester_func citus_remove_node
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
@ -691,9 +691,9 @@ run_commands_on_session_level_connection_to_node
(1 row) (1 row)
step s3-select-distributed-waiting-queries: step s3-select-distributed-waiting-queries:
SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%';
blocked_statement|current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port blocked_statement|current_statement_in_blocking_process
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (0 rows)
@ -729,7 +729,7 @@ stop_session_level_connection_to_node
(1 row) (1 row)
restore_isolation_tester_func citus_remove_node
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
@ -785,9 +785,9 @@ run_commands_on_session_level_connection_to_node
(1 row) (1 row)
step s3-select-distributed-waiting-queries: step s3-select-distributed-waiting-queries:
SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%';
blocked_statement|current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port blocked_statement|current_statement_in_blocking_process
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (0 rows)
@ -823,7 +823,7 @@ stop_session_level_connection_to_node
(1 row) (1 row)
restore_isolation_tester_func citus_remove_node
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
@ -874,11 +874,11 @@ step s2-update-ref-table:
SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1');
<waiting ...> <waiting ...>
step s3-select-distributed-waiting-queries: step s3-select-distributed-waiting-queries:
SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%';
blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port blocked_statement |current_statement_in_blocking_process
--------------------------------------------------------------------- ---------------------------------------------------------------------
UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|SELECT * FROM ref_table FOR UPDATE |coordinator_host |coordinator_host | 57636| 57636 UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|SELECT * FROM ref_table FOR UPDATE
(1 row) (1 row)
step s1-commit-worker: step s1-commit-worker:
@ -919,7 +919,7 @@ stop_session_level_connection_to_node
(1 row) (1 row)
restore_isolation_tester_func citus_remove_node
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
@ -957,13 +957,13 @@ step s1-alter-table:
ALTER TABLE ref_table ADD CONSTRAINT rf_p_key PRIMARY KEY(user_id); ALTER TABLE ref_table ADD CONSTRAINT rf_p_key PRIMARY KEY(user_id);
<waiting ...> <waiting ...>
step s3-select-distributed-waiting-queries: step s3-select-distributed-waiting-queries:
SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%';
blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port blocked_statement |current_statement_in_blocking_process
--------------------------------------------------------------------- ---------------------------------------------------------------------
ALTER TABLE ref_table ADD CONSTRAINT rf_p_key PRIMARY KEY(user_id); ALTER TABLE ref_table ADD CONSTRAINT rf_p_key PRIMARY KEY(user_id);
|INSERT INTO ref_table VALUES(8,81),(9,91)|coordinator_host |coordinator_host | 57636| 57636 |INSERT INTO ref_table VALUES(8,81),(9,91)
(1 row) (1 row)
step s2-commit-worker: step s2-commit-worker:
@ -986,7 +986,7 @@ stop_session_level_connection_to_node
(1 row) (1 row)
restore_isolation_tester_func citus_remove_node
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
@ -1003,22 +1003,22 @@ step s2-update-on-the-coordinator:
UPDATE tt1 SET value_1 = 4; UPDATE tt1 SET value_1 = 4;
<waiting ...> <waiting ...>
step s3-select-distributed-waiting-queries: step s3-select-distributed-waiting-queries:
SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%';
blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port blocked_statement |current_statement_in_blocking_process
--------------------------------------------------------------------- ---------------------------------------------------------------------
UPDATE tt1 SET value_1 = 4; UPDATE tt1 SET value_1 = 4;
| |
UPDATE tt1 SET value_1 = 4; UPDATE tt1 SET value_1 = 4;
|coordinator_host |coordinator_host | 57636| 57636
(1 row) (1 row)
step s1-commit: step s1-commit:
COMMIT; COMMIT;
step s2-update-on-the-coordinator: <... completed> step s2-update-on-the-coordinator: <... completed>
restore_isolation_tester_func citus_remove_node
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
@ -1069,11 +1069,11 @@ step s4-update-dist-table:
SELECT run_commands_on_session_level_connection_to_node('UPDATE tt1 SET value_1 = 5'); SELECT run_commands_on_session_level_connection_to_node('UPDATE tt1 SET value_1 = 5');
<waiting ...> <waiting ...>
step s3-select-distributed-waiting-queries: step s3-select-distributed-waiting-queries:
SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%';
blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port blocked_statement |current_statement_in_blocking_process
--------------------------------------------------------------------- ---------------------------------------------------------------------
UPDATE tt1 SET value_1 = 5|UPDATE tt1 SET value_1 = 4 |coordinator_host |coordinator_host | 57636| 57636 UPDATE tt1 SET value_1 = 5|UPDATE tt1 SET value_1 = 4
(1 row) (1 row)
step s1-commit-worker: step s1-commit-worker:
@ -1114,7 +1114,7 @@ stop_session_level_connection_to_node
(1 row) (1 row)
restore_isolation_tester_func citus_remove_node
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
@ -1157,11 +1157,11 @@ step s2-update-dist-table-id-1:
SELECT run_commands_on_session_level_connection_to_node('UPDATE tt1 SET value_1 = 4 WHERE user_id = 1'); SELECT run_commands_on_session_level_connection_to_node('UPDATE tt1 SET value_1 = 4 WHERE user_id = 1');
<waiting ...> <waiting ...>
step s3-select-distributed-waiting-queries: step s3-select-distributed-waiting-queries:
SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%';
blocked_statement |current_statement_in_blocking_process |waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port blocked_statement |current_statement_in_blocking_process
--------------------------------------------------------------------- ---------------------------------------------------------------------
UPDATE tt1 SET value_1 = 4 WHERE user_id = 1|UPDATE tt1 SET value_1 = 4 WHERE user_id = 1|coordinator_host |coordinator_host | 57636| 57636 UPDATE tt1 SET value_1 = 4 WHERE user_id = 1|UPDATE tt1 SET value_1 = 4 WHERE user_id = 1
(1 row) (1 row)
step s1-commit-worker: step s1-commit-worker:
@ -1194,13 +1194,13 @@ stop_session_level_connection_to_node
(1 row) (1 row)
restore_isolation_tester_func citus_remove_node
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
starting permutation: s1-begin s1-update-ref-table-from-coordinator s2-start-session-level-connection s2-update-ref-table s3-select-distributed-waiting-queries s1-commit s2-stop-connection starting permutation: s1-begin s1-update-ref-table-from-coordinator s2-start-session-level-connection s2-update-ref-table s3-select-distributed-waiting-queries s1-commit s2-stop-connection s5-begin s5-alter s6-select s3-select-distributed-waiting-queries s3-show-actual-gpids s5-rollback s8-begin s8-select s7-alter s3-select-distributed-waiting-queries s3-show-actual-gpids s8-rollback
step s1-begin: step s1-begin:
BEGIN; BEGIN;
@ -1219,13 +1219,13 @@ step s2-update-ref-table:
SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1'); SELECT run_commands_on_session_level_connection_to_node('UPDATE ref_table SET value_1 = 12 WHERE user_id = 1');
<waiting ...> <waiting ...>
step s3-select-distributed-waiting-queries: step s3-select-distributed-waiting-queries:
SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%';
blocked_statement |current_statement_in_blocking_process|waiting_node_name|blocking_node_name|waiting_node_port|blocking_node_port blocked_statement |current_statement_in_blocking_process
--------------------------------------------------------------------- ---------------------------------------------------------------------
UPDATE ref_table SET value_1 = 12 WHERE user_id = 1| UPDATE ref_table SET value_1 = 12 WHERE user_id = 1|
UPDATE ref_table SET value_1 = 15; UPDATE ref_table SET value_1 = 15;
|coordinator_host |coordinator_host | 57636| 57636
(1 row) (1 row)
step s1-commit: step s1-commit:
@ -1245,7 +1245,87 @@ stop_session_level_connection_to_node
(1 row) (1 row)
restore_isolation_tester_func step s5-begin:
BEGIN;
step s5-alter:
ALTER TABLE tt1 ADD COLUMN new_column INT;
step s6-select:
SELECT user_id FROM tt1 ORDER BY user_id DESC LIMIT 1;
<waiting ...>
step s3-select-distributed-waiting-queries:
SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%';
blocked_statement |current_statement_in_blocking_process
---------------------------------------------------------------------
SELECT user_id FROM tt1 ORDER BY user_id DESC LIMIT 1;
|
ALTER TABLE tt1 ADD COLUMN new_column INT;
(1 row)
step s3-show-actual-gpids:
SELECT global_pid > 0 as gpid_exists, query FROM citus_stat_activity WHERE state = 'active' AND query IN (SELECT blocked_statement FROM citus_lock_waits UNION SELECT current_statement_in_blocking_process FROM citus_lock_waits) ORDER BY 1 DESC;
gpid_exists|query
---------------------------------------------------------------------
f |
SELECT user_id FROM tt1 ORDER BY user_id DESC LIMIT 1;
(1 row)
step s5-rollback:
ROLLBACK;
step s6-select: <... completed>
user_id
---------------------------------------------------------------------
7
(1 row)
step s8-begin:
BEGIN;
step s8-select:
SELECT user_id FROM tt1 ORDER BY user_id DESC LIMIT 1;
user_id
---------------------------------------------------------------------
7
(1 row)
step s7-alter:
ALTER TABLE tt1 ADD COLUMN new_column INT;
<waiting ...>
step s3-select-distributed-waiting-queries:
SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%';
blocked_statement |current_statement_in_blocking_process
---------------------------------------------------------------------
ALTER TABLE tt1 ADD COLUMN new_column INT;
|
SELECT user_id FROM tt1 ORDER BY user_id DESC LIMIT 1;
(1 row)
step s3-show-actual-gpids:
SELECT global_pid > 0 as gpid_exists, query FROM citus_stat_activity WHERE state = 'active' AND query IN (SELECT blocked_statement FROM citus_lock_waits UNION SELECT current_statement_in_blocking_process FROM citus_lock_waits) ORDER BY 1 DESC;
gpid_exists|query
---------------------------------------------------------------------
t |
ALTER TABLE tt1 ADD COLUMN new_column INT;
(1 row)
step s8-rollback:
ROLLBACK;
step s7-alter: <... completed>
citus_remove_node
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)

View File

@ -1,6 +1,6 @@
Parsed test spec with 2 sessions Parsed test spec with 2 sessions
starting permutation: s1-start-session-level-connection s1-worker-begin s1-worker-select s2-coordinator-citus_stat_activity s2-coordinator-citus_dist_stat_activity s2-coordinator-citus_worker_stat_activity s1-worker-commit s1-stop-session-level-connection starting permutation: s1-start-session-level-connection s1-worker-begin s1-worker-select s2-coordinator-citus_stat_activity s2-coordinator-citus_dist_stat_activity s2-coordinator-citus_stat_activity-in-workers s1-worker-commit s1-stop-session-level-connection
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -39,7 +39,7 @@ t
(1 row) (1 row)
step s2-coordinator-citus_dist_stat_activity: step s2-coordinator-citus_dist_stat_activity:
SELECT query FROM citus_dist_stat_activity() WHERE global_pid IN ( SELECT query FROM citus_dist_stat_activity WHERE global_pid IN (
SELECT global_pid FROM citus_stat_activity WHERE query LIKE '%SELECT * FROM dist\_table%' SELECT global_pid FROM citus_stat_activity WHERE query LIKE '%SELECT * FROM dist\_table%'
) )
AND query NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND query NOT ILIKE '%run_commands_on_session_level_connection_to_node%'
@ -50,10 +50,12 @@ query
SET citus.enable_local_execution TO off; SET citus.force_max_query_parallelization TO ON; SELECT * FROM dist_table SET citus.enable_local_execution TO off; SET citus.force_max_query_parallelization TO ON; SELECT * FROM dist_table
(1 row) (1 row)
step s2-coordinator-citus_worker_stat_activity: step s2-coordinator-citus_stat_activity-in-workers:
SELECT query FROM citus_worker_stat_activity() WHERE global_pid IN ( SELECT query FROM citus_stat_activity WHERE global_pid IN (
SELECT global_pid FROM citus_stat_activity WHERE query LIKE '%SELECT * FROM dist\_table%' SELECT global_pid FROM citus_stat_activity WHERE query LIKE '%SELECT * FROM dist\_table%'
) )
AND is_worker_query = true
AND backend_type = 'client backend'
ORDER BY 1; ORDER BY 1;
query query
@ -86,7 +88,7 @@ citus_remove_node
(1 row) (1 row)
starting permutation: s1-coordinator-begin s1-coordinator-select s2-coordinator-citus_stat_activity s2-coordinator-citus_dist_stat_activity s2-coordinator-citus_worker_stat_activity s2-coordinator-get_all_active_transactions s2-coordinator-get_global_active_transactions s1-coordinator-commit starting permutation: s1-coordinator-begin s1-coordinator-select s2-coordinator-citus_stat_activity s2-coordinator-citus_dist_stat_activity s2-coordinator-citus_stat_activity-in-workers s2-coordinator-get_all_active_transactions s2-coordinator-get_global_active_transactions s1-coordinator-commit
create_distributed_table create_distributed_table
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -113,7 +115,7 @@ t
(1 row) (1 row)
step s2-coordinator-citus_dist_stat_activity: step s2-coordinator-citus_dist_stat_activity:
SELECT query FROM citus_dist_stat_activity() WHERE global_pid IN ( SELECT query FROM citus_dist_stat_activity WHERE global_pid IN (
SELECT global_pid FROM citus_stat_activity WHERE query LIKE '%SELECT * FROM dist\_table%' SELECT global_pid FROM citus_stat_activity WHERE query LIKE '%SELECT * FROM dist\_table%'
) )
AND query NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND query NOT ILIKE '%run_commands_on_session_level_connection_to_node%'
@ -128,10 +130,12 @@ query
(1 row) (1 row)
step s2-coordinator-citus_worker_stat_activity: step s2-coordinator-citus_stat_activity-in-workers:
SELECT query FROM citus_worker_stat_activity() WHERE global_pid IN ( SELECT query FROM citus_stat_activity WHERE global_pid IN (
SELECT global_pid FROM citus_stat_activity WHERE query LIKE '%SELECT * FROM dist\_table%' SELECT global_pid FROM citus_stat_activity WHERE query LIKE '%SELECT * FROM dist\_table%'
) )
AND is_worker_query = true
AND backend_type = 'client backend'
ORDER BY 1; ORDER BY 1;
query query

View File

@ -99,34 +99,37 @@ pg_sleep
(1 row) (1 row)
step s2-view-dist: step s2-view-dist:
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND query NOT ILIKE '%pg_isolation_test_session_is_blocked%' AND query NOT ILIKE '%BEGIN%' AND query NOT ILIKE '%add_node%' ORDER BY query DESC; SELECT query, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE backend_type = 'client backend' AND query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%pg_isolation_test_session_is_blocked%'), ('%BEGIN%'), ('%add_node%')) ORDER BY query DESC;
query |query_hostname |query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname query |state |wait_event_type|wait_event|usename |datname
--------------------------------------------------------------------- ---------------------------------------------------------------------
SELECT check_distributed_deadlocks(); SELECT check_distributed_deadlocks();
|coordinator_host| 57636| | 0|idle |Client |ClientRead|postgres|regression |idle |Client |ClientRead|postgres|regression
update ref_table set a = a + 1; update ref_table set a = a + 1;
|coordinator_host| 57636| | 0|idle in transaction|Client |ClientRead|postgres|regression |idle in transaction|Client |ClientRead|postgres|regression
(2 rows) (2 rows)
step s2-view-worker: step s2-view-worker:
SELECT query, query_hostname, query_hostport, distributed_query_host_name, SELECT query, state, wait_event_type, wait_event, usename, datname
distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_stat_activity
FROM citus_worker_stat_activity WHERE query NOT ILIKE ALL(VALUES
WHERE query NOT ILIKE '%pg_prepared_xacts%' AND ('%pg_prepared_xacts%'),
query NOT ILIKE '%COMMIT%' AND ('%COMMIT%'),
query NOT ILIKE '%dump_local_%' AND ('%dump_local_%'),
query NOT ILIKE '%citus_internal_local_blocked_processes%' AND ('%citus_internal_local_blocked_processes%'),
query NOT ILIKE '%add_node%' AND ('%add_node%'),
backend_type = 'client backend' ('%csa_from_one_node%'))
ORDER BY query, query_hostport DESC; AND is_worker_query = true
AND backend_type = 'client backend'
AND query != ''
ORDER BY query DESC;
query |query_hostname|query_hostport|distributed_query_host_name|distributed_query_host_port|state |wait_event_type|wait_event|usename |datname query |state |wait_event_type|wait_event|usename |datname
--------------------------------------------------------------------- ---------------------------------------------------------------------
UPDATE public.ref_table_1500777 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57638| | 0|idle in transaction|Client |ClientRead|postgres|regression UPDATE public.ref_table_1500777 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|idle in transaction|Client |ClientRead|postgres|regression
UPDATE public.ref_table_1500777 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|localhost | 57637| | 0|idle in transaction|Client |ClientRead|postgres|regression UPDATE public.ref_table_1500777 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|idle in transaction|Client |ClientRead|postgres|regression
(2 rows) (2 rows)
step s2-end: step s2-end:

View File

@ -55,6 +55,8 @@ BEGIN
RETURN localGroupId; RETURN localGroupId;
END; $$ language plpgsql VOLATILE; END; $$ language plpgsql VOLATILE;
SELECT create_distributed_function('get_local_node_id_volatile()'); SELECT create_distributed_function('get_local_node_id_volatile()');
NOTICE: procedure local_shard_execution.get_local_node_id_volatile is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -60,6 +60,8 @@ BEGIN
RETURN localGroupId; RETURN localGroupId;
END; $$ language plpgsql VOLATILE; END; $$ language plpgsql VOLATILE;
SELECT create_distributed_function('get_local_node_id_volatile()'); SELECT create_distributed_function('get_local_node_id_volatile()');
NOTICE: procedure local_shard_execution_replicated.get_local_node_id_volatile is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -2284,6 +2286,8 @@ BEGIN
END; END;
$fn$; $fn$;
SELECT create_distributed_function('register_for_event(int,int,invite_resp)'); SELECT create_distributed_function('register_for_event(int,int,invite_resp)');
NOTICE: procedure local_shard_execution_replicated.register_for_event is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -110,8 +110,8 @@ SELECT count(*) FROM history;
-- test we can replicate MX tables -- test we can replicate MX tables
SET citus.shard_replication_factor TO 1; SET citus.shard_replication_factor TO 1;
-- metadata sync will succeed even if we have rep > 1 tables -- 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 pg_catalog.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_single_shard'::regclass::oid, 0);
SELECT start_metadata_sync_to_node('localhost', :worker_1_port); SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
start_metadata_sync_to_node start_metadata_sync_to_node
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -242,8 +242,8 @@ SELECT master_remove_node('localhost', :worker_2_port);
(1 row) (1 row)
-- Removing public schema from pg_dist_object because it breaks the next tests -- 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 pg_catalog.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 = (SELECT oid FROM pg_extension WHERE extname = 'plpgsql');
-- try to manipulate node metadata via non-super user -- try to manipulate node metadata via non-super user
SET ROLE non_super_user; SET ROLE non_super_user;
SELECT 1 FROM master_add_inactive_node('localhost', :worker_2_port + 1); SELECT 1 FROM master_add_inactive_node('localhost', :worker_2_port + 1);

View File

@ -61,16 +61,6 @@ CREATE FUNCTION find_shard_interval_index(bigint)
RETURNS int RETURNS int
AS 'citus' AS 'citus'
LANGUAGE C STRICT; LANGUAGE C STRICT;
-- remove tables from pg_dist_partition, if they don't exist i.e not found in pg_class
delete from pg_dist_partition where not exists(select * from pg_class where pg_class.oid=pg_dist_partition.logicalrelid);
select 1 from run_command_on_workers($$
delete from pg_dist_partition where not exists(select * from pg_class where pg_class.oid=pg_dist_partition.logicalrelid);$$);
?column?
---------------------------------------------------------------------
1
1
(2 rows)
-- =================================================================== -- ===================================================================
-- test co-location util functions -- test co-location util functions
-- =================================================================== -- ===================================================================
@ -1048,11 +1038,11 @@ SELECT update_distributed_table_colocation('table1_group_none', colocate_with =>
(1 row) (1 row)
-- activate nodes to get rid of inconsistencies in pg_dist tables -- 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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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, 'table5_groupX'::regclass::oid, 0);
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port); SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
?column? ?column?
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -683,6 +683,8 @@ CREATE FUNCTION func_custom_param(IN param intpair, OUT total INT)
LANGUAGE SQL; LANGUAGE SQL;
SET citus.enable_metadata_sync TO OFF; SET citus.enable_metadata_sync TO OFF;
SELECT create_distributed_function('func_custom_param(intpair)'); SELECT create_distributed_function('func_custom_param(intpair)');
NOTICE: procedure function_tests.func_custom_param is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -697,6 +699,8 @@ CREATE FUNCTION func_returns_table(IN count INT)
LANGUAGE SQL; LANGUAGE SQL;
SET citus.enable_metadata_sync TO OFF; SET citus.enable_metadata_sync TO OFF;
SELECT create_distributed_function('func_returns_table(INT)'); SELECT create_distributed_function('func_returns_table(INT)');
NOTICE: procedure function_tests.func_returns_table is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -492,30 +492,17 @@ SELECT * FROM multi_extension.print_extension_changes();
-- Test downgrade to 9.4-1 from 9.5-1 -- Test downgrade to 9.4-1 from 9.5-1
ALTER EXTENSION citus UPDATE TO '9.5-1'; ALTER EXTENSION citus UPDATE TO '9.5-1';
BEGIN; -- 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
SET citus.enable_metadata_sync TO on; -- BEGIN;
SELECT master_add_node('localhost', :master_port, groupId=>0); -- SET citus.enable_metadata_sync TO on;
NOTICE: localhost:xxxxx is the coordinator and already contains metadata, skipping syncing the metadata -- SELECT master_add_node('localhost', :master_port, groupId=>0);
master_add_node -- CREATE TABLE citus_local_table (a int);
--------------------------------------------------------------------- -- SELECT create_citus_local_table('citus_local_table');
1 -- RESET citus.enable_metadata_sync;
(1 row) --
-- -- downgrade from 9.5-1 to 9.4-1 should fail as we have a citus local table
CREATE TABLE citus_local_table (a int); -- ALTER EXTENSION citus UPDATE TO '9.4-1';
SELECT create_citus_local_table('citus_local_table'); -- ROLLBACK;
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;
-- now we can downgrade as there is no citus local table -- now we can downgrade as there is no citus local table
ALTER EXTENSION citus UPDATE TO '9.4-1'; ALTER EXTENSION citus UPDATE TO '9.4-1';
-- Should be empty result since upgrade+downgrade should be a no-op -- Should be empty result since upgrade+downgrade should be a no-op
@ -1002,12 +989,16 @@ SELECT * FROM multi_extension.print_extension_changes();
previous_object | current_object previous_object | current_object
--------------------------------------------------------------------- ---------------------------------------------------------------------
function citus_disable_node(text,integer) void | function citus_disable_node(text,integer) void |
function citus_dist_stat_activity() SETOF record |
function citus_worker_stat_activity() SETOF record |
function create_distributed_function(regprocedure,text,text) void | function create_distributed_function(regprocedure,text,text) void |
function master_append_table_to_shard(bigint,text,text,integer) real | function master_append_table_to_shard(bigint,text,text,integer) real |
function master_apply_delete_command(text) integer | function master_apply_delete_command(text) integer |
function master_get_table_metadata(text) record | function master_get_table_metadata(text) record |
function worker_partition_query_result(text,text,integer,citus.distribution_type,text[],text[],boolean) SETOF record | function worker_partition_query_result(text,text,integer,citus.distribution_type,text[],text[],boolean) SETOF record |
| function citus_backend_gpid() bigint table citus.pg_dist_object |
view citus_worker_stat_activity |
| function citus_backend_gpid() bigint
| function citus_calculate_gpid(integer,integer) bigint | function citus_calculate_gpid(integer,integer) bigint
| function citus_check_cluster_node_health() SETOF record | function citus_check_cluster_node_health() SETOF record
| function citus_check_connection_to_node(text,integer) boolean | function citus_check_connection_to_node(text,integer) boolean
@ -1023,6 +1014,7 @@ SELECT * FROM multi_extension.print_extension_changes();
| function citus_shards_on_worker() SETOF record | function citus_shards_on_worker() SETOF record
| function citus_stat_activity() SETOF record | function citus_stat_activity() SETOF record
| function create_distributed_function(regprocedure,text,text,boolean) void | function create_distributed_function(regprocedure,text,text,boolean) void
| function get_nodeid_for_groupid(integer) integer
| function pg_cancel_backend(bigint) boolean | function pg_cancel_backend(bigint) boolean
| function pg_terminate_backend(bigint,bigint) boolean | function pg_terminate_backend(bigint,bigint) boolean
| function run_command_on_all_nodes(text,boolean,boolean) SETOF record | function run_command_on_all_nodes(text,boolean,boolean) SETOF record
@ -1030,8 +1022,9 @@ SELECT * FROM multi_extension.print_extension_changes();
| function worker_drop_sequence_dependency(text) void | function worker_drop_sequence_dependency(text) void
| function worker_drop_shell_table(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 | 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 | view citus_stat_activity
(30 rows) (36 rows)
DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff; DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff;
-- show running version -- show running version

View File

@ -169,6 +169,8 @@ BEGIN
END; END;
$function$; $function$;
SELECT create_distributed_function('stable_squared(int)'); SELECT create_distributed_function('stable_squared(int)');
NOTICE: procedure multi_function_evaluation.stable_squared is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -84,6 +84,8 @@ BEGIN
END; END;
$$ LANGUAGE plpgsql; $$ LANGUAGE plpgsql;
SELECT create_distributed_function('value_plus_one(int)'); SELECT create_distributed_function('value_plus_one(int)');
NOTICE: procedure multi_index_statements.value_plus_one is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -96,6 +98,8 @@ BEGIN
END; END;
$$ LANGUAGE plpgsql; $$ LANGUAGE plpgsql;
SELECT create_distributed_function('multi_index_statements_2.value_plus_one(int)'); SELECT create_distributed_function('multi_index_statements_2.value_plus_one(int)');
NOTICE: procedure multi_index_statements_2.value_plus_one is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -9,7 +9,7 @@ FROM pg_attribute
WHERE atthasmissing AND attrelid NOT IN ('pg_dist_node'::regclass, WHERE atthasmissing AND attrelid NOT IN ('pg_dist_node'::regclass,
'pg_dist_rebalance_strategy'::regclass, 'pg_dist_rebalance_strategy'::regclass,
'pg_dist_partition'::regclass, 'pg_dist_partition'::regclass,
'citus.pg_dist_object'::regclass) 'pg_dist_object'::regclass)
ORDER BY attrelid, attname; ORDER BY attrelid, attname;
attrelid | attname | atthasmissing | attmissingval attrelid | attname | atthasmissing | attmissingval
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -62,8 +62,8 @@ SELECT unnest(activate_node_snapshot()) order by 1;
unnest unnest
--------------------------------------------------------------------- ---------------------------------------------------------------------
CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION postgres 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_colocation
DELETE FROM pg_catalog.pg_dist_object
DELETE FROM pg_dist_node DELETE FROM pg_dist_node
DELETE FROM pg_dist_partition DELETE FROM pg_dist_partition
DELETE FROM pg_dist_placement 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 ALTER TABLE public.mx_test_table OWNER TO postgres
CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION 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)) 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_colocation
DELETE FROM pg_catalog.pg_dist_object
DELETE FROM pg_dist_node DELETE FROM pg_dist_node
DELETE FROM pg_dist_partition DELETE FROM pg_dist_partition
DELETE FROM pg_dist_placement 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 INDEX mx_index ON public.mx_test_table USING btree (col_2)
CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION 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)) 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_colocation
DELETE FROM pg_catalog.pg_dist_object
DELETE FROM pg_dist_node DELETE FROM pg_dist_node
DELETE FROM pg_dist_partition DELETE FROM pg_dist_partition
DELETE FROM pg_dist_placement 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 mx_testing_schema AUTHORIZATION postgres
CREATE SCHEMA IF NOT EXISTS public 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)) 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_colocation
DELETE FROM pg_catalog.pg_dist_object
DELETE FROM pg_dist_node DELETE FROM pg_dist_node
DELETE FROM pg_dist_partition DELETE FROM pg_dist_partition
DELETE FROM pg_dist_placement 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 mx_testing_schema AUTHORIZATION postgres
CREATE SCHEMA IF NOT EXISTS public 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)) 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_colocation
DELETE FROM pg_catalog.pg_dist_object
DELETE FROM pg_dist_node DELETE FROM pg_dist_node
DELETE FROM pg_dist_partition DELETE FROM pg_dist_partition
DELETE FROM pg_dist_placement 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 mx_testing_schema AUTHORIZATION postgres
CREATE SCHEMA IF NOT EXISTS public 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)) 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_colocation
DELETE FROM pg_catalog.pg_dist_object
DELETE FROM pg_dist_node DELETE FROM pg_dist_node
DELETE FROM pg_dist_partition DELETE FROM pg_dist_partition
DELETE FROM pg_dist_placement DELETE FROM pg_dist_placement
@ -1379,10 +1379,10 @@ DROP TABLE mx_table_with_small_sequence, mx_table_with_sequence;
-- owner -- owner
CREATE TABLE pg_dist_placement_temp AS SELECT * FROM pg_dist_placement; 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_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_placement;
DELETE FROM pg_dist_partition; 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 groupid AS old_worker_2_group FROM pg_dist_node WHERE nodeport = :worker_2_port \gset
SELECT master_remove_node('localhost', :worker_2_port); SELECT master_remove_node('localhost', :worker_2_port);
master_remove_node master_remove_node
@ -1459,7 +1459,7 @@ DROP TABLE mx_table;
\c - postgres - :master_port \c - postgres - :master_port
INSERT INTO pg_dist_placement SELECT * FROM pg_dist_placement_temp; INSERT INTO pg_dist_placement SELECT * FROM pg_dist_placement_temp;
INSERT INTO pg_dist_partition SELECT * FROM pg_dist_partition_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_placement_temp;
DROP TABLE pg_dist_partition_temp; DROP TABLE pg_dist_partition_temp;
DROP TABLE pg_dist_object_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.dist_table_1 (a integer)
CREATE TABLE public.mx_ref (col_1 integer, col_2 text) 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)) 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_colocation
DELETE FROM pg_catalog.pg_dist_object
DELETE FROM pg_dist_node DELETE FROM pg_dist_node
DELETE FROM pg_dist_partition DELETE FROM pg_dist_partition
DELETE FROM pg_dist_placement DELETE FROM pg_dist_placement

View File

@ -471,6 +471,8 @@ SELECT create_distributed_function('usage_access_func(usage_access_type,int[])')
ERROR: must be owner of function usage_access_func ERROR: must be owner of function usage_access_func
SET ROLE usage_access; SET ROLE usage_access;
SELECT create_distributed_function('usage_access_func(usage_access_type,int[])'); SELECT create_distributed_function('usage_access_func(usage_access_type,int[])');
NOTICE: procedure public.usage_access_func is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -191,7 +191,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg
test_proc_colocation_0 | 1410002 test_proc_colocation_0 | 1410002
(1 row) (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 proname | colocationid
--------------------------------------------------------------------- ---------------------------------------------------------------------
proc_0 | 1410002 proc_0 | 1410002
@ -226,7 +226,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg
test_proc_colocation_0 | 1410003 test_proc_colocation_0 | 1410003
(1 row) (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 proname | colocationid
--------------------------------------------------------------------- ---------------------------------------------------------------------
proc_0 | 1410003 proc_0 | 1410003
@ -272,7 +272,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg
test_proc_colocation_0 | 1410004 test_proc_colocation_0 | 1410004
(1 row) (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 proname | colocationid
--------------------------------------------------------------------- ---------------------------------------------------------------------
proc_0 | 1410004 proc_0 | 1410004
@ -305,7 +305,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg
test_proc_colocation_0 | 1410003 test_proc_colocation_0 | 1410003
(1 row) (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 proname | colocationid
--------------------------------------------------------------------- ---------------------------------------------------------------------
proc_0 | 1410003 proc_0 | 1410003
@ -359,7 +359,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg
test_proc_colocation_0 | 1410005 test_proc_colocation_0 | 1410005
(1 row) (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 proname | colocationid
--------------------------------------------------------------------- ---------------------------------------------------------------------
proc_0 | 1410005 proc_0 | 1410005
@ -389,7 +389,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg
test_proc_colocation_0 | 1410005 test_proc_colocation_0 | 1410005
(1 row) (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 proname | colocationid
--------------------------------------------------------------------- ---------------------------------------------------------------------
proc_0 | 1410005 proc_0 | 1410005
@ -440,7 +440,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg
test_proc_colocation_0 | 1410003 test_proc_colocation_0 | 1410003
(1 row) (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 proname | colocationid
--------------------------------------------------------------------- ---------------------------------------------------------------------
proc_0 | 1410003 proc_0 | 1410003
@ -465,7 +465,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg
test_proc_colocation_0 | 1410006 test_proc_colocation_0 | 1410006
(1 row) (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 proname | colocationid
--------------------------------------------------------------------- ---------------------------------------------------------------------
proc_0 | 1410003 proc_0 | 1410003

View File

@ -166,24 +166,32 @@ call mx_call_proc_custom_types('S', 'A');
-- Mark both procedures as distributed ... -- Mark both procedures as distributed ...
select create_distributed_function('mx_call_proc(int,int)'); select create_distributed_function('mx_call_proc(int,int)');
NOTICE: procedure multi_mx_call.mx_call_proc is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
select create_distributed_function('mx_call_proc_bigint(bigint,bigint)'); select create_distributed_function('mx_call_proc_bigint(bigint,bigint)');
NOTICE: procedure multi_mx_call.mx_call_proc_bigint is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
select create_distributed_function('mx_call_proc_custom_types(mx_call_enum,mx_call_enum)'); select create_distributed_function('mx_call_proc_custom_types(mx_call_enum,mx_call_enum)');
NOTICE: procedure multi_mx_call.mx_call_proc_custom_types is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
select create_distributed_function('mx_call_proc_copy(int)'); select create_distributed_function('mx_call_proc_copy(int)');
NOTICE: procedure multi_mx_call.mx_call_proc_copy is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -559,8 +567,8 @@ CREATE FUNCTION mx_call_add(int, int) RETURNS int
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function('mx_call_add(int,int)'); SELECT create_distributed_function('mx_call_add(int,int)');
DEBUG: switching to sequential query execution mode NOTICE: procedure multi_mx_call.mx_call_add is already distributed
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -166,24 +166,32 @@ call mx_call_proc_custom_types('S', 'A');
-- Mark both procedures as distributed ... -- Mark both procedures as distributed ...
select create_distributed_function('mx_call_proc(int,int)'); select create_distributed_function('mx_call_proc(int,int)');
NOTICE: procedure multi_mx_call.mx_call_proc is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
select create_distributed_function('mx_call_proc_bigint(bigint,bigint)'); select create_distributed_function('mx_call_proc_bigint(bigint,bigint)');
NOTICE: procedure multi_mx_call.mx_call_proc_bigint is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
select create_distributed_function('mx_call_proc_custom_types(mx_call_enum,mx_call_enum)'); select create_distributed_function('mx_call_proc_custom_types(mx_call_enum,mx_call_enum)');
NOTICE: procedure multi_mx_call.mx_call_proc_custom_types is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
select create_distributed_function('mx_call_proc_copy(int)'); select create_distributed_function('mx_call_proc_copy(int)');
NOTICE: procedure multi_mx_call.mx_call_proc_copy is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -559,8 +567,8 @@ CREATE FUNCTION mx_call_add(int, int) RETURNS int
DEBUG: switching to sequential query execution mode DEBUG: switching to sequential query execution mode
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands
SELECT create_distributed_function('mx_call_add(int,int)'); SELECT create_distributed_function('mx_call_add(int,int)');
DEBUG: switching to sequential query execution mode NOTICE: procedure multi_mx_call.mx_call_add is already distributed
DETAIL: A command for a distributed function is run. To make sure subsequent commands see the function correctly we need to make sure to use only one connection for all future commands DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -132,30 +132,40 @@ select mx_call_func(2, 0);
-- Mark both functions as distributed ... -- Mark both functions as distributed ...
select create_distributed_function('mx_call_func(int,int)'); select create_distributed_function('mx_call_func(int,int)');
NOTICE: procedure multi_mx_function_call_delegation.mx_call_func is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
select create_distributed_function('mx_call_func_bigint(bigint,bigint)'); select create_distributed_function('mx_call_func_bigint(bigint,bigint)');
NOTICE: procedure multi_mx_function_call_delegation.mx_call_func_bigint is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
select create_distributed_function('mx_call_func_custom_types(mx_call_enum,mx_call_enum)'); select create_distributed_function('mx_call_func_custom_types(mx_call_enum,mx_call_enum)');
NOTICE: procedure multi_mx_function_call_delegation.mx_call_func_custom_types is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
select create_distributed_function('mx_call_func_copy(int)'); select create_distributed_function('mx_call_func_copy(int)');
NOTICE: procedure multi_mx_function_call_delegation.mx_call_func_copy is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
select create_distributed_function('squares(int)'); select create_distributed_function('squares(int)');
NOTICE: procedure multi_mx_function_call_delegation.squares is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -132,30 +132,40 @@ select mx_call_func(2, 0);
-- Mark both functions as distributed ... -- Mark both functions as distributed ...
select create_distributed_function('mx_call_func(int,int)'); select create_distributed_function('mx_call_func(int,int)');
NOTICE: procedure multi_mx_function_call_delegation.mx_call_func is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
select create_distributed_function('mx_call_func_bigint(bigint,bigint)'); select create_distributed_function('mx_call_func_bigint(bigint,bigint)');
NOTICE: procedure multi_mx_function_call_delegation.mx_call_func_bigint is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
select create_distributed_function('mx_call_func_custom_types(mx_call_enum,mx_call_enum)'); select create_distributed_function('mx_call_func_custom_types(mx_call_enum,mx_call_enum)');
NOTICE: procedure multi_mx_function_call_delegation.mx_call_func_custom_types is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
select create_distributed_function('mx_call_func_copy(int)'); select create_distributed_function('mx_call_func_copy(int)');
NOTICE: procedure multi_mx_function_call_delegation.mx_call_func_copy is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
(1 row) (1 row)
select create_distributed_function('squares(int)'); select create_distributed_function('squares(int)');
NOTICE: procedure multi_mx_function_call_delegation.squares is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -24,6 +24,8 @@ CREATE FUNCTION square(int) RETURNS INT
AS $$ SELECT $1 * $1 $$ AS $$ SELECT $1 * $1 $$
LANGUAGE SQL; LANGUAGE SQL;
select create_distributed_function('square(int)'); select create_distributed_function('square(int)');
NOTICE: procedure multi_mx_insert_select_repartition.square is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -459,7 +459,7 @@ SELECT create_distributed_table('mx_old_schema.table_set_schema', 'id');
CREATE SCHEMA mx_new_schema; CREATE SCHEMA mx_new_schema;
SELECT objid::oid::regnamespace as "Distributed Schemas" 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') WHERE objid::oid::regnamespace IN ('mx_old_schema', 'mx_new_schema')
ORDER BY "Distributed Schemas"; ORDER BY "Distributed Schemas";
Distributed Schemas Distributed Schemas
@ -491,7 +491,7 @@ HINT: Connect to the coordinator and run it again.
\c - - - :master_port \c - - - :master_port
ALTER TABLE mx_old_schema.table_set_schema SET SCHEMA mx_new_schema; ALTER TABLE mx_old_schema.table_set_schema SET SCHEMA mx_new_schema;
SELECT objid::oid::regnamespace as "Distributed Schemas" 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'); WHERE objid::oid::regnamespace IN ('mx_old_schema', 'mx_new_schema');
Distributed Schemas Distributed Schemas
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -71,6 +71,7 @@ BEGIN
RETURN a*a; RETURN a*a;
END; $$ LANGUAGE PLPGSQL STABLE; END; $$ LANGUAGE PLPGSQL STABLE;
SELECT create_distributed_function('square(int)'); SELECT create_distributed_function('square(int)');
NOTICE: procedure multi_row_router_insert.square is already distributed
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -884,7 +884,7 @@ SELECT create_distributed_table('old_schema.table_set_schema', 'id');
CREATE SCHEMA new_schema; CREATE SCHEMA new_schema;
SELECT objid::oid::regnamespace as "Distributed Schemas" 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'); WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema');
Distributed Schemas Distributed Schemas
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -906,7 +906,7 @@ SELECT table_schema AS "Shards' Schema"
\c - - - :master_port \c - - - :master_port
ALTER TABLE old_schema.table_set_schema SET SCHEMA new_schema; ALTER TABLE old_schema.table_set_schema SET SCHEMA new_schema;
SELECT objid::oid::regnamespace as "Distributed Schemas" 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'); WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema');
Distributed Schemas Distributed Schemas
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -944,7 +944,7 @@ SELECT create_distributed_table('table_set_schema', 'id');
CREATE SCHEMA new_schema; CREATE SCHEMA new_schema;
SELECT objid::oid::regnamespace as "Distributed Schemas" SELECT objid::oid::regnamespace as "Distributed Schemas"
FROM citus.pg_dist_object FROM pg_catalog.pg_dist_object
WHERE objid='new_schema'::regnamespace::oid; WHERE objid='new_schema'::regnamespace::oid;
Distributed Schemas Distributed Schemas
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -965,7 +965,7 @@ SELECT table_schema AS "Shards' Schema"
\c - - - :master_port \c - - - :master_port
ALTER TABLE table_set_schema SET SCHEMA new_schema; ALTER TABLE table_set_schema SET SCHEMA new_schema;
SELECT objid::oid::regnamespace as "Distributed Schemas" SELECT objid::oid::regnamespace as "Distributed Schemas"
FROM citus.pg_dist_object FROM pg_catalog.pg_dist_object
WHERE objid='new_schema'::regnamespace::oid; WHERE objid='new_schema'::regnamespace::oid;
Distributed Schemas Distributed Schemas
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -1010,7 +1010,7 @@ SELECT create_distributed_table('table_set_schema', 'id');
CREATE SCHEMA new_schema; CREATE SCHEMA new_schema;
SELECT objid::oid::regnamespace as "Distributed Schemas" 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'); WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema');
Distributed Schemas Distributed Schemas
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -1034,7 +1034,7 @@ SELECT table_schema AS "Shards' Schema", COUNT(*) AS "Counts"
SET search_path TO old_schema; SET search_path TO old_schema;
ALTER TABLE table_set_schema SET SCHEMA new_schema; ALTER TABLE table_set_schema SET SCHEMA new_schema;
SELECT objid::oid::regnamespace as "Distributed Schemas" 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'); WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema');
Distributed Schemas Distributed Schemas
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -1392,7 +1392,7 @@ BEGIN;
ROLLBACK; ROLLBACK;
-- Clean up the created schema -- Clean up the created schema
SET client_min_messages TO WARNING; 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'); WHERE classid=2615 and objid IN (select oid from pg_namespace where nspname='run_test_schema');
pg_identify_object_as_address 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; 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 -- 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'); WHERE classid=2615 and objid IN (select oid from pg_namespace where nspname='run_test_schema');
pg_identify_object_as_address pg_identify_object_as_address
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -852,13 +852,13 @@ SELECT create_distributed_table('test_seq_dist', 'a');
(1 row) (1 row)
-- Both sequence and dependency schema should be distributed -- 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 pg_identify_object_as_address
--------------------------------------------------------------------- ---------------------------------------------------------------------
(sequence,"{test_schema_for_sequence_default_propagation,seq_10}",{}) (sequence,"{test_schema_for_sequence_default_propagation,seq_10}",{})
(1 row) (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 pg_identify_object_as_address
--------------------------------------------------------------------- ---------------------------------------------------------------------
(schema,{test_schema_for_sequence_default_propagation},{}) (schema,{test_schema_for_sequence_default_propagation},{})

View File

@ -83,7 +83,7 @@ END;$$;
CREATE OR REPLACE FUNCTION colocate_proc_with_table(procname text, tablerelid regclass, argument_index int) CREATE OR REPLACE FUNCTION colocate_proc_with_table(procname text, tablerelid regclass, argument_index int)
RETURNS void LANGUAGE plpgsql AS $$ RETURNS void LANGUAGE plpgsql AS $$
BEGIN BEGIN
update citus.pg_dist_object update pg_catalog.pg_dist_object
set distribution_argument_index = argument_index, colocationid = pg_dist_partition.colocationid set distribution_argument_index = argument_index, colocationid = pg_dist_partition.colocationid
from pg_proc, pg_dist_partition from pg_proc, pg_dist_partition
where proname = procname and oid = objid and pg_dist_partition.logicalrelid = tablerelid; where proname = procname and oid = objid and pg_dist_partition.logicalrelid = tablerelid;

View File

@ -12,27 +12,27 @@ CREATE TYPE mood AS ENUM ('sad', 'ok', 'happy');
CREATE SCHEMA test_sequence_schema; CREATE SCHEMA test_sequence_schema;
CREATE SEQUENCE test_sequence_schema.test_sequence; CREATE SEQUENCE test_sequence_schema.test_sequence;
-- show that none of the objects above are marked as distributed -- 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 pg_identify_object_as_address
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (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 pg_identify_object_as_address
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (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 pg_identify_object_as_address
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (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 pg_identify_object_as_address
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (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 pg_identify_object_as_address
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (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 -- show that schema, types, function and sequence has marked as distributed
-- on the coordinator node -- on the coordinator node
RESET ROLE; 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 pg_identify_object_as_address
--------------------------------------------------------------------- ---------------------------------------------------------------------
(schema,{local_schema},{}) (schema,{local_schema},{})
(1 row) (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 pg_identify_object_as_address
--------------------------------------------------------------------- ---------------------------------------------------------------------
(schema,{test_sequence_schema},{}) (schema,{test_sequence_schema},{})
(1 row) (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 pg_identify_object_as_address
--------------------------------------------------------------------- ---------------------------------------------------------------------
(type,{local_schema.mood},{}) (type,{local_schema.mood},{})
(1 row) (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 pg_identify_object_as_address
--------------------------------------------------------------------- ---------------------------------------------------------------------
(type,{local_schema.test_type},{}) (type,{local_schema.test_type},{})
(1 row) (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 pg_identify_object_as_address
--------------------------------------------------------------------- ---------------------------------------------------------------------
(sequence,"{test_sequence_schema,test_sequence}",{}) (sequence,"{test_sequence_schema,test_sequence}",{})
(1 row) (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 pg_identify_object_as_address
--------------------------------------------------------------------- ---------------------------------------------------------------------
(sequence,"{local_schema,dist_table_e_seq}",{}) (sequence,"{local_schema,dist_table_e_seq}",{})
(1 row) (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 pg_identify_object_as_address
--------------------------------------------------------------------- ---------------------------------------------------------------------
(function,"{local_schema,test_function}",{integer}) (function,"{local_schema,test_function}",{integer})
(1 row) (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 pg_identify_object_as_address
--------------------------------------------------------------------- ---------------------------------------------------------------------
(function,"{local_schema,plpgsql_dist_function}",{pg_catalog.text}) (function,"{local_schema,plpgsql_dist_function}",{pg_catalog.text})
(1 row) (1 row)
-- show those objects marked as distributed on metadata worker node as well -- 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 nodename | nodeport | success | result
--------------------------------------------------------------------- ---------------------------------------------------------------------
localhost | 57637 | t | (schema,{local_schema},{}) localhost | 57637 | t | (schema,{local_schema},{})
localhost | 57638 | t | (schema,{local_schema},{}) localhost | 57638 | t | (schema,{local_schema},{})
(2 rows) (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 nodename | nodeport | success | result
--------------------------------------------------------------------- ---------------------------------------------------------------------
localhost | 57637 | t | (schema,{test_sequence_schema},{}) localhost | 57637 | t | (schema,{test_sequence_schema},{})
localhost | 57638 | t | (schema,{test_sequence_schema},{}) localhost | 57638 | t | (schema,{test_sequence_schema},{})
(2 rows) (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 nodename | nodeport | success | result
--------------------------------------------------------------------- ---------------------------------------------------------------------
localhost | 57637 | t | (type,{local_schema.mood},{}) localhost | 57637 | t | (type,{local_schema.mood},{})
localhost | 57638 | t | (type,{local_schema.mood},{}) localhost | 57638 | t | (type,{local_schema.mood},{})
(2 rows) (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 nodename | nodeport | success | result
--------------------------------------------------------------------- ---------------------------------------------------------------------
localhost | 57637 | t | (type,{local_schema.test_type},{}) localhost | 57637 | t | (type,{local_schema.test_type},{})
localhost | 57638 | t | (type,{local_schema.test_type},{}) localhost | 57638 | t | (type,{local_schema.test_type},{})
(2 rows) (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 nodename | nodeport | success | result
--------------------------------------------------------------------- ---------------------------------------------------------------------
localhost | 57637 | t | (sequence,"{test_sequence_schema,test_sequence}",{}) localhost | 57637 | t | (sequence,"{test_sequence_schema,test_sequence}",{})
localhost | 57638 | t | (sequence,"{test_sequence_schema,test_sequence}",{}) localhost | 57638 | t | (sequence,"{test_sequence_schema,test_sequence}",{})
(2 rows) (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 nodename | nodeport | success | result
--------------------------------------------------------------------- ---------------------------------------------------------------------
localhost | 57637 | t | (sequence,"{local_schema,dist_table_e_seq}",{}) localhost | 57637 | t | (sequence,"{local_schema,dist_table_e_seq}",{})
localhost | 57638 | t | (sequence,"{local_schema,dist_table_e_seq}",{}) localhost | 57638 | t | (sequence,"{local_schema,dist_table_e_seq}",{})
(2 rows) (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 nodename | nodeport | success | result
--------------------------------------------------------------------- ---------------------------------------------------------------------
localhost | 57637 | t | (function,"{local_schema,test_function}",{integer}) localhost | 57637 | t | (function,"{local_schema,test_function}",{integer})
localhost | 57638 | t | (function,"{local_schema,test_function}",{integer}) localhost | 57638 | t | (function,"{local_schema,test_function}",{integer})
(2 rows) (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 nodename | nodeport | success | result
--------------------------------------------------------------------- ---------------------------------------------------------------------
localhost | 57637 | t | (function,"{local_schema,plpgsql_dist_function}",{pg_catalog.text}) 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) (2 rows)
-- Show that extension plpgsql is also marked as distributed as a dependency of plpgsl_dist_function -- 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 obj_identifier
--------------------------------------------------------------------- ---------------------------------------------------------------------
(extension,{plpgsql},{}) (extension,{plpgsql},{})
(1 row) (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 nodename | nodeport | success | result
--------------------------------------------------------------------- ---------------------------------------------------------------------
localhost | 57637 | t | (extension,{plpgsql},{}) localhost | 57637 | t | (extension,{plpgsql},{})
@ -278,12 +278,12 @@ SET citus.enable_ddl_propagation TO ON;
DROP TABLE dist_table CASCADE; DROP TABLE dist_table CASCADE;
RESET ROLE; RESET ROLE;
SET search_path TO local_schema; 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 obj_identifier
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (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 nodename | nodeport | success | result
--------------------------------------------------------------------- ---------------------------------------------------------------------
localhost | 57637 | t | 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 -- Show that altering the function's schema marks the schema distributed
CREATE SCHEMA schema_to_prop_with_function; CREATE SCHEMA schema_to_prop_with_function;
ALTER FUNCTION test_function SET 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 obj_identifier
--------------------------------------------------------------------- ---------------------------------------------------------------------
(schema,{schema_to_prop_with_function},{}) (schema,{schema_to_prop_with_function},{})
(1 row) (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 nodename | nodeport | success | result
--------------------------------------------------------------------- ---------------------------------------------------------------------
localhost | 57637 | t | (schema,{schema_to_prop_with_function},{}) 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 -- Show that dropping the function removes the metadata from pg_dist_object
-- on both coordinator and metadata worker node -- on both coordinator and metadata worker node
DROP FUNCTION schema_to_prop_with_function.test_function; 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 obj_identifier
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (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 nodename | nodeport | success | result
--------------------------------------------------------------------- ---------------------------------------------------------------------
localhost | 57637 | t | 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 -- Show that altering the type's schema marks the schema distributed
CREATE SCHEMA schema_to_prop_with_type; CREATE SCHEMA schema_to_prop_with_type;
ALTER TYPE test_type SET 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 obj_identifier
--------------------------------------------------------------------- ---------------------------------------------------------------------
(schema,{schema_to_prop_with_type},{}) (schema,{schema_to_prop_with_type},{})
(1 row) (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 nodename | nodeport | success | result
--------------------------------------------------------------------- ---------------------------------------------------------------------
localhost | 57637 | t | (schema,{schema_to_prop_with_type},{}) 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 -- on both coordinator and metadata worker node
DROP TYPE mood CASCADE; DROP TYPE mood CASCADE;
DROP TYPE schema_to_prop_with_type.test_type 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 obj_identifier
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (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 nodename | nodeport | success | result
--------------------------------------------------------------------- ---------------------------------------------------------------------
localhost | 57637 | t | localhost | 57637 | t |
@ -379,13 +379,13 @@ SELECT create_distributed_function('metadata_dist_test_proc(integer, integer)',
(1 row) (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 distribution_argument_index
--------------------------------------------------------------------- ---------------------------------------------------------------------
0 0
(1 row) (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 nodename | nodeport | success | result
--------------------------------------------------------------------- ---------------------------------------------------------------------
localhost | 57637 | t | 0 localhost | 57637 | t | 0
@ -399,13 +399,13 @@ SELECT create_distributed_function('metadata_dist_test_proc(integer, integer)',
(1 row) (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 distribution_argument_index
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (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 nodename | nodeport | success | result
--------------------------------------------------------------------- ---------------------------------------------------------------------
localhost | 57637 | t | 1 localhost | 57637 | t | 1
@ -418,12 +418,12 @@ NOTICE: drop cascades to 3 other objects
DETAIL: drop cascades to function plpgsql_dist_function(text) DETAIL: drop cascades to function plpgsql_dist_function(text)
drop cascades to table metadata_dist_test_table drop cascades to table metadata_dist_test_table
drop cascades to function metadata_dist_test_proc(integer,integer) 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 obj_identifier
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (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 nodename | nodeport | success | result
--------------------------------------------------------------------- ---------------------------------------------------------------------
localhost | 57637 | t | 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 -- Show that extension and dependent sequence also created and marked as distributed
CREATE SCHEMA extension_schema; CREATE SCHEMA extension_schema;
CREATE EXTENSION ltree WITH 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 obj_identifier
--------------------------------------------------------------------- ---------------------------------------------------------------------
(schema,{extension_schema},{}) (schema,{extension_schema},{})
(1 row) (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 nodename | nodeport | success | result
--------------------------------------------------------------------- ---------------------------------------------------------------------
localhost | 57637 | t | (schema,{extension_schema},{}) localhost | 57637 | t | (schema,{extension_schema},{})
localhost | 57638 | t | (schema,{extension_schema},{}) localhost | 57638 | t | (schema,{extension_schema},{})
(2 rows) (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 obj_identifier
--------------------------------------------------------------------- ---------------------------------------------------------------------
(extension,{ltree},{}) (extension,{ltree},{})
(1 row) (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 nodename | nodeport | success | result
--------------------------------------------------------------------- ---------------------------------------------------------------------
localhost | 57637 | t | (extension,{ltree},{}) localhost | 57637 | t | (extension,{ltree},{})
@ -467,13 +467,13 @@ SELECT create_distributed_table('extension_schema.table_to_check_object', 'id');
(1 row) (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 obj_identifier
--------------------------------------------------------------------- ---------------------------------------------------------------------
(table,"{extension_schema,table_to_check_object}",{}) (table,"{extension_schema,table_to_check_object}",{})
(1 row) (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 nodename | nodeport | success | result
--------------------------------------------------------------------- ---------------------------------------------------------------------
localhost | 57637 | t | (table,"{extension_schema,table_to_check_object}",{}) 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) (2 rows)
DROP TABLE extension_schema.table_to_check_object; 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 obj_identifier
--------------------------------------------------------------------- ---------------------------------------------------------------------
(0 rows) (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 nodename | nodeport | success | result
--------------------------------------------------------------------- ---------------------------------------------------------------------
localhost | 57637 | t | localhost | 57637 | t |

View File

@ -1232,6 +1232,8 @@ SELECT create_distributed_function('proc_with_out_param(date,int)');
ERROR: function "proc_with_out_param(date,int)" does not exist ERROR: function "proc_with_out_param(date,int)" does not exist
-- this should work -- this should work
SELECT create_distributed_function('proc_with_out_param(date)'); SELECT create_distributed_function('proc_with_out_param(date)');
NOTICE: procedure pg14.proc_with_out_param is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -6,13 +6,13 @@ SET client_min_messages TO WARNING;
-- the extension is on contrib, so should be avaliable for the regression tests -- the extension is on contrib, so should be avaliable for the regression tests
CREATE EXTENSION seg; CREATE EXTENSION seg;
-- make sure that both the schema and the extension is distributed -- 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 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); 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 -- 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
@ -67,7 +67,7 @@ BEGIN;
COMMIT; COMMIT;
-- make sure that the extension is distributed even if we run create extension in a transaction block -- 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
@ -122,7 +122,7 @@ ALTER EXTENSION isn SET SCHEMA public;
-- switch back to public schema as we set extension's schema to public -- switch back to public schema as we set extension's schema to public
SET search_path TO public; SET search_path TO public;
-- make sure that the extension is distributed -- 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
@ -282,7 +282,7 @@ BEGIN;
ROLLBACK; ROLLBACK;
-- at the end of the transaction block, we did not create isn extension in coordinator or worker nodes as we rollback'ed -- 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 -- 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
0 0
@ -319,7 +319,7 @@ SELECT count(*) FROM pg_extension WHERE extname IN ('pg_buffercache', 'isn');
-- drop extension should just work -- drop extension should just work
DROP EXTENSION seg CASCADE; 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
0 0
@ -333,7 +333,7 @@ SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname
(2 rows) (2 rows)
-- make sure that the extension is not avaliable anymore as a distributed object -- 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
0 0
@ -439,9 +439,9 @@ BEGIN;
COMMIT; COMMIT;
-- Check the pg_dist_object -- Check the pg_dist_object
SELECT pg_proc.proname as DistributedFunction 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 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; classid = 'pg_proc'::regclass;
distributedfunction distributedfunction
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -450,9 +450,9 @@ classid = 'pg_proc'::regclass;
SELECT run_command_on_workers($$ SELECT run_command_on_workers($$
SELECT count(*) SELECT count(*)
FROM citus.pg_dist_object, pg_proc FROM pg_catalog.pg_dist_object, pg_proc
WHERE pg_proc.proname = 'seg_in' and 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; classid = 'pg_proc'::regclass;
$$); $$);
run_command_on_workers run_command_on_workers
@ -468,7 +468,7 @@ SELECT 1 from master_add_node('localhost', :worker_2_port);
(1 row) (1 row)
-- make sure that both extensions are created on both nodes -- 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
2 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 -- Check the pg_dist_object on the both nodes
SELECT run_command_on_workers($$ SELECT run_command_on_workers($$
SELECT count(*) SELECT count(*)
FROM citus.pg_dist_object, pg_proc FROM pg_catalog.pg_dist_object, pg_proc
WHERE pg_proc.proname = 'seg_in' and 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; classid = 'pg_proc'::regclass;
$$); $$);
run_command_on_workers run_command_on_workers
@ -498,9 +498,9 @@ $$);
DROP EXTENSION seg CASCADE; DROP EXTENSION seg CASCADE;
-- Recheck the pg_dist_object -- Recheck the pg_dist_object
SELECT pg_proc.proname as DistributedFunction 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 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; classid = 'pg_proc'::regclass;
distributedfunction distributedfunction
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -508,9 +508,9 @@ classid = 'pg_proc'::regclass;
SELECT run_command_on_workers($$ SELECT run_command_on_workers($$
SELECT count(*) SELECT count(*)
FROM citus.pg_dist_object, pg_proc FROM pg_catalog.pg_dist_object, pg_proc
WHERE pg_proc.proname = 'seg_in' and 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; classid = 'pg_proc'::regclass;
$$); $$);
run_command_on_workers run_command_on_workers
@ -524,7 +524,7 @@ SET citus.enable_ddl_propagation TO false;
CREATE EXTENSION seg; CREATE EXTENSION seg;
SET citus.enable_ddl_propagation TO true; SET citus.enable_ddl_propagation TO true;
-- Check the extension in pg_dist_object -- 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'); objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -533,9 +533,9 @@ objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
SELECT run_command_on_workers($$ SELECT run_command_on_workers($$
SELECT count(*) SELECT count(*)
FROM citus.pg_dist_object, pg_proc FROM pg_catalog.pg_dist_object, pg_proc
WHERE pg_proc.proname = 'seg_in' and 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; classid = 'pg_proc'::regclass;
$$); $$);
run_command_on_workers run_command_on_workers
@ -551,7 +551,7 @@ SELECT create_distributed_function('seg_in(cstring)', '$1', 'test_extension_func
(1 row) (1 row)
-- Recheck the extension in pg_dist_object -- 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'); objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -559,9 +559,9 @@ objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
(1 row) (1 row)
SELECT pg_proc.proname as DistributedFunction 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 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; classid = 'pg_proc'::regclass;
distributedfunction distributedfunction
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -570,9 +570,9 @@ classid = 'pg_proc'::regclass;
SELECT run_command_on_workers($$ SELECT run_command_on_workers($$
SELECT count(*) SELECT count(*)
FROM citus.pg_dist_object, pg_proc FROM pg_catalog.pg_dist_object, pg_proc
WHERE pg_proc.proname = 'seg_in' and 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; classid = 'pg_proc'::regclass;
$$); $$);
run_command_on_workers run_command_on_workers
@ -600,7 +600,7 @@ SELECT create_distributed_function('cube(float8[], float8[])', '$1', 'test_exten
(1 row) (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'); objid = (SELECT oid FROM pg_proc WHERE prosrc = 'cube_a_f8_f8');
distribution_argument_index distribution_argument_index
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -613,7 +613,7 @@ SELECT create_distributed_function('cube(float8[], float8[])', '$2', 'test_exten
(1 row) (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'); objid = (SELECT oid FROM pg_proc WHERE prosrc = 'cube_a_f8_f8');
distribution_argument_index distribution_argument_index
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -6,13 +6,13 @@ SET client_min_messages TO WARNING;
-- the extension is on contrib, so should be avaliable for the regression tests -- the extension is on contrib, so should be avaliable for the regression tests
CREATE EXTENSION seg; CREATE EXTENSION seg;
-- make sure that both the schema and the extension is distributed -- 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
(1 row) (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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 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); 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 -- 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
@ -67,7 +67,7 @@ BEGIN;
COMMIT; COMMIT;
-- make sure that the extension is distributed even if we run create extension in a transaction block -- 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
@ -122,7 +122,7 @@ ALTER EXTENSION isn SET SCHEMA public;
-- switch back to public schema as we set extension's schema to public -- switch back to public schema as we set extension's schema to public
SET search_path TO public; SET search_path TO public;
-- make sure that the extension is distributed -- 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
1 1
@ -281,7 +281,7 @@ BEGIN;
ROLLBACK; ROLLBACK;
-- at the end of the transaction block, we did not create isn extension in coordinator or worker nodes as we rollback'ed -- 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 -- 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
0 0
@ -318,7 +318,7 @@ SELECT count(*) FROM pg_extension WHERE extname IN ('pg_buffercache', 'isn');
-- drop extension should just work -- drop extension should just work
DROP EXTENSION seg CASCADE; 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
0 0
@ -332,7 +332,7 @@ SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname
(2 rows) (2 rows)
-- make sure that the extension is not avaliable anymore as a distributed object -- 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
0 0
@ -438,9 +438,9 @@ BEGIN;
COMMIT; COMMIT;
-- Check the pg_dist_object -- Check the pg_dist_object
SELECT pg_proc.proname as DistributedFunction 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 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; classid = 'pg_proc'::regclass;
distributedfunction distributedfunction
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -449,9 +449,9 @@ classid = 'pg_proc'::regclass;
SELECT run_command_on_workers($$ SELECT run_command_on_workers($$
SELECT count(*) SELECT count(*)
FROM citus.pg_dist_object, pg_proc FROM pg_catalog.pg_dist_object, pg_proc
WHERE pg_proc.proname = 'seg_in' and 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; classid = 'pg_proc'::regclass;
$$); $$);
run_command_on_workers run_command_on_workers
@ -467,7 +467,7 @@ SELECT 1 from master_add_node('localhost', :worker_2_port);
(1 row) (1 row)
-- make sure that both extensions are created on both nodes -- 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 count
--------------------------------------------------------------------- ---------------------------------------------------------------------
2 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 -- Check the pg_dist_object on the both nodes
SELECT run_command_on_workers($$ SELECT run_command_on_workers($$
SELECT count(*) SELECT count(*)
FROM citus.pg_dist_object, pg_proc FROM pg_catalog.pg_dist_object, pg_proc
WHERE pg_proc.proname = 'seg_in' and 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; classid = 'pg_proc'::regclass;
$$); $$);
run_command_on_workers run_command_on_workers
@ -497,9 +497,9 @@ $$);
DROP EXTENSION seg CASCADE; DROP EXTENSION seg CASCADE;
-- Recheck the pg_dist_object -- Recheck the pg_dist_object
SELECT pg_proc.proname as DistributedFunction 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 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; classid = 'pg_proc'::regclass;
distributedfunction distributedfunction
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -507,9 +507,9 @@ classid = 'pg_proc'::regclass;
SELECT run_command_on_workers($$ SELECT run_command_on_workers($$
SELECT count(*) SELECT count(*)
FROM citus.pg_dist_object, pg_proc FROM pg_catalog.pg_dist_object, pg_proc
WHERE pg_proc.proname = 'seg_in' and 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; classid = 'pg_proc'::regclass;
$$); $$);
run_command_on_workers run_command_on_workers
@ -523,7 +523,7 @@ SET citus.enable_ddl_propagation TO false;
CREATE EXTENSION seg; CREATE EXTENSION seg;
SET citus.enable_ddl_propagation TO true; SET citus.enable_ddl_propagation TO true;
-- Check the extension in pg_dist_object -- 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'); objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -532,9 +532,9 @@ objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
SELECT run_command_on_workers($$ SELECT run_command_on_workers($$
SELECT count(*) SELECT count(*)
FROM citus.pg_dist_object, pg_proc FROM pg_catalog.pg_dist_object, pg_proc
WHERE pg_proc.proname = 'seg_in' and 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; classid = 'pg_proc'::regclass;
$$); $$);
run_command_on_workers run_command_on_workers
@ -550,7 +550,7 @@ SELECT create_distributed_function('seg_in(cstring)', '$1', 'test_extension_func
(1 row) (1 row)
-- Recheck the extension in pg_dist_object -- 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'); objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
count count
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -558,9 +558,9 @@ objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
(1 row) (1 row)
SELECT pg_proc.proname as DistributedFunction 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 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; classid = 'pg_proc'::regclass;
distributedfunction distributedfunction
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -569,9 +569,9 @@ classid = 'pg_proc'::regclass;
SELECT run_command_on_workers($$ SELECT run_command_on_workers($$
SELECT count(*) SELECT count(*)
FROM citus.pg_dist_object, pg_proc FROM pg_catalog.pg_dist_object, pg_proc
WHERE pg_proc.proname = 'seg_in' and 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; classid = 'pg_proc'::regclass;
$$); $$);
run_command_on_workers run_command_on_workers
@ -599,7 +599,7 @@ SELECT create_distributed_function('cube(float8[], float8[])', '$1', 'test_exten
(1 row) (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'); objid = (SELECT oid FROM pg_proc WHERE prosrc = 'cube_a_f8_f8');
distribution_argument_index distribution_argument_index
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -612,7 +612,7 @@ SELECT create_distributed_function('cube(float8[], float8[])', '$2', 'test_exten
(1 row) (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'); objid = (SELECT oid FROM pg_proc WHERE prosrc = 'cube_a_f8_f8');
distribution_argument_index distribution_argument_index
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -45,7 +45,7 @@ SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid
(1 row) (1 row)
-- verify that the aggregate is added top pg_dist_object on the new node -- 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 run_command_on_workers
--------------------------------------------------------------------- ---------------------------------------------------------------------
(localhost,57638,t,1) (localhost,57638,t,1)

View File

@ -15,6 +15,8 @@ BEGIN
END; END;
$$ language plpgsql; $$ language plpgsql;
SELECT create_distributed_function('table_returner(int)'); SELECT create_distributed_function('table_returner(int)');
NOTICE: procedure row_types.table_returner is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -29,6 +31,8 @@ BEGIN
END; END;
$$ language plpgsql; $$ language plpgsql;
SELECT create_distributed_function('record_returner(int)'); SELECT create_distributed_function('record_returner(int)');
NOTICE: procedure row_types.record_returner is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -42,6 +46,8 @@ BEGIN
END; END;
$$ language plpgsql; $$ language plpgsql;
SELECT create_distributed_function('identity_returner(anyelement)'); SELECT create_distributed_function('identity_returner(anyelement)');
NOTICE: procedure row_types.identity_returner is already distributed
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
create_distributed_function create_distributed_function
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -123,7 +123,7 @@ from
(select (select
pg_catalog.websearch_to_tsquery( pg_catalog.websearch_to_tsquery(
cast(pg_catalog.regconfigin(cast(cast(null as cstring) as cstring)) as regconfig), 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, ) as c0,
sample_0.org_id as c1, sample_0.org_id as c1,
sample_0.id as c2, sample_0.id as c2,

View File

@ -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 -- 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) 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; WHERE classid = 3602 AND objid = 'text_search.concurrent_index_config'::regconfig::oid;
pg_identify_object_as_address pg_identify_object_as_address
--------------------------------------------------------------------- ---------------------------------------------------------------------

View File

@ -53,7 +53,6 @@ ORDER BY 1;
function citus_dist_partition_cache_invalidate() function citus_dist_partition_cache_invalidate()
function citus_dist_placement_cache_invalidate() function citus_dist_placement_cache_invalidate()
function citus_dist_shard_cache_invalidate() function citus_dist_shard_cache_invalidate()
function citus_dist_stat_activity()
function citus_drain_node(text,integer,citus.shard_transfer_mode,name) function citus_drain_node(text,integer,citus.shard_transfer_mode,name)
function citus_drop_all_shards(regclass,text,text,boolean) function citus_drop_all_shards(regclass,text,text,boolean)
function citus_drop_trigger() function citus_drop_trigger()
@ -121,7 +120,6 @@ ORDER BY 1;
function citus_update_table_statistics(regclass) function citus_update_table_statistics(regclass)
function citus_validate_rebalance_strategy_functions(regproc,regproc,regproc) function citus_validate_rebalance_strategy_functions(regproc,regproc,regproc)
function citus_version() function citus_version()
function citus_worker_stat_activity()
function column_name_to_column(regclass,text) function column_name_to_column(regclass,text)
function column_to_column_name(regclass,text) function column_to_column_name(regclass,text)
function columnar.columnar_handler(internal) function columnar.columnar_handler(internal)
@ -148,6 +146,7 @@ ORDER BY 1;
function get_current_transaction_id() function get_current_transaction_id()
function get_global_active_transactions() function get_global_active_transactions()
function get_missing_time_partition_ranges(regclass,interval,timestamp with time zone,timestamp with time zone) function get_missing_time_partition_ranges(regclass,interval,timestamp with time zone,timestamp with time zone)
function get_nodeid_for_groupid(integer)
function get_rebalance_progress() function get_rebalance_progress()
function get_rebalance_table_shards_plan(regclass,real,integer,bigint[],boolean,name,real) function get_rebalance_table_shards_plan(regclass,real,integer,bigint[],boolean,name,real)
function get_shard_id_for_distribution_column(regclass,"any") function get_shard_id_for_distribution_column(regclass,"any")
@ -249,7 +248,6 @@ ORDER BY 1;
sequence pg_dist_node_nodeid_seq sequence pg_dist_node_nodeid_seq
sequence pg_dist_placement_placementid_seq sequence pg_dist_placement_placementid_seq
sequence pg_dist_shardid_seq sequence pg_dist_shardid_seq
table citus.pg_dist_object
table columnar.chunk table columnar.chunk
table columnar.chunk_group table columnar.chunk_group
table columnar.options table columnar.options
@ -259,6 +257,7 @@ ORDER BY 1;
table pg_dist_local_group table pg_dist_local_group
table pg_dist_node table pg_dist_node
table pg_dist_node_metadata table pg_dist_node_metadata
table pg_dist_object
table pg_dist_partition table pg_dist_partition
table pg_dist_placement table pg_dist_placement
table pg_dist_poolinfo table pg_dist_poolinfo
@ -277,8 +276,7 @@ ORDER BY 1;
view citus_shards_on_worker view citus_shards_on_worker
view citus_stat_activity view citus_stat_activity
view citus_stat_statements view citus_stat_statements
view citus_worker_stat_activity
view pg_dist_shard_placement view pg_dist_shard_placement
view time_partitions view time_partitions
(264 rows) (262 rows)

View File

@ -36,7 +36,7 @@ drop cascades to table upgrade_basic.t_range
-- "seg" extension -- "seg" extension
-- will not be marked as distributed -- will not be marked as distributed
-- see underlying objects -- 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 type | object_names | object_args
--------------------------------------------------------------------- ---------------------------------------------------------------------
collation | {post_11_upgrade,german_phonebook_unpropagated} | {} collation | {post_11_upgrade,german_phonebook_unpropagated} | {}

View File

@ -36,7 +36,7 @@ drop cascades to table upgrade_basic.t_range
-- "seg" extension -- "seg" extension
-- will not be marked as distributed -- will not be marked as distributed
-- see underlying objects -- 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 type | object_names | object_args
--------------------------------------------------------------------- ---------------------------------------------------------------------
database | {postgres} | {} database | {postgres} | {}

View File

@ -10,7 +10,7 @@ NOTICE: Preparing to sync the metadata to all nodes
(1 row) (1 row)
-- tables are objects with Citus 11+ -- 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 pg_identify_object_as_address
--------------------------------------------------------------------- ---------------------------------------------------------------------
(function,"{post_11_upgrade,func_in_transaction_def}",{}) (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) (6 rows)
-- on all nodes -- 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 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}\\"",{})""}") (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}\\"",{})""}")

View File

@ -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'); 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 -- 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);
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);
DROP SCHEMA test_schema_for_sequence_propagation CASCADE; DROP SCHEMA test_schema_for_sequence_propagation CASCADE;
DROP TABLE table_without_sequence; DROP TABLE table_without_sequence;

View File

@ -500,13 +500,13 @@ SELECT shardid, nodename, nodeport
-- before adding the node, add pg_dist_object entry for tables created with -- 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. -- 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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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, 'second_dustbunnies'::regclass::oid, 0);
SELECT 1 FROM master_activate_node('localhost', :worker_1_port); SELECT 1 FROM master_activate_node('localhost', :worker_1_port);
RESET client_min_messages; RESET client_min_messages;

View File

@ -1275,13 +1275,13 @@ CREATE SCHEMA test_schema_for_sequence_propagation;
CREATE SEQUENCE test_schema_for_sequence_propagation.seq_10; 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'); 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 -- 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 pg_identify_object_as_address
--------------------------------------------------------------- ---------------------------------------------------------------
(sequence,"{test_schema_for_sequence_propagation,seq_10}",{}) (sequence,"{test_schema_for_sequence_propagation,seq_10}",{})
(1 row) (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 pg_identify_object_as_address
---------------------------------------------------- ----------------------------------------------------
(schema,{test_schema_for_sequence_propagation},{}) (schema,{test_schema_for_sequence_propagation},{})

View File

@ -627,13 +627,13 @@ SELECT shardid, nodename, nodeport
-- add the node back -- add the node back
-- before adding the node, add pg_dist_object entry for tables created with -- 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. -- 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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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, 'second_dustbunnies'::regclass::oid, 0);
SELECT 1 FROM master_activate_node('localhost', :worker_1_port); SELECT 1 FROM master_activate_node('localhost', :worker_1_port);
NOTICE: Replicating postgres objects to node localhost:57637 NOTICE: Replicating postgres objects to node localhost:57637
DETAIL: There are 115 objects to replicate, depending on your environment this might take a while DETAIL: There are 115 objects to replicate, depending on your environment this might take a while

View File

@ -71,7 +71,7 @@ step "s2-sleep"
step "s2-view-dist" step "s2-view-dist"
{ {
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' and query not ILIKE '%BEGIN%' and query NOT ILIKE '%pg_catalog.pg_isolation_test_session_is_blocked%' ORDER BY query DESC; SELECT query, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%BEGIN%'), ('%pg_catalog.pg_isolation_test_session_is_blocked%')) AND backend_type = 'client backend' ORDER BY query DESC;
} }
@ -89,7 +89,7 @@ step "s3-rollback"
step "s3-view-worker" step "s3-view-worker"
{ {
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND backend_type = 'client backend' ORDER BY query DESC; SELECT query, state, wait_event_type, wait_event, usename, datname FROM citus_stat_activity WHERE query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%csa_from_one_node%')) AND is_worker_query = true AND backend_type = 'client backend' ORDER BY query DESC;
} }
// we prefer to sleep before "s2-view-dist" so that we can ensure // we prefer to sleep before "s2-view-dist" so that we can ensure

View File

@ -51,7 +51,7 @@ step "s1-print-distributed-objects"
SELECT 1 FROM master_add_node('localhost', 57638); SELECT 1 FROM master_add_node('localhost', 57638);
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';
@ -123,7 +123,7 @@ step "s2-commit"
step "s2-print-distributed-objects" step "s2-print-distributed-objects"
{ {
-- print an overview of all 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 -- print if the schema has been created
SELECT count(*) FROM pg_namespace where nspname = 'myschema'; SELECT count(*) FROM pg_namespace where nspname = 'myschema';

View File

@ -41,7 +41,7 @@ step "s1-create-extension-with-schema2"
step "s1-print" 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 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 extname from pg_extension where extname='seg'$$);
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$); SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);

View File

@ -1,6 +1,7 @@
#include "isolation_mx_common.include.spec" #include "isolation_mx_common.include.spec"
setup { setup {
SELECT citus_add_node('localhost', 57636, groupid:=0);
CREATE TABLE ref_table(user_id int, value_1 int); CREATE TABLE ref_table(user_id int, value_1 int);
SELECT create_reference_table('ref_table'); SELECT create_reference_table('ref_table');
INSERT INTO ref_table VALUES (1, 11), (2, 21), (3, 31), (4, 41), (5, 51), (6, 61), (7, 71); INSERT INTO ref_table VALUES (1, 11), (2, 21), (3, 31), (4, 41), (5, 51), (6, 61), (7, 71);
@ -17,6 +18,7 @@ teardown
DROP TABLE ref_table; DROP TABLE ref_table;
DROP TABLE tt1; DROP TABLE tt1;
SELECT citus_internal.restore_isolation_tester_func(); SELECT citus_internal.restore_isolation_tester_func();
SELECT citus_remove_node('localhost', 57636);
} }
session "s1" session "s1"
@ -159,7 +161,13 @@ session "s3"
step "s3-select-distributed-waiting-queries" step "s3-select-distributed-waiting-queries"
{ {
SELECT blocked_statement, current_statement_in_blocking_process, waiting_node_name, blocking_node_name, waiting_node_port, blocking_node_port FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%'; SELECT blocked_statement, current_statement_in_blocking_process FROM citus_lock_waits WHERE blocked_statement NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND current_statement_in_blocking_process NOT ILIKE '%run_commands_on_session_level_connection_to_node%';
}
// only works for the coordinator
step "s3-show-actual-gpids"
{
SELECT global_pid > 0 as gpid_exists, query FROM citus_stat_activity WHERE state = 'active' AND query IN (SELECT blocked_statement FROM citus_lock_waits UNION SELECT current_statement_in_blocking_process FROM citus_lock_waits) ORDER BY 1 DESC;
} }
// session s1 and s4 executes the commands on the same worker node // session s1 and s4 executes the commands on the same worker node
@ -189,6 +197,59 @@ step "s4-commit-worker"
SELECT run_commands_on_session_level_connection_to_node('COMMIT'); SELECT run_commands_on_session_level_connection_to_node('COMMIT');
} }
// on the coordinator, show that even if a backend is blocked on a DDL as the first command
// (e.g., as of today global pid has not been assigned), we can still show the blocking activity
// we use the following 4 sessions 5,6,7,8 for this purpose
session "s5"
step "s5-begin"
{
BEGIN;
}
step "s5-alter"
{
ALTER TABLE tt1 ADD COLUMN new_column INT;
}
step "s5-rollback"
{
ROLLBACK;
}
session "s6"
step "s6-select"
{
SELECT user_id FROM tt1 ORDER BY user_id DESC LIMIT 1;
}
session "s7"
step "s7-alter"
{
ALTER TABLE tt1 ADD COLUMN new_column INT;
}
session "s8"
step "s8-begin"
{
BEGIN;
}
step "s8-select"
{
SELECT user_id FROM tt1 ORDER BY user_id DESC LIMIT 1;
}
step "s8-rollback"
{
ROLLBACK;
}
permutation "s1-begin" "s1-update-ref-table-from-coordinator" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-ref-table" "s3-select-distributed-waiting-queries" "s1-commit" "s2-commit-worker" "s2-stop-connection" permutation "s1-begin" "s1-update-ref-table-from-coordinator" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-ref-table" "s3-select-distributed-waiting-queries" "s1-commit" "s2-commit-worker" "s2-stop-connection"
permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-update-ref-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-ref-table" "s3-select-distributed-waiting-queries" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-update-ref-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-ref-table" "s3-select-distributed-waiting-queries" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection"
permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-update-dist-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-dist-table" "s3-select-distributed-waiting-queries" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection" permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-update-dist-table" "s2-start-session-level-connection" "s2-begin-on-worker" "s2-update-dist-table" "s3-select-distributed-waiting-queries" "s1-commit-worker" "s2-commit-worker" "s1-stop-connection" "s2-stop-connection"
@ -212,3 +273,10 @@ permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-update-
// we can find the blocking relationship // we can find the blocking relationship
permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-update-dist-table-id-1" "s2-start-session-level-connection" "s2-update-dist-table-id-1" "s3-select-distributed-waiting-queries" "s1-commit-worker" "s1-stop-connection" "s2-stop-connection" permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-update-dist-table-id-1" "s2-start-session-level-connection" "s2-update-dist-table-id-1" "s3-select-distributed-waiting-queries" "s1-commit-worker" "s1-stop-connection" "s2-stop-connection"
permutation "s1-begin" "s1-update-ref-table-from-coordinator" "s2-start-session-level-connection" "s2-update-ref-table" "s3-select-distributed-waiting-queries" "s1-commit" "s2-stop-connection" permutation "s1-begin" "s1-update-ref-table-from-coordinator" "s2-start-session-level-connection" "s2-update-ref-table" "s3-select-distributed-waiting-queries" "s1-commit" "s2-stop-connection"
// show that we can see blocking activity even if these are the first commands in the sessions
// such that global_pids have not been assigned
// in the second permutation, s3-show-actual-gpids shows the gpid for ALTER TABLE
// because ALTER TABLE is not blocked on the parser but during the execution (hence gpid already asssigned)
"s5-begin" "s5-alter" "s6-select" "s3-select-distributed-waiting-queries" "s3-show-actual-gpids" "s5-rollback"
"s8-begin" "s8-select" "s7-alter" "s3-select-distributed-waiting-queries" "s3-show-actual-gpids" "s8-rollback"

View File

@ -69,18 +69,20 @@ step "s2-coordinator-citus_stat_activity"
step "s2-coordinator-citus_dist_stat_activity" step "s2-coordinator-citus_dist_stat_activity"
{ {
SELECT query FROM citus_dist_stat_activity() WHERE global_pid IN ( SELECT query FROM citus_dist_stat_activity WHERE global_pid IN (
SELECT global_pid FROM citus_stat_activity WHERE query LIKE '%SELECT * FROM dist\_table%' SELECT global_pid FROM citus_stat_activity WHERE query LIKE '%SELECT * FROM dist\_table%'
) )
AND query NOT ILIKE '%run_commands_on_session_level_connection_to_node%' AND query NOT ILIKE '%run_commands_on_session_level_connection_to_node%'
ORDER BY 1; ORDER BY 1;
} }
step "s2-coordinator-citus_worker_stat_activity" step "s2-coordinator-citus_stat_activity-in-workers"
{ {
SELECT query FROM citus_worker_stat_activity() WHERE global_pid IN ( SELECT query FROM citus_stat_activity WHERE global_pid IN (
SELECT global_pid FROM citus_stat_activity WHERE query LIKE '%SELECT * FROM dist\_table%' SELECT global_pid FROM citus_stat_activity WHERE query LIKE '%SELECT * FROM dist\_table%'
) )
AND is_worker_query = true
AND backend_type = 'client backend'
ORDER BY 1; ORDER BY 1;
} }
@ -101,7 +103,7 @@ step "s2-coordinator-get_global_active_transactions"
// worker - coordinator // worker - coordinator
permutation "s1-start-session-level-connection" "s1-worker-begin" "s1-worker-select" "s2-coordinator-citus_stat_activity" "s2-coordinator-citus_dist_stat_activity" "s2-coordinator-citus_worker_stat_activity" "s1-worker-commit" "s1-stop-session-level-connection" permutation "s1-start-session-level-connection" "s1-worker-begin" "s1-worker-select" "s2-coordinator-citus_stat_activity" "s2-coordinator-citus_dist_stat_activity" "s2-coordinator-citus_stat_activity-in-workers" "s1-worker-commit" "s1-stop-session-level-connection"
// coordinator - coordinator // coordinator - coordinator
permutation "s1-coordinator-begin" "s1-coordinator-select" "s2-coordinator-citus_stat_activity" "s2-coordinator-citus_dist_stat_activity" "s2-coordinator-citus_worker_stat_activity" "s2-coordinator-get_all_active_transactions" "s2-coordinator-get_global_active_transactions" "s1-coordinator-commit" permutation "s1-coordinator-begin" "s1-coordinator-select" "s2-coordinator-citus_stat_activity" "s2-coordinator-citus_dist_stat_activity" "s2-coordinator-citus_stat_activity-in-workers" "s2-coordinator-get_all_active_transactions" "s2-coordinator-get_global_active_transactions" "s1-coordinator-commit"

View File

@ -81,21 +81,24 @@ step "s2-lock-ref-table-placement-on-coordinator"
step "s2-view-dist" step "s2-view-dist"
{ {
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' AND query NOT ILIKE '%pg_isolation_test_session_is_blocked%' AND query NOT ILIKE '%BEGIN%' AND query NOT ILIKE '%add_node%' ORDER BY query DESC; SELECT query, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE backend_type = 'client backend' AND query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%pg_isolation_test_session_is_blocked%'), ('%BEGIN%'), ('%add_node%')) ORDER BY query DESC;
} }
step "s2-view-worker" step "s2-view-worker"
{ {
SELECT query, query_hostname, query_hostport, distributed_query_host_name, SELECT query, state, wait_event_type, wait_event, usename, datname
distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_stat_activity
FROM citus_worker_stat_activity WHERE query NOT ILIKE ALL(VALUES
WHERE query NOT ILIKE '%pg_prepared_xacts%' AND ('%pg_prepared_xacts%'),
query NOT ILIKE '%COMMIT%' AND ('%COMMIT%'),
query NOT ILIKE '%dump_local_%' AND ('%dump_local_%'),
query NOT ILIKE '%citus_internal_local_blocked_processes%' AND ('%citus_internal_local_blocked_processes%'),
query NOT ILIKE '%add_node%' AND ('%add_node%'),
backend_type = 'client backend' ('%csa_from_one_node%'))
ORDER BY query, query_hostport DESC; AND is_worker_query = true
AND backend_type = 'client backend'
AND query != ''
ORDER BY query DESC;
} }

View File

@ -161,7 +161,7 @@ create aggregate binstragg(text, text)(
stype=text stype=text
); );
-- verify that the aggregate is added into pg_dist_object, on each worker -- 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%';$$); 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%';$$); 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 -- 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; RESET citus.create_object_propagation;
-- drop and test outside of tx block -- drop and test outside of tx block
drop aggregate dependent_agg (float8); drop aggregate dependent_agg (float8);
-- verify that the aggregate is removed from pg_dist_object, on each worker -- 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); create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc);
--verify --verify
SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$); SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$);

View File

@ -45,7 +45,7 @@ DROP TABLE t4; -- as long as the table is using the type some operations are har
COMMIT; COMMIT;
-- verify the type is distributed -- 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 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; ALTER TYPE tt3 OWNER TO typeowner_for_disabled_object_propagation_guc;

View File

@ -180,7 +180,7 @@ SELECT create_distributed_table('streaming_table','id');
-- if not paremeters are supplied, we'd see that function doesn't have -- if not paremeters are supplied, we'd see that function doesn't have
-- distribution_argument_index and colocationid -- distribution_argument_index and colocationid
SELECT create_distributed_function('"eq_mi''xed_param_names"(macaddr, macaddr)'); 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; WHERE objid = 'eq_mi''xed_param_names(macaddr, macaddr)'::regprocedure;
-- also show that we can use the function -- 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 -- show that the colocationIds are the same
SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated 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 WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND
objects.objid = 'eq_with_param_names(macaddr, macaddr)'::regprocedure; 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 -- group preserved, because we're using the default shard creation settings
SELECT create_distributed_function('eq_with_param_names(macaddr, macaddr)', 'val1'); SELECT create_distributed_function('eq_with_param_names(macaddr, macaddr)', 'val1');
SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated 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 WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND
objects.objid = 'eq_with_param_names(macaddr, macaddr)'::regprocedure; 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 -- to coerce the values
SELECT create_distributed_function('eq8(macaddr8, macaddr8)', '$1', colocate_with:='replicated_table_func_test_4'); 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 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 WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND
objects.objid = 'eq8(macaddr8, macaddr8)'::regprocedure; objects.objid = 'eq8(macaddr8, macaddr8)'::regprocedure;
SELECT create_distributed_function('add_text(text, text)', '$1', colocate_with:='replicated_table_func_test_4'); 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 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 WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND
objects.objid = 'add_text(text, text)'::regprocedure; objects.objid = 'add_text(text, text)'::regprocedure;

View File

@ -240,3 +240,75 @@ ROLLBACK;
DROP SCHEMA drop_partitioned_table CASCADE; DROP SCHEMA drop_partitioned_table CASCADE;
SET search_path TO public; SET search_path TO public;
-- dropping the schema should drop the metadata on the workers
CREATE SCHEMA partitioning_schema;
SET search_path TO partitioning_schema;
CREATE TABLE part_table (
col timestamp
) PARTITION BY RANGE (col);
CREATE TABLE part_table_1
PARTITION OF part_table
FOR VALUES FROM ('2010-01-01') TO ('2015-01-01');
SELECT create_distributed_table('part_table', 'col');
-- show we have pg_dist_partition entries on the workers
SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_partition where exists(select * from pg_class where pg_class.oid=pg_dist_partition.logicalrelid AND relname ILIKE '%part_table%');$$);
-- show we have pg_dist_object entries on the workers
SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_object as obj where classid = 1259 AND exists(select * from pg_class where pg_class.oid=obj.objid AND relname ILIKE '%part_table%');$$);
DROP SCHEMA partitioning_schema CASCADE;
-- show we don't have pg_dist_partition entries on the workers after dropping the schema
SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_partition where exists(select * from pg_class where pg_class.oid=pg_dist_partition.logicalrelid AND relname ILIKE '%part_table%');$$);
-- show we don't have pg_dist_object entries on the workers after dropping the schema
SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_object as obj where classid = 1259 AND exists(select * from pg_class where pg_class.oid=obj.objid AND relname ILIKE '%part_table%');$$);
-- dropping the parent should drop the metadata on the workers
CREATE SCHEMA partitioning_schema;
SET search_path TO partitioning_schema;
CREATE TABLE part_table (
col timestamp
) PARTITION BY RANGE (col);
CREATE TABLE part_table_1
PARTITION OF part_table
FOR VALUES FROM ('2010-01-01') TO ('2015-01-01');
SELECT create_distributed_table('part_table', 'col');
DROP TABLE part_table;
-- show we don't have pg_dist_partition entries on the workers after dropping the parent
SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_partition where exists(select * from pg_class where pg_class.oid=pg_dist_partition.logicalrelid AND relname ILIKE '%part_table%');$$);
-- show we don't have pg_dist_object entries on the workers after dropping the parent
SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_object as obj where classid = 1259 AND exists(select * from pg_class where pg_class.oid=obj.objid AND relname ILIKE '%part_table%');$$);
SET search_path TO partitioning_schema;
CREATE TABLE part_table (
col timestamp
) PARTITION BY RANGE (col);
CREATE TABLE part_table_1
PARTITION OF part_table
FOR VALUES FROM ('2010-01-01') TO ('2015-01-01');
SELECT create_distributed_table('part_table', 'col');
DROP TABLE part_table_1;
-- show we have pg_dist_partition entries for the parent on the workers after dropping the partition
SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_partition where exists(select * from pg_class where pg_class.oid=pg_dist_partition.logicalrelid AND relname ILIKE '%part_table%');$$);
-- show we have pg_dist_object entries for the parent on the workers after dropping the partition
SELECT run_command_on_workers($$SELECT count(*) FROM pg_dist_object as obj where classid = 1259 AND exists(select * from pg_class where pg_class.oid=obj.objid AND relname ILIKE '%part_table%');$$);
-- clean-up
DROP SCHEMA partitioning_schema CASCADE;

View File

@ -18,12 +18,12 @@ END;
$$; $$;
-- Check all dependent objects and function depends on all nodes -- 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;
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;
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;
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;
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;
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;
SET citus.enable_metadata_sync TO OFF; SET citus.enable_metadata_sync TO OFF;
CREATE TYPE function_prop_type_2 AS (a int, b int); CREATE TYPE function_prop_type_2 AS (a int, b int);
@ -38,10 +38,10 @@ BEGIN
END; 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;
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;
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;
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;
-- Have a separate check for type created in transaction -- Have a separate check for type created in transaction
BEGIN; BEGIN;
@ -60,9 +60,9 @@ BEGIN
END; 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;
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;
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;
-- Check table -- Check table
CREATE TABLE function_prop_table(a int, b int); CREATE TABLE function_prop_table(a int, b int);
@ -97,8 +97,8 @@ BEGIN
END; 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;
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;
-- Views are not supported -- Views are not supported
CREATE VIEW function_prop_view AS SELECT * FROM function_prop_table; CREATE VIEW function_prop_view AS SELECT * FROM function_prop_table;
@ -133,8 +133,8 @@ BEGIN;
$$; $$;
-- Within transaction functions are not distributed -- 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;
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;
COMMIT; COMMIT;
-- Show that recreating it outside transaction distributes the function and dependencies -- Show that recreating it outside transaction distributes the function and dependencies
@ -147,10 +147,10 @@ BEGIN
END; 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;
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;
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;
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;
-- Test for SQL function with unsupported object in function body -- Test for SQL function with unsupported object in function body
CREATE TABLE table_in_sql_body(id int); 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 -- 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;
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;
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;
-- Check extension owned table -- Check extension owned table
CREATE TABLE extension_owned_table(a int); CREATE TABLE extension_owned_table(a int);
@ -202,17 +202,17 @@ BEGIN;
$$; $$;
-- Function shouldn't be propagated within transaction -- 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()); 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'); SELECT create_distributed_table('table_to_prop_func','id');
-- Function should be marked as distributed after distributing the table that depends on it -- 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; COMMIT;
-- Function should be marked as distributed on the worker after committing changes -- 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 -- Multiple functions as a default column
@ -236,20 +236,20 @@ BEGIN;
$$; $$;
-- Functions shouldn't be propagated within transaction -- 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;
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;
CREATE TABLE table_to_prop_func_2(id int, col_1 int default func_in_transaction_1() + func_in_transaction_2()); 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'); SELECT create_distributed_table('table_to_prop_func_2','id');
-- Functions should be marked as distribued after distributing the table that depends on it -- 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;
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;
COMMIT; COMMIT;
-- Functions should be marked as distributed on the worker after committing changes -- 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;
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;
-- If function has dependency on non-distributed table it should error out -- If function has dependency on non-distributed table it should error out
@ -287,16 +287,16 @@ BEGIN;
$$; $$;
-- Function shouldn't be propagated within transaction -- 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(); 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 -- 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; COMMIT;
-- Functions should be marked as distributed on the worker after committing changes -- 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 -- 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 -- 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;
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;
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()); 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'); SELECT create_distributed_table('table_to_prop_func_5', 'id');
-- Functions should be marked as distributed after adding the column -- 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;
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;
COMMIT; COMMIT;
-- Functions should be marked as distributed on the worker after committing changes -- 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;
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;
-- Adding a constraint with function check should propagate the function -- Adding a constraint with function check should propagate the function
BEGIN; BEGIN;
@ -369,17 +369,17 @@ BEGIN;
$$; $$;
-- Functions shouldn't be propagated within transaction -- 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))); 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'); SELECT create_distributed_table('table_to_prop_func_6', 'id');
-- Function should be marked as distributed after adding the column -- 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; COMMIT;
-- Function should be marked as distributed on the worker after committing changes -- 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 -- Adding a constraint with multiple functions check should propagate the function
@ -403,20 +403,20 @@ BEGIN;
$$; $$;
-- Functions shouldn't be propagated within transaction -- 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;
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;
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))); 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'); SELECT create_distributed_table('table_to_prop_func_7', 'id');
-- Function should be marked as distributed after adding the column -- 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;
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;
COMMIT; COMMIT;
-- Functions should be marked as distributed on the worker after committing changes -- 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;
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;
-- Adding a column with constraint should propagate the function -- Adding a column with constraint should propagate the function
@ -434,16 +434,16 @@ BEGIN;
$$; $$;
-- Functions shouldn't be propagated within transaction -- 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)); 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 -- 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; COMMIT;
-- Function should be marked as distributed on the worker after committing changes -- 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 -- If constraint depends on a non-distributed table it should error out
@ -479,7 +479,7 @@ BEGIN;
$$; $$;
-- Functions shouldn't be propagated within transaction -- 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 ( CREATE TABLE people (
id int, id int,
@ -489,7 +489,7 @@ BEGIN;
SELECT create_distributed_table('people', 'id'); SELECT create_distributed_table('people', 'id');
-- Show that function is distributed after distributing the table -- 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; COMMIT;
@ -505,7 +505,7 @@ BEGIN;
$$; $$;
-- Functions shouldn't be propagated within transaction -- 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_1_for_rule(id int, col_1 int);
CREATE TABLE table_2_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'); SELECT create_distributed_table('table_1_for_rule','id');
-- Functions should be distributed after distributing the table -- 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; COMMIT;
-- Function should be marked as distributed on the worker after committing changes -- 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 -- Show that functions as partitioning functions are supported
@ -535,18 +535,18 @@ BEGIN;
$$; $$;
-- Functions shouldn't be propagated within transaction -- 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)); 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'); SELECT create_distributed_table('partitioned_table_to_test_func_prop', 'id');
-- Show that function is distributed after distributing the table -- 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; COMMIT;
-- Function should be marked as distributed on the worker after committing changes -- 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 -- Test function dependency on citus local table
@ -561,14 +561,14 @@ BEGIN;
$$; $$;
-- Function shouldn't be propagated within transaction -- 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()); 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 1 FROM master_add_node('localhost', :master_port, groupid => 0);
SELECT citus_add_local_table_to_metadata('citus_local_table_to_test_func'); 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 -- 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; ROLLBACK;
-- Show that having a function dependency on exlude also works -- Show that having a function dependency on exlude also works
@ -583,17 +583,17 @@ BEGIN;
$$; $$;
-- Functions shouldn't be propagated within transaction -- 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())); 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'); SELECT create_distributed_table('exclusion_func_prop_table', 'id');
-- Function should be marked as distributed after distributing the table that depends on it -- 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; COMMIT;
-- Function should be marked as distributed on the worker after committing changes -- 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 -- Show that having a function dependency for index also works
@ -608,7 +608,7 @@ BEGIN;
$$; $$;
-- Functions shouldn't be propagated within transaction -- 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 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)); 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'); 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 -- 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; COMMIT;
-- Function should be marked as distributed on the worker after committing changes -- 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 -- Test function to function dependency
@ -648,11 +648,11 @@ BEGIN;
SELECT create_distributed_table('func_dep_table', 'a'); SELECT create_distributed_table('func_dep_table', 'a');
-- Function should be marked as distributed after distributing the table that depends on it -- 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; COMMIT;
-- Function should be marked as distributed on the worker after committing changes -- 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 -- Test function with SQL language and sequence dependency
@ -672,7 +672,7 @@ BEGIN;
$$; $$;
-- Function shouldn't be propagated within transaction -- 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 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')))); 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'); SELECT create_distributed_table('table_to_prop_seq_func','id');
-- Function should be marked as distributed after distributing the table that depends on it -- 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; COMMIT;
-- Function should be marked as distributed on the worker after committing changes -- 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 -- Show that having a dependency on another dist table work out tx
@ -709,6 +709,52 @@ CREATE TABLE table_non_for_func_dist (
b int DEFAULT non_sense_func_for_default_val(NULL::loc_for_func_dist)); b int DEFAULT non_sense_func_for_default_val(NULL::loc_for_func_dist));
SELECT create_distributed_table('table_non_for_func_dist', 'a'); SELECT create_distributed_table('table_non_for_func_dist', 'a');
SET citus.shard_replication_factor = 1;
-- test creating a colocated function
CREATE TABLE tbl_to_colocate (a int);
SELECT create_distributed_table('tbl_to_colocate', 'a');
-- first test colocating function with a ref table
CREATE TABLE tbl_to_colocate_ref (a int);
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 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 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 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 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 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 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 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 pg_catalog.pg_dist_object WHERE objid = 'func_to_colocate'::regproc;
-- Show that causing circular dependency via functions are not allowed -- Show that causing circular dependency via functions are not allowed

View File

@ -93,8 +93,8 @@ SELECT count(*) FROM history;
SET citus.shard_replication_factor TO 1; SET citus.shard_replication_factor TO 1;
-- metadata sync will succeed even if we have rep > 1 tables -- 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 pg_catalog.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_single_shard'::regclass::oid, 0);
SELECT start_metadata_sync_to_node('localhost', :worker_1_port); SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
CREATE TABLE mx_table(a int); CREATE TABLE mx_table(a int);

View File

@ -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); SELECT master_remove_node('localhost', :worker_2_port);
-- Removing public schema from pg_dist_object because it breaks the next tests -- 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 pg_catalog.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 = (SELECT oid FROM pg_extension WHERE extname = 'plpgsql');
-- try to manipulate node metadata via non-super user -- try to manipulate node metadata via non-super user
SET ROLE non_super_user; SET ROLE non_super_user;

View File

@ -66,11 +66,6 @@ CREATE FUNCTION find_shard_interval_index(bigint)
AS 'citus' AS 'citus'
LANGUAGE C STRICT; LANGUAGE C STRICT;
-- remove tables from pg_dist_partition, if they don't exist i.e not found in pg_class
delete from pg_dist_partition where not exists(select * from pg_class where pg_class.oid=pg_dist_partition.logicalrelid);
select 1 from run_command_on_workers($$
delete from pg_dist_partition where not exists(select * from pg_class where pg_class.oid=pg_dist_partition.logicalrelid);$$);
-- =================================================================== -- ===================================================================
-- test co-location util functions -- test co-location util functions
-- =================================================================== -- ===================================================================
@ -437,11 +432,11 @@ SELECT update_distributed_table_colocation('table1_group_none', colocate_with =>
SELECT update_distributed_table_colocation('table1_group_none', colocate_with => 'table3_groupE'); SELECT update_distributed_table_colocation('table1_group_none', colocate_with => 'table3_groupE');
-- activate nodes to get rid of inconsistencies in pg_dist tables -- 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 pg_catalog.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 pg_catalog.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 pg_catalog.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 pg_catalog.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, 'table5_groupX'::regclass::oid, 0);
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port); SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
SELECT 1 FROM citus_activate_node('localhost', :worker_2_port); SELECT 1 FROM citus_activate_node('localhost', :worker_2_port);

View File

@ -213,16 +213,17 @@ SELECT * FROM multi_extension.print_extension_changes();
-- Test downgrade to 9.4-1 from 9.5-1 -- Test downgrade to 9.4-1 from 9.5-1
ALTER EXTENSION citus UPDATE TO '9.5-1'; ALTER EXTENSION citus UPDATE TO '9.5-1';
BEGIN; -- 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
SET citus.enable_metadata_sync TO on; -- BEGIN;
SELECT master_add_node('localhost', :master_port, groupId=>0); -- SET citus.enable_metadata_sync TO on;
CREATE TABLE citus_local_table (a int); -- SELECT master_add_node('localhost', :master_port, groupId=>0);
SELECT create_citus_local_table('citus_local_table'); -- CREATE TABLE citus_local_table (a int);
RESET citus.enable_metadata_sync; -- 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'; -- -- downgrade from 9.5-1 to 9.4-1 should fail as we have a citus local table
ROLLBACK; -- ALTER EXTENSION citus UPDATE TO '9.4-1';
-- ROLLBACK;
-- now we can downgrade as there is no citus local table -- now we can downgrade as there is no citus local table
ALTER EXTENSION citus UPDATE TO '9.4-1'; ALTER EXTENSION citus UPDATE TO '9.4-1';

View File

@ -10,5 +10,5 @@ FROM pg_attribute
WHERE atthasmissing AND attrelid NOT IN ('pg_dist_node'::regclass, WHERE atthasmissing AND attrelid NOT IN ('pg_dist_node'::regclass,
'pg_dist_rebalance_strategy'::regclass, 'pg_dist_rebalance_strategy'::regclass,
'pg_dist_partition'::regclass, 'pg_dist_partition'::regclass,
'citus.pg_dist_object'::regclass) 'pg_dist_object'::regclass)
ORDER BY attrelid, attname; ORDER BY attrelid, attname;

View File

@ -544,10 +544,10 @@ DROP TABLE mx_table_with_small_sequence, mx_table_with_sequence;
-- owner -- owner
CREATE TABLE pg_dist_placement_temp AS SELECT * FROM pg_dist_placement; 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_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_placement;
DELETE FROM pg_dist_partition; 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 groupid AS old_worker_2_group FROM pg_dist_node WHERE nodeport = :worker_2_port \gset
SELECT master_remove_node('localhost', :worker_2_port); SELECT master_remove_node('localhost', :worker_2_port);
@ -586,7 +586,7 @@ DROP TABLE mx_table;
\c - postgres - :master_port \c - postgres - :master_port
INSERT INTO pg_dist_placement SELECT * FROM pg_dist_placement_temp; INSERT INTO pg_dist_placement SELECT * FROM pg_dist_placement_temp;
INSERT INTO pg_dist_partition SELECT * FROM pg_dist_partition_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_placement_temp;
DROP TABLE pg_dist_partition_temp; DROP TABLE pg_dist_partition_temp;
DROP TABLE pg_dist_object_temp; DROP TABLE pg_dist_object_temp;

View File

@ -67,7 +67,7 @@ END;$$;
SELECT create_distributed_function('proc_0(float8)', 'dist_key', 'test_proc_colocation_0' ); 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 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; SET client_min_messages TO DEBUG1;
CALL proc_0(1.0); CALL proc_0(1.0);
@ -81,7 +81,7 @@ CALL proc_0(1.0);
RESET client_min_messages; RESET client_min_messages;
SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('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');
-- colocatewith is not null && list_length(colocatedTableList) = 1 -- colocatewith is not null && list_length(colocatedTableList) = 1
SELECT alter_distributed_table('test_proc_colocation_0', shard_count:= 4); SELECT alter_distributed_table('test_proc_colocation_0', shard_count:= 4);
@ -94,7 +94,7 @@ CALL proc_0(1.0);
RESET client_min_messages; RESET client_min_messages;
SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('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');
-- shardCount is not null && cascade_to_colocated is true -- shardCount is not null && cascade_to_colocated is true
SELECT alter_distributed_table('test_proc_colocation_0', shard_count:= 8, cascade_to_colocated := 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; RESET client_min_messages;
SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('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');
-- colocatewith is not null && cascade_to_colocated is true -- colocatewith is not null && cascade_to_colocated is true
SELECT alter_distributed_table('test_proc_colocation_0', shard_count:= 4, cascade_to_colocated := 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; RESET client_min_messages;
SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('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');
-- try a case with more than one procedure -- try a case with more than one procedure
CREATE OR REPLACE procedure proc_1(dist_key float8) 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 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 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; SET client_min_messages TO DEBUG1;
CALL proc_0(1.0); CALL proc_0(1.0);
@ -149,14 +149,14 @@ CALL proc_1(2.0);
RESET client_min_messages; RESET client_min_messages;
SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('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;
-- case which shouldn't preserve colocation for now -- case which shouldn't preserve colocation for now
-- shardCount is not null && cascade_to_colocated is false -- 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 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 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; SET client_min_messages TO WARNING;
DROP SCHEMA mx_alter_distributed_table CASCADE; DROP SCHEMA mx_alter_distributed_table CASCADE;

View File

@ -308,7 +308,7 @@ SELECT create_distributed_table('mx_old_schema.table_set_schema', 'id');
CREATE SCHEMA mx_new_schema; CREATE SCHEMA mx_new_schema;
SELECT objid::oid::regnamespace as "Distributed Schemas" 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') WHERE objid::oid::regnamespace IN ('mx_old_schema', 'mx_new_schema')
ORDER BY "Distributed Schemas"; ORDER BY "Distributed Schemas";
\c - - - :worker_1_port \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; ALTER TABLE mx_old_schema.table_set_schema SET SCHEMA mx_new_schema;
SELECT objid::oid::regnamespace as "Distributed Schemas" 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'); WHERE objid::oid::regnamespace IN ('mx_old_schema', 'mx_new_schema');
\c - - - :worker_1_port \c - - - :worker_1_port
SELECT table_schema AS "Table's Schema" FROM information_schema.tables WHERE table_name='table_set_schema'; SELECT table_schema AS "Table's Schema" FROM information_schema.tables WHERE table_name='table_set_schema';

View File

@ -672,7 +672,7 @@ SELECT create_distributed_table('old_schema.table_set_schema', 'id');
CREATE SCHEMA new_schema; CREATE SCHEMA new_schema;
SELECT objid::oid::regnamespace as "Distributed Schemas" 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'); WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema');
\c - - - :worker_1_port \c - - - :worker_1_port
SELECT table_schema AS "Shards' Schema" 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; ALTER TABLE old_schema.table_set_schema SET SCHEMA new_schema;
SELECT objid::oid::regnamespace as "Distributed Schemas" 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'); WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema');
\c - - - :worker_1_port \c - - - :worker_1_port
SELECT table_schema AS "Shards' Schema" SELECT table_schema AS "Shards' Schema"
@ -706,7 +706,7 @@ SELECT create_distributed_table('table_set_schema', 'id');
CREATE SCHEMA new_schema; CREATE SCHEMA new_schema;
SELECT objid::oid::regnamespace as "Distributed Schemas" SELECT objid::oid::regnamespace as "Distributed Schemas"
FROM citus.pg_dist_object FROM pg_catalog.pg_dist_object
WHERE objid='new_schema'::regnamespace::oid; WHERE objid='new_schema'::regnamespace::oid;
\c - - - :worker_1_port \c - - - :worker_1_port
SELECT table_schema AS "Shards' Schema" 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; ALTER TABLE table_set_schema SET SCHEMA new_schema;
SELECT objid::oid::regnamespace as "Distributed Schemas" SELECT objid::oid::regnamespace as "Distributed Schemas"
FROM citus.pg_dist_object FROM pg_catalog.pg_dist_object
WHERE objid='new_schema'::regnamespace::oid; WHERE objid='new_schema'::regnamespace::oid;
\c - - - :worker_1_port \c - - - :worker_1_port
SELECT table_schema AS "Shards' Schema" SELECT table_schema AS "Shards' Schema"
@ -743,7 +743,7 @@ SELECT create_distributed_table('table_set_schema', 'id');
CREATE SCHEMA new_schema; CREATE SCHEMA new_schema;
SELECT objid::oid::regnamespace as "Distributed Schemas" 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'); WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema');
\c - - - :worker_1_port \c - - - :worker_1_port
SELECT table_schema AS "Shards' Schema", COUNT(*) AS "Counts" 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; ALTER TABLE table_set_schema SET SCHEMA new_schema;
SELECT objid::oid::regnamespace as "Distributed Schemas" 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'); WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema');
\c - - - :worker_1_port \c - - - :worker_1_port
SELECT table_schema AS "Shards' Schema", COUNT(*) AS "Counts" SELECT table_schema AS "Shards' Schema", COUNT(*) AS "Counts"
@ -969,9 +969,9 @@ ROLLBACK;
-- Clean up the created schema -- Clean up the created schema
SET client_min_messages TO WARNING; 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'); 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; 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 -- 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'); WHERE classid=2615 and objid IN (select oid from pg_namespace where nspname='run_test_schema');

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