Fix multi-1 tests

velioglu/tmpfuncprop
Burak Velioglu 2022-02-12 19:53:00 +03:00
parent c78ca8cbad
commit 23aa73c988
No known key found for this signature in database
GPG Key ID: F6827E620F6549C6
51 changed files with 139 additions and 336 deletions

View File

@ -898,9 +898,11 @@ NOTICE: executing the command locally: SELECT count(*) AS count FROM (SELECT in
-- a helper function which return true if the coordinated
-- trannsaction uses 2PC
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION coordinated_transaction_should_use_2PC()
RETURNS BOOL LANGUAGE C STRICT VOLATILE AS 'citus',
$$coordinated_transaction_should_use_2PC$$;
RESET citus.enable_metadata_sync;
-- a local SELECT followed by remote SELECTs
-- does not trigger 2PC
BEGIN;

View File

@ -790,6 +790,7 @@ SELECT * FROM test ORDER BY id;
DROP TABLE test;
-- verify that recreating distributed functions with TABLE params gets propagated to workers
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION func_with_return_table(int)
RETURNS TABLE (date date)
LANGUAGE plpgsql AS $$
@ -797,12 +798,14 @@ BEGIN
RETURN query SELECT '2011-01-01'::date;
END;
$$;
RESET citus.enable_metadata_sync;
SELECT create_distributed_function('func_with_return_table(int)');
create_distributed_function
---------------------------------------------------------------------
(1 row)
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION func_with_return_table(int)
RETURNS TABLE (date date)
LANGUAGE plpgsql AS $$
@ -810,6 +813,7 @@ BEGIN
RETURN query SELECT '2011-01-02'::date;
END;
$$;
RESET citus.enable_metadata_sync;
SELECT count(*) FROM
(SELECT result FROM
run_command_on_workers($$select row(pg_proc.pronargs, pg_proc.proargtypes, pg_proc.prosrc) from pg_proc where proname = 'func_with_return_table';$$)
@ -817,7 +821,7 @@ SELECT count(*) FROM
as test;
count
---------------------------------------------------------------------
1
2
(1 row)
-- verify that recreating distributed functions with OUT params gets propagated to workers

View File

@ -10,6 +10,7 @@ SELECT run_command_on_workers($$CREATE SCHEMA proc_conflict;$$);
\c - - - :worker_1_port
SET search_path TO proc_conflict;
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION existing_func(state int, i int) RETURNS int AS $$
BEGIN
RETURN state * 2 + i;
@ -19,6 +20,7 @@ CREATE AGGREGATE existing_agg(int) (
SFUNC = existing_func,
STYPE = int
);
RESET citus.enable_metadata_sync;
\c - - - :master_port
SET search_path TO proc_conflict;
CREATE FUNCTION existing_func(state int, i int) RETURNS int AS $$
@ -69,6 +71,7 @@ DROP AGGREGATE existing_agg(int) CASCADE;
DROP FUNCTION existing_func(int, int) CASCADE;
\c - - - :worker_1_port
SET search_path TO proc_conflict;
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION existing_func(state int, i int) RETURNS int AS $$
BEGIN
RETURN state * 3 + i;
@ -78,6 +81,7 @@ CREATE AGGREGATE existing_agg(int) (
SFUNC = existing_func,
STYPE = int
);
RESET citus.enable_metadata_sync;
\c - - - :master_port
SET search_path TO proc_conflict;
CREATE FUNCTION existing_func(state int, i int) RETURNS int AS $$

View File

@ -47,7 +47,6 @@ WHERE n.nspname IN ('drop_partitioned_table', 'schema1')
AND c.relkind IN ('r','p')
ORDER BY 1, 2;
\c - - - :worker_1_port
CREATE SCHEMA drop_partitioned_table;
SET search_path = drop_partitioned_table;
CREATE VIEW tables_info AS
SELECT n.nspname as "Schema",

View File

@ -223,11 +223,13 @@ EXECUTE p1(6,6,6);
DEBUG: Deferred pruning for a fast-path router query
DEBUG: Creating router plan
EXECUTE p1(7,7,7);
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION modify_fast_path_plpsql(int, int) RETURNS void as $$
BEGIN
DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2;
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
SELECT modify_fast_path_plpsql(1,1);
DEBUG: Deferred pruning for a fast-path router query
CONTEXT: SQL statement "DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2"

View File

@ -7,6 +7,7 @@
-- reference and hash-distributed version of orders, customer and part tables.
SET citus.next_shard_id TO 360000;
-- this function is dropped in Citus10, added here for tests
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass,
distribution_column text,
distribution_method citus.distribution_type)
@ -23,6 +24,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name tex
RETURNS void
AS 'citus', $$master_create_worker_shards$$
LANGUAGE C STRICT;
RESET citus.enable_metadata_sync;
CREATE TABLE lineitem (
l_orderkey bigint not null,
l_partkey integer not null,

View File

@ -697,17 +697,7 @@ SELECT create_distributed_function('func_custom_param(intpair)');
(1 row)
RESET citus.enable_metadata_sync;
SELECT deparse_and_run_on_workers($cmd$
ALTER FUNCTION func_custom_param RENAME TO func_with_custom_param;
$cmd$);
INFO: Propagating deparsed query: ALTER FUNCTION function_tests.func_custom_param(function_tests.intpair) RENAME TO func_with_custom_param;
CONTEXT: PL/pgSQL function deparse_and_run_on_workers(text) line XX at RAISE
deparse_and_run_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"ALTER FUNCTION")
(localhost,57638,t,"ALTER FUNCTION")
(2 rows)
-- a function that returns TABLE
CREATE FUNCTION func_returns_table(IN count INT)
RETURNS TABLE (x INT, y INT)
@ -721,17 +711,7 @@ SELECT create_distributed_function('func_returns_table(INT)');
(1 row)
RESET citus.enable_metadata_sync;
SELECT deparse_and_run_on_workers($cmd$
ALTER FUNCTION func_returns_table ROWS 100;
$cmd$);
INFO: Propagating deparsed query: ALTER FUNCTION function_tests.func_returns_table(integer) ROWS 100.000000;
CONTEXT: PL/pgSQL function deparse_and_run_on_workers(text) line XX at RAISE
deparse_and_run_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"ALTER FUNCTION")
(localhost,57638,t,"ALTER FUNCTION")
(2 rows)
-- clear objects
SET client_min_messages TO WARNING; -- suppress cascading objects dropping
DROP SCHEMA "CiTuS.TeeN" CASCADE;

View File

@ -71,6 +71,7 @@ DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT table1.id, ta
(1 row)
-- a function that returns a set of integers
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION next_k_integers(IN first_value INTEGER,
IN k INTEGER DEFAULT 3,
OUT result INTEGER)
@ -79,6 +80,7 @@ BEGIN
RETURN QUERY SELECT x FROM generate_series(first_value, first_value+k-1) f(x);
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
SELECT *
FROM table1 JOIN next_k_integers(3,2) next_integers ON (id = next_integers.result)
ORDER BY id ASC;
@ -223,8 +225,10 @@ SELECT raise_failed_execution_func_join($$
$$);
ERROR: Task failed to execute
-- a user-defined immutable function
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION the_answer_to_life()
RETURNS INTEGER IMMUTABLE AS 'SELECT 42' LANGUAGE SQL;
RESET citus.enable_metadata_sync;
SELECT raise_failed_execution_func_join($$
SELECT * FROM table1 JOIN the_answer_to_life() the_answer ON (id = the_answer);
$$);

View File

@ -26,10 +26,12 @@ ALTER SEQUENCE pg_catalog.pg_dist_placement_placementid_seq RESTART 100000;
SELECT nextval('pg_catalog.pg_dist_groupid_seq') AS last_group_id \gset
SELECT nextval('pg_catalog.pg_dist_node_nodeid_seq') AS last_node_id \gset
-- Create the necessary test utility function
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION activate_node_snapshot()
RETURNS text[]
LANGUAGE C STRICT
AS 'citus';
RESET citus.enable_metadata_sync;
COMMENT ON FUNCTION activate_node_snapshot()
IS 'commands to activate node snapshot';
-- Show that none of the existing tables are qualified to be MX tables
@ -89,12 +91,14 @@ SELECT unnest(activate_node_snapshot()) order by 1;
(27 rows)
-- this function is dropped in Citus10, added here for tests
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass,
distribution_column text,
distribution_method citus.distribution_type)
RETURNS void
LANGUAGE C STRICT
AS 'citus', $$master_create_distributed_table$$;
RESET citus.enable_metadata_sync;
COMMENT ON FUNCTION pg_catalog.master_create_distributed_table(table_name regclass,
distribution_column text,
distribution_method citus.distribution_type)

View File

@ -467,13 +467,6 @@ UPDATE limit_orders SET placed_at = LEAST(placed_at, now()::timestamp) WHERE id
UPDATE limit_orders SET array_of_values = 1 || array_of_values WHERE id = 246;
CREATE FUNCTION immutable_append(old_values int[], new_value int)
RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE;
\c - - - :worker_1_port
CREATE FUNCTION immutable_append(old_values int[], new_value int)
RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE;
\c - - - :worker_2_port
CREATE FUNCTION immutable_append(old_values int[], new_value int)
RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE;
\c - - - :master_port
-- immutable function calls with vars are also allowed
UPDATE limit_orders
SET array_of_values = immutable_append(array_of_values, 2) WHERE id = 246;

View File

@ -329,22 +329,15 @@ SELECT count(*) FROM pg_dist_transaction;
(1 row)
-- create a check function
SELECT * from run_command_on_workers('CREATE FUNCTION reject_large_id() RETURNS trigger AS $rli$
CREATE FUNCTION reject_large_id() RETURNS trigger AS $rli$
BEGIN
IF (NEW.id > 30) THEN
RAISE ''illegal value'';
RAISE 'illegal value';
END IF;
RETURN NEW;
END;
$rli$ LANGUAGE plpgsql;')
ORDER BY nodeport;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | CREATE FUNCTION
localhost | 57638 | t | CREATE FUNCTION
(2 rows)
$rli$ LANGUAGE plpgsql;
-- register after insert trigger
SELECT * FROM run_command_on_placements('researchers', 'CREATE CONSTRAINT TRIGGER reject_large_researcher_id AFTER INSERT ON %s DEFERRABLE INITIALLY DEFERRED FOR EACH ROW EXECUTE PROCEDURE reject_large_id()')
ORDER BY nodeport, shardid;
@ -418,14 +411,7 @@ ORDER BY nodeport, shardid;
localhost | 57638 | 1200001 | t | DROP TRIGGER
(4 rows)
SELECT * FROM run_command_on_workers('drop function reject_large_id()')
ORDER BY nodeport;
nodename | nodeport | success | result
---------------------------------------------------------------------
localhost | 57637 | t | DROP FUNCTION
localhost | 57638 | t | DROP FUNCTION
(2 rows)
DROP FUNCTION reject_large_id();
-- ALTER and copy are compatible
BEGIN;
ALTER TABLE labs ADD COLUMN motto text;
@ -522,6 +508,7 @@ AND s.logicalrelid = 'objects'::regclass;
-- create trigger on one worker to reject certain values
\c - - - :worker_2_port
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION reject_bad() RETURNS trigger AS $rb$
BEGIN
IF (NEW.name = 'BAD') THEN
@ -531,6 +518,7 @@ CREATE FUNCTION reject_bad() RETURNS trigger AS $rb$
RETURN NEW;
END;
$rb$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE CONSTRAINT TRIGGER reject_bad
AFTER INSERT ON objects_1200003
DEFERRABLE INITIALLY IMMEDIATE
@ -573,6 +561,7 @@ DELETE FROM objects;
-- there cannot be errors on different shards at different times
-- because the first failure will fail the whole transaction
\c - - - :worker_1_port
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION reject_bad() RETURNS trigger AS $rb$
BEGIN
IF (NEW.name = 'BAD') THEN
@ -582,6 +571,7 @@ CREATE FUNCTION reject_bad() RETURNS trigger AS $rb$
RETURN NEW;
END;
$rb$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE CONSTRAINT TRIGGER reject_bad
AFTER INSERT ON labs_1200002
DEFERRABLE INITIALLY IMMEDIATE
@ -850,6 +840,7 @@ SELECT * FROM reference_modifying_xacts;
-- lets fail on of the workers at before the commit time
\c - - - :worker_1_port
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION reject_bad_reference() RETURNS trigger AS $rb$
BEGIN
IF (NEW.key = 999) THEN
@ -859,6 +850,7 @@ CREATE FUNCTION reject_bad_reference() RETURNS trigger AS $rb$
RETURN NEW;
END;
$rb$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE CONSTRAINT TRIGGER reject_bad_reference
AFTER INSERT ON reference_modifying_xacts_1200006
DEFERRABLE INITIALLY IMMEDIATE
@ -938,6 +930,7 @@ INSERT INTO hash_modifying_xacts VALUES (2, 2);
ABORT;
-- lets fail one of the workers before COMMIT time for the hash table
\c - - - :worker_1_port
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION reject_bad_hash() RETURNS trigger AS $rb$
BEGIN
IF (NEW.key = 997) THEN
@ -947,6 +940,7 @@ CREATE FUNCTION reject_bad_hash() RETURNS trigger AS $rb$
RETURN NEW;
END;
$rb$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE CONSTRAINT TRIGGER reject_bad_hash
AFTER INSERT ON hash_modifying_xacts_1200007
DEFERRABLE INITIALLY IMMEDIATE

View File

@ -632,10 +632,12 @@ ERROR: permission denied for function worker_cleanup_job_schema_cache
RESET ROLE;
-- to test access to files created during repartition we will create some on worker 1
\c - - - :worker_1_port
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION citus_rm_job_directory(bigint)
RETURNS void
AS 'citus'
LANGUAGE C STRICT;
RESET citus.enable_metadata_sync;
SET ROLE full_access;
SELECT worker_hash_partition_table(42,1,'SELECT a FROM generate_series(1,100) AS a', 'a', 23, ARRAY[-2147483648, -1073741824, 0, 1073741824]::int4[]);
worker_hash_partition_table
@ -646,10 +648,12 @@ SELECT worker_hash_partition_table(42,1,'SELECT a FROM generate_series(1,100) AS
RESET ROLE;
-- all attempts for transfer are initiated from other workers
\c - - - :worker_2_port
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION citus_rm_job_directory(bigint)
RETURNS void
AS 'citus'
LANGUAGE C STRICT;
RESET citus.enable_metadata_sync;
-- super user should not be able to copy files created by a user
SELECT worker_fetch_partition_file(42, 1, 1, 1, 'localhost', :worker_1_port);
WARNING: could not open file "base/pgsql_job_cache/job_0042/task_000001/p_00001.xxxx": No such file or directory

View File

@ -4,6 +4,7 @@
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 225000;
SET citus.shard_count TO 2;
-- this function is dropped in Citus10, added here for tests
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass,
distribution_column text,
distribution_method citus.distribution_type)
@ -20,6 +21,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name tex
RETURNS void
AS 'citus', $$master_create_worker_shards$$
LANGUAGE C STRICT;
RESET citus.enable_metadata_sync;
-- Verify that a table name > 56 characters gets hashed properly.
CREATE TABLE too_long_12345678901234567890123456789012345678901234567890 (
col1 integer not null,

View File

@ -53,13 +53,7 @@ CREATE TABLE repartition_udt_other (
-- so that the OID is off.
\c - - :public_worker_1_host :worker_1_port
-- START type creation
-- ... as well as a function to use as its comparator...
CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean
AS 'select $1.i = $2.i AND $1.i2 = $2.i2;'
LANGUAGE SQL
IMMUTABLE
RETURNS NULL ON NULL INPUT;
-- ... use that function to create a custom equality operator...
-- Use function to create a custom equality operator...
CREATE OPERATOR = (
LEFTARG = test_udt,
RIGHTARG = test_udt,
@ -69,13 +63,6 @@ CREATE OPERATOR = (
);
-- ... and create a custom operator family for hash indexes...
CREATE OPERATOR FAMILY tudt_op_fam USING hash;
-- ... create a test HASH function. Though it is a poor hash function,
-- it is acceptable for our tests
CREATE FUNCTION test_udt_hash(test_udt) RETURNS int
AS 'SELECT hashtext( ($1.i + $1.i2)::text);'
LANGUAGE SQL
IMMUTABLE
RETURNS NULL ON NULL INPUT;
-- We need to define two different operator classes for the composite types
-- One uses BTREE the other uses HASH
CREATE OPERATOR CLASS tudt_op_fam_clas3
@ -88,13 +75,7 @@ FUNCTION 1 test_udt_hash(test_udt);
-- END type creation
\c - - :public_worker_2_host :worker_2_port
-- START type creation
-- ... as well as a function to use as its comparator...
CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean
AS 'select $1.i = $2.i AND $1.i2 = $2.i2;'
LANGUAGE SQL
IMMUTABLE
RETURNS NULL ON NULL INPUT;
-- ... use that function to create a custom equality operator...
-- Use function to create a custom equality operator...
CREATE OPERATOR = (
LEFTARG = test_udt,
RIGHTARG = test_udt,
@ -104,13 +85,6 @@ CREATE OPERATOR = (
);
-- ... and create a custom operator family for hash indexes...
CREATE OPERATOR FAMILY tudt_op_fam USING hash;
-- ... create a test HASH function. Though it is a poor hash function,
-- it is acceptable for our tests
CREATE FUNCTION test_udt_hash(test_udt) RETURNS int
AS 'SELECT hashtext( ($1.i + $1.i2)::text);'
LANGUAGE SQL
IMMUTABLE
RETURNS NULL ON NULL INPUT;
-- We need to define two different operator classes for the composite types
-- One uses BTREE the other uses HASH
CREATE OPERATOR CLASS tudt_op_fam_clas3

View File

@ -13,28 +13,6 @@ LANGUAGE sql IMMUTABLE AS $_$
ORDER BY 1 LIMIT 2 - MOD(array_upper($1, 1), 2)
OFFSET CEIL(array_upper($1, 1) / 2.0) - 1) sub;
$_$;
\c - - :public_worker_1_host :worker_1_port
DROP FUNCTION IF EXISTS median(double precision[]);
NOTICE: function median(pg_catalog.float8[]) does not exist, skipping
CREATE FUNCTION median(double precision[]) RETURNS double precision
LANGUAGE sql IMMUTABLE AS $_$
SELECT AVG(val) FROM
(SELECT val FROM unnest($1) val
ORDER BY 1 LIMIT 2 - MOD(array_upper($1, 1), 2)
OFFSET CEIL(array_upper($1, 1) / 2.0) - 1) sub;
$_$;
\c - - :public_worker_2_host :worker_2_port
DROP FUNCTION IF EXISTS median(double precision[]);
NOTICE: function median(pg_catalog.float8[]) does not exist, skipping
CREATE FUNCTION median(double precision[]) RETURNS double precision
LANGUAGE sql IMMUTABLE AS $_$
SELECT AVG(val) FROM
(SELECT val FROM unnest($1) val
ORDER BY 1 LIMIT 2 - MOD(array_upper($1, 1), 2)
OFFSET CEIL(array_upper($1, 1) / 2.0) - 1) sub;
$_$;
-- Run query on master
\c - - :master_host :master_port
SELECT * FROM (SELECT median(ARRAY[1,2,sum(l_suppkey)]) as median, count(*)
FROM lineitem GROUP BY l_partkey) AS a
WHERE median > 2;

View File

@ -1584,6 +1584,7 @@ DEBUG: query has a single distribution column value: 1
-- if these queries get routed, they would fail since number1() does not exist
-- on workers. This tests an exceptional case in which some local tables bypass
-- checks.
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION number1(OUT datid int)
RETURNS SETOF int
AS $$
@ -1592,6 +1593,7 @@ BEGIN
RETURN QUERY SELECT 1;
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
SELECT 1 FROM authors_reference r JOIN (
SELECT s.datid FROM number1() s LEFT JOIN pg_database d ON s.datid = d.oid
) num_db ON (r.id = num_db.datid) LIMIT 1;
@ -2297,6 +2299,7 @@ DEBUG: query has a single distribution column value: 1
(5 rows)
-- queries inside plpgsql functions could be router plannable
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION author_articles_max_id() RETURNS int AS $$
DECLARE
max_id integer;
@ -2332,6 +2335,7 @@ BEGIN
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
SELECT * FROM author_articles_id_word_count() ORDER BY 1;
DEBUG: Creating router plan
CONTEXT: SQL statement "SELECT ah.id, ah.word_count

View File

@ -1225,6 +1225,7 @@ DEBUG: query has a single distribution column value: 1
(0 rows)
-- create a dummy function to be used in filtering
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION someDummyFunction(regclass)
RETURNS text AS
$$
@ -1247,6 +1248,7 @@ BEGIN
END IF;
END;
$$LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
SET client_min_messages TO ERROR;
\set VERBOSITY terse
-- fast path router plannable, but errors
@ -1601,6 +1603,7 @@ DEBUG: Deferred pruning for a fast-path router query
DEBUG: Creating router plan
EXECUTE author_articles_update(NULL);
-- queries inside plpgsql functions could be router plannable
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION author_articles_max_id() RETURNS int AS $$
DECLARE
max_id integer;
@ -1611,6 +1614,7 @@ BEGIN
return max_id;
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
-- we don't want too many details. though we're omitting
-- "DETAIL: distribution column value:", we see it acceptable
-- since the query results verifies the correctness
@ -1655,6 +1659,7 @@ SELECT author_articles_max_id();
(1 row)
-- queries inside plpgsql functions could be router plannable
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION author_articles_max_id(int) RETURNS int AS $$
DECLARE
max_id integer;
@ -1665,6 +1670,7 @@ BEGIN
return max_id;
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
SELECT author_articles_max_id(1);
DEBUG: Deferred pruning for a fast-path router query
DEBUG: Creating router plan
@ -1714,6 +1720,7 @@ DEBUG: Creating router plan
(1 row)
-- check that function returning setof query are router plannable
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION author_articles_id_word_count() RETURNS TABLE(id bigint, word_count int) AS $$
DECLARE
BEGIN
@ -1724,6 +1731,7 @@ BEGIN
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
SELECT * FROM author_articles_id_word_count();
DEBUG: Distributed planning for a fast-path router query
DEBUG: Creating router plan
@ -1788,6 +1796,7 @@ SELECT * FROM author_articles_id_word_count();
(5 rows)
-- check that function returning setof query are router plannable
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION author_articles_id_word_count(int) RETURNS TABLE(id bigint, word_count int) AS $$
DECLARE
BEGIN
@ -1798,6 +1807,7 @@ BEGIN
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
SELECT * FROM author_articles_id_word_count(1);
DEBUG: Deferred pruning for a fast-path router query
DEBUG: Creating router plan

View File

@ -194,31 +194,6 @@ BEGIN
END;
$$
LANGUAGE 'plpgsql' IMMUTABLE;
-- create UDF in worker node 1
\c - - - :worker_1_port
CREATE OR REPLACE FUNCTION dummyFunction(theValue integer)
RETURNS text AS
$$
DECLARE
strresult text;
BEGIN
RETURN theValue * 3 / 2 + 1;
END;
$$
LANGUAGE 'plpgsql' IMMUTABLE;
-- create UDF in worker node 2
\c - - - :worker_2_port
CREATE OR REPLACE FUNCTION dummyFunction(theValue integer)
RETURNS text AS
$$
DECLARE
strresult text;
BEGIN
RETURN theValue * 3 / 2 + 1;
END;
$$
LANGUAGE 'plpgsql' IMMUTABLE;
\c - - - :master_port
-- UDF in public, table in a schema other than public, search_path is not set
SELECT dummyFunction(n_nationkey) FROM test_schema_support.nation_hash GROUP BY 1 ORDER BY 1;
dummyfunction
@ -260,33 +235,6 @@ BEGIN
END;
$$
LANGUAGE 'plpgsql' IMMUTABLE;
-- create UDF in worker node 1 in schema
\c - - - :worker_1_port
SET search_path TO test_schema_support;
CREATE OR REPLACE FUNCTION dummyFunction2(theValue integer)
RETURNS text AS
$$
DECLARE
strresult text;
BEGIN
RETURN theValue * 3 / 2 + 1;
END;
$$
LANGUAGE 'plpgsql' IMMUTABLE;
-- create UDF in worker node 2 in schema
\c - - - :worker_2_port
SET search_path TO test_schema_support;
CREATE OR REPLACE FUNCTION dummyFunction2(theValue integer)
RETURNS text AS
$$
DECLARE
strresult text;
BEGIN
RETURN theValue * 3 / 2 + 1;
END;
$$
LANGUAGE 'plpgsql' IMMUTABLE;
\c - - - :master_port
-- UDF in schema, table in a schema other than public, search_path is not set
SET search_path TO public;
SELECT test_schema_support.dummyFunction2(n_nationkey) FROM test_schema_support.nation_hash GROUP BY 1 ORDER BY 1;

View File

@ -292,11 +292,13 @@ SET citus.override_table_visibility TO false;
(1 row)
-- test DROP table with failing worker
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION fail_drop_table() RETURNS event_trigger AS $fdt$
BEGIN
RAISE 'illegal value';
END;
$fdt$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE EVENT TRIGGER fail_drop_table ON sql_drop EXECUTE PROCEDURE fail_drop_table();
\c - - - :master_port
\set VERBOSITY terse

View File

@ -62,6 +62,7 @@ SELECT * FROM mx_table ORDER BY col_1;
-- Try commands from metadata worker
\c - - - :worker_1_port
-- this function is dropped in Citus10, added here for tests
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass,
distribution_column text,
distribution_method citus.distribution_type)
@ -78,6 +79,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name tex
RETURNS void
AS 'citus', $$master_create_worker_shards$$
LANGUAGE C STRICT;
RESET citus.enable_metadata_sync;
CREATE TABLE mx_table_worker(col_1 text);
-- master_create_distributed_table
SELECT master_create_distributed_table('mx_table_worker', 'col_1', 'hash');

View File

@ -186,22 +186,6 @@ CREATE FUNCTION dintdict_lexize(internal, internal, internal, internal) RETURNS
CREATE TEXT SEARCH TEMPLATE intdict_template (LEXIZE = dintdict_lexize, INIT = dintdict_init );
CREATE TEXT SEARCH DICTIONARY intdict (TEMPLATE = intdict_template);
COMMENT ON TEXT SEARCH DICTIONARY intdict IS 'dictionary for integers';
SELECT run_command_on_workers($$
CREATE FUNCTION dintdict_init(internal) RETURNS internal AS 'dict_int.so' LANGUAGE C STRICT;
$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"CREATE FUNCTION")
(1 row)
SELECT run_command_on_workers($$
CREATE FUNCTION dintdict_lexize(internal, internal, internal, internal) RETURNS internal AS 'dict_int.so' LANGUAGE C STRICT;
$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"CREATE FUNCTION")
(1 row)
SELECT run_command_on_workers($$
CREATE TEXT SEARCH TEMPLATE intdict_template (LEXIZE = dintdict_lexize, INIT = dintdict_init );
$$);

View File

@ -186,22 +186,6 @@ CREATE FUNCTION dintdict_lexize(internal, internal, internal, internal) RETURNS
CREATE TEXT SEARCH TEMPLATE intdict_template (LEXIZE = dintdict_lexize, INIT = dintdict_init );
CREATE TEXT SEARCH DICTIONARY intdict (TEMPLATE = intdict_template);
COMMENT ON TEXT SEARCH DICTIONARY intdict IS 'dictionary for integers';
SELECT run_command_on_workers($$
CREATE FUNCTION dintdict_init(internal) RETURNS internal AS 'dict_int.so' LANGUAGE C STRICT;
$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"CREATE FUNCTION")
(1 row)
SELECT run_command_on_workers($$
CREATE FUNCTION dintdict_lexize(internal, internal, internal, internal) RETURNS internal AS 'dict_int.so' LANGUAGE C STRICT;
$$);
run_command_on_workers
---------------------------------------------------------------------
(localhost,57637,t,"CREATE FUNCTION")
(1 row)
SELECT run_command_on_workers($$
CREATE TEXT SEARCH TEMPLATE intdict_template (LEXIZE = dintdict_lexize, INIT = dintdict_init );
$$);

View File

@ -43,11 +43,13 @@ SELECT create_reference_table('numbers');
INSERT INTO numbers VALUES (20), (21);
NOTICE: executing the command locally: INSERT INTO replicate_ref_to_coordinator.numbers_8000001 AS citus_table_alias (a) VALUES (20), (21)
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION my_volatile_fn()
RETURNS INT AS $$
BEGIN
RETURN 1;
END; $$ language plpgsql VOLATILE;
RESET citus.enable_metadata_sync;
-- INSERT ... SELECT between reference tables
BEGIN;
EXPLAIN (COSTS OFF) INSERT INTO squares SELECT a, a*a FROM numbers;

View File

@ -241,6 +241,7 @@ RETURNS boolean
AS 'citus'
LANGUAGE C STRICT VOLATILE;
-- this function is dropped in Citus10, added here for tests
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass,
distribution_column text,
distribution_method citus.distribution_type)
@ -257,6 +258,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name tex
RETURNS void
AS 'citus', $$master_create_worker_shards$$
LANGUAGE C STRICT;
RESET citus.enable_metadata_sync;
SET citus.next_shard_id TO 123000;
SELECT worker_node_responsive(node_name, node_port::int)
FROM master_get_active_worker_nodes()

View File

@ -53,6 +53,7 @@ SELECT create_distributed_table('tr', 'pk');
INSERT INTO tr SELECT c, c FROM generate_series(1, 5) as c;
-- this function is dropped in Citus10, added here for tests
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass,
distribution_column text,
distribution_method citus.distribution_type)
@ -69,6 +70,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name tex
RETURNS void
AS 'citus', $$master_create_worker_shards$$
LANGUAGE C STRICT;
RESET citus.enable_metadata_sync;
CREATE TABLE t_range(id int, value_1 int);
SELECT create_distributed_table('t_range', 'id', 'range');
create_distributed_table

View File

@ -294,11 +294,13 @@ ALTER TABLE lineitem_alter_220000 DROP COLUMN first;
CREATE TABLE ddl_commands (command text UNIQUE DEFERRABLE INITIALLY DEFERRED);
-- Use an event trigger to log all DDL event tags in it
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION log_ddl_tag() RETURNS event_trigger AS $ldt$
BEGIN
INSERT INTO ddl_commands VALUES (tg_tag);
END;
$ldt$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE EVENT TRIGGER log_ddl_tag ON ddl_command_end EXECUTE PROCEDURE log_ddl_tag();
@ -499,13 +501,14 @@ CREATE TABLE trigger_table (
SELECT create_distributed_table('trigger_table', 'id');
-- first set a trigger on a shard
\c - - - :worker_1_port
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION update_value() RETURNS trigger AS $up$
BEGIN
NEW.value := 'trigger enabled';
RETURN NEW;
END;
$up$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE TRIGGER update_value
BEFORE INSERT ON trigger_table_220017

View File

@ -642,11 +642,13 @@ ALTER TABLE lineitem_alter_220000 DROP COLUMN first;
-- Create table to trigger at-xact-end (deferred) failure
CREATE TABLE ddl_commands (command text UNIQUE DEFERRABLE INITIALLY DEFERRED);
-- Use an event trigger to log all DDL event tags in it
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION log_ddl_tag() RETURNS event_trigger AS $ldt$
BEGIN
INSERT INTO ddl_commands VALUES (tg_tag);
END;
$ldt$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE EVENT TRIGGER log_ddl_tag ON ddl_command_end EXECUTE PROCEDURE log_ddl_tag();
\c - - - :master_port
-- The above trigger will cause failure at transaction end on one placement.
@ -985,12 +987,14 @@ SELECT create_distributed_table('trigger_table', 'id');
-- first set a trigger on a shard
\c - - - :worker_1_port
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION update_value() RETURNS trigger AS $up$
BEGIN
NEW.value := 'trigger enabled';
RETURN NEW;
END;
$up$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE TRIGGER update_value
BEFORE INSERT ON trigger_table_220017
FOR EACH ROW EXECUTE PROCEDURE update_value();

View File

@ -635,6 +635,8 @@ INSERT INTO citus.pg_dist_object(classid, objid, objsubid) values('pg_class'::re
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);
SELECT 1 FROM master_activate_node('localhost', :worker_1_port);
NOTICE: Replicating postgres objects to node localhost:57637
DETAIL: There are 111 objects to replicate, depending on your environment this might take a while
?column?
---------------------------------------------------------------------
1

View File

@ -373,9 +373,11 @@ inserts AS (
-- a helper function which return true if the coordinated
-- trannsaction uses 2PC
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION coordinated_transaction_should_use_2PC()
RETURNS BOOL LANGUAGE C STRICT VOLATILE AS 'citus',
$$coordinated_transaction_should_use_2PC$$;
RESET citus.enable_metadata_sync;
-- a local SELECT followed by remote SELECTs
-- does not trigger 2PC

View File

@ -462,6 +462,7 @@ SELECT * FROM test ORDER BY id;
DROP TABLE test;
-- verify that recreating distributed functions with TABLE params gets propagated to workers
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION func_with_return_table(int)
RETURNS TABLE (date date)
LANGUAGE plpgsql AS $$
@ -469,9 +470,11 @@ BEGIN
RETURN query SELECT '2011-01-01'::date;
END;
$$;
RESET citus.enable_metadata_sync;
SELECT create_distributed_function('func_with_return_table(int)');
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION func_with_return_table(int)
RETURNS TABLE (date date)
LANGUAGE plpgsql AS $$
@ -479,6 +482,7 @@ BEGIN
RETURN query SELECT '2011-01-02'::date;
END;
$$;
RESET citus.enable_metadata_sync;
SELECT count(*) FROM
(SELECT result FROM

View File

@ -6,6 +6,7 @@ SELECT run_command_on_workers($$CREATE SCHEMA proc_conflict;$$);
\c - - - :worker_1_port
SET search_path TO proc_conflict;
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION existing_func(state int, i int) RETURNS int AS $$
BEGIN
RETURN state * 2 + i;
@ -15,6 +16,7 @@ CREATE AGGREGATE existing_agg(int) (
SFUNC = existing_func,
STYPE = int
);
RESET citus.enable_metadata_sync;
\c - - - :master_port
SET search_path TO proc_conflict;
@ -59,7 +61,7 @@ DROP FUNCTION existing_func(int, int) CASCADE;
\c - - - :worker_1_port
SET search_path TO proc_conflict;
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION existing_func(state int, i int) RETURNS int AS $$
BEGIN
RETURN state * 3 + i;
@ -69,6 +71,7 @@ CREATE AGGREGATE existing_agg(int) (
SFUNC = existing_func,
STYPE = int
);
RESET citus.enable_metadata_sync;
\c - - - :master_port
SET search_path TO proc_conflict;

View File

@ -53,7 +53,6 @@ WHERE n.nspname IN ('drop_partitioned_table', 'schema1')
ORDER BY 1, 2;
\c - - - :worker_1_port
CREATE SCHEMA drop_partitioned_table;
SET search_path = drop_partitioned_table;
CREATE VIEW tables_info AS
SELECT n.nspname as "Schema",

View File

@ -101,11 +101,13 @@ EXECUTE p1(5,5,5);
EXECUTE p1(6,6,6);
EXECUTE p1(7,7,7);
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION modify_fast_path_plpsql(int, int) RETURNS void as $$
BEGIN
DELETE FROM modify_fast_path WHERE key = $1 AND value_1 = $2;
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
SELECT modify_fast_path_plpsql(1,1);
SELECT modify_fast_path_plpsql(2,2);

View File

@ -10,6 +10,7 @@
SET citus.next_shard_id TO 360000;
-- this function is dropped in Citus10, added here for tests
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass,
distribution_column text,
distribution_method citus.distribution_type)
@ -27,6 +28,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name tex
RETURNS void
AS 'citus', $$master_create_worker_shards$$
LANGUAGE C STRICT;
RESET citus.enable_metadata_sync;
CREATE TABLE lineitem (
l_orderkey bigint not null,

View File

@ -362,10 +362,7 @@ SET citus.enable_metadata_sync TO OFF;
SELECT create_distributed_function('func_custom_param(intpair)');
RESET citus.enable_metadata_sync;
SELECT deparse_and_run_on_workers($cmd$
ALTER FUNCTION func_custom_param RENAME TO func_with_custom_param;
$cmd$);
-- a function that returns TABLE
CREATE FUNCTION func_returns_table(IN count INT)
@ -376,9 +373,7 @@ SET citus.enable_metadata_sync TO OFF;
SELECT create_distributed_function('func_returns_table(INT)');
RESET citus.enable_metadata_sync;
SELECT deparse_and_run_on_workers($cmd$
ALTER FUNCTION func_returns_table ROWS 100;
$cmd$);
-- clear objects
SET client_min_messages TO WARNING; -- suppress cascading objects dropping

View File

@ -20,10 +20,12 @@ SELECT nextval('pg_catalog.pg_dist_groupid_seq') AS last_group_id \gset
SELECT nextval('pg_catalog.pg_dist_node_nodeid_seq') AS last_node_id \gset
-- Create the necessary test utility function
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION activate_node_snapshot()
RETURNS text[]
LANGUAGE C STRICT
AS 'citus';
RESET citus.enable_metadata_sync;
COMMENT ON FUNCTION activate_node_snapshot()
IS 'commands to activate node snapshot';
@ -43,12 +45,14 @@ ALTER ROLE CURRENT_USER WITH PASSWORD 'dummypassword';
SELECT unnest(activate_node_snapshot()) order by 1;
-- this function is dropped in Citus10, added here for tests
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass,
distribution_column text,
distribution_method citus.distribution_type)
RETURNS void
LANGUAGE C STRICT
AS 'citus', $$master_create_distributed_table$$;
RESET citus.enable_metadata_sync;
COMMENT ON FUNCTION pg_catalog.master_create_distributed_table(table_name regclass,
distribution_column text,
distribution_method citus.distribution_type)

View File

@ -363,16 +363,6 @@ UPDATE limit_orders SET array_of_values = 1 || array_of_values WHERE id = 246;
CREATE FUNCTION immutable_append(old_values int[], new_value int)
RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE;
\c - - - :worker_1_port
CREATE FUNCTION immutable_append(old_values int[], new_value int)
RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE;
\c - - - :worker_2_port
CREATE FUNCTION immutable_append(old_values int[], new_value int)
RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE;
\c - - - :master_port
-- immutable function calls with vars are also allowed
UPDATE limit_orders
SET array_of_values = immutable_append(array_of_values, 2) WHERE id = 246;

View File

@ -282,18 +282,16 @@ SELECT * FROM researchers WHERE lab_id = 6;
-- verify 2pc
SELECT count(*) FROM pg_dist_transaction;
-- create a check function
SELECT * from run_command_on_workers('CREATE FUNCTION reject_large_id() RETURNS trigger AS $rli$
CREATE FUNCTION reject_large_id() RETURNS trigger AS $rli$
BEGIN
IF (NEW.id > 30) THEN
RAISE ''illegal value'';
RAISE 'illegal value';
END IF;
RETURN NEW;
END;
$rli$ LANGUAGE plpgsql;')
ORDER BY nodeport;
$rli$ LANGUAGE plpgsql;
-- register after insert trigger
SELECT * FROM run_command_on_placements('researchers', 'CREATE CONSTRAINT TRIGGER reject_large_researcher_id AFTER INSERT ON %s DEFERRABLE INITIALLY DEFERRED FOR EACH ROW EXECUTE PROCEDURE reject_large_id()')
@ -344,9 +342,7 @@ SELECT * FROM researchers WHERE lab_id = 6;
SELECT * from run_command_on_placements('researchers', 'drop trigger reject_large_researcher_id on %s')
ORDER BY nodeport, shardid;
SELECT * FROM run_command_on_workers('drop function reject_large_id()')
ORDER BY nodeport;
DROP FUNCTION reject_large_id();
-- ALTER and copy are compatible
BEGIN;
ALTER TABLE labs ADD COLUMN motto text;
@ -416,6 +412,7 @@ AND s.logicalrelid = 'objects'::regclass;
-- create trigger on one worker to reject certain values
\c - - - :worker_2_port
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION reject_bad() RETURNS trigger AS $rb$
BEGIN
IF (NEW.name = 'BAD') THEN
@ -425,6 +422,7 @@ CREATE FUNCTION reject_bad() RETURNS trigger AS $rb$
RETURN NEW;
END;
$rb$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE CONSTRAINT TRIGGER reject_bad
AFTER INSERT ON objects_1200003
@ -460,6 +458,7 @@ DELETE FROM objects;
-- there cannot be errors on different shards at different times
-- because the first failure will fail the whole transaction
\c - - - :worker_1_port
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION reject_bad() RETURNS trigger AS $rb$
BEGIN
IF (NEW.name = 'BAD') THEN
@ -469,6 +468,7 @@ CREATE FUNCTION reject_bad() RETURNS trigger AS $rb$
RETURN NEW;
END;
$rb$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE CONSTRAINT TRIGGER reject_bad
AFTER INSERT ON labs_1200002
@ -668,7 +668,7 @@ SELECT * FROM reference_modifying_xacts;
-- lets fail on of the workers at before the commit time
\c - - - :worker_1_port
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION reject_bad_reference() RETURNS trigger AS $rb$
BEGIN
IF (NEW.key = 999) THEN
@ -678,6 +678,7 @@ CREATE FUNCTION reject_bad_reference() RETURNS trigger AS $rb$
RETURN NEW;
END;
$rb$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE CONSTRAINT TRIGGER reject_bad_reference
AFTER INSERT ON reference_modifying_xacts_1200006
@ -762,7 +763,7 @@ ABORT;
-- lets fail one of the workers before COMMIT time for the hash table
\c - - - :worker_1_port
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION reject_bad_hash() RETURNS trigger AS $rb$
BEGIN
IF (NEW.key = 997) THEN
@ -772,6 +773,7 @@ CREATE FUNCTION reject_bad_hash() RETURNS trigger AS $rb$
RETURN NEW;
END;
$rb$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE CONSTRAINT TRIGGER reject_bad_hash
AFTER INSERT ON hash_modifying_xacts_1200007

View File

@ -381,21 +381,24 @@ RESET ROLE;
-- to test access to files created during repartition we will create some on worker 1
\c - - - :worker_1_port
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION citus_rm_job_directory(bigint)
RETURNS void
AS 'citus'
LANGUAGE C STRICT;
RESET citus.enable_metadata_sync;
SET ROLE full_access;
SELECT worker_hash_partition_table(42,1,'SELECT a FROM generate_series(1,100) AS a', 'a', 23, ARRAY[-2147483648, -1073741824, 0, 1073741824]::int4[]);
RESET ROLE;
-- all attempts for transfer are initiated from other workers
\c - - - :worker_2_port
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION citus_rm_job_directory(bigint)
RETURNS void
AS 'citus'
LANGUAGE C STRICT;
RESET citus.enable_metadata_sync;
-- super user should not be able to copy files created by a user
SELECT worker_fetch_partition_file(42, 1, 1, 1, 'localhost', :worker_1_port);

View File

@ -7,6 +7,7 @@ ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 225000;
SET citus.shard_count TO 2;
-- this function is dropped in Citus10, added here for tests
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass,
distribution_column text,
distribution_method citus.distribution_type)
@ -24,6 +25,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name tex
RETURNS void
AS 'citus', $$master_create_worker_shards$$
LANGUAGE C STRICT;
RESET citus.enable_metadata_sync;
-- Verify that a table name > 56 characters gets hashed properly.
CREATE TABLE too_long_12345678901234567890123456789012345678901234567890 (

View File

@ -69,14 +69,7 @@ CREATE TABLE repartition_udt_other (
\c - - :public_worker_1_host :worker_1_port
-- START type creation
-- ... as well as a function to use as its comparator...
CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean
AS 'select $1.i = $2.i AND $1.i2 = $2.i2;'
LANGUAGE SQL
IMMUTABLE
RETURNS NULL ON NULL INPUT;
-- ... use that function to create a custom equality operator...
-- Use function to create a custom equality operator...
CREATE OPERATOR = (
LEFTARG = test_udt,
RIGHTARG = test_udt,
@ -88,15 +81,6 @@ CREATE OPERATOR = (
-- ... and create a custom operator family for hash indexes...
CREATE OPERATOR FAMILY tudt_op_fam USING hash;
-- ... create a test HASH function. Though it is a poor hash function,
-- it is acceptable for our tests
CREATE FUNCTION test_udt_hash(test_udt) RETURNS int
AS 'SELECT hashtext( ($1.i + $1.i2)::text);'
LANGUAGE SQL
IMMUTABLE
RETURNS NULL ON NULL INPUT;
-- We need to define two different operator classes for the composite types
-- One uses BTREE the other uses HASH
CREATE OPERATOR CLASS tudt_op_fam_clas3
@ -113,14 +97,7 @@ FUNCTION 1 test_udt_hash(test_udt);
\c - - :public_worker_2_host :worker_2_port
-- START type creation
-- ... as well as a function to use as its comparator...
CREATE FUNCTION equal_test_udt_function(test_udt, test_udt) RETURNS boolean
AS 'select $1.i = $2.i AND $1.i2 = $2.i2;'
LANGUAGE SQL
IMMUTABLE
RETURNS NULL ON NULL INPUT;
-- ... use that function to create a custom equality operator...
-- Use function to create a custom equality operator...
CREATE OPERATOR = (
LEFTARG = test_udt,
RIGHTARG = test_udt,
@ -132,15 +109,6 @@ CREATE OPERATOR = (
-- ... and create a custom operator family for hash indexes...
CREATE OPERATOR FAMILY tudt_op_fam USING hash;
-- ... create a test HASH function. Though it is a poor hash function,
-- it is acceptable for our tests
CREATE FUNCTION test_udt_hash(test_udt) RETURNS int
AS 'SELECT hashtext( ($1.i + $1.i2)::text);'
LANGUAGE SQL
IMMUTABLE
RETURNS NULL ON NULL INPUT;
-- We need to define two different operator classes for the composite types
-- One uses BTREE the other uses HASH
CREATE OPERATOR CLASS tudt_op_fam_clas3

View File

@ -18,32 +18,6 @@ LANGUAGE sql IMMUTABLE AS $_$
OFFSET CEIL(array_upper($1, 1) / 2.0) - 1) sub;
$_$;
\c - - :public_worker_1_host :worker_1_port
DROP FUNCTION IF EXISTS median(double precision[]);
CREATE FUNCTION median(double precision[]) RETURNS double precision
LANGUAGE sql IMMUTABLE AS $_$
SELECT AVG(val) FROM
(SELECT val FROM unnest($1) val
ORDER BY 1 LIMIT 2 - MOD(array_upper($1, 1), 2)
OFFSET CEIL(array_upper($1, 1) / 2.0) - 1) sub;
$_$;
\c - - :public_worker_2_host :worker_2_port
DROP FUNCTION IF EXISTS median(double precision[]);
CREATE FUNCTION median(double precision[]) RETURNS double precision
LANGUAGE sql IMMUTABLE AS $_$
SELECT AVG(val) FROM
(SELECT val FROM unnest($1) val
ORDER BY 1 LIMIT 2 - MOD(array_upper($1, 1), 2)
OFFSET CEIL(array_upper($1, 1) / 2.0) - 1) sub;
$_$;
-- Run query on master
\c - - :master_host :master_port
SELECT * FROM (SELECT median(ARRAY[1,2,sum(l_suppkey)]) as median, count(*)
FROM lineitem GROUP BY l_partkey) AS a
WHERE median > 2;

View File

@ -723,6 +723,7 @@ INTERSECT
-- if these queries get routed, they would fail since number1() does not exist
-- on workers. This tests an exceptional case in which some local tables bypass
-- checks.
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION number1(OUT datid int)
RETURNS SETOF int
AS $$
@ -731,6 +732,7 @@ BEGIN
RETURN QUERY SELECT 1;
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
SELECT 1 FROM authors_reference r JOIN (
SELECT s.datid FROM number1() s LEFT JOIN pg_database d ON s.datid = d.oid
@ -1129,6 +1131,7 @@ PREPARE author_articles(int) as
EXECUTE author_articles(1);
-- queries inside plpgsql functions could be router plannable
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION author_articles_max_id() RETURNS int AS $$
DECLARE
max_id integer;
@ -1153,6 +1156,7 @@ BEGIN
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
SELECT * FROM author_articles_id_word_count() ORDER BY 1;

View File

@ -531,6 +531,7 @@ SELECT word_count, rank() OVER (PARTITION BY author_id ORDER BY word_count)
WHERE author_id = 1 and 1=0;
-- create a dummy function to be used in filtering
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION someDummyFunction(regclass)
RETURNS text AS
$$
@ -554,6 +555,7 @@ BEGIN
END IF;
END;
$$LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
SET client_min_messages TO ERROR;
\set VERBOSITY terse
@ -685,6 +687,7 @@ EXECUTE author_articles_update(NULL);
EXECUTE author_articles_update(NULL);
-- queries inside plpgsql functions could be router plannable
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION author_articles_max_id() RETURNS int AS $$
DECLARE
max_id integer;
@ -695,6 +698,7 @@ BEGIN
return max_id;
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
-- we don't want too many details. though we're omitting
-- "DETAIL: distribution column value:", we see it acceptable
@ -709,6 +713,7 @@ SELECT author_articles_max_id();
SELECT author_articles_max_id();
-- queries inside plpgsql functions could be router plannable
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION author_articles_max_id(int) RETURNS int AS $$
DECLARE
max_id integer;
@ -719,6 +724,7 @@ BEGIN
return max_id;
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
SELECT author_articles_max_id(1);
SELECT author_articles_max_id(1);
SELECT author_articles_max_id(1);
@ -727,6 +733,7 @@ SELECT author_articles_max_id(1);
SELECT author_articles_max_id(1);
-- check that function returning setof query are router plannable
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION author_articles_id_word_count() RETURNS TABLE(id bigint, word_count int) AS $$
DECLARE
BEGIN
@ -737,6 +744,7 @@ BEGIN
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
SELECT * FROM author_articles_id_word_count();
SELECT * FROM author_articles_id_word_count();
@ -746,6 +754,7 @@ SELECT * FROM author_articles_id_word_count();
SELECT * FROM author_articles_id_word_count();
-- check that function returning setof query are router plannable
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION author_articles_id_word_count(int) RETURNS TABLE(id bigint, word_count int) AS $$
DECLARE
BEGIN
@ -756,6 +765,7 @@ BEGIN
END;
$$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
SELECT * FROM author_articles_id_word_count(1);
SELECT * FROM author_articles_id_word_count(1);
SELECT * FROM author_articles_id_word_count(1);

View File

@ -203,34 +203,6 @@ END;
$$
LANGUAGE 'plpgsql' IMMUTABLE;
-- create UDF in worker node 1
\c - - - :worker_1_port
CREATE OR REPLACE FUNCTION dummyFunction(theValue integer)
RETURNS text AS
$$
DECLARE
strresult text;
BEGIN
RETURN theValue * 3 / 2 + 1;
END;
$$
LANGUAGE 'plpgsql' IMMUTABLE;
-- create UDF in worker node 2
\c - - - :worker_2_port
CREATE OR REPLACE FUNCTION dummyFunction(theValue integer)
RETURNS text AS
$$
DECLARE
strresult text;
BEGIN
RETURN theValue * 3 / 2 + 1;
END;
$$
LANGUAGE 'plpgsql' IMMUTABLE;
\c - - - :master_port
-- UDF in public, table in a schema other than public, search_path is not set
SELECT dummyFunction(n_nationkey) FROM test_schema_support.nation_hash GROUP BY 1 ORDER BY 1;
@ -251,36 +223,6 @@ END;
$$
LANGUAGE 'plpgsql' IMMUTABLE;
-- create UDF in worker node 1 in schema
\c - - - :worker_1_port
SET search_path TO test_schema_support;
CREATE OR REPLACE FUNCTION dummyFunction2(theValue integer)
RETURNS text AS
$$
DECLARE
strresult text;
BEGIN
RETURN theValue * 3 / 2 + 1;
END;
$$
LANGUAGE 'plpgsql' IMMUTABLE;
-- create UDF in worker node 2 in schema
\c - - - :worker_2_port
SET search_path TO test_schema_support;
CREATE OR REPLACE FUNCTION dummyFunction2(theValue integer)
RETURNS text AS
$$
DECLARE
strresult text;
BEGIN
RETURN theValue * 3 / 2 + 1;
END;
$$
LANGUAGE 'plpgsql' IMMUTABLE;
\c - - - :master_port
-- UDF in schema, table in a schema other than public, search_path is not set
SET search_path TO public;
SELECT test_schema_support.dummyFunction2(n_nationkey) FROM test_schema_support.nation_hash GROUP BY 1 ORDER BY 1;

View File

@ -168,11 +168,13 @@ SET citus.override_table_visibility TO false;
-- test DROP table with failing worker
SET citus.enable_metadata_sync TO OFF;
CREATE FUNCTION fail_drop_table() RETURNS event_trigger AS $fdt$
BEGIN
RAISE 'illegal value';
END;
$fdt$ LANGUAGE plpgsql;
RESET citus.enable_metadata_sync;
CREATE EVENT TRIGGER fail_drop_table ON sql_drop EXECUTE PROCEDURE fail_drop_table();

View File

@ -51,6 +51,7 @@ SELECT * FROM mx_table ORDER BY col_1;
\c - - - :worker_1_port
-- this function is dropped in Citus10, added here for tests
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass,
distribution_column text,
distribution_method citus.distribution_type)
@ -68,6 +69,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name tex
RETURNS void
AS 'citus', $$master_create_worker_shards$$
LANGUAGE C STRICT;
RESET citus.enable_metadata_sync;
CREATE TABLE mx_table_worker(col_1 text);

View File

@ -116,13 +116,6 @@ CREATE FUNCTION dintdict_lexize(internal, internal, internal, internal) RETURNS
CREATE TEXT SEARCH TEMPLATE intdict_template (LEXIZE = dintdict_lexize, INIT = dintdict_init );
CREATE TEXT SEARCH DICTIONARY intdict (TEMPLATE = intdict_template);
COMMENT ON TEXT SEARCH DICTIONARY intdict IS 'dictionary for integers';
SELECT run_command_on_workers($$
CREATE FUNCTION dintdict_init(internal) RETURNS internal AS 'dict_int.so' LANGUAGE C STRICT;
$$);
SELECT run_command_on_workers($$
CREATE FUNCTION dintdict_lexize(internal, internal, internal, internal) RETURNS internal AS 'dict_int.so' LANGUAGE C STRICT;
$$);
SELECT run_command_on_workers($$
CREATE TEXT SEARCH TEMPLATE intdict_template (LEXIZE = dintdict_lexize, INIT = dintdict_init );

View File

@ -32,12 +32,13 @@ CREATE TABLE numbers(a int);
SELECT create_reference_table('numbers');
INSERT INTO numbers VALUES (20), (21);
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION my_volatile_fn()
RETURNS INT AS $$
BEGIN
RETURN 1;
END; $$ language plpgsql VOLATILE;
RESET citus.enable_metadata_sync;
-- INSERT ... SELECT between reference tables
BEGIN;

View File

@ -137,6 +137,7 @@ AS 'citus'
LANGUAGE C STRICT VOLATILE;
-- this function is dropped in Citus10, added here for tests
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass,
distribution_column text,
distribution_method citus.distribution_type)
@ -154,6 +155,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name tex
RETURNS void
AS 'citus', $$master_create_worker_shards$$
LANGUAGE C STRICT;
RESET citus.enable_metadata_sync;
SET citus.next_shard_id TO 123000;

View File

@ -34,6 +34,7 @@ CREATE TABLE tr(pk int, a int REFERENCES r(a) ON DELETE CASCADE ON UPDATE CASCAD
SELECT create_distributed_table('tr', 'pk');
INSERT INTO tr SELECT c, c FROM generate_series(1, 5) as c;
-- this function is dropped in Citus10, added here for tests
SET citus.enable_metadata_sync TO OFF;
CREATE OR REPLACE FUNCTION pg_catalog.master_create_distributed_table(table_name regclass,
distribution_column text,
distribution_method citus.distribution_type)
@ -50,6 +51,7 @@ CREATE OR REPLACE FUNCTION pg_catalog.master_create_worker_shards(table_name tex
RETURNS void
AS 'citus', $$master_create_worker_shards$$
LANGUAGE C STRICT;
RESET citus.enable_metadata_sync;
CREATE TABLE t_range(id int, value_1 int);
SELECT create_distributed_table('t_range', 'id', 'range');
SELECT master_create_empty_shard('t_range') as shardid1 \gset