mirror of
https://github.com/hatchet-dev/hatchet.git
synced 2025-12-30 13:19:44 -06:00
* change some olap flush settings * increase timeouts for payload wal * fix: improve performance of payload wal metrics * slight updates * more small tweaks * undo some olap changes, don't offload some payloads * remove double reads * try reducing wal poll limit * analyze v1_dag * move partition method
982 lines
26 KiB
SQL
982 lines
26 KiB
SQL
-- name: CreatePartitions :exec
|
|
SELECT
|
|
create_v1_range_partition('v1_task', @date::date),
|
|
create_v1_range_partition('v1_dag', @date::date),
|
|
create_v1_range_partition('v1_task_event', @date::date),
|
|
create_v1_range_partition('v1_log_line', @date::date),
|
|
create_v1_range_partition('v1_payload', @date::date);
|
|
|
|
-- name: EnsureTablePartitionsExist :one
|
|
WITH tomorrow_date AS (
|
|
SELECT (NOW() + INTERVAL '1 day')::date AS date
|
|
), expected_partitions AS (
|
|
SELECT
|
|
'v1_task_' || to_char((SELECT date FROM tomorrow_date), 'YYYYMMDD') AS expected_partition_name
|
|
UNION ALL
|
|
SELECT 'v1_dag_' || to_char((SELECT date FROM tomorrow_date), 'YYYYMMDD')
|
|
UNION ALL
|
|
SELECT 'v1_task_event_' || to_char((SELECT date FROM tomorrow_date), 'YYYYMMDD')
|
|
UNION ALL
|
|
SELECT 'v1_log_line_' || to_char((SELECT date FROM tomorrow_date), 'YYYYMMDD')
|
|
), partition_check AS (
|
|
SELECT
|
|
COUNT(*) AS total_tables,
|
|
COUNT(pt.tablename) AS existing_partitions
|
|
FROM expected_partitions ep
|
|
LEFT JOIN pg_catalog.pg_tables pt ON pt.tablename = ep.expected_partition_name
|
|
)
|
|
SELECT
|
|
CASE
|
|
WHEN existing_partitions = total_tables THEN TRUE
|
|
ELSE FALSE
|
|
END AS all_partitions_exist
|
|
FROM partition_check;
|
|
|
|
-- name: ListPartitionsBeforeDate :many
|
|
WITH task_partitions AS (
|
|
SELECT 'v1_task' AS parent_table, p::text as partition_name FROM get_v1_partitions_before_date('v1_task', @date::date) AS p
|
|
), dag_partitions AS (
|
|
SELECT 'v1_dag' AS parent_table, p::text as partition_name FROM get_v1_partitions_before_date('v1_dag', @date::date) AS p
|
|
), task_event_partitions AS (
|
|
SELECT 'v1_task_event' AS parent_table, p::text as partition_name FROM get_v1_partitions_before_date('v1_task_event', @date::date) AS p
|
|
), log_line_partitions AS (
|
|
SELECT 'v1_log_line' AS parent_table, p::text as partition_name FROM get_v1_partitions_before_date('v1_log_line', @date::date) AS p
|
|
), payload_partitions AS (
|
|
SELECT 'v1_payload' AS parent_table, p::text as partition_name FROM get_v1_partitions_before_date('v1_payload', @date::date) AS p
|
|
)
|
|
|
|
SELECT
|
|
*
|
|
FROM
|
|
task_partitions
|
|
|
|
UNION ALL
|
|
|
|
SELECT
|
|
*
|
|
FROM
|
|
dag_partitions
|
|
|
|
UNION ALL
|
|
|
|
SELECT
|
|
*
|
|
FROM
|
|
task_event_partitions
|
|
|
|
UNION ALL
|
|
|
|
SELECT
|
|
*
|
|
FROM
|
|
log_line_partitions
|
|
|
|
UNION ALL
|
|
|
|
SELECT
|
|
*
|
|
FROM
|
|
payload_partitions
|
|
;
|
|
|
|
-- name: DefaultTaskActivityGauge :one
|
|
SELECT
|
|
COUNT(*)
|
|
FROM
|
|
v1_queue
|
|
WHERE
|
|
tenant_id = @tenantId::uuid
|
|
AND last_active > @activeSince::timestamptz;
|
|
|
|
-- name: FlattenExternalIds :many
|
|
WITH lookup_rows AS (
|
|
SELECT
|
|
*
|
|
FROM
|
|
v1_lookup_table l
|
|
WHERE
|
|
l.external_id = ANY(@externalIds::uuid[])
|
|
AND l.tenant_id = @tenantId::uuid
|
|
), tasks_from_dags AS (
|
|
SELECT
|
|
t.id,
|
|
t.inserted_at,
|
|
t.retry_count,
|
|
t.external_id,
|
|
t.workflow_run_id,
|
|
t.additional_metadata,
|
|
t.dag_id,
|
|
t.dag_inserted_at,
|
|
t.parent_task_id,
|
|
t.child_index,
|
|
t.child_key,
|
|
d.external_id AS workflow_run_external_id
|
|
FROM
|
|
lookup_rows l
|
|
JOIN
|
|
v1_dag d ON d.id = l.dag_id AND d.inserted_at = l.inserted_at
|
|
JOIN
|
|
v1_dag_to_task dt ON dt.dag_id = d.id AND dt.dag_inserted_at = d.inserted_at
|
|
JOIN
|
|
v1_task t ON t.id = dt.task_id AND t.inserted_at = dt.task_inserted_at
|
|
WHERE
|
|
l.dag_id IS NOT NULL
|
|
)
|
|
-- Union the tasks from the lookup table with the tasks from the DAGs
|
|
SELECT
|
|
t.id,
|
|
t.inserted_at,
|
|
t.retry_count,
|
|
t.external_id,
|
|
t.workflow_run_id,
|
|
t.additional_metadata,
|
|
t.dag_id,
|
|
t.dag_inserted_at,
|
|
t.parent_task_id,
|
|
t.child_index,
|
|
t.child_key,
|
|
t.external_id AS workflow_run_external_id
|
|
FROM
|
|
lookup_rows l
|
|
JOIN
|
|
v1_task t ON t.id = l.task_id AND t.inserted_at = l.inserted_at
|
|
WHERE
|
|
l.task_id IS NOT NULL
|
|
|
|
UNION ALL
|
|
|
|
SELECT
|
|
*
|
|
FROM
|
|
tasks_from_dags;
|
|
|
|
-- name: LookupExternalIds :many
|
|
SELECT
|
|
*
|
|
FROM
|
|
v1_lookup_table
|
|
WHERE
|
|
external_id = ANY(@externalIds::uuid[])
|
|
AND tenant_id = @tenantId::uuid;
|
|
|
|
-- name: ListTasks :many
|
|
SELECT *
|
|
FROM
|
|
v1_task
|
|
WHERE
|
|
tenant_id = $1
|
|
AND id = ANY(@ids::bigint[]);
|
|
|
|
-- name: ListTaskMetas :many
|
|
SELECT
|
|
id,
|
|
inserted_at,
|
|
external_id,
|
|
retry_count,
|
|
workflow_id,
|
|
workflow_run_id
|
|
FROM
|
|
v1_task
|
|
WHERE
|
|
tenant_id = $1
|
|
AND id = ANY(@ids::bigint[]);
|
|
|
|
-- name: FailTaskAppFailure :many
|
|
-- Fails a task due to an application-level error
|
|
WITH input AS (
|
|
SELECT
|
|
*
|
|
FROM
|
|
(
|
|
SELECT
|
|
unnest(@taskIds::bigint[]) AS task_id,
|
|
unnest(@taskInsertedAts::timestamptz[]) AS task_inserted_at,
|
|
unnest(@taskRetryCounts::integer[]) AS task_retry_count,
|
|
unnest(@isNonRetryables::boolean[]) AS is_non_retryable
|
|
) AS subquery
|
|
), locked_tasks AS (
|
|
SELECT
|
|
t.id,
|
|
t.inserted_at,
|
|
t.step_id
|
|
FROM
|
|
v1_task t
|
|
JOIN
|
|
input i ON i.task_id = t.id AND i.task_inserted_at = t.inserted_at AND i.task_retry_count = t.retry_count
|
|
WHERE
|
|
t.tenant_id = @tenantId::uuid
|
|
-- order by the task id to get a stable lock order
|
|
ORDER BY
|
|
id
|
|
FOR UPDATE
|
|
), tasks_to_steps AS (
|
|
SELECT
|
|
t.id,
|
|
t.step_id,
|
|
s."retries"
|
|
FROM
|
|
locked_tasks t
|
|
JOIN
|
|
"Step" s ON s."id" = t.step_id
|
|
)
|
|
UPDATE
|
|
v1_task
|
|
SET
|
|
retry_count = retry_count + 1,
|
|
app_retry_count = app_retry_count + 1
|
|
FROM
|
|
tasks_to_steps
|
|
WHERE
|
|
(v1_task.id, v1_task.inserted_at, v1_task.retry_count) IN (
|
|
SELECT task_id, task_inserted_at, task_retry_count
|
|
FROM input
|
|
WHERE is_non_retryable = FALSE
|
|
)
|
|
AND tasks_to_steps."retries" > v1_task.app_retry_count
|
|
RETURNING
|
|
v1_task.id,
|
|
v1_task.inserted_at,
|
|
v1_task.retry_count,
|
|
v1_task.app_retry_count,
|
|
v1_task.retry_backoff_factor,
|
|
v1_task.retry_max_backoff;
|
|
|
|
-- name: FailTaskInternalFailure :many
|
|
-- Fails a task due to an application-level error
|
|
WITH input AS (
|
|
SELECT
|
|
*
|
|
FROM
|
|
(
|
|
SELECT
|
|
unnest(@taskIds::bigint[]) AS task_id,
|
|
unnest(@taskInsertedAts::timestamptz[]) AS task_inserted_at,
|
|
unnest(@taskRetryCounts::integer[]) AS task_retry_count
|
|
) AS subquery
|
|
), locked_tasks AS (
|
|
SELECT
|
|
t.id
|
|
FROM
|
|
v1_task t
|
|
-- only fail tasks which have a v1_task_runtime equivalent to the current retry count. otherwise,
|
|
-- a cancellation which deletes the v1_task_runtime might lead to a future failure event, which triggers
|
|
-- a retry.
|
|
JOIN
|
|
input i ON i.task_id = t.id AND i.task_inserted_at = t.inserted_at AND i.task_retry_count = t.retry_count
|
|
WHERE
|
|
t.tenant_id = @tenantId::uuid
|
|
-- order by the task id to get a stable lock order
|
|
ORDER BY
|
|
id
|
|
FOR UPDATE
|
|
)
|
|
UPDATE
|
|
v1_task
|
|
SET
|
|
retry_count = retry_count + 1,
|
|
internal_retry_count = internal_retry_count + 1
|
|
FROM
|
|
locked_tasks
|
|
WHERE
|
|
(v1_task.id, v1_task.inserted_at, v1_task.retry_count) IN (
|
|
SELECT task_id, task_inserted_at, task_retry_count
|
|
FROM input
|
|
)
|
|
AND @maxInternalRetries::int > v1_task.internal_retry_count
|
|
RETURNING
|
|
v1_task.id,
|
|
v1_task.inserted_at,
|
|
v1_task.retry_count;
|
|
|
|
-- name: ListTasksToTimeout :many
|
|
WITH expired_runtimes AS (
|
|
SELECT
|
|
task_id,
|
|
task_inserted_at,
|
|
retry_count,
|
|
worker_id
|
|
FROM
|
|
v1_task_runtime
|
|
WHERE
|
|
tenant_id = @tenantId::uuid
|
|
AND timeout_at <= NOW()
|
|
ORDER BY
|
|
task_id, task_inserted_at, retry_count
|
|
LIMIT
|
|
COALESCE(sqlc.narg('limit')::integer, 1000)
|
|
FOR UPDATE SKIP LOCKED
|
|
)
|
|
SELECT
|
|
v1_task.id,
|
|
v1_task.inserted_at,
|
|
v1_task.retry_count,
|
|
v1_task.step_id,
|
|
v1_task.external_id,
|
|
v1_task.workflow_run_id,
|
|
v1_task.step_timeout,
|
|
v1_task.app_retry_count,
|
|
v1_task.retry_backoff_factor,
|
|
v1_task.retry_max_backoff,
|
|
expired_runtimes.worker_id
|
|
FROM
|
|
v1_task
|
|
JOIN
|
|
expired_runtimes ON expired_runtimes.task_id = v1_task.id AND expired_runtimes.task_inserted_at = v1_task.inserted_at;
|
|
|
|
-- name: ListTasksToReassign :many
|
|
WITH tasks_on_inactive_workers AS (
|
|
SELECT
|
|
runtime.task_id,
|
|
runtime.task_inserted_at,
|
|
runtime.retry_count
|
|
FROM
|
|
"Worker" w
|
|
JOIN
|
|
v1_task_runtime runtime ON w."id" = runtime.worker_id
|
|
WHERE
|
|
w."tenantId" = @tenantId::uuid
|
|
AND w."lastHeartbeatAt" < NOW() - INTERVAL '30 seconds'
|
|
LIMIT
|
|
COALESCE(sqlc.narg('limit')::integer, 1000)
|
|
)
|
|
SELECT
|
|
v1_task.id,
|
|
v1_task.inserted_at,
|
|
v1_task.retry_count
|
|
FROM
|
|
v1_task
|
|
JOIN
|
|
tasks_on_inactive_workers lrs ON lrs.task_id = v1_task.id AND lrs.task_inserted_at = v1_task.inserted_at;
|
|
|
|
-- name: ProcessRetryQueueItems :many
|
|
WITH rqis_to_delete AS (
|
|
SELECT
|
|
*
|
|
FROM
|
|
v1_retry_queue_item rqi
|
|
WHERE
|
|
rqi.tenant_id = @tenantId::uuid
|
|
AND rqi.retry_after <= NOW()
|
|
ORDER BY
|
|
rqi.task_id, rqi.task_inserted_at, rqi.task_retry_count
|
|
LIMIT
|
|
COALESCE(sqlc.narg('limit')::integer, 1000)
|
|
FOR UPDATE SKIP LOCKED
|
|
)
|
|
DELETE FROM
|
|
v1_retry_queue_item
|
|
WHERE
|
|
(task_id, task_inserted_at, task_retry_count) IN (SELECT task_id, task_inserted_at, task_retry_count FROM rqis_to_delete)
|
|
RETURNING *;
|
|
|
|
-- name: ListMatchingTaskEvents :many
|
|
-- Lists the task events for the **latest** retry of a task, or task events which intentionally
|
|
-- aren't associated with a retry count (if the retry_count = -1).
|
|
WITH input AS (
|
|
SELECT
|
|
*
|
|
FROM
|
|
(
|
|
SELECT
|
|
unnest(@taskExternalIds::uuid[]) AS task_external_id,
|
|
-- can match any of the event types
|
|
unnest_nd_1d(@eventTypes::text[][]) AS event_types
|
|
) AS subquery
|
|
)
|
|
SELECT
|
|
t.external_id,
|
|
e.*
|
|
FROM
|
|
v1_lookup_table l
|
|
JOIN
|
|
v1_task t ON t.id = l.task_id AND t.inserted_at = l.inserted_at
|
|
JOIN
|
|
v1_task_event e ON e.tenant_id = @tenantId::uuid AND e.task_id = t.id AND e.task_inserted_at = t.inserted_at
|
|
JOIN
|
|
input i ON i.task_external_id = l.external_id AND e.event_type::text = ANY(i.event_types)
|
|
WHERE
|
|
l.tenant_id = @tenantId::uuid
|
|
AND l.external_id = ANY(@taskExternalIds::uuid[])
|
|
AND (e.retry_count = -1 OR e.retry_count = t.retry_count);
|
|
|
|
-- name: LockSignalCreatedEvents :many
|
|
-- Places a lock on the SIGNAL_CREATED events to make sure concurrent operations don't
|
|
-- modify the events.
|
|
WITH input AS (
|
|
SELECT
|
|
UNNEST(@taskIds::BIGINT[]) AS task_id,
|
|
UNNEST(@taskInsertedAts::TIMESTAMPTZ[]) AS task_inserted_at,
|
|
UNNEST(@eventKeys::TEXT[]) AS event_key
|
|
), distinct_events AS (
|
|
SELECT DISTINCT
|
|
task_id, task_inserted_at
|
|
FROM
|
|
input
|
|
), events_to_lock AS (
|
|
SELECT
|
|
e.id,
|
|
e.event_key,
|
|
e.data,
|
|
e.task_id,
|
|
e.task_inserted_at,
|
|
e.inserted_at
|
|
FROM
|
|
v1_task_event e
|
|
JOIN
|
|
distinct_events de
|
|
ON e.task_id = de.task_id
|
|
AND e.task_inserted_at = de.task_inserted_at
|
|
WHERE
|
|
e.tenant_id = @tenantId::uuid
|
|
AND e.event_type = 'SIGNAL_CREATED'
|
|
)
|
|
SELECT
|
|
e.id,
|
|
e.inserted_at,
|
|
e.event_key,
|
|
e.data
|
|
FROM
|
|
events_to_lock e
|
|
WHERE
|
|
e.event_key = ANY(SELECT event_key FROM input);
|
|
|
|
-- name: ListMatchingSignalEvents :many
|
|
WITH input AS (
|
|
SELECT
|
|
*
|
|
FROM
|
|
(
|
|
SELECT
|
|
unnest(@taskIds::bigint[]) AS task_id,
|
|
unnest(@taskInsertedAts::timestamptz[]) AS task_inserted_at,
|
|
unnest(@eventKeys::text[]) AS event_key
|
|
) AS subquery
|
|
)
|
|
SELECT
|
|
e.*
|
|
FROM
|
|
v1_task_event e
|
|
JOIN
|
|
input i ON i.task_id = e.task_id AND i.task_inserted_at = e.task_inserted_at AND i.event_key = e.event_key
|
|
WHERE
|
|
e.tenant_id = @tenantId::uuid
|
|
AND e.event_type = @eventType::v1_task_event_type;
|
|
|
|
-- name: DeleteMatchingSignalEvents :exec
|
|
WITH input AS (
|
|
SELECT
|
|
*
|
|
FROM
|
|
(
|
|
SELECT
|
|
unnest(@taskIds::bigint[]) AS task_id,
|
|
unnest(@taskInsertedAts::timestamptz[]) AS task_inserted_at,
|
|
unnest(@eventKeys::text[]) AS event_key
|
|
) AS subquery
|
|
), matching_events AS (
|
|
SELECT
|
|
e.task_id, e.task_inserted_at, e.id
|
|
FROM
|
|
v1_task_event e
|
|
JOIN
|
|
input i ON i.task_id = e.task_id AND i.task_inserted_at = e.task_inserted_at AND i.event_key = e.event_key
|
|
WHERE
|
|
e.tenant_id = @tenantId::uuid
|
|
AND e.event_type = @eventType::v1_task_event_type
|
|
ORDER BY
|
|
e.id
|
|
FOR UPDATE
|
|
)
|
|
DELETE FROM
|
|
v1_task_event
|
|
WHERE
|
|
(task_id, task_inserted_at, id) IN (SELECT task_id, task_inserted_at, id FROM matching_events);
|
|
|
|
-- name: ListTasksForReplay :many
|
|
-- Lists tasks for replay by recursively selecting all tasks that are children of the input tasks,
|
|
-- then locks the tasks for replay.
|
|
WITH RECURSIVE augmented_tasks AS (
|
|
-- First, select the tasks from the input
|
|
SELECT
|
|
id,
|
|
inserted_at,
|
|
tenant_id,
|
|
dag_id,
|
|
dag_inserted_at,
|
|
step_id
|
|
FROM
|
|
v1_task
|
|
WHERE
|
|
(id, inserted_at) IN (
|
|
SELECT
|
|
unnest(@taskIds::bigint[]),
|
|
unnest(@taskInsertedAts::timestamptz[])
|
|
)
|
|
AND tenant_id = @tenantId::uuid
|
|
|
|
UNION
|
|
|
|
-- Then, select the tasks that are children of the input tasks
|
|
SELECT
|
|
t.id,
|
|
t.inserted_at,
|
|
t.tenant_id,
|
|
t.dag_id,
|
|
t.dag_inserted_at,
|
|
t.step_id
|
|
FROM
|
|
augmented_tasks at
|
|
JOIN
|
|
"Step" s1 ON s1."id" = at.step_id
|
|
JOIN
|
|
v1_dag_to_task dt ON dt.dag_id = at.dag_id AND dt.dag_inserted_at = at.dag_inserted_at
|
|
JOIN
|
|
v1_task t ON t.id = dt.task_id AND t.inserted_at = dt.task_inserted_at
|
|
JOIN
|
|
"Step" s2 ON s2."id" = t.step_id
|
|
JOIN
|
|
"_StepOrder" so ON so."B" = s2."id" AND so."A" = s1."id"
|
|
), locked_tasks AS (
|
|
SELECT
|
|
t.id,
|
|
t.inserted_at,
|
|
t.retry_count,
|
|
t.dag_id,
|
|
t.dag_inserted_at,
|
|
t.step_readable_id,
|
|
t.step_id,
|
|
t.workflow_id,
|
|
t.external_id,
|
|
t.input,
|
|
t.additional_metadata,
|
|
t.parent_task_external_id,
|
|
t.parent_task_id,
|
|
t.parent_task_inserted_at,
|
|
t.step_index,
|
|
t.child_index,
|
|
t.child_key
|
|
FROM
|
|
v1_task t
|
|
WHERE
|
|
(t.id, t.inserted_at) IN (
|
|
SELECT
|
|
id, inserted_at
|
|
FROM
|
|
augmented_tasks
|
|
)
|
|
AND t.tenant_id = @tenantId::uuid
|
|
-- order by the task id to get a stable lock order
|
|
ORDER BY
|
|
id
|
|
FOR UPDATE
|
|
), step_orders AS (
|
|
SELECT
|
|
t.step_id,
|
|
array_agg(so."A")::uuid[] as "parents"
|
|
FROM
|
|
locked_tasks t
|
|
JOIN
|
|
"Step" s ON s."id" = t.step_id
|
|
JOIN
|
|
"_StepOrder" so ON so."B" = s."id"
|
|
GROUP BY
|
|
t.step_id
|
|
)
|
|
SELECT
|
|
t.id,
|
|
t.inserted_at,
|
|
t.retry_count,
|
|
t.dag_id,
|
|
t.dag_inserted_at,
|
|
t.step_readable_id,
|
|
t.step_id,
|
|
t.workflow_id,
|
|
t.external_id,
|
|
t.input,
|
|
t.additional_metadata,
|
|
t.parent_task_external_id,
|
|
t.parent_task_id,
|
|
t.parent_task_inserted_at,
|
|
t.step_index,
|
|
t.child_index,
|
|
t.child_key,
|
|
j."kind" as "jobKind",
|
|
COALESCE(so."parents", '{}'::uuid[]) as "parents"
|
|
FROM
|
|
locked_tasks t
|
|
JOIN
|
|
"Step" s ON s."id" = t.step_id
|
|
JOIN
|
|
"Job" j ON j."id" = s."jobId"
|
|
LEFT JOIN
|
|
step_orders so ON so.step_id = t.step_id;
|
|
|
|
-- name: ListTaskParentOutputs :many
|
|
-- Lists the outputs of parent steps for a list of tasks. This is recursive because it looks at all grandparents
|
|
-- of the tasks as well.
|
|
WITH input AS (
|
|
SELECT
|
|
UNNEST(@taskIds::BIGINT[]) AS task_id,
|
|
UNNEST(@taskInsertedAts::TIMESTAMPTZ[]) AS task_inserted_at
|
|
), task_outputs AS (
|
|
SELECT
|
|
t.id,
|
|
t.inserted_at,
|
|
e.retry_count,
|
|
t.tenant_id,
|
|
t.dag_id,
|
|
t.dag_inserted_at,
|
|
t.step_readable_id,
|
|
t.workflow_run_id,
|
|
t.step_id,
|
|
t.workflow_id,
|
|
e.id AS task_event_id,
|
|
e.inserted_at AS task_event_inserted_at,
|
|
e.data AS output
|
|
FROM
|
|
v1_task t1
|
|
JOIN
|
|
v1_dag_to_task dt ON dt.dag_id = t1.dag_id AND dt.dag_inserted_at = t1.dag_inserted_at
|
|
JOIN
|
|
v1_task t ON t.id = dt.task_id AND t.inserted_at = dt.task_inserted_at
|
|
JOIN
|
|
v1_task_event e ON e.task_id = t.id AND e.task_inserted_at = t.inserted_at AND e.event_type = 'COMPLETED'
|
|
WHERE
|
|
(t1.id, t1.inserted_at) IN (
|
|
SELECT
|
|
task_id,
|
|
task_inserted_at
|
|
FROM
|
|
input
|
|
)
|
|
AND t1.tenant_id = @tenantId::uuid
|
|
AND t1.dag_id IS NOT NULL
|
|
), max_retry_counts AS (
|
|
SELECT
|
|
id,
|
|
inserted_at,
|
|
MAX(retry_count) AS max_retry_count
|
|
FROM
|
|
task_outputs
|
|
GROUP BY
|
|
id, inserted_at
|
|
)
|
|
SELECT
|
|
DISTINCT ON (task_outputs.id, task_outputs.inserted_at, task_outputs.retry_count)
|
|
task_outputs.task_event_id,
|
|
task_outputs.task_event_inserted_at,
|
|
task_outputs.workflow_run_id,
|
|
task_outputs.output
|
|
FROM
|
|
task_outputs
|
|
JOIN
|
|
max_retry_counts mrc ON task_outputs.id = mrc.id
|
|
AND task_outputs.inserted_at = mrc.inserted_at
|
|
AND task_outputs.retry_count = mrc.max_retry_count
|
|
ORDER BY
|
|
task_outputs.id,
|
|
task_outputs.inserted_at,
|
|
task_outputs.retry_count DESC;
|
|
|
|
-- name: LockDAGsForReplay :many
|
|
-- Locks a list of DAGs for replay. Returns successfully locked DAGs which can be replayed.
|
|
SELECT
|
|
id
|
|
FROM
|
|
v1_dag
|
|
WHERE
|
|
id = ANY(@dagIds::bigint[])
|
|
AND tenant_id = @tenantId::uuid
|
|
ORDER BY id
|
|
-- We skip locked tasks because replays are the only thing that can lock a DAG for updates
|
|
FOR UPDATE SKIP LOCKED;
|
|
|
|
-- name: PreflightCheckDAGsForReplay :many
|
|
-- Checks whether DAGs can be replayed by ensuring that the length of the tasks which have been written
|
|
-- match the length of steps in the DAG. This assumes that we have a lock on DAGs so concurrent replays
|
|
-- don't interfere with each other. It also does not check for whether the tasks are running, as that's
|
|
-- checked in a different query. It returns DAGs which cannot be replayed.
|
|
WITH dags_to_step_counts AS (
|
|
SELECT
|
|
d.id,
|
|
d.external_id,
|
|
d.inserted_at,
|
|
COUNT(DISTINCT s."id") as step_count,
|
|
COUNT(DISTINCT dt.task_id) as task_count
|
|
FROM
|
|
v1_dag d
|
|
JOIN
|
|
v1_dag_to_task dt ON dt.dag_id = d.id
|
|
JOIN
|
|
"WorkflowVersion" wv ON wv."id" = d.workflow_version_id
|
|
LEFT JOIN
|
|
"Job" j ON j."workflowVersionId" = wv."id"
|
|
LEFT JOIN
|
|
"Step" s ON s."jobId" = j."id"
|
|
WHERE
|
|
d.id = ANY(@dagIds::bigint[])
|
|
AND d.tenant_id = @tenantId::uuid
|
|
GROUP BY
|
|
d.id,
|
|
d.inserted_at
|
|
)
|
|
SELECT
|
|
d.id,
|
|
d.external_id,
|
|
d.inserted_at,
|
|
d.step_count,
|
|
d.task_count
|
|
FROM
|
|
dags_to_step_counts d;
|
|
|
|
-- name: PreflightCheckTasksForReplay :many
|
|
-- Checks whether tasks can be replayed by ensuring that they don't have any active runtimes,
|
|
-- concurrency slots, or retry queue items. Returns the tasks which cannot be replayed.
|
|
WITH input AS (
|
|
SELECT
|
|
UNNEST(@taskIds::bigint[]) AS task_id,
|
|
UNNEST(@taskInsertedAts::timestamptz[]) AS task_inserted_at
|
|
), relevant_tasks AS (
|
|
SELECT *
|
|
FROM
|
|
v1_task t
|
|
JOIN
|
|
input i ON i.task_id = t.id AND i.task_inserted_at = t.inserted_at
|
|
WHERE
|
|
-- prune partitions with minInsertedAt
|
|
t.inserted_at >= @minInsertedAt::TIMESTAMPTZ
|
|
)
|
|
|
|
SELECT t.id, t.dag_id
|
|
FROM relevant_tasks t
|
|
LEFT JOIN
|
|
v1_task_runtime tr ON tr.task_id = t.id AND tr.task_inserted_at = t.inserted_at AND tr.retry_count = t.retry_count
|
|
LEFT JOIN
|
|
v1_concurrency_slot cs ON cs.task_id = t.id AND cs.task_inserted_at = t.inserted_at AND cs.task_retry_count = t.retry_count
|
|
LEFT JOIN
|
|
v1_retry_queue_item rqi ON rqi.task_id = t.id AND rqi.task_inserted_at = t.inserted_at AND rqi.task_retry_count = t.retry_count
|
|
WHERE
|
|
t.tenant_id = @tenantId::uuid
|
|
AND NOT EXISTS (
|
|
SELECT 1
|
|
FROM v1_task_event e
|
|
WHERE
|
|
-- prune partitions with minInsertedAt
|
|
e.task_inserted_at >= @minInsertedAt::TIMESTAMPTZ
|
|
AND (e.task_id, e.task_inserted_at, e.retry_count) = (t.id, t.inserted_at, t.retry_count)
|
|
AND e.event_type = ANY('{COMPLETED, FAILED, CANCELLED}'::v1_task_event_type[])
|
|
)
|
|
AND (tr.task_id IS NOT NULL OR cs.task_id IS NOT NULL OR rqi.task_id IS NOT NULL)
|
|
;
|
|
|
|
-- name: ListAllTasksInDags :many
|
|
SELECT
|
|
t.id,
|
|
t.inserted_at,
|
|
t.retry_count,
|
|
t.dag_id,
|
|
t.dag_inserted_at,
|
|
t.step_readable_id,
|
|
t.step_id,
|
|
t.workflow_id,
|
|
t.external_id
|
|
FROM
|
|
v1_task t
|
|
JOIN
|
|
v1_dag_to_task dt ON dt.task_id = t.id
|
|
WHERE
|
|
t.tenant_id = @tenantId::uuid
|
|
AND dt.dag_id = ANY(@dagIds::bigint[]);
|
|
|
|
-- name: ListTaskExpressionEvals :many
|
|
WITH input AS (
|
|
SELECT
|
|
*
|
|
FROM
|
|
(
|
|
SELECT
|
|
unnest(@taskIds::bigint[]) AS task_id,
|
|
unnest(@taskInsertedAts::timestamptz[]) AS task_inserted_at
|
|
) AS subquery
|
|
)
|
|
SELECT
|
|
*
|
|
FROM
|
|
v1_task_expression_eval te
|
|
WHERE
|
|
(task_id, task_inserted_at) IN (
|
|
SELECT
|
|
task_id,
|
|
task_inserted_at
|
|
FROM
|
|
input
|
|
);
|
|
|
|
-- name: RefreshTimeoutBy :one
|
|
WITH task AS (
|
|
SELECT
|
|
t.id,
|
|
t.inserted_at,
|
|
t.retry_count,
|
|
t.tenant_id
|
|
FROM
|
|
v1_lookup_table lt
|
|
JOIN
|
|
v1_task t ON t.id = lt.task_id AND t.inserted_at = lt.inserted_at
|
|
WHERE
|
|
lt.external_id = @externalId::uuid AND
|
|
lt.tenant_id = @tenantId::uuid
|
|
), locked_runtime AS (
|
|
SELECT
|
|
tr.task_id,
|
|
tr.task_inserted_at,
|
|
tr.retry_count,
|
|
tr.worker_id
|
|
FROM
|
|
v1_task_runtime tr
|
|
WHERE
|
|
(tr.task_id, tr.task_inserted_at, tr.retry_count) IN (SELECT id, inserted_at, retry_count FROM task)
|
|
ORDER BY
|
|
task_id, task_inserted_at, retry_count
|
|
FOR UPDATE
|
|
)
|
|
UPDATE
|
|
v1_task_runtime
|
|
SET
|
|
timeout_at = timeout_at + convert_duration_to_interval(sqlc.narg('incrementTimeoutBy')::text)
|
|
FROM
|
|
task
|
|
WHERE
|
|
(v1_task_runtime.task_id, v1_task_runtime.task_inserted_at, v1_task_runtime.retry_count) IN (SELECT id, inserted_at, retry_count FROM task)
|
|
RETURNING
|
|
v1_task_runtime.*;
|
|
|
|
-- name: ManualSlotRelease :one
|
|
WITH task AS (
|
|
SELECT
|
|
t.id,
|
|
t.inserted_at,
|
|
t.retry_count,
|
|
t.tenant_id
|
|
FROM
|
|
v1_lookup_table lt
|
|
JOIN
|
|
v1_task t ON t.id = lt.task_id AND t.inserted_at = lt.inserted_at
|
|
WHERE
|
|
lt.external_id = @externalId::uuid AND
|
|
lt.tenant_id = @tenantId::uuid
|
|
), locked_runtime AS (
|
|
SELECT
|
|
tr.task_id,
|
|
tr.task_inserted_at,
|
|
tr.retry_count,
|
|
tr.worker_id
|
|
FROM
|
|
v1_task_runtime tr
|
|
WHERE
|
|
(tr.task_id, tr.task_inserted_at, tr.retry_count) IN (SELECT id, inserted_at, retry_count FROM task)
|
|
ORDER BY
|
|
task_id, task_inserted_at, retry_count
|
|
FOR UPDATE
|
|
)
|
|
UPDATE
|
|
v1_task_runtime
|
|
SET
|
|
worker_id = NULL
|
|
FROM
|
|
task
|
|
WHERE
|
|
(v1_task_runtime.task_id, v1_task_runtime.task_inserted_at, v1_task_runtime.retry_count) IN (SELECT id, inserted_at, retry_count FROM task)
|
|
RETURNING
|
|
v1_task_runtime.*;
|
|
|
|
-- name: CleanupWorkflowConcurrencySlotsAfterInsert :exec
|
|
-- Cleans up workflow concurrency slots when tasks have been inserted in a non-QUEUED state.
|
|
-- NOTE: this comes after the insert into v1_dag_to_task and v1_lookup_table, because we case on these tables for cleanup
|
|
WITH input AS (
|
|
SELECT
|
|
UNNEST(@concurrencyParentStrategyIds::bigint[]) AS parent_strategy_id,
|
|
UNNEST(@workflowVersionIds::uuid[]) AS workflow_version_id,
|
|
UNNEST(@workflowRunIds::uuid[]) AS workflow_run_id
|
|
ORDER BY parent_strategy_id, workflow_version_id, workflow_run_id
|
|
)
|
|
SELECT
|
|
cleanup_workflow_concurrency_slots(
|
|
rec.parent_strategy_id,
|
|
rec.workflow_version_id,
|
|
rec.workflow_run_id
|
|
)
|
|
FROM
|
|
input rec;
|
|
|
|
-- name: RegisterBatch :batchexec
|
|
-- DO NOT USE: dummy query to satisfy sqlc and register Batch calls on DBTX
|
|
SELECT * FROM v1_task WHERE id = $1;
|
|
|
|
-- name: AnalyzeV1Task :exec
|
|
ANALYZE v1_task;
|
|
|
|
-- name: AnalyzeV1TaskEvent :exec
|
|
ANALYZE v1_task_event;
|
|
|
|
-- name: AnalyzeV1Dag :exec
|
|
ANALYZE v1_dag;
|
|
|
|
-- name: CleanupV1TaskRuntime :execresult
|
|
WITH locked_trs AS (
|
|
SELECT vtr.task_id, vtr.task_inserted_at, vtr.retry_count
|
|
FROM v1_task_runtime vtr
|
|
WHERE NOT EXISTS (
|
|
SELECT 1
|
|
FROM v1_task vt
|
|
WHERE vtr.task_id = vt.id
|
|
AND vtr.task_inserted_at = vt.inserted_at
|
|
)
|
|
ORDER BY vtr.task_id ASC
|
|
LIMIT @batchSize::int
|
|
FOR UPDATE SKIP LOCKED
|
|
)
|
|
DELETE FROM v1_task_runtime
|
|
WHERE (task_id, task_inserted_at, retry_count) IN (
|
|
SELECT task_id, task_inserted_at, retry_count
|
|
FROM locked_trs
|
|
);
|
|
|
|
-- name: CleanupV1ConcurrencySlot :execresult
|
|
WITH locked_cs AS (
|
|
SELECT cs.task_id, cs.task_inserted_at, cs.task_retry_count
|
|
FROM v1_concurrency_slot cs
|
|
WHERE NOT EXISTS (
|
|
SELECT 1
|
|
FROM v1_task vt
|
|
WHERE cs.task_id = vt.id
|
|
AND cs.task_inserted_at = vt.inserted_at
|
|
)
|
|
ORDER BY cs.task_id, cs.task_inserted_at, cs.task_retry_count, cs.strategy_id
|
|
LIMIT @batchSize::int
|
|
FOR UPDATE SKIP LOCKED
|
|
)
|
|
DELETE FROM v1_concurrency_slot
|
|
WHERE (task_id, task_inserted_at, task_retry_count) IN (
|
|
SELECT task_id, task_inserted_at, task_retry_count
|
|
FROM locked_cs
|
|
);
|
|
|
|
-- name: CleanupV1WorkflowConcurrencySlot :execresult
|
|
WITH active_slots AS (
|
|
SELECT DISTINCT
|
|
wcs.strategy_id,
|
|
wcs.workflow_version_id,
|
|
wcs.workflow_run_id
|
|
FROM v1_workflow_concurrency_slot wcs
|
|
ORDER BY wcs.strategy_id, wcs.workflow_version_id, wcs.workflow_run_id
|
|
LIMIT @batchSize::int
|
|
)
|
|
SELECT
|
|
cleanup_workflow_concurrency_slots(
|
|
slot.strategy_id,
|
|
slot.workflow_version_id,
|
|
slot.workflow_run_id
|
|
)
|
|
FROM active_slots slot;
|