diff --git a/src/backend/distributed/executor/local_executor.c b/src/backend/distributed/executor/local_executor.c index 5dce24ac7..0b590e629 100644 --- a/src/backend/distributed/executor/local_executor.c +++ b/src/backend/distributed/executor/local_executor.c @@ -71,8 +71,6 @@ * There are also a few limitations/trade-offs that are worth mentioning. * - The local execution on multiple shards might be slow because the execution * has to happen one task at a time (e.g., no parallelism). - * - The local execution cannot be mixed with the executors other than adaptive, - * namely task-tracker executor. * - Related with the previous item, COPY command cannot be mixed with local * execution in a transaction. The implication of that is any part of INSERT..SELECT * via coordinator cannot happen via the local execution. diff --git a/src/backend/distributed/executor/multi_server_executor.c b/src/backend/distributed/executor/multi_server_executor.c index cbe7a752b..4c60d8342 100644 --- a/src/backend/distributed/executor/multi_server_executor.c +++ b/src/backend/distributed/executor/multi_server_executor.c @@ -3,8 +3,8 @@ * multi_server_executor.c * * Function definitions for distributed task execution for adaptive - * and task-tracker executors, and routines common to both. The common - * routines are implement backend-side logic; and they trigger executions + * executor. + * routines are implemented backend-side logic; and they trigger executions * on the client-side via function hooks that they load. * * Copyright (c) Citus Data, Inc. diff --git a/src/backend/distributed/planner/distributed_planner.c b/src/backend/distributed/planner/distributed_planner.c index d2b3c4734..8ae4fe6ab 100644 --- a/src/backend/distributed/planner/distributed_planner.c +++ b/src/backend/distributed/planner/distributed_planner.c @@ -1357,8 +1357,7 @@ FinalizePlan(PlannedStmt *localPlan, DistributedPlan *distributedPlan) /* * FinalizeNonRouterPlan gets the distributed custom scan plan, and creates the - * final master select plan on the top of this distributed plan for adaptive - * and task-tracker executors. + * final master select plan on the top of this distributed plan for adaptive executor. */ static PlannedStmt * FinalizeNonRouterPlan(PlannedStmt *localPlan, DistributedPlan *distributedPlan, diff --git a/src/backend/distributed/planner/multi_physical_planner.c b/src/backend/distributed/planner/multi_physical_planner.c index b36d574c6..b0f1603df 100644 --- a/src/backend/distributed/planner/multi_physical_planner.c +++ b/src/backend/distributed/planner/multi_physical_planner.c @@ -4611,9 +4611,7 @@ MergeTaskList(MapMergeJob *mapMergeJob, List *mapTaskList, uint32 taskIdIndex) /* * If replication factor is 1, then we know that we will use the first and - * the only placement. If task tracker is used, then it will regenerate the - * query string because if there are multiple placements then it does not - * know in which placement the parent map task was successful. + * the only placement. */ StringInfo mapFetchQueryString = makeStringInfo(); appendStringInfo(mapFetchQueryString, MAP_OUTPUT_FETCH_COMMAND, diff --git a/src/backend/distributed/planner/recursive_planning.c b/src/backend/distributed/planner/recursive_planning.c index cd0ab3c28..71f396fce 100644 --- a/src/backend/distributed/planner/recursive_planning.c +++ b/src/backend/distributed/planner/recursive_planning.c @@ -25,9 +25,6 @@ * subqueries if the subquery can be executed on each shard by replacing * table names with shard names and concatenating the result. * - * - Task-tracker queries that can be executed through a tree of - * re-partitioning operations. - * * These queries have very limited SQL support and only support basic * inner joins and subqueries without joins. * diff --git a/src/backend/distributed/worker/task_tracker_protocol.c b/src/backend/distributed/worker/task_tracker_protocol.c index eade29081..fdf748de5 100644 --- a/src/backend/distributed/worker/task_tracker_protocol.c +++ b/src/backend/distributed/worker/task_tracker_protocol.c @@ -2,10 +2,8 @@ * * task_tracker_protocol.c * - * The task tracker background process runs on every worker node. The following - * routines allow for the master node to assign tasks to the task tracker, check - * these tasks' statuses, and remove these tasks when they are no longer needed. - * + * The methods in the file are deprecated. + * * Copyright (c) Citus Data, Inc. * * $Id$ @@ -25,11 +23,7 @@ PG_FUNCTION_INFO_V1(task_tracker_cleanup_job); PG_FUNCTION_INFO_V1(task_tracker_conninfo_cache_invalidate); -/* - * task_tracker_assign_task creates a new task in the shared hash or updates an - * already existing task. The function also creates a schema for the job if it - * doesn't already exist. - */ +/* This UDF is deprecated.*/ Datum task_tracker_assign_task(PG_FUNCTION_ARGS) { @@ -39,7 +33,7 @@ task_tracker_assign_task(PG_FUNCTION_ARGS) } -/* Returns the task status of an already existing task. */ +/* This UDF is deprecated.*/ Datum task_tracker_task_status(PG_FUNCTION_ARGS) { @@ -49,10 +43,7 @@ task_tracker_task_status(PG_FUNCTION_ARGS) } -/* - * task_tracker_cleanup_job finds all tasks for the given job, and cleans up - * files, connections, and shared hash enties associated with these tasks. - */ +/* This UDF is deprecated.*/ Datum task_tracker_cleanup_job(PG_FUNCTION_ARGS) { @@ -62,13 +53,7 @@ task_tracker_cleanup_job(PG_FUNCTION_ARGS) } -/* - * task_tracker_conninfo_cache_invalidate is a trigger function that signals to - * the task tracker to refresh its conn params cache after an authinfo change. - * - * NB: We decided there is little point in checking permissions here, there - * are much easier ways to waste CPU than causing cache invalidations. - */ +/* This UDF is deprecated.*/ Datum task_tracker_conninfo_cache_invalidate(PG_FUNCTION_ARGS) { diff --git a/src/backend/distributed/worker/worker_merge_protocol.c b/src/backend/distributed/worker/worker_merge_protocol.c index 2ca6552b1..1b97337f7 100644 --- a/src/backend/distributed/worker/worker_merge_protocol.c +++ b/src/backend/distributed/worker/worker_merge_protocol.c @@ -163,7 +163,7 @@ worker_repartition_cleanup(PG_FUNCTION_ARGS) /* * worker_merge_files_into_table creates a task table within the job's schema, - * which should have already been created by the task tracker protocol, and + * which should have already been created by repartition join execution, and * copies files in its task directory into this table. If the schema doesn't * exist, the function defaults to the 'public' schema. Note that, unlike * partitioning functions, this function is not always idempotent. On success, @@ -200,8 +200,8 @@ worker_merge_files_into_table(PG_FUNCTION_ARGS) } /* - * If the schema for the job isn't already created by the task tracker - * protocol, we fall to using the default 'public' schema. + * If the schema for the job isn't already created by the repartition join + * execution, we fall to using the default 'public' schema. */ bool schemaExists = JobSchemaExists(jobSchemaName); if (!schemaExists) @@ -245,18 +245,7 @@ worker_merge_files_into_table(PG_FUNCTION_ARGS) } -/* - * worker_merge_files_and_run_query creates a merge task table within the job's - * schema, which should have already been created by the task tracker protocol. - * It copies files in its task directory into this table. Then it runs final - * query to create result table of the job. - * - * Note that here we followed a different approach to create a task table for merge - * files than worker_merge_files_into_table(). In future we should unify these - * two approaches. For this purpose creating a directory_fdw extension and using - * it would make sense. Then we can merge files with a query or without query - * through directory_fdw. - */ +/* This UDF is deprecated.*/ Datum worker_merge_files_and_run_query(PG_FUNCTION_ARGS) { diff --git a/src/backend/distributed/worker/worker_partition_protocol.c b/src/backend/distributed/worker/worker_partition_protocol.c index 95854932c..b3c0d9512 100644 --- a/src/backend/distributed/worker/worker_partition_protocol.c +++ b/src/backend/distributed/worker/worker_partition_protocol.c @@ -372,10 +372,6 @@ ArrayObjectCount(ArrayType *arrayObject) StringInfo InitTaskDirectory(uint64 jobId, uint32 taskId) { - /* - * If the task tracker assigned this task (regular case), the tracker should - * have already created the job directory. - */ StringInfo jobDirectoryName = JobDirectoryName(jobId); StringInfo taskDirectoryName = TaskDirectoryName(jobId, taskId); diff --git a/src/backend/distributed/worker/worker_sql_task_protocol.c b/src/backend/distributed/worker/worker_sql_task_protocol.c index 8cbac012a..28f8cbb26 100644 --- a/src/backend/distributed/worker/worker_sql_task_protocol.c +++ b/src/backend/distributed/worker/worker_sql_task_protocol.c @@ -2,7 +2,7 @@ * * worker_sql_task_protocol.c * - * Routines for executing SQL tasks during task-tracker execution. + * Routines for executing SQL tasks. * * Copyright (c) Citus Data, Inc. * diff --git a/src/test/regress/Makefile b/src/test/regress/Makefile index 4b41d17ac..8be9fae39 100644 --- a/src/test/regress/Makefile +++ b/src/test/regress/Makefile @@ -42,7 +42,7 @@ output_files := $(patsubst $(citus_abs_srcdir)/output/%.source,expected/%.out, $ # intermediate, for muscle memory backward compatibility. check: check-full # check-full triggers all tests that ought to be run routinely -check-full: check-multi check-multi-mx check-multi-task-tracker-extra check-worker check-follower-cluster check-failure +check-full: check-multi check-multi-mx check-worker check-follower-cluster check-failure ISOLATION_DEPDIR=.deps/isolation diff --git a/src/test/regress/expected/local_shard_execution.out b/src/test/regress/expected/local_shard_execution.out index f83e12640..6f3c2b503 100644 --- a/src/test/regress/expected/local_shard_execution.out +++ b/src/test/regress/expected/local_shard_execution.out @@ -1401,7 +1401,7 @@ ROLLBACK; BEGIN; DELETE FROM distributed_table WHERE key = 500; NOTICE: executing the command locally: DELETE FROM local_shard_execution.distributed_table_1470003 distributed_table WHERE (key OPERATOR(pg_catalog.=) 500) - SET LOCAL citus.task_executor_type = 'task-tracker'; + SELECT count(*) FROM distributed_table; NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution.distributed_table_1470001 distributed_table WHERE true NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution.distributed_table_1470003 distributed_table WHERE true @@ -1414,7 +1414,7 @@ ROLLBACK; -- local execution should not be executed locally -- becase a task-tracker query has already been executed BEGIN; - SET LOCAL citus.task_executor_type = 'task-tracker'; + SET LOCAL client_min_messages TO INFO; SELECT count(*) FROM distributed_table; NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution.distributed_table_1470001 distributed_table WHERE true diff --git a/src/test/regress/expected/multi_cross_shard.out b/src/test/regress/expected/multi_cross_shard.out index 35becc66c..6489cbefb 100644 --- a/src/test/regress/expected/multi_cross_shard.out +++ b/src/test/regress/expected/multi_cross_shard.out @@ -145,7 +145,6 @@ SELECT * FROM raw_table WHERE ID = 1; 1 | 20 (2 rows) --- Task tracker query test CREATE TABLE tt1 ( id int, diff --git a/src/test/regress/expected/multi_join_order_additional.out b/src/test/regress/expected/multi_join_order_additional.out index 6dae378df..da0b0078b 100644 --- a/src/test/regress/expected/multi_join_order_additional.out +++ b/src/test/regress/expected/multi_join_order_additional.out @@ -5,7 +5,7 @@ SET citus.next_shard_id TO 650000; -- Set configuration to print table join order and pruned shards SET citus.explain_distributed_queries TO off; SET citus.log_multi_join_order TO TRUE; -SET citus.task_executor_type = 'task-tracker'; -- can't explain all queries otherwise +SET citus.task_executor_type = 'adaptive'; -- can't explain all queries otherwise SET citus.enable_repartition_joins to ON; SET citus.shard_count to 2; SET citus.shard_replication_factor to 1; diff --git a/src/test/regress/expected/multi_join_order_tpch_repartition.out b/src/test/regress/expected/multi_join_order_tpch_repartition.out index 5d7b5c2c1..d747f9f6b 100644 --- a/src/test/regress/expected/multi_join_order_tpch_repartition.out +++ b/src/test/regress/expected/multi_join_order_tpch_repartition.out @@ -5,7 +5,7 @@ SET citus.next_shard_id TO 660000; -- Enable configuration to print table join order SET citus.explain_distributed_queries TO off; SET citus.log_multi_join_order TO TRUE; -SET citus.task_executor_type = 'task-tracker'; -- can't explain all queries otherwise +SET citus.task_executor_type = 'adaptive'; -- can't explain all queries otherwise SET citus.enable_repartition_joins to ON; SET client_min_messages TO LOG; -- The following queries are basically the same as the ones in tpch_small diff --git a/src/test/regress/expected/multi_mx_repartition_udt_prepare.out b/src/test/regress/expected/multi_mx_repartition_udt_prepare.out index 9502e8fcc..0f68919db 100644 --- a/src/test/regress/expected/multi_mx_repartition_udt_prepare.out +++ b/src/test/regress/expected/multi_mx_repartition_udt_prepare.out @@ -152,7 +152,7 @@ INSERT INTO repartition_udt_other values (10, '(2,1)'::test_udt, 'foo'); INSERT INTO repartition_udt_other values (11, '(2,2)'::test_udt, 'foo'); INSERT INTO repartition_udt_other values (12, '(2,3)'::test_udt, 'foo'); SET client_min_messages = LOG; -SET citus.task_executor_type = 'task-tracker'; +SET citus.task_executor_type = 'adaptive'; SET citus.enable_repartition_joins to ON; -- Query that should result in a repartition -- join on int column, and be empty. diff --git a/src/test/regress/expected/multi_repartition_join_planning.out b/src/test/regress/expected/multi_repartition_join_planning.out index c20cbc2b8..fdabff35c 100644 --- a/src/test/regress/expected/multi_repartition_join_planning.out +++ b/src/test/regress/expected/multi_repartition_join_planning.out @@ -3,8 +3,7 @@ -- -- Tests that cover repartition join planning. Note that we explicitly start a -- transaction block here so that we don't emit debug messages with changing --- transaction ids in them. Also, we set the executor type to task tracker --- executor here, as we cannot run repartition jobs with real time executor. +-- transaction ids in them. SET citus.next_shard_id TO 690000; SET citus.enable_unique_job_ids TO off; SET citus.enable_repartition_joins to ON; diff --git a/src/test/regress/expected/multi_repartition_join_pruning.out b/src/test/regress/expected/multi_repartition_join_pruning.out index f0dced987..5bbbf60bd 100644 --- a/src/test/regress/expected/multi_repartition_join_pruning.out +++ b/src/test/regress/expected/multi_repartition_join_pruning.out @@ -1,9 +1,7 @@ -- -- MULTI_REPARTITION_JOIN_PRUNING -- --- Tests covering partition and join-pruning for repartition joins. Note that we --- set executor type to task tracker executor here, as we cannot run repartition --- jobs with real time executor. +-- Tests covering partition and join-pruning for repartition joins. SET citus.next_shard_id TO 700000; SET client_min_messages TO DEBUG2; SET citus.enable_repartition_joins to ON; diff --git a/src/test/regress/expected/multi_repartition_udt.out b/src/test/regress/expected/multi_repartition_udt.out index 4724ef887..29cb7fe0e 100644 --- a/src/test/regress/expected/multi_repartition_udt.out +++ b/src/test/regress/expected/multi_repartition_udt.out @@ -164,7 +164,7 @@ SELECT * FROM repartition_udt JOIN repartition_udt_other (0 rows) -- Query that should result in a repartition join on UDT column. -SET citus.task_executor_type = 'task-tracker'; +SET citus.task_executor_type = 'adaptive'; SET citus.log_multi_join_order = true; EXPLAIN (COSTS OFF) SELECT * FROM repartition_udt JOIN repartition_udt_other diff --git a/src/test/regress/expected/multi_simple_queries_0.out b/src/test/regress/expected/multi_simple_queries_0.out index d24ca2e1a..1f64bba84 100644 --- a/src/test/regress/expected/multi_simple_queries_0.out +++ b/src/test/regress/expected/multi_simple_queries_0.out @@ -215,12 +215,12 @@ SELECT * FROM articles, position('om' in 'Thomas') ORDER BY 2 DESC, 1 DESC, 3 DE -- subqueries are supported in WHERE clause in Citus even if the relations are not distributed SELECT * FROM articles WHERE author_id IN (SELECT id FROM authors WHERE name LIKE '%a'); -ERROR: Complex subqueries and CTEs are not supported when task_executor_type is set to 'task-tracker' +ERROR: Complex subqueries and CTEs are not supported when task_executor_type is set to 'adaptive' -- subqueries are supported in FROM clause SELECT articles.id,test.word_count FROM articles, (SELECT id, word_count FROM articles) AS test WHERE test.id = articles.id ORDER BY articles.id; -ERROR: Complex subqueries and CTEs are not supported when task_executor_type is set to 'task-tracker' +ERROR: Complex subqueries and CTEs are not supported when task_executor_type is set to 'adaptive' -- subqueries are not supported in SELECT clause SELECT a.title AS name, (SELECT a2.id FROM articles_single_shard a2 WHERE a.id = a2.id LIMIT 1) AS special_price FROM articles a; diff --git a/src/test/regress/expected/single_hash_repartition_join.out b/src/test/regress/expected/single_hash_repartition_join.out index 182dab1d2..6e12491a3 100644 --- a/src/test/regress/expected/single_hash_repartition_join.out +++ b/src/test/regress/expected/single_hash_repartition_join.out @@ -519,7 +519,7 @@ SELECT create_distributed_table('dist_1', 'a'); (1 row) INSERT INTO dist_1 SELECT x,10-x FROM generate_series(1,10) x; -SET citus.task_executor_type to 'task-tracker'; +SET citus.task_executor_type to 'adaptive'; SELECT COUNT(*) FROM dist_1 f, dist_1 s WHERE f.a = s.b; count --------------------------------------------------------------------- diff --git a/src/test/regress/expected/single_node.out b/src/test/regress/expected/single_node.out index 355c4b3a1..9d9b34610 100644 --- a/src/test/regress/expected/single_node.out +++ b/src/test/regress/expected/single_node.out @@ -266,8 +266,6 @@ SELECT count(*) FROM test WHERE false GROUP BY GROUPING SETS (x,y); (0 rows) RESET citus.task_assignment_policy; --- single node task tracker tests: -SET citus.task_executor_type to 'task-tracker'; SELECT count(*) FROM test; count --------------------------------------------------------------------- diff --git a/src/test/regress/expected/with_executors.out b/src/test/regress/expected/with_executors.out index 358fb7fb8..516474745 100644 --- a/src/test/regress/expected/with_executors.out +++ b/src/test/regress/expected/with_executors.out @@ -375,7 +375,7 @@ SELECT count(*) FROM cte, users_table where cte.count=user_id; 0 (1 row) -SET citus.task_executor_type='task-tracker'; + -- CTEs shouldn't be able to terminate a task-tracker query WITH cte_1 AS ( SELECT diff --git a/src/test/regress/sql/limit_intermediate_size.sql b/src/test/regress/sql/limit_intermediate_size.sql index adfabd1ae..6d4862443 100644 --- a/src/test/regress/sql/limit_intermediate_size.sql +++ b/src/test/regress/sql/limit_intermediate_size.sql @@ -16,7 +16,6 @@ SELECT cte.user_id, cte.value_2 FROM cte,cte2 ORDER BY 1,2 LIMIT 10; SET citus.max_intermediate_result_size TO 9; --- regular task-tracker CTE should fail WITH cte AS ( SELECT @@ -172,7 +171,7 @@ cte4 AS ( SELECT * FROM cte UNION ALL SELECT * FROM cte4 ORDER BY 1,2,3,4,5 LIMIT 5; --- regular task-tracker CTE, should work since -1 disables the limit +-- regular adaptive executor CTE, should work since -1 disables the limit WITH cte AS ( SELECT diff --git a/src/test/regress/sql/local_shard_execution.sql b/src/test/regress/sql/local_shard_execution.sql index 8f01a4485..6490a4fc0 100644 --- a/src/test/regress/sql/local_shard_execution.sql +++ b/src/test/regress/sql/local_shard_execution.sql @@ -695,19 +695,13 @@ BEGIN; ROLLBACK; --- task-tracker select execution BEGIN; DELETE FROM distributed_table WHERE key = 500; - SET LOCAL citus.task_executor_type = 'task-tracker'; - SELECT count(*) FROM distributed_table; ROLLBACK; --- local execution should not be executed locally --- becase a task-tracker query has already been executed BEGIN; - SET LOCAL citus.task_executor_type = 'task-tracker'; SET LOCAL client_min_messages TO INFO; SELECT count(*) FROM distributed_table; SET LOCAL client_min_messages TO LOG; diff --git a/src/test/regress/sql/multi_cross_shard.sql b/src/test/regress/sql/multi_cross_shard.sql index 5e716c932..fe5ce9eb5 100644 --- a/src/test/regress/sql/multi_cross_shard.sql +++ b/src/test/regress/sql/multi_cross_shard.sql @@ -90,7 +90,6 @@ SET citus.multi_task_query_log_level TO notice; -- Shouldn't log since it is a router select query SELECT * FROM raw_table WHERE ID = 1; --- Task tracker query test CREATE TABLE tt1 ( id int, @@ -113,7 +112,6 @@ INSERT INTO tt1 VALUES(2, 'Mehmet'); INSERT INTO tt2 VALUES(1, 'Ahmet', 5); INSERT INTO tt2 VALUES(2, 'Mehmet', 15); --- Should notice since it is a task-tracker query SELECT tt1.id, tt2.count from tt1,tt2 where tt1.id = tt2.id ORDER BY 1; SET citus.task_executor_type to DEFAULT; diff --git a/src/test/regress/sql/multi_join_order_additional.sql b/src/test/regress/sql/multi_join_order_additional.sql index 1d30900dd..641bfa74e 100644 --- a/src/test/regress/sql/multi_join_order_additional.sql +++ b/src/test/regress/sql/multi_join_order_additional.sql @@ -9,7 +9,6 @@ SET citus.next_shard_id TO 650000; SET citus.explain_distributed_queries TO off; SET citus.log_multi_join_order TO TRUE; -SET citus.task_executor_type = 'task-tracker'; -- can't explain all queries otherwise SET citus.enable_repartition_joins to ON; SET citus.shard_count to 2; SET citus.shard_replication_factor to 1; diff --git a/src/test/regress/sql/multi_join_order_tpch_repartition.sql b/src/test/regress/sql/multi_join_order_tpch_repartition.sql index 3060f8123..1524feaee 100644 --- a/src/test/regress/sql/multi_join_order_tpch_repartition.sql +++ b/src/test/regress/sql/multi_join_order_tpch_repartition.sql @@ -10,7 +10,6 @@ SET citus.next_shard_id TO 660000; SET citus.explain_distributed_queries TO off; SET citus.log_multi_join_order TO TRUE; -SET citus.task_executor_type = 'task-tracker'; -- can't explain all queries otherwise SET citus.enable_repartition_joins to ON; SET client_min_messages TO LOG; diff --git a/src/test/regress/sql/multi_mx_repartition_udt_prepare.sql b/src/test/regress/sql/multi_mx_repartition_udt_prepare.sql index bce22ac2c..ce0240ca4 100644 --- a/src/test/regress/sql/multi_mx_repartition_udt_prepare.sql +++ b/src/test/regress/sql/multi_mx_repartition_udt_prepare.sql @@ -182,7 +182,6 @@ INSERT INTO repartition_udt_other values (11, '(2,2)'::test_udt, 'foo'); INSERT INTO repartition_udt_other values (12, '(2,3)'::test_udt, 'foo'); SET client_min_messages = LOG; -SET citus.task_executor_type = 'task-tracker'; SET citus.enable_repartition_joins to ON; -- Query that should result in a repartition diff --git a/src/test/regress/sql/multi_mx_router_planner.sql b/src/test/regress/sql/multi_mx_router_planner.sql index 1644c2929..688a7c944 100644 --- a/src/test/regress/sql/multi_mx_router_planner.sql +++ b/src/test/regress/sql/multi_mx_router_planner.sql @@ -641,15 +641,6 @@ SET client_min_messages to 'DEBUG2'; CREATE MATERIALIZED VIEW mv_articles_hash_mx_error AS SELECT * FROM articles_hash_mx WHERE author_id in (1,2); --- router planner/executor is disabled for task-tracker executor --- following query is router plannable, but router planner is disabled - --- TODO: Uncomment once we fix task-tracker issue -----SELECT id --- FROM articles_hash_mx --- WHERE author_id = 1; - --- insert query is router plannable even under task-tracker INSERT INTO articles_hash_mx VALUES (51, 1, 'amateus', 1814); -- verify insert is successfull (not router plannable and executable) diff --git a/src/test/regress/sql/multi_partitioning.sql b/src/test/regress/sql/multi_partitioning.sql index e9d8e944d..c369da5d6 100644 --- a/src/test/regress/sql/multi_partitioning.sql +++ b/src/test/regress/sql/multi_partitioning.sql @@ -863,7 +863,6 @@ SELECT * FROM partitioning_locks ORDER BY 1, 2; SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; COMMIT; --- test locks on task-tracker SELECT BEGIN; SELECT * FROM partitioning_locks AS pl1 JOIN partitioning_locks AS pl2 ON pl1.id = pl2.ref_id ORDER BY 1, 2; SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; diff --git a/src/test/regress/sql/multi_prepare_plsql.sql b/src/test/regress/sql/multi_prepare_plsql.sql index 85542aaf3..a61be1880 100644 --- a/src/test/regress/sql/multi_prepare_plsql.sql +++ b/src/test/regress/sql/multi_prepare_plsql.sql @@ -362,8 +362,6 @@ SELECT real_time_partition_column_select(4); SELECT real_time_partition_column_select(5); SELECT real_time_partition_column_select(6); --- check task-tracker executor - CREATE FUNCTION task_tracker_non_partition_column_select(value_arg int) RETURNS TABLE(key int, value int) AS $$ DECLARE diff --git a/src/test/regress/sql/multi_prepare_sql.sql b/src/test/regress/sql/multi_prepare_sql.sql index d964a50f6..47fad0021 100644 --- a/src/test/regress/sql/multi_prepare_sql.sql +++ b/src/test/regress/sql/multi_prepare_sql.sql @@ -393,8 +393,6 @@ EXECUTE prepared_real_time_partition_column_select(4); EXECUTE prepared_real_time_partition_column_select(5); EXECUTE prepared_real_time_partition_column_select(6); --- check task-tracker executor - PREPARE prepared_task_tracker_non_partition_column_select(int) AS SELECT prepare_table.key, diff --git a/src/test/regress/sql/multi_repartition_join_planning.sql b/src/test/regress/sql/multi_repartition_join_planning.sql index 872a49c83..30cfc7156 100644 --- a/src/test/regress/sql/multi_repartition_join_planning.sql +++ b/src/test/regress/sql/multi_repartition_join_planning.sql @@ -3,8 +3,7 @@ -- -- Tests that cover repartition join planning. Note that we explicitly start a -- transaction block here so that we don't emit debug messages with changing --- transaction ids in them. Also, we set the executor type to task tracker --- executor here, as we cannot run repartition jobs with real time executor. +-- transaction ids in them. SET citus.next_shard_id TO 690000; diff --git a/src/test/regress/sql/multi_repartition_join_pruning.sql b/src/test/regress/sql/multi_repartition_join_pruning.sql index 9ff0be269..7c8f8bb36 100644 --- a/src/test/regress/sql/multi_repartition_join_pruning.sql +++ b/src/test/regress/sql/multi_repartition_join_pruning.sql @@ -1,9 +1,7 @@ -- -- MULTI_REPARTITION_JOIN_PRUNING -- --- Tests covering partition and join-pruning for repartition joins. Note that we --- set executor type to task tracker executor here, as we cannot run repartition --- jobs with real time executor. +-- Tests covering partition and join-pruning for repartition joins. SET citus.next_shard_id TO 700000; diff --git a/src/test/regress/sql/multi_repartition_udt.sql b/src/test/regress/sql/multi_repartition_udt.sql index d4f029929..c244a2b61 100644 --- a/src/test/regress/sql/multi_repartition_udt.sql +++ b/src/test/regress/sql/multi_repartition_udt.sql @@ -190,7 +190,6 @@ SELECT * FROM repartition_udt JOIN repartition_udt_other WHERE repartition_udt.pk = 1; -- Query that should result in a repartition join on UDT column. -SET citus.task_executor_type = 'task-tracker'; SET citus.log_multi_join_order = true; EXPLAIN (COSTS OFF) diff --git a/src/test/regress/sql/multi_router_planner.sql b/src/test/regress/sql/multi_router_planner.sql index 1523a751c..c55d02aab 100644 --- a/src/test/regress/sql/multi_router_planner.sql +++ b/src/test/regress/sql/multi_router_planner.sql @@ -1113,13 +1113,11 @@ CREATE MATERIALIZED VIEW mv_articles_hash_data AS SELECT * FROM articles_hash WHERE author_id in (1,2); SELECT * FROM mv_articles_hash_data ORDER BY 1, 2, 3, 4; --- router planner/executor is now enabled for task-tracker executor SELECT id FROM articles_hash WHERE author_id = 1 ORDER BY 1; --- insert query is router plannable even under task-tracker INSERT INTO articles_hash VALUES (51, 1, 'amateus', 1814), (52, 1, 'second amateus', 2824); -- verify insert is successful (not router plannable and executable) diff --git a/src/test/regress/sql/multi_router_planner_fast_path.sql b/src/test/regress/sql/multi_router_planner_fast_path.sql index 99292b00d..3c3b4a8b7 100644 --- a/src/test/regress/sql/multi_router_planner_fast_path.sql +++ b/src/test/regress/sql/multi_router_planner_fast_path.sql @@ -805,12 +805,10 @@ CREATE MATERIALIZED VIEW mv_articles_hash_empty AS SELECT * FROM mv_articles_hash_empty; --- fast-path router planner/executor is enabled for task-tracker executor SELECT id FROM articles_hash WHERE author_id = 1; --- insert query is router plannable even under task-tracker INSERT INTO articles_hash VALUES (51, 1, 'amateus', 1814), (52, 1, 'second amateus', 2824); -- verify insert is successfull (not router plannable and executable) diff --git a/src/test/regress/sql/single_hash_repartition_join.sql b/src/test/regress/sql/single_hash_repartition_join.sql index 8bcf77c68..8a0ad0ec1 100644 --- a/src/test/regress/sql/single_hash_repartition_join.sql +++ b/src/test/regress/sql/single_hash_repartition_join.sql @@ -151,7 +151,6 @@ CREATE TABLE dist_1 (a int , b int); SELECT create_distributed_table('dist_1', 'a'); INSERT INTO dist_1 SELECT x,10-x FROM generate_series(1,10) x; -SET citus.task_executor_type to 'task-tracker'; SELECT COUNT(*) FROM dist_1 f, dist_1 s WHERE f.a = s.b; SET client_min_messages TO ERROR; diff --git a/src/test/regress/sql/single_node.sql b/src/test/regress/sql/single_node.sql index e11701ef5..b0738a104 100644 --- a/src/test/regress/sql/single_node.sql +++ b/src/test/regress/sql/single_node.sql @@ -115,8 +115,6 @@ SELECT count(*) FROM test WHERE false; SELECT count(*) FROM test WHERE false GROUP BY GROUPING SETS (x,y); RESET citus.task_assignment_policy; --- single node task tracker tests: -SET citus.task_executor_type to 'task-tracker'; SELECT count(*) FROM test; -- INSERT SELECT from distributed table to local table diff --git a/src/test/regress/sql/with_executors.sql b/src/test/regress/sql/with_executors.sql index 67e0f4fa1..231df8e0c 100644 --- a/src/test/regress/sql/with_executors.sql +++ b/src/test/regress/sql/with_executors.sql @@ -1,4 +1,4 @@ --- Confirm we can use local, router, real-time, and task-tracker execution +-- Confirm we can use local, and adaptive execution CREATE SCHEMA with_executors; SET search_path TO with_executors, public; @@ -105,7 +105,7 @@ WITH cte AS ( SELECT * FROM cte WHERE uid=1 ORDER BY 2 LIMIT 5; --- CTEs should be able to use task-tracker queries +-- CTEs should be able to use adaptive executor WITH cte AS ( WITH task_tracker_1 AS ( SELECT @@ -294,8 +294,6 @@ WITH cte AS ( SELECT count(*) FROM cte, users_table where cte.count=user_id; -SET citus.task_executor_type='task-tracker'; --- CTEs shouldn't be able to terminate a task-tracker query WITH cte_1 AS ( SELECT u_table.user_id as u_id, e_table.event_type