PG16 compatibility: ruleutils and successful CREATE EXTENSION (#7087)

PG16 compatibility - Part 2

Part 1 provided successful compilation against pg16beta2.
42d956888d

This PR provides ruleutils changes with pg16beta2 and successful CREATE EXTENSION command.
Note that more changes are needed in order to have successful regression tests.
More commits are coming soon ...

For any_value changes, I referred to this commit
8ef94dc1f5
where we did something similar for PG14 support.
allow-do-block-tenant-tbl
Naisila Puka 2023-08-02 16:04:51 +03:00 committed by GitHub
parent 960a5f6104
commit 0d503dd5ac
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
22 changed files with 9614 additions and 29 deletions

1
.gitattributes vendored
View File

@ -28,6 +28,7 @@ src/backend/distributed/utils/citus_outfuncs.c -citus-style
src/backend/distributed/deparser/ruleutils_13.c -citus-style src/backend/distributed/deparser/ruleutils_13.c -citus-style
src/backend/distributed/deparser/ruleutils_14.c -citus-style src/backend/distributed/deparser/ruleutils_14.c -citus-style
src/backend/distributed/deparser/ruleutils_15.c -citus-style src/backend/distributed/deparser/ruleutils_15.c -citus-style
src/backend/distributed/deparser/ruleutils_16.c -citus-style
src/backend/distributed/commands/index_pg_source.c -citus-style src/backend/distributed/commands/index_pg_source.c -citus-style
src/include/distributed/citus_nodes.h -citus-style src/include/distributed/citus_nodes.h -citus-style

File diff suppressed because it is too large Load Diff

View File

@ -1,3 +1,6 @@
-- citus--12.0-1--12.1-1 -- citus--12.0-1--12.1-1
-- bump version to 12.1-1 -- bump version to 12.1-1
#include "udfs/citus_prepare_pg_upgrade/12.1-1.sql"
#include "udfs/citus_finish_pg_upgrade/12.1-1.sql"

View File

@ -1,2 +1,5 @@
-- citus--12.1-1--12.0-1 -- citus--12.1-1--12.0-1
-- this is an empty downgrade path since citus--12.0-1--12.1-1.sql is empty for now
-- we have modified the relevant upgrade script to include any_value changes
-- we don't need to upgrade this downgrade path for any_value changes
-- since if we are doing a Citus downgrade, not PG downgrade, then it would be no-op.

View File

@ -1,7 +1,13 @@
DO $proc$
BEGIN
-- PG16 has its own any_value, so only create it pre PG16.
IF substring(current_Setting('server_version'), '\d+')::int < 16 THEN
EXECUTE $$
CREATE OR REPLACE FUNCTION pg_catalog.any_value_agg ( anyelement, anyelement ) CREATE OR REPLACE FUNCTION pg_catalog.any_value_agg ( anyelement, anyelement )
RETURNS anyelement AS $$ RETURNS anyelement AS $agg$
SELECT CASE WHEN $1 IS NULL THEN $2 ELSE $1 END; SELECT CASE WHEN $1 IS NULL THEN $2 ELSE $1 END;
$$ LANGUAGE SQL STABLE; $agg$ LANGUAGE SQL STABLE;
CREATE AGGREGATE pg_catalog.any_value ( CREATE AGGREGATE pg_catalog.any_value (
sfunc = pg_catalog.any_value_agg, sfunc = pg_catalog.any_value_agg,
@ -12,3 +18,6 @@ CREATE AGGREGATE pg_catalog.any_value (
COMMENT ON AGGREGATE pg_catalog.any_value(anyelement) IS COMMENT ON AGGREGATE pg_catalog.any_value(anyelement) IS
'Returns the value of any row in the group. It is mostly useful when you know there will be only 1 element.'; 'Returns the value of any row in the group. It is mostly useful when you know there will be only 1 element.';
$$;
END IF;
END $proc$;

View File

@ -1,7 +1,13 @@
DO $proc$
BEGIN
-- PG16 has its own any_value, so only create it pre PG16.
IF substring(current_Setting('server_version'), '\d+')::int < 16 THEN
EXECUTE $$
CREATE OR REPLACE FUNCTION pg_catalog.any_value_agg ( anyelement, anyelement ) CREATE OR REPLACE FUNCTION pg_catalog.any_value_agg ( anyelement, anyelement )
RETURNS anyelement AS $$ RETURNS anyelement AS $agg$
SELECT CASE WHEN $1 IS NULL THEN $2 ELSE $1 END; SELECT CASE WHEN $1 IS NULL THEN $2 ELSE $1 END;
$$ LANGUAGE SQL STABLE; $agg$ LANGUAGE SQL STABLE;
CREATE AGGREGATE pg_catalog.any_value ( CREATE AGGREGATE pg_catalog.any_value (
sfunc = pg_catalog.any_value_agg, sfunc = pg_catalog.any_value_agg,
@ -12,3 +18,6 @@ CREATE AGGREGATE pg_catalog.any_value (
COMMENT ON AGGREGATE pg_catalog.any_value(anyelement) IS COMMENT ON AGGREGATE pg_catalog.any_value(anyelement) IS
'Returns the value of any row in the group. It is mostly useful when you know there will be only 1 element.'; 'Returns the value of any row in the group. It is mostly useful when you know there will be only 1 element.';
$$;
END IF;
END $proc$;

View File

@ -0,0 +1,211 @@
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;
-- PG16 has its own any_value, so only create it pre PG16.
-- We can remove this part when we drop support for PG16
IF substring(current_Setting('server_version'), '\d+')::int < 16 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 OR REPLACE FUNCTION pg_catalog.any_value_agg ( anyelement, anyelement )
RETURNS anyelement AS $$
SELECT CASE WHEN $1 IS NULL THEN $2 ELSE $1 END;
$$ LANGUAGE SQL STABLE;
CREATE AGGREGATE pg_catalog.any_value (
sfunc = pg_catalog.any_value_agg,
combinefunc = pg_catalog.any_value_agg,
basetype = anyelement,
stype = anyelement
);
COMMENT ON AGGREGATE pg_catalog.any_value(anyelement) IS
'Returns the value of any row in the group. It is mostly useful when you know there will be only 1 element.';
RESET citus.enable_ddl_propagation;
--
-- Citus creates the any_value aggregate but because of a compatibility
-- issue between pg15-pg16 -- any_value is created in PG16, we drop
-- and create it during upgrade IF upgraded version is less than 16.
-- And as Citus creates it, there needs to be a dependency to the
-- Citus extension, so we create that dependency here.
INSERT INTO pg_depend
SELECT
'pg_proc'::regclass::oid as classid,
(SELECT oid FROM pg_proc WHERE proname = 'any_value_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;
INSERT INTO pg_depend
SELECT
'pg_proc'::regclass::oid as classid,
(SELECT oid FROM pg_proc WHERE proname = 'any_value') 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;
$cmd$;
END IF;
--
-- 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;
INSERT INTO pg_catalog.pg_dist_cleanup SELECT * FROM public.pg_dist_cleanup;
INSERT INTO pg_catalog.pg_dist_schema SELECT schemaname::regnamespace, colocationid FROM public.pg_dist_schema;
-- 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;
DROP TABLE public.pg_dist_cleanup;
DROP TABLE public.pg_dist_schema;
--
-- 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);
PERFORM setval('pg_catalog.pg_dist_operationid_seq', (SELECT MAX(operation_id)+1 AS max_operation_id FROM pg_dist_cleanup), false);
PERFORM setval('pg_catalog.pg_dist_cleanup_recordid_seq', (SELECT MAX(record_id)+1 AS max_record_id FROM pg_dist_cleanup), false);
PERFORM setval('pg_catalog.pg_dist_clock_logical_seq', (SELECT last_value FROM public.pg_dist_clock_logical_seq), false);
DROP TABLE public.pg_dist_clock_logical_seq;
--
-- register triggers
--
FOR table_name IN SELECT logicalrelid FROM pg_catalog.pg_dist_partition JOIN pg_class ON (logicalrelid = oid) WHERE relkind <> 'f'
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 columnar_internal.columnar_ensure_am_depends_catalog();
-- restore pg_dist_object from the stable identifiers
TRUNCATE pg_catalog.pg_dist_object;
INSERT INTO pg_catalog.pg_dist_object (classid, objid, objsubid, distribution_argument_index, colocationid)
SELECT
address.classid,
address.objid,
address.objsubid,
naming.distribution_argument_index,
naming.colocationid
FROM
public.pg_dist_object naming,
pg_catalog.pg_get_object_address(naming.type, naming.object_names, naming.object_args) address;
DROP TABLE public.pg_dist_object;
END;
$cppu$;
COMMENT ON FUNCTION pg_catalog.citus_finish_pg_upgrade()
IS 'perform tasks to restore citus settings from a location that has been prepared before pg_upgrade';

View File

@ -51,6 +51,57 @@ BEGIN
0 as refobjsubid , 0 as refobjsubid ,
'e' as deptype; 'e' as deptype;
-- PG16 has its own any_value, so only create it pre PG16.
-- We can remove this part when we drop support for PG16
IF substring(current_Setting('server_version'), '\d+')::int < 16 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 OR REPLACE FUNCTION pg_catalog.any_value_agg ( anyelement, anyelement )
RETURNS anyelement AS $$
SELECT CASE WHEN $1 IS NULL THEN $2 ELSE $1 END;
$$ LANGUAGE SQL STABLE;
CREATE AGGREGATE pg_catalog.any_value (
sfunc = pg_catalog.any_value_agg,
combinefunc = pg_catalog.any_value_agg,
basetype = anyelement,
stype = anyelement
);
COMMENT ON AGGREGATE pg_catalog.any_value(anyelement) IS
'Returns the value of any row in the group. It is mostly useful when you know there will be only 1 element.';
RESET citus.enable_ddl_propagation;
--
-- Citus creates the any_value aggregate but because of a compatibility
-- issue between pg15-pg16 -- any_value is created in PG16, we drop
-- and create it during upgrade IF upgraded version is less than 16.
-- And as Citus creates it, there needs to be a dependency to the
-- Citus extension, so we create that dependency here.
INSERT INTO pg_depend
SELECT
'pg_proc'::regclass::oid as classid,
(SELECT oid FROM pg_proc WHERE proname = 'any_value_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;
INSERT INTO pg_depend
SELECT
'pg_proc'::regclass::oid as classid,
(SELECT oid FROM pg_proc WHERE proname = 'any_value') 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;
$cmd$;
END IF;
-- --
-- restore citus catalog tables -- restore citus catalog tables
-- --

View File

@ -0,0 +1,91 @@
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);
-- We should drop any_value because PG16 has its own any_value function
-- We can remove this part when we drop support for PG16
DELETE FROM pg_depend WHERE
objid IN (SELECT oid FROM pg_proc WHERE proname = 'any_value' OR proname = 'any_value_agg') AND
refobjid IN (select oid from pg_extension where extname = 'citus');
DROP AGGREGATE IF EXISTS pg_catalog.any_value(anyelement);
DROP FUNCTION IF EXISTS pg_catalog.any_value_agg(anyelement, anyelement);
--
-- 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;
DROP TABLE IF EXISTS public.pg_dist_cleanup;
DROP TABLE IF EXISTS public.pg_dist_schema;
DROP TABLE IF EXISTS public.pg_dist_clock_logical_seq;
--
-- 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;
CREATE TABLE public.pg_dist_cleanup AS SELECT * FROM pg_catalog.pg_dist_cleanup;
-- save names of the tenant schemas instead of their oids because the oids might change after pg upgrade
CREATE TABLE public.pg_dist_schema AS SELECT schemaid::regnamespace::text AS schemaname, colocationid FROM pg_catalog.pg_dist_schema;
-- 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;
-- sequences
CREATE TABLE public.pg_dist_clock_logical_seq AS SELECT last_value FROM pg_catalog.pg_dist_clock_logical_seq;
CREATE TABLE public.pg_dist_rebalance_strategy AS SELECT
name,
default_strategy,
shard_cost_function::regprocedure::text,
node_capacity_function::regprocedure::text,
shard_allowed_on_node_function::regprocedure::text,
default_threshold,
minimum_threshold,
improvement_threshold
FROM pg_catalog.pg_dist_rebalance_strategy;
-- store upgrade stable identifiers on pg_dist_object catalog
CREATE TABLE public.pg_dist_object AS SELECT
address.type,
address.object_names,
address.object_args,
objects.distribution_argument_index,
objects.colocationid
FROM pg_catalog.pg_dist_object objects,
pg_catalog.pg_identify_object_as_address(objects.classid, objects.objid, objects.objsubid) address;
END;
$cppu$;
COMMENT ON FUNCTION pg_catalog.citus_prepare_pg_upgrade()
IS 'perform tasks to copy citus settings to a location that could later be restored after pg_upgrade is done';

View File

@ -17,6 +17,15 @@ BEGIN
DROP AGGREGATE IF EXISTS array_cat_agg(anyarray); DROP AGGREGATE IF EXISTS array_cat_agg(anyarray);
DROP AGGREGATE IF EXISTS array_cat_agg(anycompatiblearray); DROP AGGREGATE IF EXISTS array_cat_agg(anycompatiblearray);
-- We should drop any_value because PG16 has its own any_value function
-- We can remove this part when we drop support for PG16
DELETE FROM pg_depend WHERE
objid IN (SELECT oid FROM pg_proc WHERE proname = 'any_value' OR proname = 'any_value_agg') AND
refobjid IN (select oid from pg_extension where extname = 'citus');
DROP AGGREGATE IF EXISTS pg_catalog.any_value(anyelement);
DROP FUNCTION IF EXISTS pg_catalog.any_value_agg(anyelement, anyelement);
-- --
-- Drop existing backup tables -- Drop existing backup tables
-- --

View File

@ -39,7 +39,7 @@
#include "distributed/version_compat.h" #include "distributed/version_compat.h"
/* forward declaration of background worker entrypoint */ /* forward declaration of background worker entrypoint */
extern void LockAcquireHelperMain(Datum main_arg); extern PGDLLEXPORT void LockAcquireHelperMain(Datum main_arg);
/* forward declaration of helper functions */ /* forward declaration of helper functions */
static void lock_acquire_helper_sigterm(SIGNAL_ARGS); static void lock_acquire_helper_sigterm(SIGNAL_ARGS);

View File

@ -274,14 +274,14 @@ extern ChunkData * CreateEmptyChunkData(uint32 columnCount, bool *columnMask,
uint32 chunkGroupRowCount); uint32 chunkGroupRowCount);
extern void FreeChunkData(ChunkData *chunkData); extern void FreeChunkData(ChunkData *chunkData);
extern uint64 ColumnarTableRowCount(Relation relation); extern uint64 ColumnarTableRowCount(Relation relation);
extern const char * CompressionTypeStr(CompressionType type); extern PGDLLEXPORT const char * CompressionTypeStr(CompressionType type);
/* columnar_metadata_tables.c */ /* columnar_metadata_tables.c */
extern void InitColumnarOptions(Oid regclass); extern PGDLLEXPORT void InitColumnarOptions(Oid regclass);
extern void SetColumnarOptions(Oid regclass, ColumnarOptions *options); extern PGDLLEXPORT void SetColumnarOptions(Oid regclass, ColumnarOptions *options);
extern bool DeleteColumnarTableOptions(Oid regclass, bool missingOk); extern PGDLLEXPORT bool DeleteColumnarTableOptions(Oid regclass, bool missingOk);
extern bool ReadColumnarOptions(Oid regclass, ColumnarOptions *options); extern PGDLLEXPORT bool ReadColumnarOptions(Oid regclass, ColumnarOptions *options);
extern bool IsColumnarTableAmTable(Oid relationId); extern PGDLLEXPORT bool IsColumnarTableAmTable(Oid relationId);
/* columnar_metadata_tables.c */ /* columnar_metadata_tables.c */
extern void DeleteMetadataRows(RelFileLocator relfilelocator); extern void DeleteMetadataRows(RelFileLocator relfilelocator);

View File

@ -56,7 +56,7 @@ extern TableScanDesc columnar_beginscan_extended(Relation relation, Snapshot sna
uint32 flags, Bitmapset *attr_needed, uint32 flags, Bitmapset *attr_needed,
List *scanQual); List *scanQual);
extern int64 ColumnarScanChunkGroupsFiltered(ColumnarScanDesc columnarScanDesc); extern int64 ColumnarScanChunkGroupsFiltered(ColumnarScanDesc columnarScanDesc);
extern bool ColumnarSupportsIndexAM(char *indexAMName); extern PGDLLEXPORT bool ColumnarSupportsIndexAM(char *indexAMName);
extern bool IsColumnarTableAmTable(Oid relationId); extern bool IsColumnarTableAmTable(Oid relationId);
extern void CheckCitusColumnarCreateExtensionStmt(Node *parseTree); extern void CheckCitusColumnarCreateExtensionStmt(Node *parseTree);
extern void CheckCitusColumnarAlterExtensionStmt(Node *parseTree); extern void CheckCitusColumnarAlterExtensionStmt(Node *parseTree);

View File

@ -102,8 +102,8 @@ typedef struct ParallelTasksPerNodeEntry
extern BackgroundWorkerHandle * StartCitusBackgroundTaskQueueMonitor(Oid database, extern BackgroundWorkerHandle * StartCitusBackgroundTaskQueueMonitor(Oid database,
Oid extensionOwner); Oid extensionOwner);
extern void CitusBackgroundTaskQueueMonitorMain(Datum arg); extern PGDLLEXPORT void CitusBackgroundTaskQueueMonitorMain(Datum arg);
extern void CitusBackgroundTaskExecutor(Datum main_arg); extern PGDLLEXPORT void CitusBackgroundTaskExecutor(Datum main_arg);
extern Datum citus_job_cancel(PG_FUNCTION_ARGS); extern Datum citus_job_cancel(PG_FUNCTION_ARGS);
extern Datum citus_job_wait(PG_FUNCTION_ARGS); extern Datum citus_job_wait(PG_FUNCTION_ARGS);

View File

@ -29,6 +29,6 @@ extern void MaintenanceDaemonShmemInit(void);
extern void InitializeMaintenanceDaemonBackend(void); extern void InitializeMaintenanceDaemonBackend(void);
extern bool LockCitusExtension(void); extern bool LockCitusExtension(void);
extern void CitusMaintenanceDaemonMain(Datum main_arg); extern PGDLLEXPORT void CitusMaintenanceDaemonMain(Datum main_arg);
#endif /* MAINTENANCED_H */ #endif /* MAINTENANCED_H */

View File

@ -119,7 +119,7 @@ extern List * InterTableRelationshipOfRelationCommandList(Oid relationId);
extern List * DetachPartitionCommandList(void); extern List * DetachPartitionCommandList(void);
extern void SyncNodeMetadataToNodes(void); extern void SyncNodeMetadataToNodes(void);
extern BackgroundWorkerHandle * SpawnSyncNodeMetadataToNodes(Oid database, Oid owner); extern BackgroundWorkerHandle * SpawnSyncNodeMetadataToNodes(Oid database, Oid owner);
extern void SyncNodeMetadataToNodesMain(Datum main_arg); extern PGDLLEXPORT void SyncNodeMetadataToNodesMain(Datum main_arg);
extern void SignalMetadataSyncDaemon(Oid database, int sig); extern void SignalMetadataSyncDaemon(Oid database, int sig);
extern bool ShouldInitiateMetadataSync(bool *lockFailure); extern bool ShouldInitiateMetadataSync(bool *lockFailure);
extern List * SequenceDependencyCommandList(Oid relationId); extern List * SequenceDependencyCommandList(Oid relationId);

View File

@ -14,5 +14,6 @@
#define PG_VERSION_14 140000 #define PG_VERSION_14 140000
#define PG_VERSION_15 150000 #define PG_VERSION_15 150000
#define PG_VERSION_16 160000 #define PG_VERSION_16 160000
#define PG_VERSION_17 170000
#endif /* PG_VERSION_CONSTANTS */ #endif /* PG_VERSION_CONSTANTS */

View File

@ -17,10 +17,10 @@
#define MAX_SHARD_COUNT 64000 #define MAX_SHARD_COUNT 64000
#define MAX_SHARD_REPLICATION_FACTOR 100 #define MAX_SHARD_REPLICATION_FACTOR 100
extern ColumnarSupportsIndexAM_type extern_ColumnarSupportsIndexAM; extern PGDLLEXPORT ColumnarSupportsIndexAM_type extern_ColumnarSupportsIndexAM;
extern CompressionTypeStr_type extern_CompressionTypeStr; extern PGDLLEXPORT CompressionTypeStr_type extern_CompressionTypeStr;
extern IsColumnarTableAmTable_type extern_IsColumnarTableAmTable; extern PGDLLEXPORT IsColumnarTableAmTable_type extern_IsColumnarTableAmTable;
extern ReadColumnarOptions_type extern_ReadColumnarOptions; extern PGDLLEXPORT ReadColumnarOptions_type extern_ReadColumnarOptions;
extern void StartupCitusBackend(void); extern void StartupCitusBackend(void);
extern const char * GetClientMinMessageLevelNameForValue(int minMessageLevel); extern const char * GetClientMinMessageLevelNameForValue(int minMessageLevel);

View File

@ -59,7 +59,9 @@ BEGIN
SELECT p.description previous_object, c.description current_object SELECT p.description previous_object, c.description current_object
FROM current_objects c FULL JOIN prev_objects p FROM current_objects c FULL JOIN prev_objects p
ON p.description = c.description ON p.description = c.description
WHERE p.description is null OR c.description is null; WHERE (p.description is null OR c.description is null)
AND c.description IS DISTINCT FROM 'function any_value(anyelement) anyelement'
AND c.description IS DISTINCT FROM 'function any_value_agg(anyelement,anyelement) anyelement';
DROP TABLE prev_objects; DROP TABLE prev_objects;
ALTER TABLE current_objects RENAME TO prev_objects; ALTER TABLE current_objects RENAME TO prev_objects;
@ -132,8 +134,6 @@ SELECT * FROM multi_extension.print_extension_changes();
--------------------------------------------------------------------- ---------------------------------------------------------------------
| event trigger citus_cascade_to_partition | event trigger citus_cascade_to_partition
| function alter_role_if_exists(text,text) boolean | function alter_role_if_exists(text,text) boolean
| function any_value(anyelement) anyelement
| function any_value_agg(anyelement,anyelement) anyelement
| function array_cat_agg(anycompatiblearray) anycompatiblearray | function array_cat_agg(anycompatiblearray) anycompatiblearray
| function assign_distributed_transaction_id(integer,bigint,timestamp with time zone) void | function assign_distributed_transaction_id(integer,bigint,timestamp with time zone) void
| function authinfo_valid(text) boolean | function authinfo_valid(text) boolean
@ -318,7 +318,7 @@ SELECT * FROM multi_extension.print_extension_changes();
| view citus_stat_statements | view citus_stat_statements
| view citus_worker_stat_activity | view citus_worker_stat_activity
| view pg_dist_shard_placement | view pg_dist_shard_placement
(188 rows) (186 rows)
-- Test downgrade to 9.2-2 from 9.2-4 -- Test downgrade to 9.2-2 from 9.2-4
ALTER EXTENSION citus UPDATE TO '9.2-4'; ALTER EXTENSION citus UPDATE TO '9.2-4';

View File

@ -5,6 +5,8 @@ WHERE refclassid = 'pg_catalog.pg_extension'::pg_catalog.regclass
AND refobjid = e.oid AND refobjid = e.oid
AND deptype = 'e' AND deptype = 'e'
AND e.extname='citus' AND e.extname='citus'
AND pg_catalog.pg_describe_object(classid, objid, 0) != 'function any_value(anyelement)'
AND pg_catalog.pg_describe_object(classid, objid, 0) != 'function any_value_agg(anyelement,anyelement)'
ORDER BY 1; ORDER BY 1;
description description
--------------------------------------------------------------------- ---------------------------------------------------------------------
@ -13,8 +15,6 @@ ORDER BY 1;
function alter_old_partitions_set_access_method(regclass,timestamp with time zone,name) function alter_old_partitions_set_access_method(regclass,timestamp with time zone,name)
function alter_role_if_exists(text,text) function alter_role_if_exists(text,text)
function alter_table_set_access_method(regclass,text) function alter_table_set_access_method(regclass,text)
function any_value(anyelement)
function any_value_agg(anyelement,anyelement)
function array_cat_agg(anycompatiblearray) function array_cat_agg(anycompatiblearray)
function assign_distributed_transaction_id(integer,bigint,timestamp with time zone) function assign_distributed_transaction_id(integer,bigint,timestamp with time zone)
function authinfo_valid(text) function authinfo_valid(text)
@ -338,5 +338,5 @@ ORDER BY 1;
view citus_stat_tenants_local view citus_stat_tenants_local
view pg_dist_shard_placement view pg_dist_shard_placement
view time_partitions view time_partitions
(330 rows) (328 rows)

View File

@ -63,7 +63,9 @@ BEGIN
SELECT p.description previous_object, c.description current_object SELECT p.description previous_object, c.description current_object
FROM current_objects c FULL JOIN prev_objects p FROM current_objects c FULL JOIN prev_objects p
ON p.description = c.description ON p.description = c.description
WHERE p.description is null OR c.description is null; WHERE (p.description is null OR c.description is null)
AND c.description IS DISTINCT FROM 'function any_value(anyelement) anyelement'
AND c.description IS DISTINCT FROM 'function any_value_agg(anyelement,anyelement) anyelement';
DROP TABLE prev_objects; DROP TABLE prev_objects;
ALTER TABLE current_objects RENAME TO prev_objects; ALTER TABLE current_objects RENAME TO prev_objects;

View File

@ -5,4 +5,6 @@ WHERE refclassid = 'pg_catalog.pg_extension'::pg_catalog.regclass
AND refobjid = e.oid AND refobjid = e.oid
AND deptype = 'e' AND deptype = 'e'
AND e.extname='citus' AND e.extname='citus'
AND pg_catalog.pg_describe_object(classid, objid, 0) != 'function any_value(anyelement)'
AND pg_catalog.pg_describe_object(classid, objid, 0) != 'function any_value_agg(anyelement,anyelement)'
ORDER BY 1; ORDER BY 1;