mirror of https://github.com/citusdata/citus.git
Merge branch 'master' into velioglu/cyclic_dep
commit
9b583fdb0d
|
@ -11,11 +11,13 @@
|
|||
#include "postgres.h"
|
||||
#include "miscadmin.h"
|
||||
|
||||
|
||||
#include "distributed/commands/utility_hook.h"
|
||||
#include "distributed/commands.h"
|
||||
#include "distributed/metadata_utility.h"
|
||||
#include "distributed/coordinator_protocol.h"
|
||||
#include "distributed/metadata_sync.h"
|
||||
#include "distributed/multi_partitioning_utils.h"
|
||||
#include "distributed/worker_transaction.h"
|
||||
#include "utils/builtins.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
|
||||
* have any workers with metadata. Also, the function errors out if called
|
||||
* 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
|
||||
MasterRemoveDistributedTableMetadataFromWorkers(Oid relationId, char *schemaName,
|
||||
|
@ -146,6 +152,16 @@ MasterRemoveDistributedTableMetadataFromWorkers(Oid relationId, char *schemaName
|
|||
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 */
|
||||
char *deleteDistributionCommand = DistributionDeleteCommand(schemaName, tableName);
|
||||
SendCommandToWorkersWithMetadata(deleteDistributionCommand);
|
||||
|
|
|
@ -69,6 +69,10 @@
|
|||
(strncmp(arg, prefix, strlen(prefix)) == 0)
|
||||
|
||||
/* forward declaration for helper functions*/
|
||||
static bool RecreateSameNonColocatedFunction(ObjectAddress functionAddress,
|
||||
char *distributionArgumentName,
|
||||
bool colocateWithTableNameDefault,
|
||||
bool *forceDelegationAddress);
|
||||
static void ErrorIfAnyNodeDoesNotHaveMetadata(void);
|
||||
static char * GetAggregateDDLCommand(const RegProcedure funcOid, bool useCreateOrReplace);
|
||||
static char * GetFunctionAlterOwnerCommand(const RegProcedure funcOid);
|
||||
|
@ -128,6 +132,7 @@ create_distributed_function(PG_FUNCTION_ARGS)
|
|||
|
||||
char *distributionArgumentName = NULL;
|
||||
char *colocateWithTableName = NULL;
|
||||
bool colocateWithTableNameDefault = false;
|
||||
bool *forceDelegationAddress = NULL;
|
||||
bool forceDelegation = false;
|
||||
ObjectAddress extensionAddress = { 0 };
|
||||
|
@ -167,8 +172,13 @@ create_distributed_function(PG_FUNCTION_ARGS)
|
|||
colocateWithText = PG_GETARG_TEXT_P(2);
|
||||
colocateWithTableName = text_to_cstring(colocateWithText);
|
||||
|
||||
if (pg_strncasecmp(colocateWithTableName, "default", NAMEDATALEN) == 0)
|
||||
{
|
||||
colocateWithTableNameDefault = true;
|
||||
}
|
||||
|
||||
/* check if the colocation belongs to a reference table */
|
||||
if (pg_strncasecmp(colocateWithTableName, "default", NAMEDATALEN) != 0)
|
||||
if (!colocateWithTableNameDefault)
|
||||
{
|
||||
Oid colocationRelationId = ResolveRelationId(colocateWithText, false);
|
||||
colocatedWithReferenceTable = IsCitusTableType(colocationRelationId,
|
||||
|
@ -192,6 +202,20 @@ create_distributed_function(PG_FUNCTION_ARGS)
|
|||
|
||||
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
|
||||
* 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
|
||||
* of the worker nodes does not have the metadata.
|
||||
|
|
|
@ -220,8 +220,9 @@ MarkObjectDistributedLocally(const ObjectAddress *distAddress)
|
|||
ObjectIdGetDatum(distAddress->objectId),
|
||||
Int32GetDatum(distAddress->objectSubId)
|
||||
};
|
||||
char *insertQuery = "INSERT INTO citus.pg_dist_object (classid, objid, objsubid) "
|
||||
"VALUES ($1, $2, $3) ON CONFLICT DO NOTHING";
|
||||
char *insertQuery =
|
||||
"INSERT INTO pg_catalog.pg_dist_object (classid, objid, objsubid) "
|
||||
"VALUES ($1, $2, $3) ON CONFLICT DO NOTHING";
|
||||
int spiStatus = ExecuteCommandAsSuperuser(insertQuery, paramCount, paramTypes,
|
||||
paramValues);
|
||||
if (spiStatus < 0)
|
||||
|
@ -335,7 +336,7 @@ UnmarkObjectDistributed(const ObjectAddress *address)
|
|||
Int32GetDatum(address->objectSubId)
|
||||
};
|
||||
|
||||
char *deleteQuery = "DELETE FROM citus.pg_dist_object WHERE classid = $1 AND "
|
||||
char *deleteQuery = "DELETE FROM pg_catalog.pg_dist_object WHERE classid = $1 AND "
|
||||
"objid = $2 AND objsubid = $3";
|
||||
|
||||
int spiStatus = ExecuteCommandAsSuperuser(deleteQuery, paramCount, paramTypes,
|
||||
|
|
|
@ -247,8 +247,13 @@ static void GetPartitionTypeInputInfo(char *partitionKeyString, char partitionMe
|
|||
Oid *intervalTypeId, int32 *intervalTypeMod);
|
||||
static void CachedNamespaceLookup(const char *nspname, Oid *cachedOid);
|
||||
static void CachedRelationLookup(const char *relationName, Oid *cachedOid);
|
||||
static void CachedRelationLookupExtended(const char *relationName, Oid *cachedOid,
|
||||
bool missing_ok);
|
||||
static void CachedRelationNamespaceLookup(const char *relationName, Oid relnamespace,
|
||||
Oid *cachedOid);
|
||||
static void CachedRelationNamespaceLookupExtended(const char *relationName,
|
||||
Oid renamespace, Oid *cachedOid,
|
||||
bool missing_ok);
|
||||
static ShardPlacement * ResolveGroupShardPlacement(
|
||||
GroupShardPlacement *groupShardPlacement, CitusTableCacheEntry *tableEntry,
|
||||
int shardIndex);
|
||||
|
@ -2321,8 +2326,37 @@ CitusCatalogNamespaceId(void)
|
|||
Oid
|
||||
DistObjectRelationId(void)
|
||||
{
|
||||
CachedRelationNamespaceLookup("pg_dist_object", CitusCatalogNamespaceId(),
|
||||
&MetadataCache.distObjectRelationId);
|
||||
/*
|
||||
* In older versions pg_dist_object was living in the `citus` namespace, With Citus 11
|
||||
* this has been moved to pg_dist_catalog.
|
||||
*
|
||||
* During upgrades it could therefore be that we simply need to look in the old
|
||||
* catalog. Since we expect to find it most of the time in the pg_catalog schema from
|
||||
* now on we will start there.
|
||||
*
|
||||
* even after the table has been moved, the oid's stay the same, so we don't have to
|
||||
* invalidate the cache after a move
|
||||
*
|
||||
* Note: during testing we also up/downgrade the extension, and sometimes interact
|
||||
* with the database when the schema and the binary are not in sync. Hance we always
|
||||
* allow the catalog to be missing on our first lookup. The error message might
|
||||
* therefore become misleading as it will complain about citus.pg_dist_object not
|
||||
* being found when called too early.
|
||||
*/
|
||||
CachedRelationLookupExtended("pg_dist_object",
|
||||
&MetadataCache.distObjectRelationId,
|
||||
true);
|
||||
if (!OidIsValid(MetadataCache.distObjectRelationId))
|
||||
{
|
||||
/*
|
||||
* We can only ever reach here while we are creating/altering our extension before
|
||||
* the table is moved to pg_catalog.
|
||||
*/
|
||||
CachedRelationNamespaceLookupExtended("pg_dist_object",
|
||||
CitusCatalogNamespaceId(),
|
||||
&MetadataCache.distObjectRelationId,
|
||||
false);
|
||||
}
|
||||
|
||||
return MetadataCache.distObjectRelationId;
|
||||
}
|
||||
|
@ -2332,9 +2366,38 @@ DistObjectRelationId(void)
|
|||
Oid
|
||||
DistObjectPrimaryKeyIndexId(void)
|
||||
{
|
||||
CachedRelationNamespaceLookup("pg_dist_object_pkey",
|
||||
CitusCatalogNamespaceId(),
|
||||
&MetadataCache.distObjectPrimaryKeyIndexId);
|
||||
/*
|
||||
* In older versions pg_dist_object was living in the `citus` namespace, With Citus 11
|
||||
* this has been moved to pg_dist_catalog.
|
||||
*
|
||||
* During upgrades it could therefore be that we simply need to look in the old
|
||||
* catalog. Since we expect to find it most of the time in the pg_catalog schema from
|
||||
* now on we will start there.
|
||||
*
|
||||
* even after the table has been moved, the oid's stay the same, so we don't have to
|
||||
* invalidate the cache after a move
|
||||
*
|
||||
* Note: during testing we also up/downgrade the extension, and sometimes interact
|
||||
* with the database when the schema and the binary are not in sync. Hance we always
|
||||
* allow the catalog to be missing on our first lookup. The error message might
|
||||
* therefore become misleading as it will complain about citus.pg_dist_object not
|
||||
* being found when called too early.
|
||||
*/
|
||||
CachedRelationLookupExtended("pg_dist_object_pkey",
|
||||
&MetadataCache.distObjectPrimaryKeyIndexId,
|
||||
true);
|
||||
|
||||
if (!OidIsValid(MetadataCache.distObjectPrimaryKeyIndexId))
|
||||
{
|
||||
/*
|
||||
* We can only ever reach here while we are creating/altering our extension before
|
||||
* the table is moved to pg_catalog.
|
||||
*/
|
||||
CachedRelationNamespaceLookupExtended("pg_dist_object_pkey",
|
||||
CitusCatalogNamespaceId(),
|
||||
&MetadataCache.distObjectPrimaryKeyIndexId,
|
||||
false);
|
||||
}
|
||||
|
||||
return MetadataCache.distObjectPrimaryKeyIndexId;
|
||||
}
|
||||
|
@ -4591,9 +4654,30 @@ CachedRelationLookup(const char *relationName, Oid *cachedOid)
|
|||
}
|
||||
|
||||
|
||||
/*
|
||||
* CachedRelationLookupExtended performs a cached lookup for the relation
|
||||
* relationName, with the result cached in *cachedOid. Will _not_ throw an error when
|
||||
* missing_ok is set to true.
|
||||
*/
|
||||
static void
|
||||
CachedRelationLookupExtended(const char *relationName, Oid *cachedOid, bool missing_ok)
|
||||
{
|
||||
CachedRelationNamespaceLookupExtended(relationName, PG_CATALOG_NAMESPACE, cachedOid,
|
||||
missing_ok);
|
||||
}
|
||||
|
||||
|
||||
static void
|
||||
CachedRelationNamespaceLookup(const char *relationName, Oid relnamespace,
|
||||
Oid *cachedOid)
|
||||
{
|
||||
CachedRelationNamespaceLookupExtended(relationName, relnamespace, cachedOid, false);
|
||||
}
|
||||
|
||||
|
||||
static void
|
||||
CachedRelationNamespaceLookupExtended(const char *relationName, Oid relnamespace,
|
||||
Oid *cachedOid, bool missing_ok)
|
||||
{
|
||||
/* force callbacks to be registered, so we always get notified upon changes */
|
||||
InitializeCaches();
|
||||
|
@ -4602,7 +4686,7 @@ CachedRelationNamespaceLookup(const char *relationName, Oid relnamespace,
|
|||
{
|
||||
*cachedOid = get_relname_relid(relationName, relnamespace);
|
||||
|
||||
if (*cachedOid == InvalidOid)
|
||||
if (*cachedOid == InvalidOid && !missing_ok)
|
||||
{
|
||||
ereport(ERROR, (errmsg(
|
||||
"cache lookup failed for %s, called too early?",
|
||||
|
|
|
@ -2226,6 +2226,19 @@ CitusAuthHook(Port *port, int status)
|
|||
"regular client connections",
|
||||
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 */
|
||||
|
|
|
@ -22,26 +22,29 @@
|
|||
#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_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/citus_isolation_test_session_is_blocked/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_backend_gpid/11.0-1.sql"
|
||||
|
||||
CREATE VIEW citus.citus_worker_stat_activity AS
|
||||
SELECT * FROM pg_catalog.citus_worker_stat_activity();
|
||||
ALTER VIEW citus.citus_worker_stat_activity SET SCHEMA pg_catalog;
|
||||
GRANT SELECT ON pg_catalog.citus_worker_stat_activity TO PUBLIC;
|
||||
|
||||
DROP FUNCTION IF EXISTS pg_catalog.citus_dist_stat_activity() CASCADE;
|
||||
DROP FUNCTION IF EXISTS pg_catalog.citus_worker_stat_activity() CASCADE;
|
||||
#include "udfs/citus_dist_stat_activity/11.0-1.sql"
|
||||
|
||||
CREATE VIEW citus.citus_dist_stat_activity AS
|
||||
SELECT * FROM pg_catalog.citus_dist_stat_activity();
|
||||
ALTER VIEW citus.citus_dist_stat_activity SET SCHEMA pg_catalog;
|
||||
GRANT SELECT ON pg_catalog.citus_dist_stat_activity TO PUBLIC;
|
||||
-- a very simple helper function defined for citus_lock_waits
|
||||
CREATE OR REPLACE FUNCTION get_nodeid_for_groupid(groupIdInput int) RETURNS int AS $$
|
||||
DECLARE
|
||||
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/pg_cancel_backend/11.0-1.sql"
|
||||
|
@ -87,5 +90,7 @@ $$;
|
|||
|
||||
#include "udfs/citus_finalize_upgrade_to_citus11/11.0-1.sql"
|
||||
|
||||
#include "udfs/run_command_on_all_nodes/11.0-1.sql"
|
||||
#include "udfs/citus_stat_activity/11.0-1.sql"
|
||||
ALTER TABLE citus.pg_dist_object SET SCHEMA pg_catalog;
|
||||
GRANT SELECT ON pg_catalog.pg_dist_object TO public;
|
||||
#include "udfs/citus_prepare_pg_upgrade/11.0-1.sql"
|
||||
#include "udfs/citus_finish_pg_upgrade/11.0-1.sql"
|
||||
|
|
|
@ -1,4 +1,6 @@
|
|||
-- citus--11.0-1--10.2-4
|
||||
REVOKE SELECT ON pg_catalog.pg_dist_object FROM public;
|
||||
ALTER TABLE pg_catalog.pg_dist_object SET SCHEMA citus;
|
||||
|
||||
DROP FUNCTION pg_catalog.create_distributed_function(regprocedure, text, text, bool);
|
||||
|
||||
|
@ -124,8 +126,8 @@ DROP VIEW pg_catalog.citus_lock_waits;
|
|||
DROP FUNCTION citus_internal_local_blocked_processes;
|
||||
DROP FUNCTION citus_internal_global_blocked_processes;
|
||||
|
||||
DROP VIEW pg_catalog.citus_dist_stat_activity;
|
||||
DROP FUNCTION pg_catalog.citus_dist_stat_activity;
|
||||
DROP VIEW IF EXISTS 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,
|
||||
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;
|
||||
|
||||
SET search_path = 'pg_catalog';
|
||||
DROP VIEW citus_worker_stat_activity;
|
||||
DROP FUNCTION citus_worker_stat_activity;
|
||||
DROP VIEW IF EXISTS 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,
|
||||
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_calculate_gpid(integer,integer);
|
||||
DROP FUNCTION pg_catalog.citus_backend_gpid();
|
||||
DROP FUNCTION get_nodeid_for_groupid(integer);
|
||||
|
||||
RESET search_path;
|
||||
|
||||
|
|
|
@ -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,
|
||||
OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name,
|
||||
OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET,
|
||||
OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz,
|
||||
OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text,
|
||||
OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text, OUT global_pid int8)
|
||||
RETURNS SETOF RECORD
|
||||
LANGUAGE C STRICT AS 'MODULE_PATHNAME',
|
||||
$$citus_dist_stat_activity$$;
|
||||
CREATE OR REPLACE VIEW citus.citus_dist_stat_activity AS
|
||||
SELECT * FROM citus_stat_activity
|
||||
WHERE is_worker_query = false;
|
||||
|
||||
COMMENT ON FUNCTION pg_catalog.citus_dist_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int,
|
||||
OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name,
|
||||
OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET,
|
||||
OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz,
|
||||
OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text,
|
||||
OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text, OUT global_pid int8)
|
||||
IS 'returns distributed transaction activity on distributed tables';
|
||||
ALTER VIEW citus.citus_dist_stat_activity SET SCHEMA pg_catalog;
|
||||
GRANT SELECT ON pg_catalog.citus_dist_stat_activity TO PUBLIC;
|
||||
|
|
|
@ -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,
|
||||
OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name,
|
||||
OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET,
|
||||
OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz,
|
||||
OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text,
|
||||
OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text, OUT global_pid int8)
|
||||
RETURNS SETOF RECORD
|
||||
LANGUAGE C STRICT AS 'MODULE_PATHNAME',
|
||||
$$citus_dist_stat_activity$$;
|
||||
CREATE OR REPLACE VIEW citus.citus_dist_stat_activity AS
|
||||
SELECT * FROM citus_stat_activity
|
||||
WHERE is_worker_query = false;
|
||||
|
||||
COMMENT ON FUNCTION pg_catalog.citus_dist_stat_activity(OUT query_hostname text, OUT query_hostport int, OUT distributed_query_host_name text, OUT distributed_query_host_port int,
|
||||
OUT transaction_number int8, OUT transaction_stamp timestamptz, OUT datid oid, OUT datname name,
|
||||
OUT pid int, OUT usesysid oid, OUT usename name, OUT application_name text, OUT client_addr INET,
|
||||
OUT client_hostname TEXT, OUT client_port int, OUT backend_start timestamptz, OUT xact_start timestamptz,
|
||||
OUT query_start timestamptz, OUT state_change timestamptz, OUT wait_event_type text, OUT wait_event text,
|
||||
OUT state text, OUT backend_xid xid, OUT backend_xmin xid, OUT query text, OUT backend_type text, OUT global_pid int8)
|
||||
IS 'returns distributed transaction activity on distributed tables';
|
||||
ALTER VIEW citus.citus_dist_stat_activity SET SCHEMA pg_catalog;
|
||||
GRANT SELECT ON pg_catalog.citus_dist_stat_activity TO PUBLIC;
|
||||
|
|
|
@ -12,22 +12,15 @@ 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;
|
||||
|
||||
|
|
|
@ -0,0 +1,151 @@
|
|||
CREATE OR REPLACE FUNCTION pg_catalog.citus_finish_pg_upgrade()
|
||||
RETURNS void
|
||||
LANGUAGE plpgsql
|
||||
SET search_path = pg_catalog
|
||||
AS $cppu$
|
||||
DECLARE
|
||||
table_name regclass;
|
||||
command text;
|
||||
trigger_name text;
|
||||
BEGIN
|
||||
|
||||
|
||||
IF substring(current_Setting('server_version'), '\d+')::int >= 14 THEN
|
||||
EXECUTE $cmd$
|
||||
-- disable propagation to prevent EnsureCoordinator errors
|
||||
-- the aggregate created here does not depend on Citus extension (yet)
|
||||
-- since we add the dependency with the next command
|
||||
SET citus.enable_ddl_propagation TO OFF;
|
||||
CREATE AGGREGATE array_cat_agg(anycompatiblearray) (SFUNC = array_cat, STYPE = anycompatiblearray);
|
||||
COMMENT ON AGGREGATE array_cat_agg(anycompatiblearray)
|
||||
IS 'concatenate input arrays into a single array';
|
||||
RESET citus.enable_ddl_propagation;
|
||||
$cmd$;
|
||||
ELSE
|
||||
EXECUTE $cmd$
|
||||
SET citus.enable_ddl_propagation TO OFF;
|
||||
CREATE AGGREGATE array_cat_agg(anyarray) (SFUNC = array_cat, STYPE = anyarray);
|
||||
COMMENT ON AGGREGATE array_cat_agg(anyarray)
|
||||
IS 'concatenate input arrays into a single array';
|
||||
RESET citus.enable_ddl_propagation;
|
||||
$cmd$;
|
||||
END IF;
|
||||
|
||||
--
|
||||
-- Citus creates the array_cat_agg but because of a compatibility
|
||||
-- issue between pg13-pg14, we drop and create it during upgrade.
|
||||
-- And as Citus creates it, there needs to be a dependency to the
|
||||
-- Citus extension, so we create that dependency here.
|
||||
-- We are not using:
|
||||
-- ALTER EXENSION citus DROP/CREATE AGGREGATE array_cat_agg
|
||||
-- because we don't have an easy way to check if the aggregate
|
||||
-- exists with anyarray type or anycompatiblearray type.
|
||||
|
||||
INSERT INTO pg_depend
|
||||
SELECT
|
||||
'pg_proc'::regclass::oid as classid,
|
||||
(SELECT oid FROM pg_proc WHERE proname = 'array_cat_agg') as objid,
|
||||
0 as objsubid,
|
||||
'pg_extension'::regclass::oid as refclassid,
|
||||
(select oid from pg_extension where extname = 'citus') as refobjid,
|
||||
0 as refobjsubid ,
|
||||
'e' as deptype;
|
||||
|
||||
--
|
||||
-- restore citus catalog tables
|
||||
--
|
||||
INSERT INTO pg_catalog.pg_dist_partition SELECT * FROM public.pg_dist_partition;
|
||||
INSERT INTO pg_catalog.pg_dist_shard SELECT * FROM public.pg_dist_shard;
|
||||
INSERT INTO pg_catalog.pg_dist_placement SELECT * FROM public.pg_dist_placement;
|
||||
INSERT INTO pg_catalog.pg_dist_node_metadata SELECT * FROM public.pg_dist_node_metadata;
|
||||
INSERT INTO pg_catalog.pg_dist_node SELECT * FROM public.pg_dist_node;
|
||||
INSERT INTO pg_catalog.pg_dist_local_group SELECT * FROM public.pg_dist_local_group;
|
||||
INSERT INTO pg_catalog.pg_dist_transaction SELECT * FROM public.pg_dist_transaction;
|
||||
INSERT INTO pg_catalog.pg_dist_colocation SELECT * FROM public.pg_dist_colocation;
|
||||
-- enterprise catalog tables
|
||||
INSERT INTO pg_catalog.pg_dist_authinfo SELECT * FROM public.pg_dist_authinfo;
|
||||
INSERT INTO pg_catalog.pg_dist_poolinfo SELECT * FROM public.pg_dist_poolinfo;
|
||||
|
||||
INSERT INTO pg_catalog.pg_dist_rebalance_strategy SELECT
|
||||
name,
|
||||
default_strategy,
|
||||
shard_cost_function::regprocedure::regproc,
|
||||
node_capacity_function::regprocedure::regproc,
|
||||
shard_allowed_on_node_function::regprocedure::regproc,
|
||||
default_threshold,
|
||||
minimum_threshold,
|
||||
improvement_threshold
|
||||
FROM public.pg_dist_rebalance_strategy;
|
||||
|
||||
--
|
||||
-- drop backup tables
|
||||
--
|
||||
DROP TABLE public.pg_dist_authinfo;
|
||||
DROP TABLE public.pg_dist_colocation;
|
||||
DROP TABLE public.pg_dist_local_group;
|
||||
DROP TABLE public.pg_dist_node;
|
||||
DROP TABLE public.pg_dist_node_metadata;
|
||||
DROP TABLE public.pg_dist_partition;
|
||||
DROP TABLE public.pg_dist_placement;
|
||||
DROP TABLE public.pg_dist_poolinfo;
|
||||
DROP TABLE public.pg_dist_shard;
|
||||
DROP TABLE public.pg_dist_transaction;
|
||||
DROP TABLE public.pg_dist_rebalance_strategy;
|
||||
|
||||
--
|
||||
-- reset sequences
|
||||
--
|
||||
PERFORM setval('pg_catalog.pg_dist_shardid_seq', (SELECT MAX(shardid)+1 AS max_shard_id FROM pg_dist_shard), false);
|
||||
PERFORM setval('pg_catalog.pg_dist_placement_placementid_seq', (SELECT MAX(placementid)+1 AS max_placement_id FROM pg_dist_placement), false);
|
||||
PERFORM setval('pg_catalog.pg_dist_groupid_seq', (SELECT MAX(groupid)+1 AS max_group_id FROM pg_dist_node), false);
|
||||
PERFORM setval('pg_catalog.pg_dist_node_nodeid_seq', (SELECT MAX(nodeid)+1 AS max_node_id FROM pg_dist_node), false);
|
||||
PERFORM setval('pg_catalog.pg_dist_colocationid_seq', (SELECT MAX(colocationid)+1 AS max_colocation_id FROM pg_dist_colocation), false);
|
||||
|
||||
--
|
||||
-- register triggers
|
||||
--
|
||||
FOR table_name IN SELECT logicalrelid FROM pg_catalog.pg_dist_partition
|
||||
LOOP
|
||||
trigger_name := 'truncate_trigger_' || table_name::oid;
|
||||
command := 'create trigger ' || trigger_name || ' after truncate on ' || table_name || ' execute procedure pg_catalog.citus_truncate_trigger()';
|
||||
EXECUTE command;
|
||||
command := 'update pg_trigger set tgisinternal = true where tgname = ' || quote_literal(trigger_name);
|
||||
EXECUTE command;
|
||||
END LOOP;
|
||||
|
||||
--
|
||||
-- set dependencies
|
||||
--
|
||||
INSERT INTO pg_depend
|
||||
SELECT
|
||||
'pg_class'::regclass::oid as classid,
|
||||
p.logicalrelid::regclass::oid as objid,
|
||||
0 as objsubid,
|
||||
'pg_extension'::regclass::oid as refclassid,
|
||||
(select oid from pg_extension where extname = 'citus') as refobjid,
|
||||
0 as refobjsubid ,
|
||||
'n' as deptype
|
||||
FROM pg_catalog.pg_dist_partition p;
|
||||
|
||||
-- set dependencies for columnar table access method
|
||||
PERFORM citus_internal.columnar_ensure_am_depends_catalog();
|
||||
|
||||
-- restore pg_dist_object from the stable identifiers
|
||||
TRUNCATE pg_catalog.pg_dist_object;
|
||||
INSERT INTO pg_catalog.pg_dist_object (classid, objid, objsubid, distribution_argument_index, colocationid)
|
||||
SELECT
|
||||
address.classid,
|
||||
address.objid,
|
||||
address.objsubid,
|
||||
naming.distribution_argument_index,
|
||||
naming.colocationid
|
||||
FROM
|
||||
public.pg_dist_object naming,
|
||||
pg_catalog.pg_get_object_address(naming.type, naming.object_names, naming.object_args) address;
|
||||
|
||||
DROP TABLE public.pg_dist_object;
|
||||
END;
|
||||
$cppu$;
|
||||
|
||||
COMMENT ON FUNCTION pg_catalog.citus_finish_pg_upgrade()
|
||||
IS 'perform tasks to restore citus settings from a location that has been prepared before pg_upgrade';
|
|
@ -131,8 +131,8 @@ BEGIN
|
|||
PERFORM citus_internal.columnar_ensure_am_depends_catalog();
|
||||
|
||||
-- restore pg_dist_object from the stable identifiers
|
||||
TRUNCATE citus.pg_dist_object;
|
||||
INSERT INTO citus.pg_dist_object (classid, objid, objsubid, distribution_argument_index, colocationid)
|
||||
TRUNCATE pg_catalog.pg_dist_object;
|
||||
INSERT INTO pg_catalog.pg_dist_object (classid, objid, objsubid, distribution_argument_index, colocationid)
|
||||
SELECT
|
||||
address.classid,
|
||||
address.objid,
|
||||
|
|
|
@ -2,43 +2,44 @@ SET search_path = 'pg_catalog';
|
|||
|
||||
CREATE VIEW citus.citus_lock_waits AS
|
||||
WITH
|
||||
citus_dist_stat_activity AS
|
||||
(
|
||||
SELECT * FROM citus_dist_stat_activity
|
||||
unique_global_wait_edges_with_calculated_gpids AS (
|
||||
SELECT
|
||||
-- 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
|
||||
(
|
||||
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
|
||||
citus_dist_stat_activity.*, (CASE citus_dist_stat_activity.distributed_query_host_name WHEN 'coordinator_host' THEN 0 ELSE pg_dist_node.nodeid END) as initiator_node_id
|
||||
FROM
|
||||
citus_dist_stat_activity LEFT JOIN pg_dist_node
|
||||
ON
|
||||
citus_dist_stat_activity.distributed_query_host_name = pg_dist_node.nodename AND
|
||||
citus_dist_stat_activity.distributed_query_host_port = pg_dist_node.nodeport
|
||||
-- if global_pid is NULL, it is most likely that a backend is blocked on a DDL
|
||||
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
|
||||
)
|
||||
SELECT
|
||||
waiting.global_pid as waiting_gpid,
|
||||
blocking.global_pid as blocking_gpid,
|
||||
waiting.pid AS waiting_pid,
|
||||
blocking.pid AS blocking_pid,
|
||||
waiting.query AS blocked_statement,
|
||||
blocking.query AS current_statement_in_blocking_process,
|
||||
waiting.initiator_node_id AS waiting_node_id,
|
||||
blocking.initiator_node_id AS blocking_node_id,
|
||||
waiting.distributed_query_host_name AS waiting_node_name,
|
||||
blocking.distributed_query_host_name AS blocking_node_name,
|
||||
waiting.distributed_query_host_port AS waiting_node_port,
|
||||
blocking.distributed_query_host_port AS blocking_node_port
|
||||
waiting.global_pid as waiting_gpid,
|
||||
blocking.global_pid as blocking_gpid,
|
||||
waiting.query AS blocked_statement,
|
||||
blocking.query AS current_statement_in_blocking_process,
|
||||
waiting.nodeid AS waiting_nodeid,
|
||||
blocking.nodeid AS blocking_nodeid
|
||||
FROM
|
||||
unique_global_wait_edges
|
||||
JOIN
|
||||
citus_dist_stat_activity_with_node_id waiting ON (unique_global_wait_edges.waiting_global_pid = waiting.global_pid)
|
||||
JOIN
|
||||
citus_dist_stat_activity_with_node_id blocking ON (unique_global_wait_edges.blocking_global_pid = blocking.global_pid);
|
||||
unique_global_wait_edges
|
||||
JOIN
|
||||
citus_dist_stat_activity_with_calculated_gpids waiting ON (unique_global_wait_edges.waiting_global_pid = waiting.global_pid)
|
||||
JOIN
|
||||
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;
|
||||
GRANT SELECT ON pg_catalog.citus_lock_waits TO PUBLIC;
|
||||
|
|
|
@ -2,43 +2,44 @@ SET search_path = 'pg_catalog';
|
|||
|
||||
CREATE VIEW citus.citus_lock_waits AS
|
||||
WITH
|
||||
citus_dist_stat_activity AS
|
||||
(
|
||||
SELECT * FROM citus_dist_stat_activity
|
||||
unique_global_wait_edges_with_calculated_gpids AS (
|
||||
SELECT
|
||||
-- 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
|
||||
(
|
||||
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
|
||||
citus_dist_stat_activity.*, (CASE citus_dist_stat_activity.distributed_query_host_name WHEN 'coordinator_host' THEN 0 ELSE pg_dist_node.nodeid END) as initiator_node_id
|
||||
FROM
|
||||
citus_dist_stat_activity LEFT JOIN pg_dist_node
|
||||
ON
|
||||
citus_dist_stat_activity.distributed_query_host_name = pg_dist_node.nodename AND
|
||||
citus_dist_stat_activity.distributed_query_host_port = pg_dist_node.nodeport
|
||||
-- if global_pid is NULL, it is most likely that a backend is blocked on a DDL
|
||||
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
|
||||
)
|
||||
SELECT
|
||||
waiting.global_pid as waiting_gpid,
|
||||
blocking.global_pid as blocking_gpid,
|
||||
waiting.pid AS waiting_pid,
|
||||
blocking.pid AS blocking_pid,
|
||||
waiting.query AS blocked_statement,
|
||||
blocking.query AS current_statement_in_blocking_process,
|
||||
waiting.initiator_node_id AS waiting_node_id,
|
||||
blocking.initiator_node_id AS blocking_node_id,
|
||||
waiting.distributed_query_host_name AS waiting_node_name,
|
||||
blocking.distributed_query_host_name AS blocking_node_name,
|
||||
waiting.distributed_query_host_port AS waiting_node_port,
|
||||
blocking.distributed_query_host_port AS blocking_node_port
|
||||
waiting.global_pid as waiting_gpid,
|
||||
blocking.global_pid as blocking_gpid,
|
||||
waiting.query AS blocked_statement,
|
||||
blocking.query AS current_statement_in_blocking_process,
|
||||
waiting.nodeid AS waiting_nodeid,
|
||||
blocking.nodeid AS blocking_nodeid
|
||||
FROM
|
||||
unique_global_wait_edges
|
||||
JOIN
|
||||
citus_dist_stat_activity_with_node_id waiting ON (unique_global_wait_edges.waiting_global_pid = waiting.global_pid)
|
||||
JOIN
|
||||
citus_dist_stat_activity_with_node_id blocking ON (unique_global_wait_edges.blocking_global_pid = blocking.global_pid);
|
||||
unique_global_wait_edges
|
||||
JOIN
|
||||
citus_dist_stat_activity_with_calculated_gpids waiting ON (unique_global_wait_edges.waiting_global_pid = waiting.global_pid)
|
||||
JOIN
|
||||
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;
|
||||
GRANT SELECT ON pg_catalog.citus_lock_waits TO PUBLIC;
|
||||
|
|
|
@ -0,0 +1,74 @@
|
|||
CREATE OR REPLACE FUNCTION pg_catalog.citus_prepare_pg_upgrade()
|
||||
RETURNS void
|
||||
LANGUAGE plpgsql
|
||||
SET search_path = pg_catalog
|
||||
AS $cppu$
|
||||
BEGIN
|
||||
|
||||
DELETE FROM pg_depend WHERE
|
||||
objid IN (SELECT oid FROM pg_proc WHERE proname = 'array_cat_agg') AND
|
||||
refobjid IN (select oid from pg_extension where extname = 'citus');
|
||||
--
|
||||
-- We are dropping the aggregates because postgres 14 changed
|
||||
-- array_cat type from anyarray to anycompatiblearray. When
|
||||
-- upgrading to pg14, specifically when running pg_restore on
|
||||
-- array_cat_agg we would get an error. So we drop the aggregate
|
||||
-- and create the right one on citus_finish_pg_upgrade.
|
||||
|
||||
DROP AGGREGATE IF EXISTS array_cat_agg(anyarray);
|
||||
DROP AGGREGATE IF EXISTS array_cat_agg(anycompatiblearray);
|
||||
--
|
||||
-- Drop existing backup tables
|
||||
--
|
||||
DROP TABLE IF EXISTS public.pg_dist_partition;
|
||||
DROP TABLE IF EXISTS public.pg_dist_shard;
|
||||
DROP TABLE IF EXISTS public.pg_dist_placement;
|
||||
DROP TABLE IF EXISTS public.pg_dist_node_metadata;
|
||||
DROP TABLE IF EXISTS public.pg_dist_node;
|
||||
DROP TABLE IF EXISTS public.pg_dist_local_group;
|
||||
DROP TABLE IF EXISTS public.pg_dist_transaction;
|
||||
DROP TABLE IF EXISTS public.pg_dist_colocation;
|
||||
DROP TABLE IF EXISTS public.pg_dist_authinfo;
|
||||
DROP TABLE IF EXISTS public.pg_dist_poolinfo;
|
||||
DROP TABLE IF EXISTS public.pg_dist_rebalance_strategy;
|
||||
DROP TABLE IF EXISTS public.pg_dist_object;
|
||||
|
||||
--
|
||||
-- backup citus catalog tables
|
||||
--
|
||||
CREATE TABLE public.pg_dist_partition AS SELECT * FROM pg_catalog.pg_dist_partition;
|
||||
CREATE TABLE public.pg_dist_shard AS SELECT * FROM pg_catalog.pg_dist_shard;
|
||||
CREATE TABLE public.pg_dist_placement AS SELECT * FROM pg_catalog.pg_dist_placement;
|
||||
CREATE TABLE public.pg_dist_node_metadata AS SELECT * FROM pg_catalog.pg_dist_node_metadata;
|
||||
CREATE TABLE public.pg_dist_node AS SELECT * FROM pg_catalog.pg_dist_node;
|
||||
CREATE TABLE public.pg_dist_local_group AS SELECT * FROM pg_catalog.pg_dist_local_group;
|
||||
CREATE TABLE public.pg_dist_transaction AS SELECT * FROM pg_catalog.pg_dist_transaction;
|
||||
CREATE TABLE public.pg_dist_colocation AS SELECT * FROM pg_catalog.pg_dist_colocation;
|
||||
-- enterprise catalog tables
|
||||
CREATE TABLE public.pg_dist_authinfo AS SELECT * FROM pg_catalog.pg_dist_authinfo;
|
||||
CREATE TABLE public.pg_dist_poolinfo AS SELECT * FROM pg_catalog.pg_dist_poolinfo;
|
||||
CREATE TABLE public.pg_dist_rebalance_strategy AS SELECT
|
||||
name,
|
||||
default_strategy,
|
||||
shard_cost_function::regprocedure::text,
|
||||
node_capacity_function::regprocedure::text,
|
||||
shard_allowed_on_node_function::regprocedure::text,
|
||||
default_threshold,
|
||||
minimum_threshold,
|
||||
improvement_threshold
|
||||
FROM pg_catalog.pg_dist_rebalance_strategy;
|
||||
|
||||
-- store upgrade stable identifiers on pg_dist_object catalog
|
||||
CREATE TABLE public.pg_dist_object AS SELECT
|
||||
address.type,
|
||||
address.object_names,
|
||||
address.object_args,
|
||||
objects.distribution_argument_index,
|
||||
objects.colocationid
|
||||
FROM pg_catalog.pg_dist_object objects,
|
||||
pg_catalog.pg_identify_object_as_address(objects.classid, objects.objid, objects.objsubid) address;
|
||||
END;
|
||||
$cppu$;
|
||||
|
||||
COMMENT ON FUNCTION pg_catalog.citus_prepare_pg_upgrade()
|
||||
IS 'perform tasks to copy citus settings to a location that could later be restored after pg_upgrade is done';
|
|
@ -65,7 +65,7 @@ BEGIN
|
|||
address.object_args,
|
||||
objects.distribution_argument_index,
|
||||
objects.colocationid
|
||||
FROM citus.pg_dist_object objects,
|
||||
FROM pg_catalog.pg_dist_object objects,
|
||||
pg_catalog.pg_identify_object_as_address(objects.classid, objects.objid, objects.objsubid) address;
|
||||
END;
|
||||
$cppu$;
|
||||
|
|
|
@ -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';
|
|
@ -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';
|
|
@ -188,7 +188,7 @@ run_commands_on_session_level_connection_to_node(PG_FUNCTION_ARGS)
|
|||
|
||||
/*
|
||||
* override_backend_data_command_originator is a wrapper around
|
||||
* OverrideBackendDataDistributedCommandOriginator().
|
||||
* SetBackendDataDistributedCommandOriginator().
|
||||
*/
|
||||
Datum
|
||||
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);
|
||||
|
||||
OverrideBackendDataDistributedCommandOriginator(distributedCommandOriginator);
|
||||
SetBackendDataDistributedCommandOriginator(distributedCommandOriginator);
|
||||
|
||||
PG_RETURN_VOID();
|
||||
}
|
||||
|
|
|
@ -721,6 +721,7 @@ UnSetGlobalPID(void)
|
|||
MyBackendData->globalPID = 0;
|
||||
MyBackendData->databaseId = 0;
|
||||
MyBackendData->userId = 0;
|
||||
MyBackendData->distributedCommandOriginator = false;
|
||||
|
||||
SpinLockRelease(&MyBackendData->mutex);
|
||||
}
|
||||
|
@ -850,12 +851,16 @@ AssignGlobalPID(void)
|
|||
|
||||
|
||||
/*
|
||||
* OverrideBackendDataDistributedCommandOriginator should only be used for isolation testing.
|
||||
* See how it is used in the relevant functions.
|
||||
* SetBackendDataDistributedCommandOriginator is used to set the distributedCommandOriginator
|
||||
* field on MyBackendData.
|
||||
*/
|
||||
void
|
||||
OverrideBackendDataDistributedCommandOriginator(bool distributedCommandOriginator)
|
||||
SetBackendDataDistributedCommandOriginator(bool distributedCommandOriginator)
|
||||
{
|
||||
if (!MyBackendData)
|
||||
{
|
||||
return;
|
||||
}
|
||||
SpinLockAcquire(&MyBackendData->mutex);
|
||||
MyBackendData->distributedCommandOriginator =
|
||||
distributedCommandOriginator;
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -29,6 +29,7 @@
|
|||
#include "distributed/coordinator_protocol.h"
|
||||
#include "distributed/metadata_cache.h"
|
||||
#include "distributed/metadata/distobject.h"
|
||||
#include "distributed/multi_partitioning_utils.h"
|
||||
#include "foreign/foreign.h"
|
||||
#include "utils/builtins.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_sequence_dependency);
|
||||
|
||||
|
||||
static void WorkerDropDistributedTable(Oid relationId);
|
||||
#if PG_VERSION_NUM < PG_VERSION_13
|
||||
static long deleteDependencyRecordsForSpecific(Oid classId, Oid objectId, char deptype,
|
||||
Oid refclassId, Oid refobjectId);
|
||||
#endif
|
||||
|
||||
|
||||
/*
|
||||
* 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
|
||||
|
@ -64,8 +66,6 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS)
|
|||
text *relationName = PG_GETARG_TEXT_P(0);
|
||||
Oid relationId = ResolveRelationId(relationName, true);
|
||||
|
||||
ObjectAddress distributedTableObject = { InvalidOid, InvalidOid, 0 };
|
||||
|
||||
if (!OidIsValid(relationId))
|
||||
{
|
||||
ereport(NOTICE, (errmsg("relation %s does not exist, skipping",
|
||||
|
@ -75,8 +75,45 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS)
|
|||
|
||||
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 */
|
||||
Relation distributedRelation = relation_open(relationId, AccessShareLock);
|
||||
|
||||
|
@ -86,9 +123,7 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS)
|
|||
relation_close(distributedRelation, AccessShareLock);
|
||||
|
||||
/* prepare distributedTableObject for dropping the table */
|
||||
distributedTableObject.classId = RelationRelationId;
|
||||
distributedTableObject.objectId = relationId;
|
||||
distributedTableObject.objectSubId = 0;
|
||||
ObjectAddress distributedTableObject = { RelationRelationId, relationId, 0 };
|
||||
|
||||
/* Drop dependent sequences from pg_dist_object */
|
||||
#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 */
|
||||
List *shardList = LoadShardList(relationId);
|
||||
uint64 *shardIdPointer = NULL;
|
||||
foreach_ptr(shardIdPointer, shardList)
|
||||
{
|
||||
|
@ -140,8 +176,6 @@ worker_drop_distributed_table(PG_FUNCTION_ARGS)
|
|||
|
||||
/* delete the row from pg_dist_partition */
|
||||
DeletePartitionRow(relationId);
|
||||
|
||||
PG_RETURN_VOID();
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -55,8 +55,8 @@ extern void UnSetGlobalPID(void);
|
|||
extern void AssignDistributedTransactionId(void);
|
||||
extern void AssignGlobalPID(void);
|
||||
extern uint64 GetGlobalPID(void);
|
||||
extern void OverrideBackendDataDistributedCommandOriginator(bool
|
||||
distributedCommandOriginator);
|
||||
extern void SetBackendDataDistributedCommandOriginator(bool
|
||||
distributedCommandOriginator);
|
||||
extern uint64 ExtractGlobalPID(char *applicationName);
|
||||
extern int ExtractNodeIdFromGlobalPID(uint64 globalPID, bool missingOk);
|
||||
extern int ExtractProcessIdFromGlobalPID(uint64 globalPID);
|
||||
|
|
|
@ -86,7 +86,7 @@ extern void SyncDeleteColocationGroupToNodes(uint32 colocationId);
|
|||
#define DELETE_ALL_NODES "DELETE FROM pg_dist_node"
|
||||
#define DELETE_ALL_PLACEMENTS "DELETE FROM pg_dist_placement"
|
||||
#define DELETE_ALL_SHARDS "DELETE FROM pg_dist_shard"
|
||||
#define DELETE_ALL_DISTRIBUTED_OBJECTS "DELETE FROM citus.pg_dist_object"
|
||||
#define DELETE_ALL_DISTRIBUTED_OBJECTS "DELETE FROM pg_catalog.pg_dist_object"
|
||||
#define DELETE_ALL_PARTITIONS "DELETE FROM pg_dist_partition"
|
||||
#define DELETE_ALL_COLOCATION "DELETE FROM pg_catalog.pg_dist_colocation"
|
||||
#define REMOVE_ALL_SHELL_TABLES_COMMAND \
|
||||
|
|
|
@ -40,12 +40,16 @@ create aggregate sum2_strict (int) (
|
|||
combinefunc = sum2_sfunc_strict
|
||||
);
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -96,12 +100,16 @@ create aggregate psum_strict(int, int)(
|
|||
initcond=0
|
||||
);
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -283,7 +291,7 @@ create aggregate binstragg(text, text)(
|
|||
stype=text
|
||||
);
|
||||
-- verify that the aggregate is added into pg_dist_object, on each worker
|
||||
SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.binstragg'::regproc;$$);
|
||||
SELECT run_command_on_workers($$SELECT count(*) from pg_catalog.pg_dist_object where objid = 'aggregate_support.binstragg'::regproc;$$);
|
||||
run_command_on_workers
|
||||
---------------------------------------------------------------------
|
||||
(localhost,57637,t,1)
|
||||
|
@ -298,6 +306,8 @@ SELECT run_command_on_workers($$select count(*) from pg_aggregate where aggfnoid
|
|||
(2 rows)
|
||||
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -527,12 +537,16 @@ CREATE AGGREGATE last (
|
|||
combinefunc = last_agg
|
||||
);
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -590,6 +604,8 @@ SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid
|
|||
(2 rows)
|
||||
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -614,6 +630,8 @@ create aggregate array_collect_sort(el int) (
|
|||
initcond = '{}'
|
||||
);
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -1139,7 +1157,7 @@ SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid
|
|||
(2 rows)
|
||||
|
||||
-- verify that the aggregate is added into pg_dist_object, on each worker
|
||||
SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$);
|
||||
SELECT run_command_on_workers($$SELECT count(*) from pg_catalog.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$);
|
||||
run_command_on_workers
|
||||
---------------------------------------------------------------------
|
||||
(localhost,57637,t,1)
|
||||
|
@ -1150,7 +1168,7 @@ RESET citus.create_object_propagation;
|
|||
-- drop and test outside of tx block
|
||||
drop aggregate dependent_agg (float8);
|
||||
-- verify that the aggregate is removed from pg_dist_object, on each worker
|
||||
SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$);
|
||||
SELECT run_command_on_workers($$SELECT count(*) from pg_catalog.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$);
|
||||
run_command_on_workers
|
||||
---------------------------------------------------------------------
|
||||
(localhost,57637,f,"ERROR: function ""aggregate_support.dependent_agg"" does not exist")
|
||||
|
|
|
@ -11,6 +11,8 @@ BEGIN
|
|||
RETURN localGroupId;
|
||||
END; $$ language plpgsql 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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -24,6 +26,8 @@ BEGIN
|
|||
RETURN localGroupId;
|
||||
END; $$ language plpgsql VOLATILE;
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
|
|
@ -21,6 +21,8 @@ BEGIN
|
|||
RETURN localGroupId;
|
||||
END; $$ language plpgsql 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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
|
|
@ -21,6 +21,8 @@ BEGIN
|
|||
RETURN localGroupId;
|
||||
END; $$ language plpgsql 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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
|
|
@ -77,7 +77,7 @@ SELECT create_distributed_table('t4','a');
|
|||
DROP TABLE t4; -- as long as the table is using the type some operations are hard to force
|
||||
COMMIT;
|
||||
-- verify the type is distributed
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = 'disabled_object_propagation.tt3'::regtype::oid;
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = 'disabled_object_propagation.tt3'::regtype::oid;
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
|
|
@ -33,6 +33,8 @@ BEGIN
|
|||
END;
|
||||
$$;
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -193,7 +195,7 @@ SELECT create_distributed_function('"eq_mi''xed_param_names"(macaddr, macaddr)')
|
|||
|
||||
(1 row)
|
||||
|
||||
SELECT distribution_argument_index is NULL, colocationid is NULL from citus.pg_dist_object
|
||||
SELECT distribution_argument_index is NULL, colocationid is NULL from pg_catalog.pg_dist_object
|
||||
WHERE objid = 'eq_mi''xed_param_names(macaddr, macaddr)'::regprocedure;
|
||||
?column? | ?column?
|
||||
---------------------------------------------------------------------
|
||||
|
@ -486,6 +488,8 @@ AS 'select $1 = $2;'
|
|||
IMMUTABLE
|
||||
RETURNS NULL ON NULL INPUT;
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -648,7 +652,7 @@ SELECT create_distributed_function('eq_with_param_names(macaddr, macaddr)', '$1'
|
|||
|
||||
-- show that the colocationIds are the same
|
||||
SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated
|
||||
FROM pg_dist_partition, citus.pg_dist_object as objects
|
||||
FROM pg_dist_partition, pg_catalog.pg_dist_object as objects
|
||||
WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND
|
||||
objects.objid = 'eq_with_param_names(macaddr, macaddr)'::regprocedure;
|
||||
table_and_function_colocated
|
||||
|
@ -665,7 +669,7 @@ SELECT create_distributed_function('eq_with_param_names(macaddr, macaddr)', 'val
|
|||
(1 row)
|
||||
|
||||
SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated
|
||||
FROM pg_dist_partition, citus.pg_dist_object as objects
|
||||
FROM pg_dist_partition, pg_catalog.pg_dist_object as objects
|
||||
WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND
|
||||
objects.objid = 'eq_with_param_names(macaddr, macaddr)'::regprocedure;
|
||||
table_and_function_colocated
|
||||
|
@ -700,7 +704,7 @@ SELECT create_distributed_function('eq8(macaddr8, macaddr8)', '$1', colocate_wit
|
|||
(1 row)
|
||||
|
||||
SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated
|
||||
FROM pg_dist_partition, citus.pg_dist_object as objects
|
||||
FROM pg_dist_partition, pg_catalog.pg_dist_object as objects
|
||||
WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND
|
||||
objects.objid = 'eq8(macaddr8, macaddr8)'::regprocedure;
|
||||
table_and_function_colocated
|
||||
|
@ -715,7 +719,7 @@ SELECT create_distributed_function('add_text(text, text)', '$1', colocate_with:=
|
|||
(1 row)
|
||||
|
||||
SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated
|
||||
FROM pg_dist_partition, citus.pg_dist_object as objects
|
||||
FROM pg_dist_partition, pg_catalog.pg_dist_object as objects
|
||||
WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND
|
||||
objects.objid = 'add_text(text, text)'::regprocedure;
|
||||
table_and_function_colocated
|
||||
|
@ -797,6 +801,8 @@ BEGIN
|
|||
END;
|
||||
$$;
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -824,6 +830,8 @@ CREATE OR REPLACE FUNCTION func_with_out_param(a int, out b int)
|
|||
RETURNS int
|
||||
LANGUAGE sql AS $$ select 1; $$;
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -869,6 +877,8 @@ SELECT create_distributed_function('func_with_inout_param(int)');
|
|||
ERROR: function "func_with_inout_param(int)" does not exist
|
||||
-- this should work
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -893,6 +903,8 @@ CREATE OR REPLACE FUNCTION func_with_variadic_param(a int, variadic b int[])
|
|||
LANGUAGE sql AS $$ select 1; $$;
|
||||
-- this should work
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -923,6 +935,8 @@ $BODY$
|
|||
LANGUAGE plpgsql VOLATILE
|
||||
COST 100;
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -961,6 +975,8 @@ $BODY$
|
|||
LANGUAGE plpgsql VOLATILE
|
||||
COST 100;
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -997,6 +1013,8 @@ SELECT create_distributed_function('proc_with_variadic_param(date)');
|
|||
ERROR: function "proc_with_variadic_param(date)" does not exist
|
||||
-- this should work
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -1028,6 +1046,8 @@ SELECT create_distributed_function('proc_with_inout_param(date)');
|
|||
ERROR: function "proc_with_inout_param(date)" does not exist
|
||||
-- this should work
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
|
|
@ -26,6 +26,8 @@ CREATE AGGREGATE existing_agg(int) (
|
|||
STYPE = 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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -87,6 +89,8 @@ CREATE AGGREGATE existing_agg(int) (
|
|||
STYPE = 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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
|
|
@ -395,3 +395,117 @@ NOTICE: issuing ROLLBACK
|
|||
DROP SCHEMA drop_partitioned_table CASCADE;
|
||||
NOTICE: drop cascades to 3 other objects
|
||||
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
|
||||
|
|
|
@ -307,8 +307,8 @@ $$ LANGUAGE plpgsql;
|
|||
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
|
||||
SELECT create_distributed_function('func_calls_forcepush_func()');
|
||||
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
|
||||
NOTICE: procedure forcepushdown_schema.func_calls_forcepush_func is already distributed
|
||||
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
|
||||
create_distributed_function
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -1182,6 +1182,7 @@ END;
|
|||
$$
|
||||
LANGUAGE plpgsql STABLE;
|
||||
SELECT create_distributed_function('dist_func(int, int)');
|
||||
NOTICE: procedure insert_select_repartition.dist_func is already distributed
|
||||
create_distributed_function
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
|
|
@ -32,32 +32,32 @@ pg_sleep
|
|||
(1 row)
|
||||
|
||||
step s2-view-dist:
|
||||
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' 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;
|
||||
|coordinator_host| 57636|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression
|
||||
|idle in transaction|Client |ClientRead|postgres|regression
|
||||
(1 row)
|
||||
|
||||
step s3-view-worker:
|
||||
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' 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', '
|
||||
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', '
|
||||
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', '
|
||||
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', '
|
||||
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)
|
||||
|
||||
step s2-rollback:
|
||||
|
@ -102,21 +102,21 @@ pg_sleep
|
|||
(1 row)
|
||||
|
||||
step s2-view-dist:
|
||||
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' 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);
|
||||
|coordinator_host| 57636|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression
|
||||
|idle in transaction|Client |ClientRead|postgres|regression
|
||||
(1 row)
|
||||
|
||||
step s3-view-worker:
|
||||
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' 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)
|
||||
|
||||
step s2-rollback:
|
||||
|
@ -166,24 +166,24 @@ pg_sleep
|
|||
(1 row)
|
||||
|
||||
step s2-view-dist:
|
||||
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' 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;
|
||||
|coordinator_host| 57636|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression
|
||||
|idle in transaction|Client |ClientRead|postgres|regression
|
||||
(1 row)
|
||||
|
||||
step s3-view-worker:
|
||||
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' 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_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_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_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_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|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|idle in transaction|Client |ClientRead|postgres|regression
|
||||
(4 rows)
|
||||
|
||||
step s2-rollback:
|
||||
|
@ -233,21 +233,21 @@ pg_sleep
|
|||
(1 row)
|
||||
|
||||
step s2-view-dist:
|
||||
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' 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;
|
||||
|coordinator_host| 57636|coordinator_host | 57636|idle in transaction|Client |ClientRead|postgres|regression
|
||||
|idle in transaction|Client |ClientRead|postgres|regression
|
||||
(1 row)
|
||||
|
||||
step s3-view-worker:
|
||||
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' 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)
|
||||
|
||||
step s2-rollback:
|
||||
|
|
|
@ -9,7 +9,7 @@ starting permutation: s1-print-distributed-objects s1-begin s1-add-worker s2-pub
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -102,7 +102,7 @@ create_distributed_table
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -180,7 +180,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-begin s1-add-work
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -279,7 +279,7 @@ step s2-commit:
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -357,7 +357,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-begin s2-public-s
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -456,7 +456,7 @@ step s1-commit:
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -534,7 +534,7 @@ starting permutation: s1-print-distributed-objects s1-begin s1-add-worker s2-cre
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -628,7 +628,7 @@ create_distributed_table
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -707,7 +707,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-begin s1-add-work
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -807,7 +807,7 @@ step s2-commit:
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -886,7 +886,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-begin s2-create-s
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -986,7 +986,7 @@ step s1-commit:
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -1065,7 +1065,7 @@ starting permutation: s1-print-distributed-objects s2-create-schema s1-begin s2-
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -1165,7 +1165,7 @@ step s2-commit:
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -1244,7 +1244,7 @@ starting permutation: s1-print-distributed-objects s1-add-worker s2-create-schem
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -1358,7 +1358,7 @@ step s3-commit:
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -1438,7 +1438,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-begin s3-begin s1
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -1560,7 +1560,7 @@ step s2-commit:
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -1641,7 +1641,7 @@ starting permutation: s1-print-distributed-objects s1-begin s1-add-worker s2-pub
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -1726,7 +1726,7 @@ step s1-commit:
|
|||
step s2-create-type: <... completed>
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -1804,7 +1804,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-public-schema s2-
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -1888,7 +1888,7 @@ step s1-commit:
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -1966,7 +1966,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-begin s2-create-s
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -2069,7 +2069,7 @@ step s1-commit:
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -2149,7 +2149,7 @@ starting permutation: s1-print-distributed-objects s1-begin s1-add-worker s2-pub
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -2254,7 +2254,7 @@ wait_until_metadata_sync
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -2332,7 +2332,7 @@ starting permutation: s1-print-distributed-objects s1-begin s2-public-schema s2-
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -2444,7 +2444,7 @@ wait_until_metadata_sync
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -2522,7 +2522,7 @@ starting permutation: s1-print-distributed-objects s2-begin s2-create-schema s2-
|
|||
step s1-print-distributed-objects:
|
||||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
@ -2635,7 +2635,7 @@ wait_until_metadata_sync
|
|||
|
||||
step s2-print-distributed-objects:
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
SELECT run_command_on_workers($$SELECT count(*) FROM pg_namespace where nspname = 'myschema';$$);
|
||||
|
|
|
@ -20,7 +20,7 @@ step s1-commit:
|
|||
|
||||
step s2-create-extension-version-11: <... completed>
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
@ -81,7 +81,7 @@ step s1-commit:
|
|||
|
||||
step s2-alter-extension-update-to-version-12: <... completed>
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
@ -150,7 +150,7 @@ step s1-commit:
|
|||
|
||||
step s2-drop-extension: <... completed>
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
@ -206,7 +206,7 @@ step s1-commit:
|
|||
|
||||
step s2-create-extension-with-schema1: <... completed>
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
@ -267,7 +267,7 @@ step s1-commit:
|
|||
|
||||
step s2-drop-extension: <... completed>
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
@ -338,7 +338,7 @@ step s1-commit:
|
|||
|
||||
step s2-alter-extension-set-schema3: <... completed>
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
@ -406,7 +406,7 @@ step s1-commit:
|
|||
|
||||
step s2-create-extension-with-schema1: <... completed>
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
@ -481,7 +481,7 @@ step s2-commit:
|
|||
COMMIT;
|
||||
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
@ -564,7 +564,7 @@ step s1-add-node-1: <... completed>
|
|||
(1 row)
|
||||
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
@ -633,7 +633,7 @@ step s1-remove-node-1: <... completed>
|
|||
(1 row)
|
||||
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
@ -688,7 +688,7 @@ step s2-commit:
|
|||
COMMIT;
|
||||
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
@ -763,7 +763,7 @@ step s1-remove-node-1: <... completed>
|
|||
(1 row)
|
||||
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
@ -830,7 +830,7 @@ step s2-commit:
|
|||
COMMIT;
|
||||
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
@ -909,7 +909,7 @@ step s1-add-node-1: <... completed>
|
|||
(1 row)
|
||||
|
||||
step s1-print:
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
|
|
@ -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
|
||||
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');
|
||||
<waiting ...>
|
||||
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 = 15;
|
||||
|coordinator_host |coordinator_host | 57636| 57636
|
||||
|
||||
(1 row)
|
||||
|
||||
step s1-commit:
|
||||
|
@ -61,7 +61,7 @@ stop_session_level_connection_to_node
|
|||
|
||||
(1 row)
|
||||
|
||||
restore_isolation_tester_func
|
||||
citus_remove_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(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');
|
||||
<waiting ...>
|
||||
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)
|
||||
|
||||
step s1-commit-worker:
|
||||
|
@ -157,7 +157,7 @@ stop_session_level_connection_to_node
|
|||
|
||||
(1 row)
|
||||
|
||||
restore_isolation_tester_func
|
||||
citus_remove_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(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');
|
||||
<waiting ...>
|
||||
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)
|
||||
|
||||
step s1-commit-worker:
|
||||
|
@ -253,7 +253,7 @@ stop_session_level_connection_to_node
|
|||
|
||||
(1 row)
|
||||
|
||||
restore_isolation_tester_func
|
||||
citus_remove_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(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');
|
||||
<waiting ...>
|
||||
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)
|
||||
|
||||
step s1-commit-worker:
|
||||
|
@ -349,7 +349,7 @@ stop_session_level_connection_to_node
|
|||
|
||||
(1 row)
|
||||
|
||||
restore_isolation_tester_func
|
||||
citus_remove_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(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');
|
||||
<waiting ...>
|
||||
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)
|
||||
|
||||
step s1-commit-worker:
|
||||
|
@ -445,7 +445,7 @@ stop_session_level_connection_to_node
|
|||
|
||||
(1 row)
|
||||
|
||||
restore_isolation_tester_func
|
||||
citus_remove_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
@ -501,9 +501,9 @@ run_commands_on_session_level_connection_to_node
|
|||
(1 row)
|
||||
|
||||
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)
|
||||
|
||||
|
@ -539,7 +539,7 @@ stop_session_level_connection_to_node
|
|||
|
||||
(1 row)
|
||||
|
||||
restore_isolation_tester_func
|
||||
citus_remove_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(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');
|
||||
<waiting ...>
|
||||
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)
|
||||
|
||||
step s1-commit-worker:
|
||||
|
@ -635,7 +635,7 @@ stop_session_level_connection_to_node
|
|||
|
||||
(1 row)
|
||||
|
||||
restore_isolation_tester_func
|
||||
citus_remove_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
@ -691,9 +691,9 @@ run_commands_on_session_level_connection_to_node
|
|||
(1 row)
|
||||
|
||||
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)
|
||||
|
||||
|
@ -729,7 +729,7 @@ stop_session_level_connection_to_node
|
|||
|
||||
(1 row)
|
||||
|
||||
restore_isolation_tester_func
|
||||
citus_remove_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
@ -785,9 +785,9 @@ run_commands_on_session_level_connection_to_node
|
|||
(1 row)
|
||||
|
||||
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)
|
||||
|
||||
|
@ -823,7 +823,7 @@ stop_session_level_connection_to_node
|
|||
|
||||
(1 row)
|
||||
|
||||
restore_isolation_tester_func
|
||||
citus_remove_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(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');
|
||||
<waiting ...>
|
||||
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)
|
||||
|
||||
step s1-commit-worker:
|
||||
|
@ -919,7 +919,7 @@ stop_session_level_connection_to_node
|
|||
|
||||
(1 row)
|
||||
|
||||
restore_isolation_tester_func
|
||||
citus_remove_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
@ -957,13 +957,13 @@ step s1-alter-table:
|
|||
ALTER TABLE ref_table ADD CONSTRAINT rf_p_key PRIMARY KEY(user_id);
|
||||
<waiting ...>
|
||||
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);
|
||||
|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)
|
||||
|
||||
step s2-commit-worker:
|
||||
|
@ -986,7 +986,7 @@ stop_session_level_connection_to_node
|
|||
|
||||
(1 row)
|
||||
|
||||
restore_isolation_tester_func
|
||||
citus_remove_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
@ -1003,22 +1003,22 @@ step s2-update-on-the-coordinator:
|
|||
UPDATE tt1 SET value_1 = 4;
|
||||
<waiting ...>
|
||||
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;
|
||||
|coordinator_host |coordinator_host | 57636| 57636
|
||||
|
||||
(1 row)
|
||||
|
||||
step s1-commit:
|
||||
COMMIT;
|
||||
|
||||
step s2-update-on-the-coordinator: <... completed>
|
||||
restore_isolation_tester_func
|
||||
citus_remove_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(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');
|
||||
<waiting ...>
|
||||
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)
|
||||
|
||||
step s1-commit-worker:
|
||||
|
@ -1114,7 +1114,7 @@ stop_session_level_connection_to_node
|
|||
|
||||
(1 row)
|
||||
|
||||
restore_isolation_tester_func
|
||||
citus_remove_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(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');
|
||||
<waiting ...>
|
||||
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)
|
||||
|
||||
step s1-commit-worker:
|
||||
|
@ -1194,13 +1194,13 @@ stop_session_level_connection_to_node
|
|||
|
||||
(1 row)
|
||||
|
||||
restore_isolation_tester_func
|
||||
citus_remove_node
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(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:
|
||||
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');
|
||||
<waiting ...>
|
||||
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 = 15;
|
||||
|coordinator_host |coordinator_host | 57636| 57636
|
||||
|
||||
(1 row)
|
||||
|
||||
step s1-commit:
|
||||
|
@ -1245,7 +1245,87 @@ stop_session_level_connection_to_node
|
|||
|
||||
(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,6 +1,6 @@
|
|||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -39,7 +39,7 @@ t
|
|||
(1 row)
|
||||
|
||||
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%'
|
||||
)
|
||||
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
|
||||
(1 row)
|
||||
|
||||
step s2-coordinator-citus_worker_stat_activity:
|
||||
SELECT query FROM citus_worker_stat_activity() WHERE global_pid IN (
|
||||
step s2-coordinator-citus_stat_activity-in-workers:
|
||||
SELECT query FROM citus_stat_activity WHERE global_pid IN (
|
||||
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;
|
||||
|
||||
query
|
||||
|
@ -86,7 +88,7 @@ citus_remove_node
|
|||
(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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -113,7 +115,7 @@ t
|
|||
(1 row)
|
||||
|
||||
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%'
|
||||
)
|
||||
AND query NOT ILIKE '%run_commands_on_session_level_connection_to_node%'
|
||||
|
@ -128,10 +130,12 @@ query
|
|||
|
||||
(1 row)
|
||||
|
||||
step s2-coordinator-citus_worker_stat_activity:
|
||||
SELECT query FROM citus_worker_stat_activity() WHERE global_pid IN (
|
||||
step s2-coordinator-citus_stat_activity-in-workers:
|
||||
SELECT query FROM citus_stat_activity WHERE global_pid IN (
|
||||
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;
|
||||
|
||||
query
|
||||
|
|
|
@ -99,34 +99,37 @@ pg_sleep
|
|||
(1 row)
|
||||
|
||||
step s2-view-dist:
|
||||
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' 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();
|
||||
|coordinator_host| 57636| | 0|idle |Client |ClientRead|postgres|regression
|
||||
|idle |Client |ClientRead|postgres|regression
|
||||
|
||||
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)
|
||||
|
||||
step s2-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
|
||||
query NOT ILIKE '%dump_local_%' AND
|
||||
query NOT ILIKE '%citus_internal_local_blocked_processes%' AND
|
||||
query NOT ILIKE '%add_node%' AND
|
||||
backend_type = 'client backend'
|
||||
ORDER BY query, query_hostport 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%'),
|
||||
('%dump_local_%'),
|
||||
('%citus_internal_local_blocked_processes%'),
|
||||
('%add_node%'),
|
||||
('%csa_from_one_node%'))
|
||||
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)|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
|
||||
UPDATE public.ref_table_1500777 ref_table SET a = (a OPERATOR(pg_catalog.+) 1)|idle in transaction|Client |ClientRead|postgres|regression
|
||||
(2 rows)
|
||||
|
||||
step s2-end:
|
||||
|
|
|
@ -55,6 +55,8 @@ BEGIN
|
|||
RETURN localGroupId;
|
||||
END; $$ language plpgsql 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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
|
|
@ -60,6 +60,8 @@ BEGIN
|
|||
RETURN localGroupId;
|
||||
END; $$ language plpgsql 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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -2284,6 +2286,8 @@ BEGIN
|
|||
END;
|
||||
$fn$;
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
|
|
@ -110,8 +110,8 @@ SELECT count(*) FROM history;
|
|||
-- test we can replicate MX tables
|
||||
SET citus.shard_replication_factor TO 1;
|
||||
-- metadata sync will succeed even if we have rep > 1 tables
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles_single_shard'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles_single_shard'::regclass::oid, 0);
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
start_metadata_sync_to_node
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -242,8 +242,8 @@ SELECT master_remove_node('localhost', :worker_2_port);
|
|||
(1 row)
|
||||
|
||||
-- Removing public schema from pg_dist_object because it breaks the next tests
|
||||
DELETE FROM citus.pg_dist_object WHERE objid = 'public'::regnamespace::oid;
|
||||
DELETE FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'plpgsql');
|
||||
DELETE FROM pg_catalog.pg_dist_object WHERE objid = 'public'::regnamespace::oid;
|
||||
DELETE FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'plpgsql');
|
||||
-- try to manipulate node metadata via non-super user
|
||||
SET ROLE non_super_user;
|
||||
SELECT 1 FROM master_add_inactive_node('localhost', :worker_2_port + 1);
|
||||
|
|
|
@ -61,16 +61,6 @@ CREATE FUNCTION find_shard_interval_index(bigint)
|
|||
RETURNS int
|
||||
AS 'citus'
|
||||
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
|
||||
-- ===================================================================
|
||||
|
@ -1048,11 +1038,11 @@ SELECT update_distributed_table_colocation('table1_group_none', colocate_with =>
|
|||
(1 row)
|
||||
|
||||
-- activate nodes to get rid of inconsistencies in pg_dist tables
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table1_group1'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table2_group1'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table3_group2'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table4_group2'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table5_groupX'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table1_group1'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table2_group1'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table3_group2'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table4_group2'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table5_groupX'::regclass::oid, 0);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
?column?
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -683,6 +683,8 @@ CREATE FUNCTION func_custom_param(IN param intpair, OUT total INT)
|
|||
LANGUAGE SQL;
|
||||
SET citus.enable_metadata_sync TO OFF;
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -697,6 +699,8 @@ CREATE FUNCTION func_returns_table(IN count INT)
|
|||
LANGUAGE SQL;
|
||||
SET citus.enable_metadata_sync TO OFF;
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
|
|
@ -492,30 +492,17 @@ SELECT * FROM multi_extension.print_extension_changes();
|
|||
|
||||
-- Test downgrade to 9.4-1 from 9.5-1
|
||||
ALTER EXTENSION citus UPDATE TO '9.5-1';
|
||||
BEGIN;
|
||||
SET citus.enable_metadata_sync TO on;
|
||||
SELECT master_add_node('localhost', :master_port, groupId=>0);
|
||||
NOTICE: localhost:xxxxx is the coordinator and already contains metadata, skipping syncing the metadata
|
||||
master_add_node
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
CREATE TABLE citus_local_table (a int);
|
||||
SELECT create_citus_local_table('citus_local_table');
|
||||
NOTICE: create_citus_local_table is deprecated in favour of citus_add_local_table_to_metadata
|
||||
create_citus_local_table
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
RESET citus.enable_metadata_sync;
|
||||
-- downgrade from 9.5-1 to 9.4-1 should fail as we have a citus local table
|
||||
ALTER EXTENSION citus UPDATE TO '9.4-1';
|
||||
ERROR: citus local tables are introduced in Citus 9.5
|
||||
HINT: To downgrade Citus to an older version, you should first convert each citus local table to a postgres table by executing SELECT undistribute_table("%s")
|
||||
CONTEXT: PL/pgSQL function inline_code_block line XX at RAISE
|
||||
ROLLBACK;
|
||||
-- TODO: This test should be moved to a valid downgrade testing suite where the downgrade is done, both on the schema and the binaries. Later changes in Citus made a C vs Schema discrepancy error here
|
||||
-- BEGIN;
|
||||
-- SET citus.enable_metadata_sync TO on;
|
||||
-- SELECT master_add_node('localhost', :master_port, groupId=>0);
|
||||
-- CREATE TABLE citus_local_table (a int);
|
||||
-- SELECT create_citus_local_table('citus_local_table');
|
||||
-- RESET citus.enable_metadata_sync;
|
||||
--
|
||||
-- -- downgrade from 9.5-1 to 9.4-1 should fail as we have a citus local table
|
||||
-- ALTER EXTENSION citus UPDATE TO '9.4-1';
|
||||
-- ROLLBACK;
|
||||
-- now we can downgrade as there is no citus local table
|
||||
ALTER EXTENSION citus UPDATE TO '9.4-1';
|
||||
-- Should be empty result since upgrade+downgrade should be a no-op
|
||||
|
@ -1002,12 +989,16 @@ SELECT * FROM multi_extension.print_extension_changes();
|
|||
previous_object | current_object
|
||||
---------------------------------------------------------------------
|
||||
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 master_append_table_to_shard(bigint,text,text,integer) real |
|
||||
function master_apply_delete_command(text) integer |
|
||||
function master_get_table_metadata(text) record |
|
||||
function worker_partition_query_result(text,text,integer,citus.distribution_type,text[],text[],boolean) SETOF record |
|
||||
| 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_check_cluster_node_health() SETOF record
|
||||
| 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_stat_activity() SETOF record
|
||||
| function create_distributed_function(regprocedure,text,text,boolean) void
|
||||
| function get_nodeid_for_groupid(integer) integer
|
||||
| function pg_cancel_backend(bigint) boolean
|
||||
| function pg_terminate_backend(bigint,bigint) boolean
|
||||
| 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_shell_table(text) void
|
||||
| function worker_partition_query_result(text,text,integer,citus.distribution_type,text[],text[],boolean,boolean,boolean) SETOF record
|
||||
| table pg_dist_object
|
||||
| view citus_stat_activity
|
||||
(30 rows)
|
||||
(36 rows)
|
||||
|
||||
DROP TABLE multi_extension.prev_objects, multi_extension.extension_diff;
|
||||
-- show running version
|
||||
|
|
|
@ -169,6 +169,8 @@ BEGIN
|
|||
END;
|
||||
$function$;
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
|
|
@ -84,6 +84,8 @@ BEGIN
|
|||
END;
|
||||
$$ LANGUAGE plpgsql;
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -96,6 +98,8 @@ BEGIN
|
|||
END;
|
||||
$$ LANGUAGE plpgsql;
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
|
|
@ -9,7 +9,7 @@ FROM pg_attribute
|
|||
WHERE atthasmissing AND attrelid NOT IN ('pg_dist_node'::regclass,
|
||||
'pg_dist_rebalance_strategy'::regclass,
|
||||
'pg_dist_partition'::regclass,
|
||||
'citus.pg_dist_object'::regclass)
|
||||
'pg_dist_object'::regclass)
|
||||
ORDER BY attrelid, attname;
|
||||
attrelid | attname | atthasmissing | attmissingval
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -62,8 +62,8 @@ SELECT unnest(activate_node_snapshot()) order by 1;
|
|||
unnest
|
||||
---------------------------------------------------------------------
|
||||
CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION postgres
|
||||
DELETE FROM citus.pg_dist_object
|
||||
DELETE FROM pg_catalog.pg_dist_colocation
|
||||
DELETE FROM pg_catalog.pg_dist_object
|
||||
DELETE FROM pg_dist_node
|
||||
DELETE FROM pg_dist_partition
|
||||
DELETE FROM pg_dist_placement
|
||||
|
@ -131,8 +131,8 @@ SELECT unnest(activate_node_snapshot()) order by 1;
|
|||
ALTER TABLE public.mx_test_table OWNER TO postgres
|
||||
CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION postgres
|
||||
CREATE TABLE public.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('public.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass))
|
||||
DELETE FROM citus.pg_dist_object
|
||||
DELETE FROM pg_catalog.pg_dist_colocation
|
||||
DELETE FROM pg_catalog.pg_dist_object
|
||||
DELETE FROM pg_dist_node
|
||||
DELETE FROM pg_dist_partition
|
||||
DELETE FROM pg_dist_placement
|
||||
|
@ -180,8 +180,8 @@ SELECT unnest(activate_node_snapshot()) order by 1;
|
|||
CREATE INDEX mx_index ON public.mx_test_table USING btree (col_2)
|
||||
CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION postgres
|
||||
CREATE TABLE public.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('public.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass))
|
||||
DELETE FROM citus.pg_dist_object
|
||||
DELETE FROM pg_catalog.pg_dist_colocation
|
||||
DELETE FROM pg_catalog.pg_dist_object
|
||||
DELETE FROM pg_dist_node
|
||||
DELETE FROM pg_dist_partition
|
||||
DELETE FROM pg_dist_placement
|
||||
|
@ -231,8 +231,8 @@ SELECT unnest(activate_node_snapshot()) order by 1;
|
|||
CREATE SCHEMA IF NOT EXISTS mx_testing_schema AUTHORIZATION postgres
|
||||
CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION postgres
|
||||
CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass))
|
||||
DELETE FROM citus.pg_dist_object
|
||||
DELETE FROM pg_catalog.pg_dist_colocation
|
||||
DELETE FROM pg_catalog.pg_dist_object
|
||||
DELETE FROM pg_dist_node
|
||||
DELETE FROM pg_dist_partition
|
||||
DELETE FROM pg_dist_placement
|
||||
|
@ -288,8 +288,8 @@ SELECT unnest(activate_node_snapshot()) order by 1;
|
|||
CREATE SCHEMA IF NOT EXISTS mx_testing_schema AUTHORIZATION postgres
|
||||
CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION postgres
|
||||
CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass))
|
||||
DELETE FROM citus.pg_dist_object
|
||||
DELETE FROM pg_catalog.pg_dist_colocation
|
||||
DELETE FROM pg_catalog.pg_dist_object
|
||||
DELETE FROM pg_dist_node
|
||||
DELETE FROM pg_dist_partition
|
||||
DELETE FROM pg_dist_placement
|
||||
|
@ -338,8 +338,8 @@ SELECT unnest(activate_node_snapshot()) order by 1;
|
|||
CREATE SCHEMA IF NOT EXISTS mx_testing_schema AUTHORIZATION postgres
|
||||
CREATE SCHEMA IF NOT EXISTS public AUTHORIZATION postgres
|
||||
CREATE TABLE mx_testing_schema.mx_test_table (col_1 integer, col_2 text NOT NULL, col_3 bigint DEFAULT nextval('mx_testing_schema.mx_test_table_col_3_seq'::regclass) NOT NULL, col_4 bigint DEFAULT nextval('public.user_defined_seq'::regclass))
|
||||
DELETE FROM citus.pg_dist_object
|
||||
DELETE FROM pg_catalog.pg_dist_colocation
|
||||
DELETE FROM pg_catalog.pg_dist_object
|
||||
DELETE FROM pg_dist_node
|
||||
DELETE FROM pg_dist_partition
|
||||
DELETE FROM pg_dist_placement
|
||||
|
@ -1379,10 +1379,10 @@ DROP TABLE mx_table_with_small_sequence, mx_table_with_sequence;
|
|||
-- owner
|
||||
CREATE TABLE pg_dist_placement_temp AS SELECT * FROM pg_dist_placement;
|
||||
CREATE TABLE pg_dist_partition_temp AS SELECT * FROM pg_dist_partition;
|
||||
CREATE TABLE pg_dist_object_temp AS SELECT * FROM citus.pg_dist_object;
|
||||
CREATE TABLE pg_dist_object_temp AS SELECT * FROM pg_catalog.pg_dist_object;
|
||||
DELETE FROM pg_dist_placement;
|
||||
DELETE FROM pg_dist_partition;
|
||||
DELETE FROM citus.pg_dist_object;
|
||||
DELETE FROM pg_catalog.pg_dist_object;
|
||||
SELECT groupid AS old_worker_2_group FROM pg_dist_node WHERE nodeport = :worker_2_port \gset
|
||||
SELECT master_remove_node('localhost', :worker_2_port);
|
||||
master_remove_node
|
||||
|
@ -1459,7 +1459,7 @@ DROP TABLE mx_table;
|
|||
\c - postgres - :master_port
|
||||
INSERT INTO pg_dist_placement SELECT * FROM pg_dist_placement_temp;
|
||||
INSERT INTO pg_dist_partition SELECT * FROM pg_dist_partition_temp;
|
||||
INSERT INTO citus.pg_dist_object SELECT * FROM pg_dist_object_temp ON CONFLICT ON CONSTRAINT pg_dist_object_pkey DO NOTHING;
|
||||
INSERT INTO pg_catalog.pg_dist_object SELECT * FROM pg_dist_object_temp ON CONFLICT ON CONSTRAINT pg_dist_object_pkey DO NOTHING;
|
||||
DROP TABLE pg_dist_placement_temp;
|
||||
DROP TABLE pg_dist_partition_temp;
|
||||
DROP TABLE pg_dist_object_temp;
|
||||
|
@ -1875,8 +1875,8 @@ SELECT unnest(activate_node_snapshot()) order by 1;
|
|||
CREATE TABLE public.dist_table_1 (a integer)
|
||||
CREATE TABLE public.mx_ref (col_1 integer, col_2 text)
|
||||
CREATE TABLE public.test_table (id integer DEFAULT worker_nextval('public.mx_test_sequence_0'::regclass), id2 integer DEFAULT worker_nextval('public.mx_test_sequence_1'::regclass))
|
||||
DELETE FROM citus.pg_dist_object
|
||||
DELETE FROM pg_catalog.pg_dist_colocation
|
||||
DELETE FROM pg_catalog.pg_dist_object
|
||||
DELETE FROM pg_dist_node
|
||||
DELETE FROM pg_dist_partition
|
||||
DELETE FROM pg_dist_placement
|
||||
|
|
|
@ -471,6 +471,8 @@ SELECT create_distributed_function('usage_access_func(usage_access_type,int[])')
|
|||
ERROR: must be owner of function usage_access_func
|
||||
SET ROLE usage_access;
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
|
|
@ -191,7 +191,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg
|
|||
test_proc_colocation_0 | 1410002
|
||||
(1 row)
|
||||
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
proname | colocationid
|
||||
---------------------------------------------------------------------
|
||||
proc_0 | 1410002
|
||||
|
@ -226,7 +226,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg
|
|||
test_proc_colocation_0 | 1410003
|
||||
(1 row)
|
||||
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
proname | colocationid
|
||||
---------------------------------------------------------------------
|
||||
proc_0 | 1410003
|
||||
|
@ -272,7 +272,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg
|
|||
test_proc_colocation_0 | 1410004
|
||||
(1 row)
|
||||
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
proname | colocationid
|
||||
---------------------------------------------------------------------
|
||||
proc_0 | 1410004
|
||||
|
@ -305,7 +305,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg
|
|||
test_proc_colocation_0 | 1410003
|
||||
(1 row)
|
||||
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
proname | colocationid
|
||||
---------------------------------------------------------------------
|
||||
proc_0 | 1410003
|
||||
|
@ -359,7 +359,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg
|
|||
test_proc_colocation_0 | 1410005
|
||||
(1 row)
|
||||
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
proname | colocationid
|
||||
---------------------------------------------------------------------
|
||||
proc_0 | 1410005
|
||||
|
@ -389,7 +389,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg
|
|||
test_proc_colocation_0 | 1410005
|
||||
(1 row)
|
||||
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname;
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname;
|
||||
proname | colocationid
|
||||
---------------------------------------------------------------------
|
||||
proc_0 | 1410005
|
||||
|
@ -440,7 +440,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg
|
|||
test_proc_colocation_0 | 1410003
|
||||
(1 row)
|
||||
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname;
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname;
|
||||
proname | colocationid
|
||||
---------------------------------------------------------------------
|
||||
proc_0 | 1410003
|
||||
|
@ -465,7 +465,7 @@ SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::reg
|
|||
test_proc_colocation_0 | 1410006
|
||||
(1 row)
|
||||
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname;
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname;
|
||||
proname | colocationid
|
||||
---------------------------------------------------------------------
|
||||
proc_0 | 1410003
|
||||
|
|
|
@ -166,24 +166,32 @@ call mx_call_proc_custom_types('S', 'A');
|
|||
|
||||
-- Mark both procedures as distributed ...
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -559,8 +567,8 @@ CREATE FUNCTION mx_call_add(int, int) RETURNS int
|
|||
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
|
||||
SELECT create_distributed_function('mx_call_add(int,int)');
|
||||
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
|
||||
NOTICE: procedure multi_mx_call.mx_call_add is already distributed
|
||||
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
|
||||
create_distributed_function
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
|
|
@ -166,24 +166,32 @@ call mx_call_proc_custom_types('S', 'A');
|
|||
|
||||
-- Mark both procedures as distributed ...
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -559,8 +567,8 @@ CREATE FUNCTION mx_call_add(int, int) RETURNS int
|
|||
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
|
||||
SELECT create_distributed_function('mx_call_add(int,int)');
|
||||
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
|
||||
NOTICE: procedure multi_mx_call.mx_call_add is already distributed
|
||||
DETAIL: Citus distributes procedures with CREATE [PROCEDURE|FUNCTION|AGGREGATE] commands
|
||||
create_distributed_function
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
|
|
@ -132,30 +132,40 @@ select mx_call_func(2, 0);
|
|||
|
||||
-- Mark both functions as distributed ...
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
|
|
@ -132,30 +132,40 @@ select mx_call_func(2, 0);
|
|||
|
||||
-- Mark both functions as distributed ...
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
(1 row)
|
||||
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
|
|
@ -24,6 +24,8 @@ CREATE FUNCTION square(int) RETURNS INT
|
|||
AS $$ SELECT $1 * $1 $$
|
||||
LANGUAGE SQL;
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
|
|
@ -459,7 +459,7 @@ SELECT create_distributed_table('mx_old_schema.table_set_schema', 'id');
|
|||
|
||||
CREATE SCHEMA mx_new_schema;
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid::oid::regnamespace IN ('mx_old_schema', 'mx_new_schema')
|
||||
ORDER BY "Distributed Schemas";
|
||||
Distributed Schemas
|
||||
|
@ -491,7 +491,7 @@ HINT: Connect to the coordinator and run it again.
|
|||
\c - - - :master_port
|
||||
ALTER TABLE mx_old_schema.table_set_schema SET SCHEMA mx_new_schema;
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid::oid::regnamespace IN ('mx_old_schema', 'mx_new_schema');
|
||||
Distributed Schemas
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -71,6 +71,7 @@ BEGIN
|
|||
RETURN a*a;
|
||||
END; $$ LANGUAGE PLPGSQL STABLE;
|
||||
SELECT create_distributed_function('square(int)');
|
||||
NOTICE: procedure multi_row_router_insert.square is already distributed
|
||||
create_distributed_function
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
|
|
@ -884,7 +884,7 @@ SELECT create_distributed_table('old_schema.table_set_schema', 'id');
|
|||
|
||||
CREATE SCHEMA new_schema;
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema');
|
||||
Distributed Schemas
|
||||
---------------------------------------------------------------------
|
||||
|
@ -906,7 +906,7 @@ SELECT table_schema AS "Shards' Schema"
|
|||
\c - - - :master_port
|
||||
ALTER TABLE old_schema.table_set_schema SET SCHEMA new_schema;
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema');
|
||||
Distributed Schemas
|
||||
---------------------------------------------------------------------
|
||||
|
@ -944,7 +944,7 @@ SELECT create_distributed_table('table_set_schema', 'id');
|
|||
|
||||
CREATE SCHEMA new_schema;
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid='new_schema'::regnamespace::oid;
|
||||
Distributed Schemas
|
||||
---------------------------------------------------------------------
|
||||
|
@ -965,7 +965,7 @@ SELECT table_schema AS "Shards' Schema"
|
|||
\c - - - :master_port
|
||||
ALTER TABLE table_set_schema SET SCHEMA new_schema;
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid='new_schema'::regnamespace::oid;
|
||||
Distributed Schemas
|
||||
---------------------------------------------------------------------
|
||||
|
@ -1010,7 +1010,7 @@ SELECT create_distributed_table('table_set_schema', 'id');
|
|||
|
||||
CREATE SCHEMA new_schema;
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema');
|
||||
Distributed Schemas
|
||||
---------------------------------------------------------------------
|
||||
|
@ -1034,7 +1034,7 @@ SELECT table_schema AS "Shards' Schema", COUNT(*) AS "Counts"
|
|||
SET search_path TO old_schema;
|
||||
ALTER TABLE table_set_schema SET SCHEMA new_schema;
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema');
|
||||
Distributed Schemas
|
||||
---------------------------------------------------------------------
|
||||
|
@ -1392,7 +1392,7 @@ BEGIN;
|
|||
ROLLBACK;
|
||||
-- Clean up the created schema
|
||||
SET client_min_messages TO WARNING;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object
|
||||
WHERE classid=2615 and objid IN (select oid from pg_namespace where nspname='run_test_schema');
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
|
@ -1401,7 +1401,7 @@ SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dis
|
|||
|
||||
DROP SCHEMA run_test_schema, test_schema_support_join_1, test_schema_support_join_2, "Citus'Teen123", "CiTUS.TEEN2", bar, test_schema_support CASCADE;
|
||||
-- verify that the dropped schema is removed from worker's pg_dist_object
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object
|
||||
WHERE classid=2615 and objid IN (select oid from pg_namespace where nspname='run_test_schema');
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -852,13 +852,13 @@ SELECT create_distributed_table('test_seq_dist', 'a');
|
|||
(1 row)
|
||||
|
||||
-- Both sequence and dependency schema should be distributed
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation.seq_10'::regclass);
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation.seq_10'::regclass);
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(sequence,"{test_schema_for_sequence_default_propagation,seq_10}",{})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation'::regnamespace);
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object WHERE objid IN ('test_schema_for_sequence_default_propagation'::regnamespace);
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(schema,{test_schema_for_sequence_default_propagation},{})
|
||||
|
|
|
@ -83,7 +83,7 @@ END;$$;
|
|||
CREATE OR REPLACE FUNCTION colocate_proc_with_table(procname text, tablerelid regclass, argument_index int)
|
||||
RETURNS void LANGUAGE plpgsql AS $$
|
||||
BEGIN
|
||||
update citus.pg_dist_object
|
||||
update pg_catalog.pg_dist_object
|
||||
set distribution_argument_index = argument_index, colocationid = pg_dist_partition.colocationid
|
||||
from pg_proc, pg_dist_partition
|
||||
where proname = procname and oid = objid and pg_dist_partition.logicalrelid = tablerelid;
|
||||
|
|
|
@ -12,27 +12,27 @@ CREATE TYPE mood AS ENUM ('sad', 'ok', 'happy');
|
|||
CREATE SCHEMA test_sequence_schema;
|
||||
CREATE SEQUENCE test_sequence_schema.test_sequence;
|
||||
-- show that none of the objects above are marked as distributed
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema'::regnamespace::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema'::regnamespace::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.mood'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.mood'::regtype::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
@ -97,105 +97,105 @@ SELECT create_distributed_function('plpgsql_dist_function(text)');
|
|||
-- show that schema, types, function and sequence has marked as distributed
|
||||
-- on the coordinator node
|
||||
RESET ROLE;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema'::regnamespace::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema'::regnamespace::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(schema,{local_schema},{})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'test_sequence_schema'::regnamespace::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'test_sequence_schema'::regnamespace::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(schema,{test_sequence_schema},{})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.mood'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.mood'::regtype::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(type,{local_schema.mood},{})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(type,{local_schema.test_type},{})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(sequence,"{test_sequence_schema,test_sequence}",{})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.dist_table_e_seq'::regclass::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.dist_table_e_seq'::regclass::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(sequence,"{local_schema,dist_table_e_seq}",{})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{local_schema,test_function}",{integer})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.plpgsql_dist_function'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.plpgsql_dist_function'::regproc::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{local_schema,plpgsql_dist_function}",{pg_catalog.text})
|
||||
(1 row)
|
||||
|
||||
-- show those objects marked as distributed on metadata worker node as well
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema'::regnamespace::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema'::regnamespace::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (schema,{local_schema},{})
|
||||
localhost | 57638 | t | (schema,{local_schema},{})
|
||||
(2 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'test_sequence_schema'::regnamespace::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'test_sequence_schema'::regnamespace::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (schema,{test_sequence_schema},{})
|
||||
localhost | 57638 | t | (schema,{test_sequence_schema},{})
|
||||
(2 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.mood'::regtype::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.mood'::regtype::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (type,{local_schema.mood},{})
|
||||
localhost | 57638 | t | (type,{local_schema.mood},{})
|
||||
(2 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_type'::regtype::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (type,{local_schema.test_type},{})
|
||||
localhost | 57638 | t | (type,{local_schema.test_type},{})
|
||||
(2 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'test_sequence_schema.test_sequence'::regclass::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (sequence,"{test_sequence_schema,test_sequence}",{})
|
||||
localhost | 57638 | t | (sequence,"{test_sequence_schema,test_sequence}",{})
|
||||
(2 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.dist_table_e_seq'::regclass::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.dist_table_e_seq'::regclass::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (sequence,"{local_schema,dist_table_e_seq}",{})
|
||||
localhost | 57638 | t | (sequence,"{local_schema,dist_table_e_seq}",{})
|
||||
(2 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.test_function'::regproc::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (function,"{local_schema,test_function}",{integer})
|
||||
localhost | 57638 | t | (function,"{local_schema,test_function}",{integer})
|
||||
(2 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'local_schema.plpgsql_dist_function'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'local_schema.plpgsql_dist_function'::regproc::oid;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (function,"{local_schema,plpgsql_dist_function}",{pg_catalog.text})
|
||||
|
@ -203,13 +203,13 @@ SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(clas
|
|||
(2 rows)
|
||||
|
||||
-- Show that extension plpgsql is also marked as distributed as a dependency of plpgsl_dist_function
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{plpgsql}%';
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{plpgsql}%';
|
||||
obj_identifier
|
||||
---------------------------------------------------------------------
|
||||
(extension,{plpgsql},{})
|
||||
(1 row)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{plpgsql}%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{plpgsql}%';$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (extension,{plpgsql},{})
|
||||
|
@ -278,12 +278,12 @@ SET citus.enable_ddl_propagation TO ON;
|
|||
DROP TABLE dist_table CASCADE;
|
||||
RESET ROLE;
|
||||
SET search_path TO local_schema;
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%dist_table_e_seq%';
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%dist_table_e_seq%';
|
||||
obj_identifier
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%dist_table_e_seq%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%dist_table_e_seq%';$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t |
|
||||
|
@ -293,13 +293,13 @@ SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_
|
|||
-- Show that altering the function's schema marks the schema distributed
|
||||
CREATE SCHEMA schema_to_prop_with_function;
|
||||
ALTER FUNCTION test_function SET SCHEMA schema_to_prop_with_function;
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_function}%';
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_function}%';
|
||||
obj_identifier
|
||||
---------------------------------------------------------------------
|
||||
(schema,{schema_to_prop_with_function},{})
|
||||
(1 row)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_function}%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_function}%';$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (schema,{schema_to_prop_with_function},{})
|
||||
|
@ -309,12 +309,12 @@ SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_
|
|||
-- Show that dropping the function removes the metadata from pg_dist_object
|
||||
-- on both coordinator and metadata worker node
|
||||
DROP FUNCTION schema_to_prop_with_function.test_function;
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_function%';
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_function%';
|
||||
obj_identifier
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_function%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_function%';$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t |
|
||||
|
@ -324,13 +324,13 @@ SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_
|
|||
-- Show that altering the type's schema marks the schema distributed
|
||||
CREATE SCHEMA schema_to_prop_with_type;
|
||||
ALTER TYPE test_type SET SCHEMA schema_to_prop_with_type;
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_type}%';
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_type}%';
|
||||
obj_identifier
|
||||
---------------------------------------------------------------------
|
||||
(schema,{schema_to_prop_with_type},{})
|
||||
(1 row)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_type}%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{schema_to_prop_with_type}%';$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (schema,{schema_to_prop_with_type},{})
|
||||
|
@ -341,12 +341,12 @@ SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_
|
|||
-- on both coordinator and metadata worker node
|
||||
DROP TYPE mood CASCADE;
|
||||
DROP TYPE schema_to_prop_with_type.test_type CASCADE;
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_type%' or obj_identifier::text like '%mood%';
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_type%' or obj_identifier::text like '%mood%';
|
||||
obj_identifier
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_type%' or obj_identifier::text like '%mood%'$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%test_type%' or obj_identifier::text like '%mood%'$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t |
|
||||
|
@ -379,13 +379,13 @@ SELECT create_distributed_function('metadata_dist_test_proc(integer, integer)',
|
|||
|
||||
(1 row)
|
||||
|
||||
SELECT distribution_argument_index FROM citus.pg_dist_object WHERE objid = 'metadata_dist_test_proc'::regproc;
|
||||
SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE objid = 'metadata_dist_test_proc'::regproc;
|
||||
distribution_argument_index
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
(1 row)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT distribution_argument_index FROM citus.pg_dist_object WHERE objid = 'local_schema.metadata_dist_test_proc'::regproc;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE objid = 'local_schema.metadata_dist_test_proc'::regproc;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | 0
|
||||
|
@ -399,13 +399,13 @@ SELECT create_distributed_function('metadata_dist_test_proc(integer, integer)',
|
|||
|
||||
(1 row)
|
||||
|
||||
SELECT distribution_argument_index FROM citus.pg_dist_object WHERE objid = 'metadata_dist_test_proc'::regproc;
|
||||
SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE objid = 'metadata_dist_test_proc'::regproc;
|
||||
distribution_argument_index
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$ SELECT distribution_argument_index FROM citus.pg_dist_object WHERE objid = 'local_schema.metadata_dist_test_proc'::regproc;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$ SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE objid = 'local_schema.metadata_dist_test_proc'::regproc;$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | 1
|
||||
|
@ -418,12 +418,12 @@ NOTICE: drop cascades to 3 other objects
|
|||
DETAIL: drop cascades to function plpgsql_dist_function(text)
|
||||
drop cascades to table metadata_dist_test_table
|
||||
drop cascades to function metadata_dist_test_proc(integer,integer)
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{local_schema}%';
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{local_schema}%';
|
||||
obj_identifier
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{local_schema}%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{local_schema}%';$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t |
|
||||
|
@ -433,26 +433,26 @@ SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_
|
|||
-- Show that extension and dependent sequence also created and marked as distributed
|
||||
CREATE SCHEMA extension_schema;
|
||||
CREATE EXTENSION ltree WITH SCHEMA extension_schema;
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{extension_schema}%';
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{extension_schema}%';
|
||||
obj_identifier
|
||||
---------------------------------------------------------------------
|
||||
(schema,{extension_schema},{})
|
||||
(1 row)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{extension_schema}%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{extension_schema}%';$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (schema,{extension_schema},{})
|
||||
localhost | 57638 | t | (schema,{extension_schema},{})
|
||||
(2 rows)
|
||||
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{ltree}%';
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{ltree}%';
|
||||
obj_identifier
|
||||
---------------------------------------------------------------------
|
||||
(extension,{ltree},{})
|
||||
(1 row)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{ltree}%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%{ltree}%';$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (extension,{ltree},{})
|
||||
|
@ -467,13 +467,13 @@ SELECT create_distributed_table('extension_schema.table_to_check_object', 'id');
|
|||
|
||||
(1 row)
|
||||
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';
|
||||
obj_identifier
|
||||
---------------------------------------------------------------------
|
||||
(table,"{extension_schema,table_to_check_object}",{})
|
||||
(1 row)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t | (table,"{extension_schema,table_to_check_object}",{})
|
||||
|
@ -481,12 +481,12 @@ SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_
|
|||
(2 rows)
|
||||
|
||||
DROP TABLE extension_schema.table_to_check_object;
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';
|
||||
SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';
|
||||
obj_identifier
|
||||
---------------------------------------------------------------------
|
||||
(0 rows)
|
||||
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from citus.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT * FROM (SELECT pg_identify_object_as_address(classid, objid, objsubid) as obj_identifier from pg_catalog.pg_dist_object) as obj_identifiers where obj_identifier::text like '%table_to_check_object%';$$) ORDER BY 1,2;
|
||||
nodename | nodeport | success | result
|
||||
---------------------------------------------------------------------
|
||||
localhost | 57637 | t |
|
||||
|
|
|
@ -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
|
||||
-- this should work
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
|
|
@ -6,13 +6,13 @@ SET client_min_messages TO WARNING;
|
|||
-- the extension is on contrib, so should be avaliable for the regression tests
|
||||
CREATE EXTENSION seg;
|
||||
-- make sure that both the schema and the extension is distributed
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -34,7 +34,7 @@ SELECT count(*) from pg_dist_partition where logicalrelid='extension''test.test_
|
|||
|
||||
CREATE TYPE two_segs AS (seg_1 seg, seg_2 seg);
|
||||
-- verify that the type that depends on the extension is also marked as distributed
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_type WHERE typname = 'two_segs' AND typnamespace = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test'));
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_type WHERE typname = 'two_segs' AND typnamespace = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test'));
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -67,7 +67,7 @@ BEGIN;
|
|||
|
||||
COMMIT;
|
||||
-- make sure that the extension is distributed even if we run create extension in a transaction block
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -122,7 +122,7 @@ ALTER EXTENSION isn SET SCHEMA public;
|
|||
-- switch back to public schema as we set extension's schema to public
|
||||
SET search_path TO public;
|
||||
-- make sure that the extension is distributed
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -282,7 +282,7 @@ BEGIN;
|
|||
ROLLBACK;
|
||||
-- at the end of the transaction block, we did not create isn extension in coordinator or worker nodes as we rollback'ed
|
||||
-- make sure that the extension is not distributed
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -319,7 +319,7 @@ SELECT count(*) FROM pg_extension WHERE extname IN ('pg_buffercache', 'isn');
|
|||
|
||||
-- drop extension should just work
|
||||
DROP EXTENSION seg CASCADE;
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -333,7 +333,7 @@ SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname
|
|||
(2 rows)
|
||||
|
||||
-- make sure that the extension is not avaliable anymore as a distributed object
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn'));
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn'));
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -439,9 +439,9 @@ BEGIN;
|
|||
COMMIT;
|
||||
-- Check the pg_dist_object
|
||||
SELECT pg_proc.proname as DistributedFunction
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
distributedfunction
|
||||
---------------------------------------------------------------------
|
||||
|
@ -450,9 +450,9 @@ classid = 'pg_proc'::regclass;
|
|||
|
||||
SELECT run_command_on_workers($$
|
||||
SELECT count(*)
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
$$);
|
||||
run_command_on_workers
|
||||
|
@ -468,7 +468,7 @@ SELECT 1 from master_add_node('localhost', :worker_2_port);
|
|||
(1 row)
|
||||
|
||||
-- make sure that both extensions are created on both nodes
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid IN (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn'));
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid IN (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn'));
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
2
|
||||
|
@ -484,9 +484,9 @@ SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname
|
|||
-- Check the pg_dist_object on the both nodes
|
||||
SELECT run_command_on_workers($$
|
||||
SELECT count(*)
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
$$);
|
||||
run_command_on_workers
|
||||
|
@ -498,9 +498,9 @@ $$);
|
|||
DROP EXTENSION seg CASCADE;
|
||||
-- Recheck the pg_dist_object
|
||||
SELECT pg_proc.proname as DistributedFunction
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
distributedfunction
|
||||
---------------------------------------------------------------------
|
||||
|
@ -508,9 +508,9 @@ classid = 'pg_proc'::regclass;
|
|||
|
||||
SELECT run_command_on_workers($$
|
||||
SELECT count(*)
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
$$);
|
||||
run_command_on_workers
|
||||
|
@ -524,7 +524,7 @@ SET citus.enable_ddl_propagation TO false;
|
|||
CREATE EXTENSION seg;
|
||||
SET citus.enable_ddl_propagation TO true;
|
||||
-- Check the extension in pg_dist_object
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND
|
||||
objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -533,9 +533,9 @@ objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
|||
|
||||
SELECT run_command_on_workers($$
|
||||
SELECT count(*)
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
$$);
|
||||
run_command_on_workers
|
||||
|
@ -551,7 +551,7 @@ SELECT create_distributed_function('seg_in(cstring)', '$1', 'test_extension_func
|
|||
(1 row)
|
||||
|
||||
-- Recheck the extension in pg_dist_object
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND
|
||||
objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -559,9 +559,9 @@ objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
|||
(1 row)
|
||||
|
||||
SELECT pg_proc.proname as DistributedFunction
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
distributedfunction
|
||||
---------------------------------------------------------------------
|
||||
|
@ -570,9 +570,9 @@ classid = 'pg_proc'::regclass;
|
|||
|
||||
SELECT run_command_on_workers($$
|
||||
SELECT count(*)
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
$$);
|
||||
run_command_on_workers
|
||||
|
@ -600,7 +600,7 @@ SELECT create_distributed_function('cube(float8[], float8[])', '$1', 'test_exten
|
|||
|
||||
(1 row)
|
||||
|
||||
SELECT distribution_argument_index FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND
|
||||
SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND
|
||||
objid = (SELECT oid FROM pg_proc WHERE prosrc = 'cube_a_f8_f8');
|
||||
distribution_argument_index
|
||||
---------------------------------------------------------------------
|
||||
|
@ -613,7 +613,7 @@ SELECT create_distributed_function('cube(float8[], float8[])', '$2', 'test_exten
|
|||
|
||||
(1 row)
|
||||
|
||||
SELECT distribution_argument_index FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND
|
||||
SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND
|
||||
objid = (SELECT oid FROM pg_proc WHERE prosrc = 'cube_a_f8_f8');
|
||||
distribution_argument_index
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -6,13 +6,13 @@ SET client_min_messages TO WARNING;
|
|||
-- the extension is on contrib, so should be avaliable for the regression tests
|
||||
CREATE EXTENSION seg;
|
||||
-- make sure that both the schema and the extension is distributed
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
(1 row)
|
||||
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -34,7 +34,7 @@ SELECT count(*) from pg_dist_partition where logicalrelid='extension''test.test_
|
|||
|
||||
CREATE TYPE two_segs AS (seg_1 seg, seg_2 seg);
|
||||
-- verify that the type that depends on the extension is also marked as distributed
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_type WHERE typname = 'two_segs' AND typnamespace = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test'));
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_type WHERE typname = 'two_segs' AND typnamespace = (SELECT oid FROM pg_namespace WHERE nspname = 'extension''test'));
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -67,7 +67,7 @@ BEGIN;
|
|||
|
||||
COMMIT;
|
||||
-- make sure that the extension is distributed even if we run create extension in a transaction block
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -122,7 +122,7 @@ ALTER EXTENSION isn SET SCHEMA public;
|
|||
-- switch back to public schema as we set extension's schema to public
|
||||
SET search_path TO public;
|
||||
-- make sure that the extension is distributed
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
1
|
||||
|
@ -281,7 +281,7 @@ BEGIN;
|
|||
ROLLBACK;
|
||||
-- at the end of the transaction block, we did not create isn extension in coordinator or worker nodes as we rollback'ed
|
||||
-- make sure that the extension is not distributed
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'isn');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -318,7 +318,7 @@ SELECT count(*) FROM pg_extension WHERE extname IN ('pg_buffercache', 'isn');
|
|||
|
||||
-- drop extension should just work
|
||||
DROP EXTENSION seg CASCADE;
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -332,7 +332,7 @@ SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname
|
|||
(2 rows)
|
||||
|
||||
-- make sure that the extension is not avaliable anymore as a distributed object
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn'));
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn'));
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
0
|
||||
|
@ -438,9 +438,9 @@ BEGIN;
|
|||
COMMIT;
|
||||
-- Check the pg_dist_object
|
||||
SELECT pg_proc.proname as DistributedFunction
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
distributedfunction
|
||||
---------------------------------------------------------------------
|
||||
|
@ -449,9 +449,9 @@ classid = 'pg_proc'::regclass;
|
|||
|
||||
SELECT run_command_on_workers($$
|
||||
SELECT count(*)
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
$$);
|
||||
run_command_on_workers
|
||||
|
@ -467,7 +467,7 @@ SELECT 1 from master_add_node('localhost', :worker_2_port);
|
|||
(1 row)
|
||||
|
||||
-- make sure that both extensions are created on both nodes
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid IN (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn'));
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid IN (SELECT oid FROM pg_extension WHERE extname IN ('seg', 'isn'));
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
2
|
||||
|
@ -483,9 +483,9 @@ SELECT run_command_on_workers($$SELECT count(*) FROM pg_extension WHERE extname
|
|||
-- Check the pg_dist_object on the both nodes
|
||||
SELECT run_command_on_workers($$
|
||||
SELECT count(*)
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
$$);
|
||||
run_command_on_workers
|
||||
|
@ -497,9 +497,9 @@ $$);
|
|||
DROP EXTENSION seg CASCADE;
|
||||
-- Recheck the pg_dist_object
|
||||
SELECT pg_proc.proname as DistributedFunction
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
distributedfunction
|
||||
---------------------------------------------------------------------
|
||||
|
@ -507,9 +507,9 @@ classid = 'pg_proc'::regclass;
|
|||
|
||||
SELECT run_command_on_workers($$
|
||||
SELECT count(*)
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
$$);
|
||||
run_command_on_workers
|
||||
|
@ -523,7 +523,7 @@ SET citus.enable_ddl_propagation TO false;
|
|||
CREATE EXTENSION seg;
|
||||
SET citus.enable_ddl_propagation TO true;
|
||||
-- Check the extension in pg_dist_object
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND
|
||||
objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -532,9 +532,9 @@ objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
|||
|
||||
SELECT run_command_on_workers($$
|
||||
SELECT count(*)
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
$$);
|
||||
run_command_on_workers
|
||||
|
@ -550,7 +550,7 @@ SELECT create_distributed_function('seg_in(cstring)', '$1', 'test_extension_func
|
|||
(1 row)
|
||||
|
||||
-- Recheck the extension in pg_dist_object
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_extension'::pg_catalog.regclass AND
|
||||
objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
@ -558,9 +558,9 @@ objid = (SELECT oid FROM pg_extension WHERE extname = 'seg');
|
|||
(1 row)
|
||||
|
||||
SELECT pg_proc.proname as DistributedFunction
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
distributedfunction
|
||||
---------------------------------------------------------------------
|
||||
|
@ -569,9 +569,9 @@ classid = 'pg_proc'::regclass;
|
|||
|
||||
SELECT run_command_on_workers($$
|
||||
SELECT count(*)
|
||||
FROM citus.pg_dist_object, pg_proc
|
||||
FROM pg_catalog.pg_dist_object, pg_proc
|
||||
WHERE pg_proc.proname = 'seg_in' and
|
||||
pg_proc.oid = citus.pg_dist_object.objid and
|
||||
pg_proc.oid = pg_catalog.pg_dist_object.objid and
|
||||
classid = 'pg_proc'::regclass;
|
||||
$$);
|
||||
run_command_on_workers
|
||||
|
@ -599,7 +599,7 @@ SELECT create_distributed_function('cube(float8[], float8[])', '$1', 'test_exten
|
|||
|
||||
(1 row)
|
||||
|
||||
SELECT distribution_argument_index FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND
|
||||
SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND
|
||||
objid = (SELECT oid FROM pg_proc WHERE prosrc = 'cube_a_f8_f8');
|
||||
distribution_argument_index
|
||||
---------------------------------------------------------------------
|
||||
|
@ -612,7 +612,7 @@ SELECT create_distributed_function('cube(float8[], float8[])', '$2', 'test_exten
|
|||
|
||||
(1 row)
|
||||
|
||||
SELECT distribution_argument_index FROM citus.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND
|
||||
SELECT distribution_argument_index FROM pg_catalog.pg_dist_object WHERE classid = 'pg_catalog.pg_proc'::pg_catalog.regclass AND
|
||||
objid = (SELECT oid FROM pg_proc WHERE prosrc = 'cube_a_f8_f8');
|
||||
distribution_argument_index
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -45,7 +45,7 @@ SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid
|
|||
(1 row)
|
||||
|
||||
-- verify that the aggregate is added top pg_dist_object on the new node
|
||||
SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'propagate_foreign_server.array_agg'::regproc;$$);
|
||||
SELECT run_command_on_workers($$SELECT count(*) from pg_catalog.pg_dist_object where objid = 'propagate_foreign_server.array_agg'::regproc;$$);
|
||||
run_command_on_workers
|
||||
---------------------------------------------------------------------
|
||||
(localhost,57638,t,1)
|
||||
|
|
|
@ -15,6 +15,8 @@ BEGIN
|
|||
END;
|
||||
$$ language plpgsql;
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -29,6 +31,8 @@ BEGIN
|
|||
END;
|
||||
$$ language plpgsql;
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
@ -42,6 +46,8 @@ BEGIN
|
|||
END;
|
||||
$$ language plpgsql;
|
||||
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
|
||||
---------------------------------------------------------------------
|
||||
|
||||
|
|
|
@ -123,7 +123,7 @@ from
|
|||
(select
|
||||
pg_catalog.websearch_to_tsquery(
|
||||
cast(pg_catalog.regconfigin(cast(cast(null as cstring) as cstring)) as regconfig),
|
||||
cast((select type from citus.pg_dist_object limit 1 offset 1) as text)
|
||||
cast((select type from pg_catalog.pg_dist_object limit 1 offset 1) as text)
|
||||
) as c0,
|
||||
sample_0.org_id as c1,
|
||||
sample_0.id as c2,
|
||||
|
|
|
@ -419,7 +419,7 @@ SELECT * FROM run_command_on_workers($$ SELECT 'text_search.concurrent_index_con
|
|||
|
||||
-- verify the objid is correctly committed locally due to the somewhat convoluted commit and new transaction starting when creating an index concurrently
|
||||
SELECT pg_catalog.pg_identify_object_as_address(classid, objid, objsubid)
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE classid = 3602 AND objid = 'text_search.concurrent_index_config'::regconfig::oid;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -53,7 +53,6 @@ ORDER BY 1;
|
|||
function citus_dist_partition_cache_invalidate()
|
||||
function citus_dist_placement_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_drop_all_shards(regclass,text,text,boolean)
|
||||
function citus_drop_trigger()
|
||||
|
@ -121,7 +120,6 @@ ORDER BY 1;
|
|||
function citus_update_table_statistics(regclass)
|
||||
function citus_validate_rebalance_strategy_functions(regproc,regproc,regproc)
|
||||
function citus_version()
|
||||
function citus_worker_stat_activity()
|
||||
function column_name_to_column(regclass,text)
|
||||
function column_to_column_name(regclass,text)
|
||||
function columnar.columnar_handler(internal)
|
||||
|
@ -148,6 +146,7 @@ ORDER BY 1;
|
|||
function get_current_transaction_id()
|
||||
function get_global_active_transactions()
|
||||
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_table_shards_plan(regclass,real,integer,bigint[],boolean,name,real)
|
||||
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_placement_placementid_seq
|
||||
sequence pg_dist_shardid_seq
|
||||
table citus.pg_dist_object
|
||||
table columnar.chunk
|
||||
table columnar.chunk_group
|
||||
table columnar.options
|
||||
|
@ -259,6 +257,7 @@ ORDER BY 1;
|
|||
table pg_dist_local_group
|
||||
table pg_dist_node
|
||||
table pg_dist_node_metadata
|
||||
table pg_dist_object
|
||||
table pg_dist_partition
|
||||
table pg_dist_placement
|
||||
table pg_dist_poolinfo
|
||||
|
@ -277,8 +276,7 @@ ORDER BY 1;
|
|||
view citus_shards_on_worker
|
||||
view citus_stat_activity
|
||||
view citus_stat_statements
|
||||
view citus_worker_stat_activity
|
||||
view pg_dist_shard_placement
|
||||
view time_partitions
|
||||
(264 rows)
|
||||
(262 rows)
|
||||
|
||||
|
|
|
@ -36,7 +36,7 @@ drop cascades to table upgrade_basic.t_range
|
|||
-- "seg" extension
|
||||
-- will not be marked as distributed
|
||||
-- see underlying objects
|
||||
SELECT i.* FROM citus.pg_dist_object, pg_identify_object_as_address(classid, objid, objsubid) i ORDER BY 1, 2, 3;
|
||||
SELECT i.* FROM pg_catalog.pg_dist_object, pg_identify_object_as_address(classid, objid, objsubid) i ORDER BY 1, 2, 3;
|
||||
type | object_names | object_args
|
||||
---------------------------------------------------------------------
|
||||
collation | {post_11_upgrade,german_phonebook_unpropagated} | {}
|
||||
|
|
|
@ -36,7 +36,7 @@ drop cascades to table upgrade_basic.t_range
|
|||
-- "seg" extension
|
||||
-- will not be marked as distributed
|
||||
-- see underlying objects
|
||||
SELECT i.* FROM citus.pg_dist_object, pg_identify_object_as_address(classid, objid, objsubid) i ORDER BY 1, 2, 3;
|
||||
SELECT i.* FROM pg_catalog.pg_dist_object, pg_identify_object_as_address(classid, objid, objsubid) i ORDER BY 1, 2, 3;
|
||||
type | object_names | object_args
|
||||
---------------------------------------------------------------------
|
||||
database | {postgres} | {}
|
||||
|
|
|
@ -10,7 +10,7 @@ NOTICE: Preparing to sync the metadata to all nodes
|
|||
(1 row)
|
||||
|
||||
-- tables are objects with Citus 11+
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object WHERE objid IN ('post_11_upgrade'::regnamespace, 'post_11_upgrade.part_table'::regclass, 'post_11_upgrade.sensors'::regclass, 'post_11_upgrade.func_in_transaction_def'::regproc, 'post_11_upgrade.partial_index_test_config'::regconfig, 'post_11_upgrade.my_type'::regtype) ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object WHERE objid IN ('post_11_upgrade'::regnamespace, 'post_11_upgrade.part_table'::regclass, 'post_11_upgrade.sensors'::regclass, 'post_11_upgrade.func_in_transaction_def'::regproc, 'post_11_upgrade.partial_index_test_config'::regconfig, 'post_11_upgrade.my_type'::regtype) ORDER BY 1;
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------------
|
||||
(function,"{post_11_upgrade,func_in_transaction_def}",{})
|
||||
|
@ -22,7 +22,7 @@ SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dis
|
|||
(6 rows)
|
||||
|
||||
-- on all nodes
|
||||
SELECT run_command_on_workers($$SELECT array_agg(pg_identify_object_as_address(classid, objid, objsubid)) FROM citus.pg_dist_object WHERE objid IN ('post_11_upgrade'::regnamespace, 'post_11_upgrade.part_table'::regclass, 'post_11_upgrade.sensors'::regclass, 'post_11_upgrade.func_in_transaction_def'::regproc, 'post_11_upgrade.partial_index_test_config'::regconfig, 'post_11_upgrade.my_type'::regtype) ORDER BY 1;$$) ORDER BY 1;
|
||||
SELECT run_command_on_workers($$SELECT array_agg(pg_identify_object_as_address(classid, objid, objsubid)) FROM pg_catalog.pg_dist_object WHERE objid IN ('post_11_upgrade'::regnamespace, 'post_11_upgrade.part_table'::regclass, 'post_11_upgrade.sensors'::regclass, 'post_11_upgrade.func_in_transaction_def'::regproc, 'post_11_upgrade.partial_index_test_config'::regconfig, 'post_11_upgrade.my_type'::regtype) ORDER BY 1;$$) ORDER BY 1;
|
||||
run_command_on_workers
|
||||
---------------------------------------------------------------------
|
||||
(localhost,57636,t,"{""(type,{post_11_upgrade.my_type},{})"",""(function,\\""{post_11_upgrade,func_in_transaction_def}\\"",{})"",""(table,\\""{post_11_upgrade,part_table}\\"",{})"",""(table,\\""{post_11_upgrade,sensors}\\"",{})"",""(schema,{post_11_upgrade},{})"",""(\\""text search configuration\\"",\\""{post_11_upgrade,partial_index_test_config}\\"",{})""}")
|
||||
|
|
|
@ -651,8 +651,8 @@ CREATE SEQUENCE test_schema_for_sequence_propagation.seq_10;
|
|||
ALTER TABLE table_without_sequence ADD COLUMN x BIGINT DEFAULT nextval('test_schema_for_sequence_propagation.seq_10');
|
||||
|
||||
-- Should be distributed along with the sequence
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation.seq_10'::regclass);
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation'::regnamespace);
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation.seq_10'::regclass);
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation'::regnamespace);
|
||||
|
||||
DROP SCHEMA test_schema_for_sequence_propagation CASCADE;
|
||||
DROP TABLE table_without_sequence;
|
||||
|
|
|
@ -500,13 +500,13 @@ SELECT shardid, nodename, nodeport
|
|||
-- before adding the node, add pg_dist_object entry for tables created with
|
||||
-- master_create_distributed_table as we don't have the entry for them.
|
||||
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'objects'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'customer_with_special_\\_character'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, '1_customer'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'packed_numbers_hash'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'super_packed_numbers_hash'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table_to_distribute'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'second_dustbunnies'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'objects'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'customer_with_special_\\_character'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, '1_customer'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'packed_numbers_hash'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'super_packed_numbers_hash'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table_to_distribute'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'second_dustbunnies'::regclass::oid, 0);
|
||||
|
||||
SELECT 1 FROM master_activate_node('localhost', :worker_1_port);
|
||||
RESET client_min_messages;
|
||||
|
|
|
@ -1275,13 +1275,13 @@ CREATE SCHEMA test_schema_for_sequence_propagation;
|
|||
CREATE SEQUENCE test_schema_for_sequence_propagation.seq_10;
|
||||
ALTER TABLE table_without_sequence ADD COLUMN x BIGINT DEFAULT nextval('test_schema_for_sequence_propagation.seq_10');
|
||||
-- Should be distributed along with the sequence
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation.seq_10'::regclass);
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation.seq_10'::regclass);
|
||||
pg_identify_object_as_address
|
||||
---------------------------------------------------------------
|
||||
(sequence,"{test_schema_for_sequence_propagation,seq_10}",{})
|
||||
(1 row)
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation'::regnamespace);
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object WHERE objid IN ('test_schema_for_sequence_propagation'::regnamespace);
|
||||
pg_identify_object_as_address
|
||||
----------------------------------------------------
|
||||
(schema,{test_schema_for_sequence_propagation},{})
|
||||
|
|
|
@ -627,13 +627,13 @@ SELECT shardid, nodename, nodeport
|
|||
-- add the node back
|
||||
-- before adding the node, add pg_dist_object entry for tables created with
|
||||
-- master_create_distributed_table as we don't have the entry for them.
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'objects'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'customer_with_special_\\_character'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, '1_customer'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'packed_numbers_hash'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'super_packed_numbers_hash'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table_to_distribute'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'second_dustbunnies'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'objects'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'customer_with_special_\\_character'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, '1_customer'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'packed_numbers_hash'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'super_packed_numbers_hash'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table_to_distribute'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'second_dustbunnies'::regclass::oid, 0);
|
||||
SELECT 1 FROM master_activate_node('localhost', :worker_1_port);
|
||||
NOTICE: Replicating postgres objects to node localhost:57637
|
||||
DETAIL: There are 115 objects to replicate, depending on your environment this might take a while
|
||||
|
|
|
@ -71,7 +71,7 @@ step "s2-sleep"
|
|||
|
||||
step "s2-view-dist"
|
||||
{
|
||||
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' 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"
|
||||
{
|
||||
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
|
||||
|
|
|
@ -51,7 +51,7 @@ step "s1-print-distributed-objects"
|
|||
SELECT 1 FROM master_add_node('localhost', 57638);
|
||||
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
|
@ -123,7 +123,7 @@ step "s2-commit"
|
|||
step "s2-print-distributed-objects"
|
||||
{
|
||||
-- print an overview of all distributed objects
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object ORDER BY 1;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object ORDER BY 1;
|
||||
|
||||
-- print if the schema has been created
|
||||
SELECT count(*) FROM pg_namespace where nspname = 'myschema';
|
||||
|
|
|
@ -41,7 +41,7 @@ step "s1-create-extension-with-schema2"
|
|||
|
||||
step "s1-print"
|
||||
{
|
||||
select count(*) from citus.pg_dist_object ;
|
||||
select count(*) from pg_catalog.pg_dist_object ;
|
||||
select extname, extversion, nspname from pg_extension, pg_namespace where pg_namespace.oid=pg_extension.extnamespace and extname='seg';
|
||||
SELECT run_command_on_workers($$select extname from pg_extension where extname='seg'$$);
|
||||
SELECT run_command_on_workers($$select extversion from pg_extension where extname='seg'$$);
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
#include "isolation_mx_common.include.spec"
|
||||
|
||||
setup {
|
||||
SELECT citus_add_node('localhost', 57636, groupid:=0);
|
||||
CREATE TABLE ref_table(user_id int, value_1 int);
|
||||
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);
|
||||
|
@ -17,6 +18,7 @@ teardown
|
|||
DROP TABLE ref_table;
|
||||
DROP TABLE tt1;
|
||||
SELECT citus_internal.restore_isolation_tester_func();
|
||||
SELECT citus_remove_node('localhost', 57636);
|
||||
}
|
||||
|
||||
session "s1"
|
||||
|
@ -159,7 +161,13 @@ session "s3"
|
|||
|
||||
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
|
||||
|
@ -189,6 +197,59 @@ step "s4-commit-worker"
|
|||
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-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"
|
||||
|
@ -212,3 +273,10 @@ permutation "s1-start-session-level-connection" "s1-begin-on-worker" "s1-update-
|
|||
// 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-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"
|
||||
|
|
|
@ -69,18 +69,20 @@ step "s2-coordinator-citus_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%'
|
||||
)
|
||||
AND query NOT ILIKE '%run_commands_on_session_level_connection_to_node%'
|
||||
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%'
|
||||
)
|
||||
AND is_worker_query = true
|
||||
AND backend_type = 'client backend'
|
||||
ORDER BY 1;
|
||||
}
|
||||
|
||||
|
@ -101,7 +103,7 @@ step "s2-coordinator-get_global_active_transactions"
|
|||
|
||||
|
||||
// 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
|
||||
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"
|
||||
|
|
|
@ -81,21 +81,24 @@ step "s2-lock-ref-table-placement-on-coordinator"
|
|||
|
||||
step "s2-view-dist"
|
||||
{
|
||||
SELECT query, query_hostname, query_hostport, distributed_query_host_name, distributed_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' 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"
|
||||
{
|
||||
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
|
||||
query NOT ILIKE '%dump_local_%' AND
|
||||
query NOT ILIKE '%citus_internal_local_blocked_processes%' AND
|
||||
query NOT ILIKE '%add_node%' AND
|
||||
backend_type = 'client backend'
|
||||
ORDER BY query, query_hostport 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%'),
|
||||
('%dump_local_%'),
|
||||
('%citus_internal_local_blocked_processes%'),
|
||||
('%add_node%'),
|
||||
('%csa_from_one_node%'))
|
||||
AND is_worker_query = true
|
||||
AND backend_type = 'client backend'
|
||||
AND query != ''
|
||||
ORDER BY query DESC;
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -161,7 +161,7 @@ create aggregate binstragg(text, text)(
|
|||
stype=text
|
||||
);
|
||||
-- verify that the aggregate is added into pg_dist_object, on each worker
|
||||
SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.binstragg'::regproc;$$);
|
||||
SELECT run_command_on_workers($$SELECT count(*) from pg_catalog.pg_dist_object where objid = 'aggregate_support.binstragg'::regproc;$$);
|
||||
|
||||
SELECT run_command_on_workers($$select count(*) from pg_aggregate where aggfnoid::text like '%binstragg%';$$);
|
||||
|
||||
|
@ -588,14 +588,14 @@ COMMIT;
|
|||
SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$);
|
||||
|
||||
-- verify that the aggregate is added into pg_dist_object, on each worker
|
||||
SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$);
|
||||
SELECT run_command_on_workers($$SELECT count(*) from pg_catalog.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$);
|
||||
|
||||
RESET citus.create_object_propagation;
|
||||
|
||||
-- drop and test outside of tx block
|
||||
drop aggregate dependent_agg (float8);
|
||||
-- verify that the aggregate is removed from pg_dist_object, on each worker
|
||||
SELECT run_command_on_workers($$SELECT count(*) from citus.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$);
|
||||
SELECT run_command_on_workers($$SELECT count(*) from pg_catalog.pg_dist_object where objid = 'aggregate_support.dependent_agg'::regproc;$$);
|
||||
create aggregate dependent_agg (float8) (stype=dummy_tbl, sfunc=dummy_fnc);
|
||||
--verify
|
||||
SELECT run_command_on_workers($$select aggfnoid from pg_aggregate where aggfnoid::text like '%dependent_agg%';$$);
|
||||
|
|
|
@ -45,7 +45,7 @@ DROP TABLE t4; -- as long as the table is using the type some operations are har
|
|||
COMMIT;
|
||||
|
||||
-- verify the type is distributed
|
||||
SELECT count(*) FROM citus.pg_dist_object WHERE objid = 'disabled_object_propagation.tt3'::regtype::oid;
|
||||
SELECT count(*) FROM pg_catalog.pg_dist_object WHERE objid = 'disabled_object_propagation.tt3'::regtype::oid;
|
||||
|
||||
ALTER TYPE tt3 ADD ATTRIBUTE c int, DROP ATTRIBUTE b, ALTER ATTRIBUTE a SET DATA TYPE text COLLATE "POSIX";
|
||||
ALTER TYPE tt3 OWNER TO typeowner_for_disabled_object_propagation_guc;
|
||||
|
|
|
@ -180,7 +180,7 @@ SELECT create_distributed_table('streaming_table','id');
|
|||
-- if not paremeters are supplied, we'd see that function doesn't have
|
||||
-- distribution_argument_index and colocationid
|
||||
SELECT create_distributed_function('"eq_mi''xed_param_names"(macaddr, macaddr)');
|
||||
SELECT distribution_argument_index is NULL, colocationid is NULL from citus.pg_dist_object
|
||||
SELECT distribution_argument_index is NULL, colocationid is NULL from pg_catalog.pg_dist_object
|
||||
WHERE objid = 'eq_mi''xed_param_names(macaddr, macaddr)'::regprocedure;
|
||||
|
||||
-- also show that we can use the function
|
||||
|
@ -380,7 +380,7 @@ SELECT create_distributed_function('eq_with_param_names(macaddr, macaddr)', '$1'
|
|||
|
||||
-- show that the colocationIds are the same
|
||||
SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated
|
||||
FROM pg_dist_partition, citus.pg_dist_object as objects
|
||||
FROM pg_dist_partition, pg_catalog.pg_dist_object as objects
|
||||
WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND
|
||||
objects.objid = 'eq_with_param_names(macaddr, macaddr)'::regprocedure;
|
||||
|
||||
|
@ -388,7 +388,7 @@ WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass
|
|||
-- group preserved, because we're using the default shard creation settings
|
||||
SELECT create_distributed_function('eq_with_param_names(macaddr, macaddr)', 'val1');
|
||||
SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated
|
||||
FROM pg_dist_partition, citus.pg_dist_object as objects
|
||||
FROM pg_dist_partition, pg_catalog.pg_dist_object as objects
|
||||
WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND
|
||||
objects.objid = 'eq_with_param_names(macaddr, macaddr)'::regprocedure;
|
||||
|
||||
|
@ -405,13 +405,13 @@ SELECT create_distributed_function('eq_with_param_names(macaddr, macaddr)', colo
|
|||
-- to coerce the values
|
||||
SELECT create_distributed_function('eq8(macaddr8, macaddr8)', '$1', colocate_with:='replicated_table_func_test_4');
|
||||
SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated
|
||||
FROM pg_dist_partition, citus.pg_dist_object as objects
|
||||
FROM pg_dist_partition, pg_catalog.pg_dist_object as objects
|
||||
WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND
|
||||
objects.objid = 'eq8(macaddr8, macaddr8)'::regprocedure;
|
||||
|
||||
SELECT create_distributed_function('add_text(text, text)', '$1', colocate_with:='replicated_table_func_test_4');
|
||||
SELECT pg_dist_partition.colocationid = objects.colocationid as table_and_function_colocated
|
||||
FROM pg_dist_partition, citus.pg_dist_object as objects
|
||||
FROM pg_dist_partition, pg_catalog.pg_dist_object as objects
|
||||
WHERE pg_dist_partition.logicalrelid = 'replicated_table_func_test_4'::regclass AND
|
||||
objects.objid = 'add_text(text, text)'::regprocedure;
|
||||
|
||||
|
|
|
@ -240,3 +240,75 @@ ROLLBACK;
|
|||
|
||||
DROP SCHEMA drop_partitioned_table CASCADE;
|
||||
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;
|
||||
|
|
|
@ -18,12 +18,12 @@ END;
|
|||
$$;
|
||||
|
||||
-- Check all dependent objects and function depends on all nodes
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema'::regnamespace::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.function_prop_type'::regtype::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_1'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
SET citus.enable_metadata_sync TO OFF;
|
||||
CREATE TYPE function_prop_type_2 AS (a int, b int);
|
||||
|
@ -38,10 +38,10 @@ BEGIN
|
|||
END;
|
||||
$$;
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_2'::regtype::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_2'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
-- Have a separate check for type created in transaction
|
||||
BEGIN;
|
||||
|
@ -60,9 +60,9 @@ BEGIN
|
|||
END;
|
||||
$$;
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_3'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.function_prop_type_3'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_3'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
-- Check table
|
||||
CREATE TABLE function_prop_table(a int, b int);
|
||||
|
@ -97,8 +97,8 @@ BEGIN
|
|||
END;
|
||||
$$;
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_6'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
-- Views are not supported
|
||||
CREATE VIEW function_prop_view AS SELECT * FROM function_prop_table;
|
||||
|
@ -133,8 +133,8 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Within transaction functions are not distributed
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Show that recreating it outside transaction distributes the function and dependencies
|
||||
|
@ -147,10 +147,10 @@ BEGIN
|
|||
END;
|
||||
$$;
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regtype::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
-- Test for SQL function with unsupported object in function body
|
||||
CREATE TABLE table_in_sql_body(id int);
|
||||
|
@ -163,9 +163,9 @@ $$
|
|||
$$;
|
||||
|
||||
-- Show that only function has propagated, since the table is not resolved as dependency
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regclass::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.type_in_transaction'::regclass::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.max_of_table'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
-- Check extension owned table
|
||||
CREATE TABLE extension_owned_table(a int);
|
||||
|
@ -202,17 +202,17 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Function shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid;
|
||||
|
||||
CREATE TABLE table_to_prop_func(id int, col_1 int default func_in_transaction_def());
|
||||
SELECT create_distributed_table('table_to_prop_func','id');
|
||||
|
||||
-- Function should be marked as distributed after distributing the table that depends on it
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
|
||||
-- Multiple functions as a default column
|
||||
|
@ -236,20 +236,20 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid;
|
||||
|
||||
CREATE TABLE table_to_prop_func_2(id int, col_1 int default func_in_transaction_1() + func_in_transaction_2());
|
||||
SELECT create_distributed_table('table_to_prop_func_2','id');
|
||||
|
||||
-- Functions should be marked as distribued after distributing the table that depends on it
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Functions should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_1'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_2'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
|
||||
-- If function has dependency on non-distributed table it should error out
|
||||
|
@ -287,16 +287,16 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Function shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid;
|
||||
|
||||
ALTER TABLE table_to_prop_func_4 ADD COLUMN col_1 int default function_propagation_schema.func_in_transaction_4();
|
||||
|
||||
-- Function should be marked as distributed after adding the column
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Functions should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_4'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
|
||||
-- Adding a column with default function depending on non-distributable table should fail
|
||||
|
@ -342,20 +342,20 @@ BEGIN;
|
|||
|
||||
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid;
|
||||
|
||||
CREATE TABLE table_to_prop_func_5(id int, col_1 int default func_in_transaction_5(), col_2 int default func_in_transaction_6());
|
||||
SELECT create_distributed_table('table_to_prop_func_5', 'id');
|
||||
|
||||
-- Functions should be marked as distributed after adding the column
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Functions should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_5'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_6'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
-- Adding a constraint with function check should propagate the function
|
||||
BEGIN;
|
||||
|
@ -369,17 +369,17 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid;
|
||||
|
||||
CREATE TABLE table_to_prop_func_6(id int, col_1 int check (function_propagation_schema.func_in_transaction_7(col_1)));
|
||||
SELECT create_distributed_table('table_to_prop_func_6', 'id');
|
||||
|
||||
-- Function should be marked as distributed after adding the column
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_7'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
|
||||
-- Adding a constraint with multiple functions check should propagate the function
|
||||
|
@ -403,20 +403,20 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid;
|
||||
|
||||
CREATE TABLE table_to_prop_func_7(id int, col_1 int check (function_propagation_schema.func_in_transaction_8(col_1) and function_propagation_schema.func_in_transaction_9(col_1)));
|
||||
SELECT create_distributed_table('table_to_prop_func_7', 'id');
|
||||
|
||||
-- Function should be marked as distributed after adding the column
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Functions should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_8'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_9'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
|
||||
-- Adding a column with constraint should propagate the function
|
||||
|
@ -434,16 +434,16 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid;
|
||||
|
||||
ALTER TABLE table_to_prop_func_8 ADD CONSTRAINT col1_check CHECK (function_propagation_schema.func_in_transaction_10(col_1));
|
||||
|
||||
-- Function should be marked as distributed after adding the constraint
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_10'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
|
||||
-- If constraint depends on a non-distributed table it should error out
|
||||
|
@ -479,7 +479,7 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_generated_always'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_generated_always'::regproc::oid;
|
||||
|
||||
CREATE TABLE people (
|
||||
id int,
|
||||
|
@ -489,7 +489,7 @@ BEGIN;
|
|||
SELECT create_distributed_table('people', 'id');
|
||||
|
||||
-- Show that function is distributed after distributing the table
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_generated_always'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_generated_always'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
|
||||
|
@ -505,7 +505,7 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid;
|
||||
|
||||
CREATE TABLE table_1_for_rule(id int, col_1 int);
|
||||
CREATE TABLE table_2_for_rule(id int, col_1 int);
|
||||
|
@ -515,11 +515,11 @@ BEGIN;
|
|||
SELECT create_distributed_table('table_1_for_rule','id');
|
||||
|
||||
-- Functions should be distributed after distributing the table
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_for_rule'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
|
||||
-- Show that functions as partitioning functions are supported
|
||||
|
@ -535,18 +535,18 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid;
|
||||
|
||||
CREATE TABLE partitioned_table_to_test_func_prop(id INT, a INT) PARTITION BY RANGE (non_sense_func_for_partitioning(id));
|
||||
|
||||
SELECT create_distributed_table('partitioned_table_to_test_func_prop', 'id');
|
||||
|
||||
-- Show that function is distributed after distributing the table
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.non_sense_func_for_partitioning'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
|
||||
-- Test function dependency on citus local table
|
||||
|
@ -561,14 +561,14 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Function shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_for_local_table'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_for_local_table'::regproc::oid;
|
||||
|
||||
CREATE TABLE citus_local_table_to_test_func(l1 int DEFAULT func_in_transaction_for_local_table());
|
||||
SELECT 1 FROM master_add_node('localhost', :master_port, groupid => 0);
|
||||
SELECT citus_add_local_table_to_metadata('citus_local_table_to_test_func');
|
||||
|
||||
-- Function should be marked as distributed after distributing the table that depends on it
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_for_local_table'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_for_local_table'::regproc::oid;
|
||||
ROLLBACK;
|
||||
|
||||
-- Show that having a function dependency on exlude also works
|
||||
|
@ -583,17 +583,17 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid;
|
||||
|
||||
CREATE TABLE exclusion_func_prop_table (id int, EXCLUDE USING btree (id WITH =) WHERE (exclude_bool_func()));
|
||||
SELECT create_distributed_table('exclusion_func_prop_table', 'id');
|
||||
|
||||
-- Function should be marked as distributed after distributing the table that depends on it
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.exclude_bool_func'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
|
||||
-- Show that having a function dependency for index also works
|
||||
|
@ -608,7 +608,7 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Functions shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid;
|
||||
|
||||
CREATE TABLE table_to_check_func_index_dep (id int, col_2 int);
|
||||
CREATE INDEX on table_to_check_func_index_dep(col_2) WHERE (func_for_index_predicate(col_2));
|
||||
|
@ -616,11 +616,11 @@ BEGIN;
|
|||
SELECT create_distributed_table('table_to_check_func_index_dep', 'id');
|
||||
|
||||
-- Function should be marked as distributed after distributing the table that depends on it
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_for_index_predicate'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
|
||||
-- Test function to function dependency
|
||||
|
@ -648,11 +648,11 @@ BEGIN;
|
|||
SELECT create_distributed_table('func_dep_table', 'a');
|
||||
|
||||
-- Function should be marked as distributed after distributing the table that depends on it
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_func_dep_1'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_for_func_dep_1'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_for_func_dep_1'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_for_func_dep_1'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
|
||||
-- Test function with SQL language and sequence dependency
|
||||
|
@ -672,7 +672,7 @@ BEGIN;
|
|||
$$;
|
||||
|
||||
-- Function shouldn't be propagated within transaction
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid;
|
||||
|
||||
CREATE SEQUENCE myseq;
|
||||
CREATE TABLE table_to_prop_seq_func(id int, col_1 bigint default func_in_transaction_def_with_func(func_in_transaction_def_with_seq(nextval('myseq'))));
|
||||
|
@ -680,11 +680,11 @@ BEGIN;
|
|||
SELECT create_distributed_table('table_to_prop_seq_func','id');
|
||||
|
||||
-- Function should be marked as distributed after distributing the table that depends on it
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid;
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid;
|
||||
COMMIT;
|
||||
|
||||
-- Function should be marked as distributed on the worker after committing changes
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from citus.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid;$$) ORDER BY 1,2;
|
||||
SELECT * FROM run_command_on_workers($$SELECT pg_identify_object_as_address(classid, objid, objsubid) from pg_catalog.pg_dist_object where objid = 'function_propagation_schema.func_in_transaction_def_with_seq'::regproc::oid;$$) ORDER BY 1,2;
|
||||
|
||||
|
||||
-- Show that having a dependency on another dist table work out tx
|
||||
|
@ -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));
|
||||
|
||||
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
|
||||
|
|
|
@ -93,8 +93,8 @@ SELECT count(*) FROM history;
|
|||
SET citus.shard_replication_factor TO 1;
|
||||
|
||||
-- metadata sync will succeed even if we have rep > 1 tables
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles_single_shard'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'public.articles_single_shard'::regclass::oid, 0);
|
||||
SELECT start_metadata_sync_to_node('localhost', :worker_1_port);
|
||||
|
||||
CREATE TABLE mx_table(a int);
|
||||
|
|
|
@ -108,8 +108,8 @@ SELECT run_command_on_workers('GRANT ALL ON SCHEMA citus TO node_metadata_user')
|
|||
SELECT master_remove_node('localhost', :worker_2_port);
|
||||
|
||||
-- Removing public schema from pg_dist_object because it breaks the next tests
|
||||
DELETE FROM citus.pg_dist_object WHERE objid = 'public'::regnamespace::oid;
|
||||
DELETE FROM citus.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'plpgsql');
|
||||
DELETE FROM pg_catalog.pg_dist_object WHERE objid = 'public'::regnamespace::oid;
|
||||
DELETE FROM pg_catalog.pg_dist_object WHERE objid = (SELECT oid FROM pg_extension WHERE extname = 'plpgsql');
|
||||
|
||||
-- try to manipulate node metadata via non-super user
|
||||
SET ROLE non_super_user;
|
||||
|
|
|
@ -66,11 +66,6 @@ CREATE FUNCTION find_shard_interval_index(bigint)
|
|||
AS 'citus'
|
||||
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
|
||||
-- ===================================================================
|
||||
|
@ -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');
|
||||
|
||||
-- activate nodes to get rid of inconsistencies in pg_dist tables
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table1_group1'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table2_group1'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table3_group2'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table4_group2'::regclass::oid, 0);
|
||||
INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table5_groupX'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table1_group1'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table2_group1'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table3_group2'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table4_group2'::regclass::oid, 0);
|
||||
INSERT INTO pg_catalog.pg_dist_object(classid, objid, objsubid) values('pg_class'::regclass::oid, 'table5_groupX'::regclass::oid, 0);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_1_port);
|
||||
SELECT 1 FROM citus_activate_node('localhost', :worker_2_port);
|
||||
|
||||
|
|
|
@ -213,16 +213,17 @@ SELECT * FROM multi_extension.print_extension_changes();
|
|||
-- Test downgrade to 9.4-1 from 9.5-1
|
||||
ALTER EXTENSION citus UPDATE TO '9.5-1';
|
||||
|
||||
BEGIN;
|
||||
SET citus.enable_metadata_sync TO on;
|
||||
SELECT master_add_node('localhost', :master_port, groupId=>0);
|
||||
CREATE TABLE citus_local_table (a int);
|
||||
SELECT create_citus_local_table('citus_local_table');
|
||||
RESET citus.enable_metadata_sync;
|
||||
|
||||
-- downgrade from 9.5-1 to 9.4-1 should fail as we have a citus local table
|
||||
ALTER EXTENSION citus UPDATE TO '9.4-1';
|
||||
ROLLBACK;
|
||||
-- TODO: This test should be moved to a valid downgrade testing suite where the downgrade is done, both on the schema and the binaries. Later changes in Citus made a C vs Schema discrepancy error here
|
||||
-- BEGIN;
|
||||
-- SET citus.enable_metadata_sync TO on;
|
||||
-- SELECT master_add_node('localhost', :master_port, groupId=>0);
|
||||
-- CREATE TABLE citus_local_table (a int);
|
||||
-- SELECT create_citus_local_table('citus_local_table');
|
||||
-- RESET citus.enable_metadata_sync;
|
||||
--
|
||||
-- -- downgrade from 9.5-1 to 9.4-1 should fail as we have a citus local table
|
||||
-- ALTER EXTENSION citus UPDATE TO '9.4-1';
|
||||
-- ROLLBACK;
|
||||
|
||||
-- now we can downgrade as there is no citus local table
|
||||
ALTER EXTENSION citus UPDATE TO '9.4-1';
|
||||
|
|
|
@ -10,5 +10,5 @@ FROM pg_attribute
|
|||
WHERE atthasmissing AND attrelid NOT IN ('pg_dist_node'::regclass,
|
||||
'pg_dist_rebalance_strategy'::regclass,
|
||||
'pg_dist_partition'::regclass,
|
||||
'citus.pg_dist_object'::regclass)
|
||||
'pg_dist_object'::regclass)
|
||||
ORDER BY attrelid, attname;
|
||||
|
|
|
@ -544,10 +544,10 @@ DROP TABLE mx_table_with_small_sequence, mx_table_with_sequence;
|
|||
-- owner
|
||||
CREATE TABLE pg_dist_placement_temp AS SELECT * FROM pg_dist_placement;
|
||||
CREATE TABLE pg_dist_partition_temp AS SELECT * FROM pg_dist_partition;
|
||||
CREATE TABLE pg_dist_object_temp AS SELECT * FROM citus.pg_dist_object;
|
||||
CREATE TABLE pg_dist_object_temp AS SELECT * FROM pg_catalog.pg_dist_object;
|
||||
DELETE FROM pg_dist_placement;
|
||||
DELETE FROM pg_dist_partition;
|
||||
DELETE FROM citus.pg_dist_object;
|
||||
DELETE FROM pg_catalog.pg_dist_object;
|
||||
SELECT groupid AS old_worker_2_group FROM pg_dist_node WHERE nodeport = :worker_2_port \gset
|
||||
SELECT master_remove_node('localhost', :worker_2_port);
|
||||
|
||||
|
@ -586,7 +586,7 @@ DROP TABLE mx_table;
|
|||
\c - postgres - :master_port
|
||||
INSERT INTO pg_dist_placement SELECT * FROM pg_dist_placement_temp;
|
||||
INSERT INTO pg_dist_partition SELECT * FROM pg_dist_partition_temp;
|
||||
INSERT INTO citus.pg_dist_object SELECT * FROM pg_dist_object_temp ON CONFLICT ON CONSTRAINT pg_dist_object_pkey DO NOTHING;
|
||||
INSERT INTO pg_catalog.pg_dist_object SELECT * FROM pg_dist_object_temp ON CONFLICT ON CONSTRAINT pg_dist_object_pkey DO NOTHING;
|
||||
DROP TABLE pg_dist_placement_temp;
|
||||
DROP TABLE pg_dist_partition_temp;
|
||||
DROP TABLE pg_dist_object_temp;
|
||||
|
|
|
@ -67,7 +67,7 @@ END;$$;
|
|||
SELECT create_distributed_function('proc_0(float8)', 'dist_key', 'test_proc_colocation_0' );
|
||||
|
||||
SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('test_proc_colocation_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
|
||||
SET client_min_messages TO DEBUG1;
|
||||
CALL proc_0(1.0);
|
||||
|
@ -81,7 +81,7 @@ CALL proc_0(1.0);
|
|||
RESET client_min_messages;
|
||||
|
||||
SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('test_proc_colocation_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
|
||||
-- colocatewith is not null && list_length(colocatedTableList) = 1
|
||||
SELECT alter_distributed_table('test_proc_colocation_0', shard_count:= 4);
|
||||
|
@ -94,7 +94,7 @@ CALL proc_0(1.0);
|
|||
RESET client_min_messages;
|
||||
|
||||
SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('test_proc_colocation_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
|
||||
-- shardCount is not null && cascade_to_colocated is true
|
||||
SELECT alter_distributed_table('test_proc_colocation_0', shard_count:= 8, cascade_to_colocated := true);
|
||||
|
@ -104,7 +104,7 @@ CALL proc_0(1.0);
|
|||
RESET client_min_messages;
|
||||
|
||||
SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('test_proc_colocation_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
|
||||
-- colocatewith is not null && cascade_to_colocated is true
|
||||
SELECT alter_distributed_table('test_proc_colocation_0', shard_count:= 4, cascade_to_colocated := true);
|
||||
|
@ -117,7 +117,7 @@ CALL proc_0(1.0);
|
|||
RESET client_min_messages;
|
||||
|
||||
SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('test_proc_colocation_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0');
|
||||
|
||||
-- try a case with more than one procedure
|
||||
CREATE OR REPLACE procedure proc_1(dist_key float8)
|
||||
|
@ -134,7 +134,7 @@ END;$$;
|
|||
SELECT create_distributed_function('proc_1(float8)', 'dist_key', 'test_proc_colocation_0' );
|
||||
|
||||
SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('test_proc_colocation_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname;
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname;
|
||||
|
||||
SET client_min_messages TO DEBUG1;
|
||||
CALL proc_0(1.0);
|
||||
|
@ -149,14 +149,14 @@ CALL proc_1(2.0);
|
|||
RESET client_min_messages;
|
||||
|
||||
SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('test_proc_colocation_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname;
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname;
|
||||
|
||||
-- case which shouldn't preserve colocation for now
|
||||
-- shardCount is not null && cascade_to_colocated is false
|
||||
SELECT alter_distributed_table('test_proc_colocation_0', shard_count:= 18, cascade_to_colocated := false);
|
||||
|
||||
SELECT logicalrelid, colocationid FROM pg_dist_partition WHERE logicalrelid::regclass::text IN ('test_proc_colocation_0');
|
||||
SELECT proname, colocationid FROM pg_proc JOIN citus.pg_dist_object ON pg_proc.oid = citus.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname;
|
||||
SELECT proname, colocationid FROM pg_proc JOIN pg_catalog.pg_dist_object ON pg_proc.oid = pg_catalog.pg_dist_object.objid WHERE proname IN ('proc_0', 'proc_1') ORDER BY proname;
|
||||
|
||||
SET client_min_messages TO WARNING;
|
||||
DROP SCHEMA mx_alter_distributed_table CASCADE;
|
||||
|
|
|
@ -308,7 +308,7 @@ SELECT create_distributed_table('mx_old_schema.table_set_schema', 'id');
|
|||
CREATE SCHEMA mx_new_schema;
|
||||
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid::oid::regnamespace IN ('mx_old_schema', 'mx_new_schema')
|
||||
ORDER BY "Distributed Schemas";
|
||||
\c - - - :worker_1_port
|
||||
|
@ -325,7 +325,7 @@ ALTER SCHEMA mx_old_schema RENAME TO temp_mx_old_schema;
|
|||
ALTER TABLE mx_old_schema.table_set_schema SET SCHEMA mx_new_schema;
|
||||
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid::oid::regnamespace IN ('mx_old_schema', 'mx_new_schema');
|
||||
\c - - - :worker_1_port
|
||||
SELECT table_schema AS "Table's Schema" FROM information_schema.tables WHERE table_name='table_set_schema';
|
||||
|
|
|
@ -672,7 +672,7 @@ SELECT create_distributed_table('old_schema.table_set_schema', 'id');
|
|||
CREATE SCHEMA new_schema;
|
||||
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema');
|
||||
\c - - - :worker_1_port
|
||||
SELECT table_schema AS "Shards' Schema"
|
||||
|
@ -685,7 +685,7 @@ SELECT table_schema AS "Shards' Schema"
|
|||
ALTER TABLE old_schema.table_set_schema SET SCHEMA new_schema;
|
||||
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema');
|
||||
\c - - - :worker_1_port
|
||||
SELECT table_schema AS "Shards' Schema"
|
||||
|
@ -706,7 +706,7 @@ SELECT create_distributed_table('table_set_schema', 'id');
|
|||
CREATE SCHEMA new_schema;
|
||||
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid='new_schema'::regnamespace::oid;
|
||||
\c - - - :worker_1_port
|
||||
SELECT table_schema AS "Shards' Schema"
|
||||
|
@ -719,7 +719,7 @@ SELECT table_schema AS "Shards' Schema"
|
|||
ALTER TABLE table_set_schema SET SCHEMA new_schema;
|
||||
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid='new_schema'::regnamespace::oid;
|
||||
\c - - - :worker_1_port
|
||||
SELECT table_schema AS "Shards' Schema"
|
||||
|
@ -743,7 +743,7 @@ SELECT create_distributed_table('table_set_schema', 'id');
|
|||
CREATE SCHEMA new_schema;
|
||||
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema');
|
||||
\c - - - :worker_1_port
|
||||
SELECT table_schema AS "Shards' Schema", COUNT(*) AS "Counts"
|
||||
|
@ -757,7 +757,7 @@ SET search_path TO old_schema;
|
|||
ALTER TABLE table_set_schema SET SCHEMA new_schema;
|
||||
|
||||
SELECT objid::oid::regnamespace as "Distributed Schemas"
|
||||
FROM citus.pg_dist_object
|
||||
FROM pg_catalog.pg_dist_object
|
||||
WHERE objid::oid::regnamespace IN ('old_schema', 'new_schema');
|
||||
\c - - - :worker_1_port
|
||||
SELECT table_schema AS "Shards' Schema", COUNT(*) AS "Counts"
|
||||
|
@ -969,9 +969,9 @@ ROLLBACK;
|
|||
-- Clean up the created schema
|
||||
SET client_min_messages TO WARNING;
|
||||
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object
|
||||
WHERE classid=2615 and objid IN (select oid from pg_namespace where nspname='run_test_schema');
|
||||
DROP SCHEMA run_test_schema, test_schema_support_join_1, test_schema_support_join_2, "Citus'Teen123", "CiTUS.TEEN2", bar, test_schema_support CASCADE;
|
||||
-- verify that the dropped schema is removed from worker's pg_dist_object
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM citus.pg_dist_object
|
||||
SELECT pg_identify_object_as_address(classid, objid, objsubid) FROM pg_catalog.pg_dist_object
|
||||
WHERE classid=2615 and objid IN (select oid from pg_namespace where nspname='run_test_schema');
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue