Skip to content

Commit

Permalink
Swap CeleryExecutor over to use TaskSDK for execution.
Browse files Browse the repository at this point in the history
Some points of note about this PR:

- Logging is changed in Celery, but only for Airflow 3

  Celery does it's own "capture stdout" logging, which conflicts with the ones
  we do in the TaskSDK, so we disable that; but to not change anything for
  Airflow 3.

- Simplify task SDK logging redirection

  As part of this discovery that Celery captures stdout/stderr itself (and
  before disabling that) I discovered a simpler way to re-open the
  stdin/out/err so that the implementation needs fewer/no special casing.

- Make JSON task logs more readable by giving them a consistent/useful order

  We re-order (by re-creating) the event_dict so that timestamp, level, and
  then even are always the first items in the dict

- Makes the CeleryExecutor understand the concept of "workloads" instead a
  command tuple.

  This change isn't done in the best way, but until Kube executor is swapped
  over (and possibly the other in-tree executors, such as ECS) we need to
  support both styles concurrently.

  The change should be done in such a way that the provider still works with
  Airflow v2, if it's running on that version.

- Upgrade Celery

  This turned out to not be 100% necessary but it does fix some deprecation
  warnings when running on Python 3.12

- Ensure that the forked process in TaskSDK _never ever_ exits

  Again, this isn't possible usually, but since the setup step of `_fork_main`
  died, it didn't call `os._exit()`, and was caught further up, which meant
  the process stayed alive as it never closed the sockets properly. We put and
  extra safety try/except block in place to catch that

I have not yet included a newsfragment for changing the executor interface as
the old style is _currently_ still supported.
  • Loading branch information
ashb committed Jan 29, 2025
1 parent e3baba0 commit 8263c94
Show file tree
Hide file tree
Showing 10 changed files with 201 additions and 79 deletions.
41 changes: 32 additions & 9 deletions airflow/executors/base_executor.py
Original file line number Diff line number Diff line change
Expand Up @@ -223,7 +223,12 @@ def has_task(self, task_instance: TaskInstance) -> bool:
:param task_instance: TaskInstance
:return: True if the task is known to this executor
"""
return task_instance.key in self.queued_tasks or task_instance.key in self.running
return (
task_instance.id in self.queued_tasks
or task_instance.id in self.running
or task_instance.key in self.queued_tasks
or task_instance.key in self.running
)

def sync(self) -> None:
"""
Expand Down Expand Up @@ -319,6 +324,20 @@ def order_queued_tasks_by_priority(self) -> list[tuple[TaskInstanceKey, QueuedTa
:return: List of tuples from the queued_tasks according to the priority.
"""
from airflow.executors import workloads

if not self.queued_tasks:
return []

kind = next(iter(self.queued_tasks.values()))
if isinstance(kind, workloads.BaseActivity):
# V3 + new executor that supports workloads
return sorted(
self.queued_tasks.items(),
key=lambda x: x[1].ti.priority_weight,
reverse=True,
)

return sorted(
self.queued_tasks.items(),
key=lambda x: x[1][1],
Expand All @@ -332,12 +351,12 @@ def trigger_tasks(self, open_slots: int) -> None:
:param open_slots: Number of open slots
"""
span = Trace.get_current_span()
sorted_queue = self.order_queued_tasks_by_priority()
task_tuples = []
workloads = []

for _ in range(min((open_slots, len(self.queued_tasks)))):
key, (command, _, queue, ti) = sorted_queue.pop(0)
key, item = sorted_queue.pop(0)

# If a task makes it here but is still understood by the executor
# to be running, it generally means that the task has been killed
Expand Down Expand Up @@ -375,15 +394,19 @@ def trigger_tasks(self, open_slots: int) -> None:
else:
if key in self.attempts:
del self.attempts[key]
task_tuples.append((key, command, queue, ti.executor_config))
if span.is_recording():
span.add_event(
name="task to trigger",
attributes={"command": str(command), "conf": str(ti.executor_config)},
)
# TODO: TaskSDK: Compat, remove when KubeExecutor is fully moved over to TaskSDK too.
# TODO: TaskSDK: We need to minimum version requirements on executors with Airflow 3.
# How/where do we do that? Executor loader?
if hasattr(self, "_process_workloads"):
workloads.append(item)
else:
(command, _, queue, ti) = item
task_tuples.append((key, command, queue, getattr(ti, "executor_config", None)))

if task_tuples:
self._process_tasks(task_tuples)
elif workloads:
self._process_workloads(workloads) # type: ignore[attr-defined]

@add_span
def _process_tasks(self, task_tuples: list[TaskTuple]) -> None:
Expand Down
4 changes: 3 additions & 1 deletion airflow/jobs/scheduler_job_runner.py
Original file line number Diff line number Diff line change
Expand Up @@ -836,7 +836,6 @@ def process_executor_events(
)
if info is not None:
msg += " Extra info: %s" % info # noqa: RUF100, UP031, flynt
cls.logger().error(msg)
session.add(Log(event="state mismatch", extra=msg, task_instance=ti.key))

# Get task from the Serialized DAG
Expand All @@ -849,6 +848,9 @@ def process_executor_events(
continue
ti.task = task
if task.on_retry_callback or task.on_failure_callback:
# Only log the error/extra info here, since the `ti.handle_failure()` path will log it
# too, which would lead to double logging
cls.logger().error(msg)
request = TaskCallbackRequest(
full_filepath=ti.dag_model.fileloc,
ti=ti,
Expand Down
2 changes: 1 addition & 1 deletion generated/provider_dependencies.json
Original file line number Diff line number Diff line change
Expand Up @@ -335,7 +335,7 @@
"celery": {
"deps": [
"apache-airflow>=2.9.0",
"celery[redis]>=5.3.0,<6,!=5.3.3,!=5.3.2",
"celery[redis]>=5.4.0,<6",
"flower>=1.0.0",
"google-re2>=1.0"
],
Expand Down
4 changes: 1 addition & 3 deletions providers/celery/pyproject.toml
Original file line number Diff line number Diff line change
Expand Up @@ -59,9 +59,7 @@ dependencies = [
# The Celery is known to introduce problems when upgraded to a MAJOR version. Airflow Core
# Uses Celery for CeleryExecutor, and we also know that Kubernetes Python client follows SemVer
# (https://docs.celeryq.dev/en/stable/contributing.html?highlight=semver#versions).
# Make sure that the limit here is synchronized with [celery] extra in the airflow core
# The 5.3.3/5.3.2 limit comes from https://github.com/celery/celery/issues/8470
"celery[redis]>=5.3.0,<6,!=5.3.3,!=5.3.2",
"celery[redis]>=5.4.0,<6",
"flower>=1.0.0",
"google-re2>=1.0",
]
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -154,6 +154,11 @@ def worker(args):
# This needs to be imported locally to not trigger Providers Manager initialization
from airflow.providers.celery.executors.celery_executor import app as celery_app

if AIRFLOW_V_3_0_PLUS:
from airflow.sdk.log import configure_logging

configure_logging(output=sys.stdout.buffer)

# Disable connection pool so that celery worker does not hold an unnecessary db connection
settings.reconfigure_orm(disable_connection_pool=True)
if not settings.validate_session():
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@
from airflow.configuration import conf
from airflow.exceptions import AirflowProviderDeprecationWarning, AirflowTaskTimeout
from airflow.executors.base_executor import BaseExecutor
from airflow.providers.celery.version_compat import AIRFLOW_V_2_8_PLUS
from airflow.providers.celery.version_compat import AIRFLOW_V_2_8_PLUS, AIRFLOW_V_3_0_PLUS
from airflow.stats import Stats
from airflow.utils.state import TaskInstanceState
from celery import states as celery_states
Expand All @@ -67,6 +67,9 @@
if TYPE_CHECKING:
import argparse

from sqlalchemy.orm import Session

from airflow.executors import workloads
from airflow.executors.base_executor import CommandType, TaskTuple
from airflow.models.taskinstance import TaskInstance
from airflow.models.taskinstancekey import TaskInstanceKey
Expand Down Expand Up @@ -228,6 +231,11 @@ class CeleryExecutor(BaseExecutor):
supports_ad_hoc_ti_run: bool = True
supports_sentry: bool = True

if TYPE_CHECKING and AIRFLOW_V_3_0_PLUS:
# In the v3 path, we store workloads, not commands as strings.
# TODO: TaskSDK: move this type change into BaseExecutor
queued_tasks: dict[TaskInstanceKey, workloads.All] # type: ignore[assignment]

def __init__(self):
super().__init__()

Expand Down Expand Up @@ -256,10 +264,25 @@ def _num_tasks_per_send_process(self, to_send_count: int) -> int:
return max(1, math.ceil(to_send_count / self._sync_parallelism))

def _process_tasks(self, task_tuples: list[TaskTuple]) -> None:
# Airflow V2 version
from airflow.providers.celery.executors.celery_executor_utils import execute_command

task_tuples_to_send = [task_tuple[:3] + (execute_command,) for task_tuple in task_tuples]
first_task = next(t[3] for t in task_tuples_to_send)

self._send_tasks(task_tuples_to_send)

def _process_workloads(self, workloads: list[workloads.All]) -> None:
# Airflow V3 version
from airflow.providers.celery.executors.celery_executor_utils import execute_workload

tasks = [
(workload.ti.key, (workload.model_dump_json(),), workload.ti.queue, execute_workload)
for workload in workloads
]
self._send_tasks(tasks)

def _send_tasks(self, task_tuples_to_send):
first_task = next(t[-1] for t in task_tuples_to_send)

# Celery state queries will stuck if we do not use one same backend
# for all tasks.
Expand Down Expand Up @@ -359,7 +382,7 @@ def update_task_state(self, key: TaskInstanceKey, state: str, info: Any) -> None
self.success(key, info)
elif state in (celery_states.FAILURE, celery_states.REVOKED):
self.fail(key, info)
elif state in (celery_states.STARTED, celery_states.PENDING):
elif state in (celery_states.STARTED, celery_states.PENDING, celery_states.RETRY):
pass
else:
self.log.info("Unexpected state for %s: %s", key, state)
Expand Down Expand Up @@ -416,6 +439,10 @@ def try_adopt_task_instances(self, tis: Sequence[TaskInstance]) -> Sequence[Task
for celery_task_id, (state, info) in states_by_celery_task_id.items():
result, ti = celery_tasks[celery_task_id]
result.backend = cached_celery_backend
if isinstance(result.result, BaseException):
e = result.result
# Log the exception we got from the remote end
self.log.warning("Task %s tailed with error", ti.key, exc_info=e)

# Set the correct elements of the state dicts, then update this
# like we just queried it.
Expand Down Expand Up @@ -475,6 +502,10 @@ def get_cli_commands() -> list[GroupCommand]:
),
]

def queue_workload(self, workload: workloads.ExecuteTask, session: Session | None) -> None:
ti = workload.ti
self.queued_tasks[ti.key] = workload


def _get_parser() -> argparse.ArgumentParser:
"""
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,8 +40,9 @@
from airflow.configuration import conf
from airflow.exceptions import AirflowException, AirflowProviderDeprecationWarning, AirflowTaskTimeout
from airflow.executors.base_executor import BaseExecutor
from airflow.providers.celery.version_compat import AIRFLOW_V_3_0_PLUS
from airflow.sdk.definitions._internal.dag_parsing_context import _airflow_parsing_context_manager
from airflow.stats import Stats
from airflow.utils.dag_parsing_context import _airflow_parsing_context_manager
from airflow.utils.log.logging_mixin import LoggingMixin
from airflow.utils.net import get_hostname
from airflow.utils.providers_configuration_loader import providers_configuration_loaded
Expand Down Expand Up @@ -125,21 +126,54 @@ def on_celery_import_modules(*args, **kwargs):
import kubernetes.client # noqa: F401


@app.task
def execute_command(command_to_exec: CommandType) -> None:
"""Execute command."""
dag_id, task_id = BaseExecutor.validate_airflow_tasks_run_command(command_to_exec)
# Once Celery5 is out of beta, we can pass `pydantic=True` to the decorator and it will handle the validation
# and deserialization for us
@app.task(name="execute_workload")
def execute_workload(input: str) -> None:
from pydantic import TypeAdapter

from airflow.configuration import conf
from airflow.executors import workloads
from airflow.sdk.execution_time.supervisor import supervise

decoder = TypeAdapter(workloads.All)
workload = decoder.validate_json(input)

celery_task_id = app.current_task.request.id
log.info("[%s] Executing command in Celery: %s", celery_task_id, command_to_exec)
with _airflow_parsing_context_manager(dag_id=dag_id, task_id=task_id):
try:
if settings.EXECUTE_TASKS_NEW_PYTHON_INTERPRETER:
_execute_in_subprocess(command_to_exec, celery_task_id)
else:
_execute_in_fork(command_to_exec, celery_task_id)
except Exception:
Stats.incr("celery.execute_command.failure")
raise

if not isinstance(workload, workloads.ExecuteTask):
raise ValueError(f"CeleryExecutor does not now how to handle {type(workload)}")

log.info("[%s] Executing workload in Celery: %s", celery_task_id, workload)

supervise(
# This is the "wrong" ti type, but it duck types the same. TODO: Create a protocol for this.
ti=workload.ti, # type: ignore[arg-type]
dag_rel_path=workload.dag_rel_path,
bundle_info=workload.bundle_info,
token=workload.token,
server=conf.get("workers", "execution_api_server_url", fallback="http://localhost:9091/execution/"),
log_path=workload.log_path,
)


if not AIRFLOW_V_3_0_PLUS:

@app.task
def execute_command(command_to_exec: CommandType) -> None:
"""Execute command."""
dag_id, task_id = BaseExecutor.validate_airflow_tasks_run_command(command_to_exec)
celery_task_id = app.current_task.request.id
log.info("[%s] Executing command in Celery: %s", celery_task_id, command_to_exec)
with _airflow_parsing_context_manager(dag_id=dag_id, task_id=task_id):
try:
if settings.EXECUTE_TASKS_NEW_PYTHON_INTERPRETER:
_execute_in_subprocess(command_to_exec, celery_task_id)
else:
_execute_in_fork(command_to_exec, celery_task_id)
except Exception:
Stats.incr("celery.execute_command.failure")
raise


def _execute_in_fork(command_to_exec: CommandType, celery_task_id: str | None = None) -> None:
Expand Down Expand Up @@ -213,15 +247,15 @@ def send_task_to_executor(
task_tuple: TaskInstanceInCelery,
) -> tuple[TaskInstanceKey, CommandType, AsyncResult | ExceptionWithTraceback]:
"""Send task to executor."""
key, command, queue, task_to_run = task_tuple
key, args, queue, task_to_run = task_tuple
try:
with timeout(seconds=OPERATION_TIMEOUT):
result = task_to_run.apply_async(args=[command], queue=queue)
result = task_to_run.apply_async(args=args, queue=queue)
except (Exception, AirflowTaskTimeout) as e:
exception_traceback = f"Celery Task ID: {key}\n{traceback.format_exc()}"
result = ExceptionWithTraceback(e, exception_traceback)

return key, command, result
return key, args, result


def fetch_celery_task_state(async_result: AsyncResult) -> tuple[str, str | ExceptionWithTraceback, Any]:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@

from airflow.configuration import conf
from airflow.exceptions import AirflowConfigException, AirflowException
from airflow.providers.celery.version_compat import AIRFLOW_V_3_0_PLUS


def _broker_supports_visibility_timeout(url):
Expand Down Expand Up @@ -67,7 +68,7 @@ def _broker_supports_visibility_timeout(url):
result_backend = conf.get_mandatory_value("celery", "RESULT_BACKEND")
else:
log.debug("Value for celery result_backend not found. Using sql_alchemy_conn with db+ prefix.")
result_backend = f'db+{conf.get("database", "SQL_ALCHEMY_CONN")}'
result_backend = f"db+{conf.get('database', 'SQL_ALCHEMY_CONN')}"

extra_celery_config = conf.getjson("celery", "extra_celery_config", fallback={})

Expand All @@ -81,6 +82,9 @@ def _broker_supports_visibility_timeout(url):
"task_track_started": conf.getboolean("celery", "task_track_started", fallback=True),
"broker_url": broker_url,
"broker_transport_options": broker_transport_options,
"broker_connection_retry_on_startup": conf.getboolean(
"celery", "broker_connection_retry_on_startup", fallback=True
),
"result_backend": result_backend,
"database_engine_options": conf.getjson(
"celery", "result_backend_sqlalchemy_engine_options", fallback={}
Expand All @@ -90,6 +94,11 @@ def _broker_supports_visibility_timeout(url):
**(extra_celery_config if isinstance(extra_celery_config, dict) else {}),
}

# In order to not change anything pre Task Execution API, we leave this setting as it was (unset) in Airflow2
if AIRFLOW_V_3_0_PLUS:
DEFAULT_CELERY_CONFIG.setdefault("worker_redirect_stdouts", False)
DEFAULT_CELERY_CONFIG.setdefault("worker_hijack_root_logger", False)


def _get_celery_ssl_active() -> bool:
try:
Expand Down Expand Up @@ -126,9 +135,7 @@ def _get_celery_ssl_active() -> bool:
DEFAULT_CELERY_CONFIG["broker_use_ssl"] = broker_use_ssl
except AirflowConfigException:
raise AirflowException(
"AirflowConfigException: SSL_ACTIVE is True, "
"please ensure SSL_KEY, "
"SSL_CERT and SSL_CACERT are set"
"AirflowConfigException: SSL_ACTIVE is True, please ensure SSL_KEY, SSL_CERT and SSL_CACERT are set"
)
except Exception as e:
raise AirflowException(
Expand Down
Loading

0 comments on commit 8263c94

Please sign in to comment.