remove task-tracker in comments, documentation

pull/4042/head
Sait Talha Nisanci 2020-07-20 12:12:04 +03:00
parent a3dc8fe2b5
commit 4308d867d9
40 changed files with 33 additions and 116 deletions

View File

@ -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.

View File

@ -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.

View File

@ -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,

View File

@ -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,

View File

@ -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.
*

View File

@ -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)
{

View File

@ -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)
{

View File

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

View File

@ -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.
*

View File

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

View File

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

View File

@ -145,7 +145,6 @@ SELECT * FROM raw_table WHERE ID = 1;
1 | 20
(2 rows)
-- Task tracker query test
CREATE TABLE tt1
(
id int,

View File

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

View File

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

View File

@ -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.

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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)

View File

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

View File

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

View File

@ -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,

View File

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

View File

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

View File

@ -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)

View File

@ -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)

View File

@ -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)

View File

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

View File

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

View File

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