From e6d2b94e28c2ef48e92d8d6e0b16cf9183f5be3c Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 9 Jan 2023 23:35:23 -0800 Subject: [PATCH 01/96] Enable trigger logging fix tests fix tests fix tests remove changes that were not necessary review comments last cleanup for removal of is_trigger simplify better diff remove the is_trigger param simplify simplify Add trigger logs web UI --- .../api_connexion/endpoints/log_endpoint.py | 12 +- airflow/api_connexion/openapi/v1.yaml | 6 + airflow/cli/cli_parser.py | 1 + airflow/cli/commands/triggerer_command.py | 30 +- airflow/config_templates/config.yml | 8 + airflow/config_templates/default_airflow.cfg | 4 + airflow/jobs/triggerer_job.py | 224 ++++++++- airflow/models/taskinstance.py | 22 +- airflow/models/trigger.py | 14 +- .../amazon/aws/log/s3_task_handler.py | 67 ++- .../microsoft/azure/log/wasb_task_handler.py | 12 +- airflow/triggers/base.py | 13 +- airflow/triggers/temporal.py | 13 +- airflow/utils/log/file_task_handler.py | 196 +++++--- airflow/utils/log/log_reader.py | 17 +- airflow/utils/log/trigger_handler.py | 134 ++++++ airflow/utils/serve_logs.py | 8 +- airflow/www/static/js/api/useGridData.ts | 2 + airflow/www/static/js/api/useTaskLog.ts | 6 +- .../static/js/components/TabWithTooltip.tsx | 47 ++ .../dag/details/taskInstance/Logs/index.tsx | 3 + .../js/dag/details/taskInstance/index.tsx | 91 +++- airflow/www/static/js/dag/grid/index.test.tsx | 1 + airflow/www/static/js/types/api-generated.ts | 2 + airflow/www/static/js/types/index.ts | 1 + airflow/www/views.py | 5 + .../triggerer/triggerer-deployment.yaml | 35 +- .../triggerer/triggerer-networkpolicy.yaml | 55 +++ .../triggerer/triggerer-service.yaml | 48 ++ chart/values.schema.json | 51 ++ chart/values.yaml | 19 +- tests/charts/test_basic_helm_chart.py | 10 +- tests/charts/test_extra_env_env_from.py | 3 +- tests/charts/test_rbac.py | 3 +- tests/charts/test_triggerer.py | 40 +- tests/jobs/test_triggerer_job.py | 56 ++- tests/jobs/test_triggerer_job_logging.py | 445 ++++++++++++++++++ .../amazon/aws/log/test_s3_task_handler.py | 32 +- tests/utils/log/test_log_reader.py | 16 +- tests/utils/test_log_handlers.py | 37 +- tests/www/views/test_views_grid.py | 5 + tests/www/views/test_views_tasks.py | 23 +- 42 files changed, 1637 insertions(+), 180 deletions(-) create mode 100644 airflow/utils/log/trigger_handler.py create mode 100644 airflow/www/static/js/components/TabWithTooltip.tsx create mode 100644 chart/templates/triggerer/triggerer-networkpolicy.yaml create mode 100644 chart/templates/triggerer/triggerer-service.yaml create mode 100644 tests/jobs/test_triggerer_job_logging.py diff --git a/airflow/api_connexion/endpoints/log_endpoint.py b/airflow/api_connexion/endpoints/log_endpoint.py index 388b164727e90..54ed462fe0168 100644 --- a/airflow/api_connexion/endpoints/log_endpoint.py +++ b/airflow/api_connexion/endpoints/log_endpoint.py @@ -21,6 +21,7 @@ from flask import Response, request from itsdangerous.exc import BadSignature from itsdangerous.url_safe import URLSafeSerializer +from sqlalchemy.orm import joinedload from sqlalchemy.orm.session import Session from airflow.api_connexion import security @@ -31,6 +32,7 @@ from airflow.models import TaskInstance from airflow.security import permissions from airflow.utils.airflow_flask_app import get_airflow_app +from airflow.utils.log.file_task_handler import LogType from airflow.utils.log.log_reader import TaskLogReader from airflow.utils.session import NEW_SESSION, provide_session @@ -52,6 +54,7 @@ def get_log( full_content: bool = False, map_index: int = -1, token: str | None = None, + log_type: LogType | None = None, session: Session = NEW_SESSION, ) -> APIResponse: """Get logs for specific task instance.""" @@ -73,6 +76,9 @@ def get_log( metadata["download_logs"] = False task_log_reader = TaskLogReader() + if log_type == LogType.TRIGGER and not task_log_reader.supports_triggerer: + raise BadRequest("Task log handler does not support trigger logging.") + if not task_log_reader.supports_read: raise BadRequest("Task log handler does not support read logs.") ti = ( @@ -84,6 +90,8 @@ def get_log( TaskInstance.map_index == map_index, ) .join(TaskInstance.dag_run) + .options(joinedload("trigger")) + .options(joinedload("trigger.triggerer_job")) .one_or_none() ) if ti is None: @@ -102,12 +110,12 @@ def get_log( # return_type would be either the above two or None logs: Any if return_type == "application/json" or return_type is None: # default - logs, metadata = task_log_reader.read_log_chunks(ti, task_try_number, metadata) + logs, metadata = task_log_reader.read_log_chunks(ti, task_try_number, metadata, log_type=log_type) logs = logs[0] if task_try_number is not None else logs # we must have token here, so we can safely ignore it token = URLSafeSerializer(key).dumps(metadata) # type: ignore[assignment] return logs_schema.dump(LogResponseObject(continuation_token=token, content=logs)) # text/plain. Stream - logs = task_log_reader.read_log_stream(ti, task_try_number, metadata) + logs = task_log_reader.read_log_stream(ti, task_try_number, metadata, log_type=log_type) return Response(logs, headers={"Content-Type": return_type}) diff --git a/airflow/api_connexion/openapi/v1.yaml b/airflow/api_connexion/openapi/v1.yaml index f2cc9b3af20d8..1f62eb0d6241a 100644 --- a/airflow/api_connexion/openapi/v1.yaml +++ b/airflow/api_connexion/openapi/v1.yaml @@ -1656,6 +1656,12 @@ paths: - $ref: '#/components/parameters/FullContent' - $ref: '#/components/parameters/FilterMapIndex' - $ref: '#/components/parameters/ContinuationToken' + - name: log_type + in: query + schema: + type: string + required: false + get: summary: Get logs diff --git a/airflow/cli/cli_parser.py b/airflow/cli/cli_parser.py index 8056bf2712777..6a78b57904b99 100644 --- a/airflow/cli/cli_parser.py +++ b/airflow/cli/cli_parser.py @@ -2005,6 +2005,7 @@ class GroupCommand(NamedTuple): ARG_LOG_FILE, ARG_CAPACITY, ARG_VERBOSE, + ARG_SKIP_SERVE_LOGS, ), ), ActionCommand( diff --git a/airflow/cli/commands/triggerer_command.py b/airflow/cli/commands/triggerer_command.py index 64755f3830291..c4e4921e99d9a 100644 --- a/airflow/cli/commands/triggerer_command.py +++ b/airflow/cli/commands/triggerer_command.py @@ -18,14 +18,36 @@ from __future__ import annotations import signal +from contextlib import contextmanager +from multiprocessing import Process +from typing import Generator import daemon from daemon.pidfile import TimeoutPIDLockFile from airflow import settings +from airflow.configuration import conf from airflow.jobs.triggerer_job import TriggererJob from airflow.utils import cli as cli_utils from airflow.utils.cli import setup_locations, setup_logging, sigint_handler, sigquit_handler +from airflow.utils.serve_logs import serve_logs + + +@contextmanager +def _serve_logs(skip_serve_logs: bool = False) -> Generator[None, None, None]: + """Starts serve_logs sub-process""" + sub_proc = None + if skip_serve_logs is False: + from functools import partial + + port = conf.getint("logging", "trigger_log_server_port", fallback=8794) + sub_proc = Process(target=partial(serve_logs, port=port)) + sub_proc.start() + try: + yield + finally: + if sub_proc: + sub_proc.terminate() @cli_utils.action_cli @@ -44,18 +66,18 @@ def triggerer(args): stdout_handle.truncate(0) stderr_handle.truncate(0) - ctx = daemon.DaemonContext( + daemon_context = daemon.DaemonContext( pidfile=TimeoutPIDLockFile(pid, -1), files_preserve=[handle], stdout=stdout_handle, stderr=stderr_handle, umask=int(settings.DAEMON_UMASK, 8), ) - with ctx: + with daemon_context, _serve_logs(args.skip_serve_logs): job.run() - else: signal.signal(signal.SIGINT, sigint_handler) signal.signal(signal.SIGTERM, sigint_handler) signal.signal(signal.SIGQUIT, sigquit_handler) - job.run() + with _serve_logs(args.skip_serve_logs): + job.run() diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml index 1fed76a25cece..7da4c176bd70e 100644 --- a/airflow/config_templates/config.yml +++ b/airflow/config_templates/config.yml @@ -788,6 +788,14 @@ logging: type: string example: ~ default: "8793" + trigger_log_server_port: + description: | + Port to serve logs from for triggerer. See worker_log_server_port description + for more info. + version_added: 2.6.0 + type: string + example: ~ + default: "8794" metrics: description: | StatsD (https://github.com/etsy/statsd) integration settings. diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index 5a6e03e404465..243bf50a86599 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -434,6 +434,10 @@ extra_logger_names = # visible from the main web server to connect into the workers. worker_log_server_port = 8793 +# Port to serve logs from for triggerer. See worker_log_server_port description +# for more info. +trigger_log_server_port = 8794 + [metrics] # StatsD (https://github.com/etsy/statsd) integration settings. diff --git a/airflow/jobs/triggerer_job.py b/airflow/jobs/triggerer_job.py index 1c62a476bf000..2d7c2ffff4526 100644 --- a/airflow/jobs/triggerer_job.py +++ b/airflow/jobs/triggerer_job.py @@ -17,26 +17,176 @@ from __future__ import annotations import asyncio +import logging import os import signal import sys import threading import time +import warnings from collections import deque -from typing import Deque +from queue import SimpleQueue +from typing import TYPE_CHECKING, Deque from sqlalchemy import func from airflow.configuration import conf from airflow.jobs.base_job import BaseJob from airflow.models.trigger import Trigger +from airflow.settings import DONOT_MODIFY_HANDLERS from airflow.stats import Stats from airflow.triggers.base import BaseTrigger, TriggerEvent from airflow.typing_compat import TypedDict +from airflow.utils.log.file_task_handler import FileTaskHandler from airflow.utils.log.logging_mixin import LoggingMixin +from airflow.utils.log.trigger_handler import ( + DropTriggerLogsFilter, + LocalQueueHandler, + TriggererHandlerWrapper, + TriggerMetadataFilter, + ctx_close_handler, + ctx_indiv_trigger, + ctx_task_instance, + ctx_trigger_id, +) from airflow.utils.module_loading import import_string from airflow.utils.session import provide_session +if TYPE_CHECKING: + from airflow.models import TaskInstance + +USING_TRIGGERER_HANDLER_WRAPPER = False +""" +If this value is true, trigger logging is configured to use TriggerHandlerWrapper + +:meta :private +""" + +logger = logging.getLogger(__name__) + + +def configure_trigger_log_handler(): + """ + Configure logging such that each trigger logs to its own file and + can be exposed through the airflow webserver. + + Generally speaking, we take the log handler configured for logger ``airflow.task``, + wrap it with TriggerHandlerWrapper, and set it as the handler for root logger. + + If there already is a handler configured for the root logger + and it supports triggers, we wrap it instead. + + :meta private: + """ + global USING_TRIGGERER_HANDLER_WRAPPER + + def supports_triggerer(handler): + return getattr(handler, "supports_triggerer", False) + + def get_task_handler_from_logger(logger_): + for h in logger_.handlers: + if isinstance(h, FileTaskHandler) and not supports_triggerer(h): + warnings.warn( + f"Handler {h.__class__.__name__} does not support " + "individual trigger logging. Please check the release notes " + "for your provider to see if a newer version supports " + "individual trigger logging." + ) + if supports_triggerer(h): + return h + + def find_suitable_task_handler(): + # check root logger then check airflow.task to see if a handler + # suitable for use with TriggerHandlerWrapper (has supports_triggerer + # attr, likely inherits from FileTaskHandler) + h = get_task_handler_from_logger(root_logger) + if not h: + # try to use handler configured from airflow task + logger.debug("No task logger configured for root logger; trying `airflow.task`.") + h = get_task_handler_from_logger(logging.getLogger("airflow.task")) + if h: + logger.debug("Using logging configuration from `airflow.task`") + if not h: + warnings.warn("Could not find log handler suitable for individual trigger logging.") + return None + return h + + def filter_trigger_logs_from_other_root_handlers(new_hdlr): + # we add context vars to log records emitted for individual triggerer logging + # we want these records to be processed by our special trigger handler wrapper + # but not by any other handlers, so we filter out these messages from + # other handlers by adding DropTriggerLogsFilter + # we could consider only adding this filter to the default console logger + # so as to leave other custom handlers alone + for h in root_logger.handlers: + if h is not new_hdlr: + h.addFilter(DropTriggerLogsFilter()) + + def add_handler_wrapper_to_root(base_handler): + # first make sure we remove from root logger if it happens to be there + # it could have come from root or airflow.task, but we only need + # to make sure we remove from root, since messages will not flow + # through airflow.task + if base_handler in root_logger.handlers: + root_logger.removeHandler(base_handler) + + logger.info("Setting up TriggererHandlerWrapper with handler %s", base_handler) + h = TriggererHandlerWrapper(base_handler=base_handler, level=base_handler.level) + # just extra cautious, checking if user manually configured it there + if h not in root_logger.handlers: + root_logger.addHandler(h) + return h + + root_logger = logging.getLogger() + task_handler = find_suitable_task_handler() + if not task_handler: + return None + if TYPE_CHECKING: + assert isinstance(task_handler, FileTaskHandler) + wrapper_handler = add_handler_wrapper_to_root(task_handler) + filter_trigger_logs_from_other_root_handlers(wrapper_handler) + USING_TRIGGERER_HANDLER_WRAPPER = True + return None + + +def setup_queue_listener(): + """ + Route log messages to a queue and process them with QueueListener. + + Airflow task handlers make blocking I/O calls. + We replace trigger log handlers, with LocalQueueHandler, + which sends log records to a queue. + Then we start a QueueListener in a thread, which is configured + to consume the queue and pass the records to the handlers as + originally configured. This keeps the handler I/O out of the + async event loop. + + :meta private: + """ + queue = SimpleQueue() + root_logger = logging.getLogger() + + handlers: list[logging.Handler] = [] + + queue_handler = LocalQueueHandler(queue) + queue_handler.addFilter(TriggerMetadataFilter()) + + root_logger.addHandler(queue_handler) + for h in root_logger.handlers[:]: + if h is not queue_handler and "pytest" not in h.__module__: + root_logger.removeHandler(h) + handlers.append(h) + + this_logger = logging.getLogger(__name__) + if handlers: + this_logger.info("Setting up logging queue listener with handlers %s", handlers) + listener = logging.handlers.QueueListener(queue, *handlers, respect_handler_level=True) + listener.start() + return listener + else: + this_logger.warning("Unable to set up individual trigger logging") + return None + class TriggererJob(BaseJob): """ @@ -62,6 +212,10 @@ def __init__(self, capacity=None, *args, **kwargs): else: raise ValueError(f"Capacity number {capacity} is invalid") + if not DONOT_MODIFY_HANDLERS: + configure_trigger_log_handler() + self.listener = setup_queue_listener() + # Set up runner async thread self.runner = TriggerRunner() @@ -87,12 +241,19 @@ def on_kill(self): """ self.runner.stop = True + def _kill_listener(self): + if self.listener: + for h in self.listener.handlers: + h.close() + self.listener.stop() + def _exit_gracefully(self, signum, frame) -> None: """Helper method to clean up processor_agent to avoid leaving orphan processes.""" # The first time, try to exit nicely if not self.runner.stop: self.log.info("Exiting gracefully upon receiving signal %s", signum) self.runner.stop = True + self._kill_listener() else: self.log.warning("Forcing exit due to second exit signal %s", signum) sys.exit(os.EX_SOFTWARE) @@ -100,6 +261,9 @@ def _exit_gracefully(self, signum, frame) -> None: def _execute(self) -> None: self.log.info("Starting the triggerer") try: + # set job_id so that it can be used in log file names + self.runner.job_id = self.id + # Kick off runner thread self.runner.start() # Start our own DB loop in the main thread @@ -224,6 +388,7 @@ def __init__(self): self.to_cancel = deque() self.events = deque() self.failed_triggers = deque() + self.job_id = None def run(self): """Sync entrypoint - just runs arun in an async loop.""" @@ -245,11 +410,10 @@ async def arun(self): await self.cleanup_finished_triggers() # Sleep for a bit await asyncio.sleep(1) - # Every minute, log status if at least one trigger is running. + # Every minute, log status if time.time() - last_status >= 60: count = len(self.triggers) - if count > 0: - self.log.info("%i triggers currently running", count) + self.log.info("%i triggers currently running", count) last_status = time.time() # Wait for watchdog to complete await watchdog @@ -349,15 +513,29 @@ async def block_watchdog(self): ) Stats.incr("triggers.blocked_main_thread") - # Async trigger logic + @staticmethod + def set_individual_trigger_logging(trigger): + """ + Setting these context vars allows log messages for individual triggers + to be routed to distinct files and filtered from triggerer stdout. + """ + # set logging context vars for routing to appropriate handler + ctx_task_instance.set(trigger.task_instance) + ctx_trigger_id.set(trigger.trigger_id) + ctx_close_handler.set(False) + + # mark that we're in the context of an individual trigger so log records can be filtered + ctx_indiv_trigger.set(True) async def run_trigger(self, trigger_id, trigger): """ Wrapper which runs an actual trigger (they are async generators) and pushes their events into our outbound event deque. """ - self.log.info("Trigger %s starting", self.triggers[trigger_id]["name"]) + name = self.triggers[trigger_id]["name"] + self.log.info("trigger %s starting", name) try: + self.set_individual_trigger_logging(trigger) async for event in trigger.run(): self.log.info("Trigger %s fired: %s", self.triggers[trigger_id]["name"], event) self.triggers[trigger_id]["events"] += 1 @@ -368,8 +546,18 @@ async def run_trigger(self, trigger_id, trigger): # allow triggers a chance to cleanup, either in that case or if # they exit cleanly. trigger.cleanup() + self.close_handler(trigger) - # Main-thread sync API + # unsetting ctx_indiv_trigger var restores stdout logging + ctx_indiv_trigger.set(None) + self.log.info("trigger %s completed", name) + + @staticmethod + def close_handler(trigger): + if not USING_TRIGGERER_HANDLER_WRAPPER: + return + ctx_close_handler.set(True) + trigger.log.log(level=100, msg="close handler") def update_triggers(self, requested_trigger_ids: set[int]): """ @@ -404,16 +592,34 @@ def update_triggers(self, requested_trigger_ids: set[int]): continue # Resolve trigger record into an actual class instance try: - trigger_class = self.get_trigger_by_classpath(new_triggers[new_id].classpath) + new_trigger_orm = new_triggers[new_id] + trigger_class = self.get_trigger_by_classpath(new_trigger_orm.classpath) except BaseException as e: # Either the trigger code or the path to it is bad. Fail the trigger. self.failed_triggers.append((new_id, e)) continue - self.to_create.append((new_id, trigger_class(**new_triggers[new_id].kwargs))) + new_trigger_instance = trigger_class(**new_trigger_orm.kwargs) + self.set_trigger_logging_metadata(new_trigger_orm.task_instance, new_id, new_trigger_instance) + self.to_create.append((new_id, new_trigger_instance)) # Enqueue orphaned triggers for cancellation for old_id in cancel_trigger_ids: self.to_cancel.append(old_id) + def set_trigger_logging_metadata(self, ti: TaskInstance, trigger_id, trigger): + """ + Set up logging for triggers + + We want to ensure that each trigger logs to its own file and that the log messages are not + propagated to parent loggers. + + :meta private: + """ + if ti: # can be None in tests + ti.is_trigger_log_context = True + trigger.task_instance = ti + trigger.triggerer_job_id = self.job_id + trigger.trigger_id = trigger_id + def get_trigger_by_classpath(self, classpath: str) -> type[BaseTrigger]: """ Gets a trigger class by its classpath ("path.to.module.classname"). diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index e8d9fe8ceec47..b3031d0610d53 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -439,7 +439,7 @@ class TaskInstance(Base, LoggingMixin): viewonly=True, ) - trigger = relationship("Trigger", uselist=False) + trigger = relationship("Trigger", uselist=False, back_populates="task_instance") triggerer_job = association_proxy("trigger", "triggerer_job") dag_run = relationship("DagRun", back_populates="task_instances", lazy="joined", innerjoin=True) rendered_task_instance_fields = relationship("RenderedTaskInstanceFields", lazy="noload", uselist=False) @@ -512,6 +512,26 @@ def __init__( # can be changed when calling 'run' self.test_mode = False + self._is_trigger_log_context: bool = False + """ + Flag to indicate to FileTaskHandler that logging context should be set up for + trigger logging. + """ + + @property + def is_trigger_log_context(self): + """ + Flag to indicate to FileTaskHandler that logging context should be set up for + trigger logging. + + :meta private: + """ + return getattr(self, "_is_trigger_log_context", False) + + @is_trigger_log_context.setter + def is_trigger_log_context(self, value): + self._is_trigger_log_context = value + @staticmethod def insert_mapping(run_id: str, task: Operator, map_index: int) -> dict[str, Any]: """:meta private:""" diff --git a/airflow/models/trigger.py b/airflow/models/trigger.py index 57d2ac8f2600a..0a4e83440a77a 100644 --- a/airflow/models/trigger.py +++ b/airflow/models/trigger.py @@ -21,7 +21,7 @@ from typing import Any, Iterable from sqlalchemy import Column, Integer, String, func, or_ -from sqlalchemy.orm import relationship +from sqlalchemy.orm import joinedload, relationship from airflow.models.base import Base from airflow.models.taskinstance import TaskInstance @@ -65,6 +65,8 @@ class Trigger(Base): uselist=False, ) + task_instance = relationship("TaskInstance", back_populates="trigger", lazy="joined", uselist=False) + def __init__(self, classpath: str, kwargs: dict[str, Any], created_date: datetime.datetime | None = None): super().__init__() self.classpath = classpath @@ -87,7 +89,15 @@ def bulk_fetch(cls, ids: Iterable[int], session=None) -> dict[int, Trigger]: Fetches all of the Triggers by ID and returns a dict mapping ID -> Trigger instance """ - return {obj.id: obj for obj in session.query(cls).filter(cls.id.in_(ids)).all()} + return { + obj.id: obj + for obj in session.query(cls) + .filter(cls.id.in_(ids)) + .options(joinedload("task_instance")) + .options(joinedload("task_instance.trigger")) + .options(joinedload("task_instance.trigger.triggerer_job")) + .all() + } @classmethod @provide_session diff --git a/airflow/providers/amazon/aws/log/s3_task_handler.py b/airflow/providers/amazon/aws/log/s3_task_handler.py index 8535277b13e70..9eb6be85c8d14 100644 --- a/airflow/providers/amazon/aws/log/s3_task_handler.py +++ b/airflow/providers/amazon/aws/log/s3_task_handler.py @@ -26,6 +26,15 @@ from airflow.utils.log.file_task_handler import FileTaskHandler from airflow.utils.log.logging_mixin import LoggingMixin +try: + from airflow.utils.log.file_task_handler import LogType +except ImportError: + # todo: remove when min airflow version >= 2.6 + class LogType: # type: ignore[no-redef] + """For backcompat""" + + TRIGGER = object() + class S3TaskHandler(FileTaskHandler, LoggingMixin): """ @@ -53,9 +62,13 @@ def set_context(self, ti): super().set_context(ti) # Local location and remote location is needed to open and # upload local log file to S3 remote storage. - self.log_relative_path = self._render_filename(ti, ti.try_number) - self.upload_on_close = not ti.raw - + log_relative_path = self._render_filename(ti, ti.try_number) + # use getattr for compat with airflow < 2.6 + is_trigger_log_context = getattr(ti, "is_trigger_log_context", False) + if is_trigger_log_context: + log_relative_path = self.add_triggerer_suffix(log_relative_path, ti.triggerer_job.id) + self.log_relative_path = log_relative_path + self.upload_on_close = is_trigger_log_context or not ti.raw # Clear the file first so that duplicate data is not uploaded # when re-using the same path (e.g. with rescheduled sensors) if self.upload_on_close: @@ -86,7 +99,7 @@ def close(self): # Mark closed so we don't double write if close is called twice self.closed = True - def _read(self, ti, try_number, metadata=None): + def _read(self, ti, try_number, metadata=None, *, log_type=None): """ Read logs of given task instance and try_number from S3 remote storage. If failed, read the log from task instance host machine. @@ -99,28 +112,44 @@ def _read(self, ti, try_number, metadata=None): # Explicitly getting log relative path is necessary as the given # task instance might be different than task instance passed in # in set_context method. - log_relative_path = self._render_filename(ti, try_number) - remote_loc = os.path.join(self.remote_base, log_relative_path) - - log_exists = False log = "" + keys = [] + worker_log_rel_path = self._render_filename(ti, try_number) + if log_type == LogType.TRIGGER: + if ti.triggerer_job: + # triggerer currently running; skip s3 read and try to read from triggerer log server + return super()._read(ti, try_number, metadata, log_type=log_type) + trigger_log_rel_prefix = self.add_triggerer_suffix(worker_log_rel_path) + bucket, prefix = self.hook.parse_s3_url( + s3url=os.path.join(self.remote_base, trigger_log_rel_prefix) + ) + keys = self.hook.list_keys(bucket_name=bucket, prefix=prefix) + if keys: + keys = [f"s3://{bucket}/{key}" for key in keys] + else: + log += f"*** No logs found for triggerer; ti=%s {ti}\n" + else: + remote_loc = os.path.join(self.remote_base, worker_log_rel_path) + try: + if self.s3_log_exists(remote_loc): + keys = [remote_loc] + except Exception as error: + self.log.exception("Failed to verify remote log exists %s.", remote_loc) + log += f"*** Failed to verify remote log exists {remote_loc}.\n{error}\n" - try: - log_exists = self.s3_log_exists(remote_loc) - except Exception as error: - self.log.exception("Failed to verify remote log exists %s.", remote_loc) - log = f"*** Failed to verify remote log exists {remote_loc}.\n{error}\n" - - if log_exists: + if keys: # If S3 remote file exists, we do not fetch logs from task instance # local machine even if there are errors reading remote logs, as # returned remote_log will contain error messages. - remote_log = self.s3_read(remote_loc, return_error=True) - log = f"*** Reading remote log from {remote_loc}.\n{remote_log}\n" + for key in keys: + remote_log = self.s3_read(key, return_error=True) + log += f"*** Reading remote log from {key}.\n{remote_log}\n" return log, {"end_of_log": True} else: log += "*** Falling back to local log\n" - local_log, metadata = super()._read(ti, try_number, metadata) + # todo: after min airflow version >= 2.6, just forward log_type to super()._read + kwargs = {"log_type": log_type} if log_type else {} + local_log, metadata = super()._read(ti, try_number, metadata, **kwargs) return log + local_log, metadata def s3_log_exists(self, remote_log_location: str) -> bool: @@ -172,7 +201,7 @@ def s3_write(self, log: str, remote_log_location: str, append: bool = True, max_ # Default to a single retry attempt because s3 upload failures are # rare but occasionally occur. Multiple retry attempts are unlikely - # to help as they usually indicate non-empheral errors. + # to help as they usually indicate non-ephemeral errors. for try_num in range(1 + max_retry): try: self.hook.load_string( diff --git a/airflow/providers/microsoft/azure/log/wasb_task_handler.py b/airflow/providers/microsoft/azure/log/wasb_task_handler.py index 8c0fe220830ef..b8cce38edaf5d 100644 --- a/airflow/providers/microsoft/azure/log/wasb_task_handler.py +++ b/airflow/providers/microsoft/azure/log/wasb_task_handler.py @@ -70,6 +70,11 @@ def hook(self): ) return None + @cached_property + def supports_triggerer(self): + """Not implemented yet.""" + return False + def set_context(self, ti) -> None: super().set_context(ti) # Local location and remote location is needed to open and @@ -105,7 +110,7 @@ def close(self) -> None: self.closed = True def _read( - self, ti, try_number: int, metadata: dict[str, Any] | None = None + self, ti, try_number: int, metadata: dict[str, Any] | None = None, *, log_type=None ) -> tuple[str, dict[str, bool]]: """ Read logs of given task instance and try_number from Wasb remote storage. @@ -113,8 +118,9 @@ def _read( :param ti: task instance object :param try_number: task instance try_number to read logs from - :param metadata: log metadata, - can be used for steaming log reading and auto-tailing. + :param metadata: log metadata, can be used for steaming log reading and auto-tailing. + :param log_type: Currently unused. In future will control whether to retrieve + logs from triggerer or worker. """ # Explicitly getting log relative path is necessary as the given # task instance might be different than task instance passed in diff --git a/airflow/triggers/base.py b/airflow/triggers/base.py index 06bce36a4c73c..f616df66a08c7 100644 --- a/airflow/triggers/base.py +++ b/airflow/triggers/base.py @@ -37,7 +37,18 @@ class BaseTrigger(abc.ABC, LoggingMixin): """ def __init__(self, **kwargs): - pass + + # these values are set by triggerer when preparing to run the instance + # when run, they are injected into logger record. + self.task_instance = None + self.trigger_id = None + + def _set_context(self, context): + """ + This method, part of LoggingMixin, is used mainly for configuration of logging + for tasks, but is not used for triggers. + """ + raise NotImplementedError @abc.abstractmethod def serialize(self) -> tuple[str, dict[str, Any]]: diff --git a/airflow/triggers/temporal.py b/airflow/triggers/temporal.py index 3967940a7e9ba..d7f8ab5299bfd 100644 --- a/airflow/triggers/temporal.py +++ b/airflow/triggers/temporal.py @@ -58,13 +58,20 @@ async def run(self): "the number of seconds until the time" in case the system clock changes unexpectedly, or handles a DST change poorly. """ - # Sleep an hour at a time while it's more than 2 hours away - while (self.moment - timezone.utcnow()).total_seconds() > 2 * 3600: - await asyncio.sleep(3600) + # Sleep in successively smaller increments starting from 1 hour down to 10 seconds at a time + self.log.info("trigger starting") + for step in 3600, 60, 10: + seconds_remaining = (self.moment - timezone.utcnow()).total_seconds() + while seconds_remaining > 2 * step: + self.log.info(f"{int(seconds_remaining)} seconds remaining; sleeping {step} seconds") + await asyncio.sleep(step) + seconds_remaining = (self.moment - timezone.utcnow()).total_seconds() # Sleep a second at a time otherwise while self.moment > timezone.utcnow(): + self.log.info("sleeping 1 second...") await asyncio.sleep(1) # Send our single event and then we're done + self.log.info("yielding event with payload %r", self.moment) yield TriggerEvent(self.moment) diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index 09fbbbe0976be..a1c5638adb97b 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -18,15 +18,18 @@ """File logging handler for tasks.""" from __future__ import annotations +import inspect import logging import os import warnings from contextlib import suppress +from enum import Enum from pathlib import Path from typing import TYPE_CHECKING, Any from urllib.parse import urljoin -from airflow.configuration import AirflowConfigException, conf +from airflow.compat.functools import cached_property +from airflow.configuration import conf from airflow.exceptions import RemovedInAirflow3Warning from airflow.utils.context import Context from airflow.utils.helpers import parse_template_string, render_template_to_string @@ -39,6 +42,37 @@ from airflow.models import TaskInstance +class LogType(str, Enum): + """ + Type of service from which we retrieve logs. + + :meta private: + """ + + TRIGGER = "trigger" + WORKER = "worker" + + +def _fetch_logs_from_service(url, log_relative_path): + import httpx + + from airflow.utils.jwt_signer import JWTSigner + + timeout = conf.getint("webserver", "log_fetch_timeout_sec", fallback=None) + signer = JWTSigner( + secret_key=conf.get("webserver", "secret_key"), + expiration_time_in_seconds=conf.getint("webserver", "log_request_clock_grace", fallback=30), + audience="task-instance-logs", + ) + response = httpx.get( + url, + timeout=timeout, + headers={"Authorization": signer.generate_signed_token({"filename": log_relative_path})}, + ) + response.encoding = "utf-8" + return response + + class FileTaskHandler(logging.Handler): """ FileTaskHandler is a python log handler that handles and reads @@ -88,6 +122,32 @@ def set_context(self, ti: TaskInstance) -> None | SetContextPropagate: self.handler.setLevel(self.level) return SetContextPropagate.MAINTAIN_PROPAGATE if self.maintain_propagate else None + @staticmethod + def add_triggerer_suffix(full_path, job_id=None): + """ + Helper for deriving trigger log filename from task log filename. + + E.g. given /path/to/file.log returns /path/to/file.log.trigger.123.log, where 123 + is the triggerer id. We use the triggerer ID instead of trigger ID to distinguish + the files because, rarely, the same trigger could get picked up by two different + triggerer instances. + """ + full_path = Path(full_path).as_posix() + full_path += f".{LogType.TRIGGER}" + if job_id: + full_path += f".{job_id}.log" + return full_path + + @cached_property + def supports_triggerer(self): + """ + If true, this handler has been updated to support individual logging as implemented + in triggerer_job. + + :meta private: + """ + return "log_type" in inspect.signature(self._read).parameters.keys() + def emit(self, record): if self.handler: self.handler.emit(record) @@ -101,6 +161,7 @@ def close(self): self.handler.close() def _render_filename(self, ti: TaskInstance, try_number: int) -> str: + """Returns the worker log filename.""" with create_session() as session: dag_run = ti.get_dagrun(session=session) template = dag_run.get_log_template(session=session).filename @@ -164,7 +225,14 @@ def _should_check_k8s(queue): return True return False - def _read(self, ti: TaskInstance, try_number: int, metadata: dict[str, Any] | None = None): + def _read( + self, + ti: TaskInstance, + try_number: int, + metadata: dict[str, Any] | None = None, + *, + log_type: LogType | None = LogType.WORKER, + ): """ Template method that contains custom logic of reading logs given the try_number. @@ -178,7 +246,7 @@ def _read(self, ti: TaskInstance, try_number: int, metadata: dict[str, Any] | No which was retrieved in previous calls, this part will be skipped and only following test returned to be added to tail. - + :param log_type: controls whether to fetch worker or trigger logs for task :return: log message as a string and metadata. Following attributes are used in metadata: end_of_log: Boolean, True if end of log is reached or False @@ -186,25 +254,24 @@ def _read(self, ti: TaskInstance, try_number: int, metadata: dict[str, Any] | No This is determined by the status of the TaskInstance log_pos: (absolute) Char position to which the log is retrieved """ - from airflow.utils.jwt_signer import JWTSigner - # Task instance here might be different from task instance when # initializing the handler. Thus explicitly getting log location # is needed to get correct log path. - log_relative_path = self._render_filename(ti, try_number) - location = os.path.join(self.local_base, log_relative_path) + worker_log_rel_path = self._render_filename(ti, try_number) + worker_log_full_path = Path(self.local_base, worker_log_rel_path) + local_log_files = self._get_local_log_files(worker_log_path=worker_log_full_path, log_type=log_type) log = "" - if os.path.exists(location): + if local_log_files: try: - with open(location, encoding="utf-8", errors="surrogateescape") as file: - log += f"*** Reading local file: {location}\n" - log += "".join(file.readlines()) + for file in local_log_files: + log += f"*** Reading local file: {file}\n" + log += Path(file).read_text() except Exception as e: - log = f"*** Failed to load local log file: {location}\n" + log = f"*** Failed to load local log files: {local_log_files}\n" log += f"*** {str(e)}\n" return log, {"end_of_log": True} - elif self._should_check_k8s(ti.queue): + elif self._should_check_k8s(ti.queue) and log_type != LogType.TRIGGER: try: from airflow.kubernetes.kube_client import get_kube_client from airflow.kubernetes.pod_generator import PodGenerator @@ -245,49 +312,25 @@ def _read(self, ti: TaskInstance, try_number: int, metadata: dict[str, Any] | No log += f"*** Unable to fetch logs from worker pod {ti.hostname} ***\n{str(f)}\n\n" return log, {"end_of_log": True} else: - import httpx - - url = self._get_log_retrieval_url(ti, log_relative_path) - log += f"*** Log file does not exist: {location}\n" - log += f"*** Fetching from: {url}\n" + log += "*** Logs not found locally\n" try: - timeout = None # No timeout - try: - timeout = conf.getint("webserver", "log_fetch_timeout_sec") - except (AirflowConfigException, ValueError): - pass - - signer = JWTSigner( - secret_key=conf.get("webserver", "secret_key"), - expiration_time_in_seconds=conf.getint( - "webserver", "log_request_clock_grace", fallback=30 - ), - audience="task-instance-logs", - ) - response = httpx.get( - url, - timeout=timeout, - headers={"Authorization": signer.generate_signed_token({"filename": log_relative_path})}, - ) - response.encoding = "utf-8" - + url, rel_path = self._get_log_retrieval_url(ti, worker_log_rel_path, log_type=log_type) + log += f"*** Fetching from: {url}\n" + response = _fetch_logs_from_service(url, rel_path) if response.status_code == 403: log += ( "*** !!!! Please make sure that all your Airflow components (e.g. " - "schedulers, webservers and workers) have " + "schedulers, webservers, workers and triggerer) have " "the same 'secret_key' configured in 'webserver' section and " "time is synchronized on all your machines (for example with ntpd) !!!!!\n***" - ) - log += ( "*** See more at https://airflow.apache.org/docs/apache-airflow/" "stable/configurations-ref.html#secret-key\n***" ) # Check if the resource was properly fetched response.raise_for_status() - log += "\n" + response.text except Exception as e: - log += f"*** Failed to fetch log file from worker. {str(e)}\n" + log += f"*** Failed to fetch log file from {log_type}. {str(e)}\n" return log, {"end_of_log": True} # Process tailing if log is not at it's end @@ -296,9 +339,21 @@ def _read(self, ti: TaskInstance, try_number: int, metadata: dict[str, Any] | No if metadata and "log_pos" in metadata: previous_chars = metadata["log_pos"] log = log[previous_chars:] # Cut off previously passed log test as new tail - return log, {"end_of_log": end_of_log, "log_pos": log_pos} + def _get_local_log_files(self, worker_log_path: Path, log_type: str | None = None): + """ + If trigger log type, get sublogs. Otherwise, get requested path. + + We take worker log path because the trigger logs are always named the same as + the worker logs but with an additional suffix appended. + """ + if log_type == LogType.TRIGGER: + prefix = self.add_triggerer_suffix(worker_log_path.name, job_id=None) + return list(worker_log_path.parent.rglob(prefix + "*.*")) + elif worker_log_path.exists(): + return [worker_log_path] + @staticmethod def _get_pod_namespace(ti: TaskInstance): pod_override = ti.executor_config.get("pod_override") @@ -307,30 +362,39 @@ def _get_pod_namespace(ti: TaskInstance): namespace = pod_override.metadata.namespace return namespace or conf.get("kubernetes_executor", "namespace", fallback="default") - @staticmethod - def _get_log_retrieval_url(ti: TaskInstance, log_relative_path: str) -> str: - url = urljoin( - f"http://{ti.hostname}:{conf.get('logging', 'WORKER_LOG_SERVER_PORT')}/log/", + def _get_log_retrieval_url( + self, ti: TaskInstance, log_relative_path: str, log_type: LogType | None = None + ) -> tuple[str, str]: + """Given TI, generate URL with which to fetch logs from service log server.""" + if log_type == LogType.TRIGGER: + if not ti.triggerer_job: + raise RuntimeError("Could not build triggerer log URL; no triggerer job.") + config_key = "triggerer_log_server_port" + hostname = ti.triggerer_job.hostname + log_relative_path = self.add_triggerer_suffix(log_relative_path, job_id=ti.triggerer_job.id) + else: + hostname = ti.hostname + config_key = "worker_log_server_port" + return ( + urljoin(f"http://{hostname}:{conf.get('logging', config_key)}/log/", log_relative_path), log_relative_path, ) - return url - def read(self, task_instance, try_number=None, metadata=None): + def read(self, task_instance, try_number=None, metadata=None, log_type=None): """ Read logs of given task instance from local machine. :param task_instance: task instance object :param try_number: task instance try_number to read logs from. If None it returns all logs separated by try_number - :param metadata: log metadata, - can be used for steaming log reading and auto-tailing. + :param metadata: log metadata, can be used for steaming log reading and auto-tailing. + :param log_type: Whether to retrieve logs for triggerer or worker. :return: a list of listed tuples which order log string by host """ # Task instance increments its try number when it starts to run. # So the log for a particular task try will only show up when # try number gets incremented in DB, i.e logs produced the time # after cli run and before try_number + 1 in DB will not be displayed. - if try_number is None: next_try = task_instance.next_try_number try_numbers = list(range(1, next_try)) @@ -344,8 +408,11 @@ def read(self, task_instance, try_number=None, metadata=None): logs = [""] * len(try_numbers) metadata_array = [{}] * len(try_numbers) + + # subclasses implement _read and may not have log_type, which was added recently + kwargs = {"log_type": log_type} if log_type == LogType.TRIGGER else {} for i, try_number_element in enumerate(try_numbers): - log, out_metadata = self._read(task_instance, try_number_element, metadata) + log, out_metadata = self._read(task_instance, try_number_element, metadata, **kwargs) # es_task_handler return logs grouped by host. wrap other handler returning log string # with default/ empty host so that UI can render the response in the same way logs[i] = log if self._read_grouped_logs() else [(task_instance.hostname, log)] @@ -389,21 +456,12 @@ def _init_file(self, ti): :param ti: task instance object :return: relative log path of the given task instance """ - # To handle log writing when tasks are impersonated, the log files need to - # be writable by the user that runs the Airflow command and the user - # that is impersonated. This is mainly to handle corner cases with the - # SubDagOperator. When the SubDagOperator is run, all of the operators - # run under the impersonated user and create appropriate log files - # as the impersonated user. However, if the user manually runs tasks - # of the SubDagOperator through the UI, then the log files are created - # by the user that runs the Airflow command. For example, the Airflow - # run command may be run by the `airflow_sudoable` user, but the Airflow - # tasks may be run by the `airflow` user. If the log files are not - # writable by both users, then it's possible that re-running a task - # via the UI (or vice versa) results in a permission error as the task - # tries to write to a log file created by the other user. - relative_path = self._render_filename(ti, ti.try_number) - full_path = os.path.join(self.local_base, relative_path) + local_relative_path = self._render_filename(ti, ti.try_number) + full_path = os.path.join(self.local_base, local_relative_path) + if ti.is_trigger_log_context is True: + # if this is true, we're invoked via set_context in the context of + # setting up individual trigger logging. return trigger log path. + full_path = self.add_triggerer_suffix(full_path=full_path, job_id=ti.triggerer_job.id) self._prepare_log_folder(Path(full_path).parent) if not os.path.exists(full_path): diff --git a/airflow/utils/log/log_reader.py b/airflow/utils/log/log_reader.py index 5cc8b9377e186..9b95bd5e1e010 100644 --- a/airflow/utils/log/log_reader.py +++ b/airflow/utils/log/log_reader.py @@ -38,7 +38,7 @@ class TaskLogReader: """Time to sleep between loops while waiting for more logs""" def read_log_chunks( - self, ti: TaskInstance, try_number: int | None, metadata + self, ti: TaskInstance, try_number: int | None, metadata, *, log_type=None ) -> tuple[list[tuple[tuple[str, str]]], dict[str, str]]: """ Reads chunks of Task Instance logs. @@ -47,6 +47,7 @@ def read_log_chunks( :param try_number: If provided, logs for the given try will be returned. Otherwise, logs from all attempts are returned. :param metadata: A dictionary containing information about how to read the task log + :param log_type: Either trigger or worker, to determine where to read logs from. The following is an example of how to use this method to read log: @@ -59,11 +60,14 @@ def read_log_chunks( contain information about the task log which can enable you read logs to the end. """ - logs, metadatas = self.log_handler.read(ti, try_number, metadata=metadata) + kwargs = {"log_type": log_type} if self.supports_triggerer else {} + logs, metadatas = self.log_handler.read(ti, try_number, metadata=metadata, **kwargs) metadata = metadatas[0] return logs, metadata - def read_log_stream(self, ti: TaskInstance, try_number: int | None, metadata: dict) -> Iterator[str]: + def read_log_stream( + self, ti: TaskInstance, try_number: int | None, metadata: dict, *, log_type=None + ) -> Iterator[str]: """ Used to continuously read log to the end @@ -82,7 +86,8 @@ def read_log_stream(self, ti: TaskInstance, try_number: int | None, metadata: di metadata.pop("offset", None) metadata.pop("log_pos", None) while True: - logs, metadata = self.read_log_chunks(ti, current_try_number, metadata) + kwargs = {"log_type": log_type} if self.supports_triggerer else {} + logs, metadata = self.read_log_chunks(ti, current_try_number, metadata, **kwargs) for host, log in logs[0]: yield "\n".join([host or "", log]) + "\n" if "end_of_log" not in metadata or ( @@ -108,6 +113,10 @@ def supports_read(self): """Checks if a read operation is supported by a current log handler.""" return hasattr(self.log_handler, "read") + @property + def supports_triggerer(self): + return getattr(self.log_handler, "supports_triggerer", False) + @property def supports_external_link(self) -> bool: """Check if the logging handler supports external links (e.g. to Elasticsearch, Stackdriver, etc).""" diff --git a/airflow/utils/log/trigger_handler.py b/airflow/utils/log/trigger_handler.py new file mode 100644 index 0000000000000..f9f3f9cde1a05 --- /dev/null +++ b/airflow/utils/log/trigger_handler.py @@ -0,0 +1,134 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from __future__ import annotations + +import asyncio +import logging +from contextvars import ContextVar +from copy import copy +from logging.handlers import QueueHandler + +from airflow.utils.log.file_task_handler import FileTaskHandler + +ctx_task_instance: ContextVar = ContextVar("task_instance") +ctx_trigger_id: ContextVar = ContextVar("trigger_id") +ctx_close_handler: ContextVar = ContextVar("close_handler") +ctx_indiv_trigger: ContextVar = ContextVar("__individual_trigger") + + +class TriggerMetadataFilter(logging.Filter): + """ + Injects TI key, triggerer job_id, and trigger_id into the log record. + + :meta private: + """ + + def filter(self, record): + for var in ( + ctx_task_instance, + ctx_trigger_id, + ctx_close_handler, + ctx_indiv_trigger, + ): + val = var.get(None) + if val is not None: + setattr(record, var.name, val) + return True + + +class DropTriggerLogsFilter(logging.Filter): + """ + If record has non-empty attr trigger_id, filter the record. + + :meta private: + """ + + def filter(self, record): + return getattr(record, ctx_indiv_trigger.name, None) is None + + +class TriggererHandlerWrapper(logging.Handler): + """ + Wrap inheritors of FileTaskHandler and direct log messages + to them based on trigger_id. + + :meta private: + """ + + def __init__(self, base_handler: FileTaskHandler, level=logging.NOTSET): + super().__init__(level=level) + self.base_handler: FileTaskHandler = base_handler + self.handlers: dict[int, FileTaskHandler] = {} + + def _make_handler(self, ti): + h = copy(self.base_handler) + h.set_context(ti=ti) + return h + + def _get_or_create_handler(self, trigger_id, ti): + if trigger_id not in self.handlers: + self.handlers[trigger_id] = self._make_handler(ti) + return self.handlers[trigger_id] + + def emit(self, record): + h = self._get_or_create_handler(record.trigger_id, record.task_instance) + h.emit(record) + + def handle(self, record): + if not getattr(record, ctx_indiv_trigger.name, None): + return False + if record.close_handler: + self.close_one(record.trigger_id) + return False + emit = self.filter(record) + if emit: + self.emit(record) + return emit + + def close_one(self, trigger_id): + h = self.handlers.get(trigger_id) + if h: + h.close() + del self.handlers[trigger_id] + + def flush(self): + for _, h in self.handlers.items(): + h.flush() + + def close(self): + for trigger_id in list(self.handlers.keys()): + h = self.handlers[trigger_id] + h.close() + del self.handlers[trigger_id] + + +class LocalQueueHandler(QueueHandler): + """ + Send messages to queue. + + :meta private: + """ + + def emit(self, record: logging.LogRecord) -> None: + # There is no need to call `prepare` because queue is in same process. + try: + self.enqueue(record) + except asyncio.CancelledError: + raise + except Exception: + self.handleError(record) diff --git a/airflow/utils/serve_logs.py b/airflow/utils/serve_logs.py index 755733916461e..d647fd74d38cc 100644 --- a/airflow/utils/serve_logs.py +++ b/airflow/utils/serve_logs.py @@ -138,19 +138,19 @@ def load(self): return self.application -def serve_logs(): +def serve_logs(port=None): """Serves logs generated by Worker""" setproctitle("airflow serve-logs") wsgi_app = create_app() - worker_log_server_port = conf.getint("logging", "WORKER_LOG_SERVER_PORT") + port = port or conf.getint("logging", "WORKER_LOG_SERVER_PORT") # If dual stack is available and IPV6_V6ONLY is not enabled on the socket # then when IPV6 is bound to it will also bind to IPV4 automatically if getattr(socket, "has_dualstack_ipv6", lambda: False)(): - bind_option = GunicornOption("bind", f"[::]:{worker_log_server_port}") + bind_option = GunicornOption("bind", f"[::]:{port}") else: - bind_option = GunicornOption("bind", f"0.0.0.0:{worker_log_server_port}") + bind_option = GunicornOption("bind", f"0.0.0.0:{port}") options = [bind_option, GunicornOption("workers", 2)] StandaloneGunicornApplication(wsgi_app, options).run() diff --git a/airflow/www/static/js/api/useGridData.ts b/airflow/www/static/js/api/useGridData.ts index 091127fee2401..6dd7649189f81 100644 --- a/airflow/www/static/js/api/useGridData.ts +++ b/airflow/www/static/js/api/useGridData.ts @@ -40,6 +40,7 @@ export interface GridData { dagRuns: DagRun[]; groups: Task; ordering: RunOrdering; + readerSupportsTriggerer: boolean; } export const emptyGridData: GridData = { @@ -50,6 +51,7 @@ export const emptyGridData: GridData = { instances: [], }, ordering: [], + readerSupportsTriggerer: false, }; const formatOrdering = (data: GridData) => ({ diff --git a/airflow/www/static/js/api/useTaskLog.ts b/airflow/www/static/js/api/useTaskLog.ts index bbb6395878b4f..25366015b8419 100644 --- a/airflow/www/static/js/api/useTaskLog.ts +++ b/airflow/www/static/js/api/useTaskLog.ts @@ -32,7 +32,7 @@ interface Props extends API.GetLogVariables { } const useTaskLog = ({ - dagId, dagRunId, taskId, taskTryNumber, mapIndex, fullContent, state, + dagId, dagRunId, taskId, taskTryNumber, mapIndex, fullContent, state, logType, }: Props) => { let url: string = ''; const [isPreviousStatePending, setPrevState] = useState(true); @@ -55,14 +55,14 @@ const useTaskLog = ({ const expectingLogs = isStatePending || isPreviousStatePending; return useQuery( - ['taskLogs', dagId, dagRunId, taskId, mapIndex, taskTryNumber, fullContent], + ['taskLogs', dagId, dagRunId, taskId, mapIndex, taskTryNumber, fullContent, logType], () => { setPrevState(isStatePending); return axios.get( url, { headers: { Accept: 'text/plain' }, - params: { map_index: mapIndex, full_content: fullContent }, + params: { map_index: mapIndex, full_content: fullContent, log_type: logType }, }, ); }, diff --git a/airflow/www/static/js/components/TabWithTooltip.tsx b/airflow/www/static/js/components/TabWithTooltip.tsx new file mode 100644 index 0000000000000..c27b5f6b84517 --- /dev/null +++ b/airflow/www/static/js/components/TabWithTooltip.tsx @@ -0,0 +1,47 @@ +/*! + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +import React from 'react'; +import { + Box, + useTab, + useMultiStyleConfig, + Button, + TabProps, +} from '@chakra-ui/react'; + +const TabWithTooltip = React.forwardRef((props, ref) => { + const tabProps = useTab({ ...props, ref }); + const styles = useMultiStyleConfig('Tabs', tabProps); + + return ( + + + + ); +}); + +export default TabWithTooltip; diff --git a/airflow/www/static/js/dag/details/taskInstance/Logs/index.tsx b/airflow/www/static/js/dag/details/taskInstance/Logs/index.tsx index 0139d8e22392a..6c4323fb532b3 100644 --- a/airflow/www/static/js/dag/details/taskInstance/Logs/index.tsx +++ b/airflow/www/static/js/dag/details/taskInstance/Logs/index.tsx @@ -89,6 +89,7 @@ interface Props { executionDate: DagRun['executionDate']; tryNumber: TaskInstance['tryNumber']; state?: TaskInstance['state']; + logType?: string; } const Logs = ({ @@ -99,6 +100,7 @@ const Logs = ({ executionDate, tryNumber, state, + logType, }: Props) => { const [internalIndexes, externalIndexes] = getLinkIndexes(tryNumber); const [selectedTryNumber, setSelectedTryNumber] = useState(); @@ -117,6 +119,7 @@ const Logs = ({ taskTryNumber, fullContent: shouldRequestFullContent, state, + logType, }); const params = new URLSearchParamsWrapper({ diff --git a/airflow/www/static/js/dag/details/taskInstance/index.tsx b/airflow/www/static/js/dag/details/taskInstance/index.tsx index 894b22a5c8122..cf8e71e7fefe6 100644 --- a/airflow/www/static/js/dag/details/taskInstance/index.tsx +++ b/airflow/www/static/js/dag/details/taskInstance/index.tsx @@ -19,7 +19,7 @@ /* global localStorage */ -import React, { useRef, useState } from 'react'; +import React, { useEffect, useRef, useState } from 'react'; import { Box, Text, @@ -28,6 +28,7 @@ import { Tab, TabPanels, TabPanel, + Tooltip, } from '@chakra-ui/react'; import { useGridData, useTaskInstance } from 'src/api'; @@ -36,6 +37,8 @@ import useOffsetHeight from 'src/utils/useOffsetHeight'; import type { DagRun, TaskInstance as TaskInstanceType } from 'src/types'; import type { SelectionProps } from 'src/dag/useSelection'; import NotesAccordion from 'src/dag/details/NotesAccordion'; +import { useContainerRef } from 'src/context/containerRef'; +import TabWithTooltip from 'src/components/TabWithTooltip'; import ExtraLinks from './ExtraLinks'; import Logs from './Logs'; @@ -61,7 +64,8 @@ const TaskInstance = ({ }: Props) => { const isMapIndexDefined = !(mapIndex === undefined); const actionsMapIndexes = isMapIndexDefined ? [mapIndex] : []; - const { data: { dagRuns, groups } } = useGridData(); + const { data: { dagRuns, groups, readerSupportsTriggerer } } = useGridData(); + const containerRef = useContainerRef(); const detailsRef = useRef(null); const offsetHeight = useOffsetHeight(detailsRef); @@ -92,6 +96,32 @@ const TaskInstance = ({ setPreferedTabIndex(index); }; + let triggerLogsTooltipProps; + if (!readerSupportsTriggerer) { + triggerLogsTooltipProps = { + label: 'The configured log handler does not support reading trigger logs.', + isDisabled: false, + }; + } else if (!instance?.hasDeferred) { + triggerLogsTooltipProps = { + label: 'This task has no deferrals.', + isDisabled: false, + }; + } else { + triggerLogsTooltipProps = { + label: '', + isDisabled: true, + }; + } + const disableTriggerLogs = !(readerSupportsTriggerer && instance?.hasDeferred === true); + + useEffect(() => { + // Reset preferred tab if it is disabled + if (disableTriggerLogs && preferedTabIndex === 2) { + setPreferedTabIndex(1); + } + }, [disableTriggerLogs, preferedTabIndex]); + if (!group || !run || !instance) return null; let isPreferedTabDisplayed = false; @@ -103,6 +133,9 @@ const TaskInstance = ({ case 1: isPreferedTabDisplayed = !isGroup || (isGroup && !!isMapped); break; + case 2: // triggerer logs + isPreferedTabDisplayed = !isGroup || (isGroup && !!isMapped); + break; default: isPreferedTabDisplayed = false; } @@ -149,6 +182,19 @@ const TaskInstance = ({ Logs )} + + {!isGroupOrMappedTaskSummary && ( + + + Triggerer logs + + + )} )} + {/* Triggerer Logs Tab */} + {!isGroupOrMappedTaskSummary && ( + + {instance?.hasDeferred === true && ( + + )} + + )} + {/* Mapped Task Instances Tab */} - { - isMappedTaskSummary && !isGroup && ( - - onSelect({ runId, taskId, mapIndex: row.values.mapIndex })} - /> - - ) - } + {isMappedTaskSummary && !isGroup && ( + + onSelect({ runId, taskId, mapIndex: row.values.mapIndex })} + /> + + )} + diff --git a/airflow/www/static/js/dag/grid/index.test.tsx b/airflow/www/static/js/dag/grid/index.test.tsx index 4475a75a345a7..8b85b88466d94 100644 --- a/airflow/www/static/js/dag/grid/index.test.tsx +++ b/airflow/www/static/js/dag/grid/index.test.tsx @@ -107,6 +107,7 @@ const mockGridData = { }, ], ordering: ['dataIntervalStart'], + readerSupportsTriggerer: true, } as useGridDataModule.GridData; const EXPAND = 'Expand all task groups'; diff --git a/airflow/www/static/js/types/api-generated.ts b/airflow/www/static/js/types/api-generated.ts index 8841cd7225ddd..e79dd651ca34f 100644 --- a/airflow/www/static/js/types/api-generated.ts +++ b/airflow/www/static/js/types/api-generated.ts @@ -493,6 +493,7 @@ export interface paths { * If passed, it will specify the location from which the download should be continued. */ token?: components["parameters"]["ContinuationToken"]; + log_type?: string; }; }; }; @@ -3791,6 +3792,7 @@ export interface operations { * If passed, it will specify the location from which the download should be continued. */ token?: components["parameters"]["ContinuationToken"]; + log_type?: string; }; }; responses: { diff --git a/airflow/www/static/js/types/index.ts b/airflow/www/static/js/types/index.ts index 524e09a92a3a1..4a119bc23d07b 100644 --- a/airflow/www/static/js/types/index.ts +++ b/airflow/www/static/js/types/index.ts @@ -72,6 +72,7 @@ interface TaskInstance { tryNumber?: number; triggererJob?: Job; trigger?: Trigger; + hasDeferred?: boolean | null; // comes from grid_data not API note: string | null; } diff --git a/airflow/www/views.py b/airflow/www/views.py index 8618a61fa66ae..af4843d539351 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -145,6 +145,8 @@ "} else {xLabel = d3.time.format('%H:%M, %d %b')(new Date(parseInt(d)));} return xLabel;}" ) +task_log_reader = TaskLogReader() + def truncate_task_duration(task_duration): """ @@ -264,6 +266,7 @@ def dag_to_grid(dag: DagModel, dag_runs: Sequence[DagRun], session: Session): TaskInstance.task_id, TaskInstance.run_id, TaskInstance.state, + func.max(case((TaskInstance.next_kwargs.is_not(None), 1), else_=0)).label("has_deferred"), TaskInstance._try_number, func.min(TaskInstanceNote.content).label("note"), func.count(func.coalesce(TaskInstance.state, sqla.literal("no_status"))).label("state_count"), @@ -293,6 +296,7 @@ def _get_summary(task_instance): "end_date": task_instance.end_date, "try_number": wwwutils.get_try_count(task_instance._try_number, task_instance.state), "note": task_instance.note, + "has_deferred": bool(task_instance.has_deferred), } def _mapped_summary(ti_summaries): @@ -3617,6 +3621,7 @@ def grid_data(self): "groups": dag_to_grid(dag, dag_runs, session), "dag_runs": encoded_runs, "ordering": dag.timetable.run_ordering, + "reader_supports_triggerer": task_log_reader.supports_triggerer, } # avoid spaces to reduce payload size return ( diff --git a/chart/templates/triggerer/triggerer-deployment.yaml b/chart/templates/triggerer/triggerer-deployment.yaml index 78310394e91c9..6707b87bdaf7d 100644 --- a/chart/templates/triggerer/triggerer-deployment.yaml +++ b/chart/templates/triggerer/triggerer-deployment.yaml @@ -20,13 +20,14 @@ ################################# {{- if semverCompare ">=2.2.0" .Values.airflowVersion }} {{- if .Values.triggerer.enabled }} +{{- $persistence := .Values.triggerer.persistence.enabled }} {{- $nodeSelector := or .Values.triggerer.nodeSelector .Values.nodeSelector }} {{- $affinity := or .Values.triggerer.affinity .Values.affinity }} {{- $tolerations := or .Values.triggerer.tolerations .Values.tolerations }} {{- $topologySpreadConstraints := or .Values.triggerer.topologySpreadConstraints .Values.topologySpreadConstraints }} {{- $revisionHistoryLimit := or .Values.triggerer.revisionHistoryLimit .Values.revisionHistoryLimit }} {{- $securityContext := include "airflowSecurityContext" (list . .Values.triggerer) }} -kind: Deployment +kind: {{ if $persistence }}StatefulSet{{ else }}Deployment{{ end }} apiVersion: apps/v1 metadata: name: {{ .Release.Name }}-triggerer @@ -44,6 +45,9 @@ metadata: {{- toYaml .Values.triggerer.annotations | nindent 4 }} {{- end }} spec: + {{- if $persistence }} + serviceName: {{ .Release.Name }}-triggerer + {{- end }} replicas: {{ .Values.triggerer.replicas }} {{- if $revisionHistoryLimit }} revisionHistoryLimit: {{ $revisionHistoryLimit }} @@ -53,7 +57,11 @@ spec: tier: airflow component: triggerer release: {{ .Release.Name }} - {{- if .Values.triggerer.strategy }} + {{- if and $persistence .Values.triggerer.updateStrategy }} + updateStrategy: + {{- toYaml .Values.triggerer.updateStrategy | nindent 4 }} + {{- end }} + {{- if and (not $persistence) (.Values.triggerer.strategy) }} strategy: {{- toYaml .Values.triggerer.strategy | nindent 4 }} {{- end }} @@ -193,6 +201,9 @@ spec: {{- else }} {{- include "triggerer_liveness_check_command" . | nindent 16 }} {{- end }} + ports: + - name: triggerer-logs + containerPort: {{ .Values.ports.triggererLogs }} {{- if and (.Values.dags.gitSync.enabled) (not .Values.dags.persistence.enabled) }} {{- include "git_sync_container" . | indent 8 }} {{- end }} @@ -225,13 +236,29 @@ spec: {{- if .Values.triggerer.extraVolumes }} {{- toYaml .Values.triggerer.extraVolumes | nindent 8 }} {{- end }} - {{- if .Values.logs.persistence.enabled }} +{{- if .Values.logs.persistence.enabled }} - name: logs persistentVolumeClaim: claimName: {{ template "airflow_logs_volume_claim" . }} - {{- else }} +{{- else if not $persistence }} - name: logs emptyDir: {} +{{- else }} + volumeClaimTemplates: + - metadata: + name: logs + {{- if .Values.triggerer.persistence.annotations }} + annotations: + {{- toYaml .Values.triggerer.persistence.annotations | nindent 10 }} + {{- end }} + spec: + {{- if .Values.triggerer.persistence.storageClassName }} + storageClassName: {{ .Values.triggerer.persistence.storageClassName }} + {{- end }} + accessModes: ["ReadWriteOnce"] + resources: + requests: + storage: {{ .Values.triggerer.persistence.size }} {{- end }} {{- end }} {{- end }} diff --git a/chart/templates/triggerer/triggerer-networkpolicy.yaml b/chart/templates/triggerer/triggerer-networkpolicy.yaml new file mode 100644 index 0000000000000..e9d919607b78a --- /dev/null +++ b/chart/templates/triggerer/triggerer-networkpolicy.yaml @@ -0,0 +1,55 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +################################ +## Airflow triggerer NetworkPolicy +################################# +{{- if .Values.networkPolicies.enabled }} +{{- if .Values.triggerer.enabled }} +apiVersion: networking.k8s.io/v1 +kind: NetworkPolicy +metadata: + name: {{ .Release.Name }}-triggerer-policy + labels: + tier: airflow + component: airflow-triggerer-policy + release: {{ .Release.Name }} + chart: "{{ .Chart.Name }}-{{ .Chart.Version }}" + heritage: {{ .Release.Service }} +{{- if or (.Values.labels) (.Values.triggerer.labels) }} +{{- mustMerge .Values.triggerer.labels .Values.labels | toYaml | nindent 4 }} +{{- end }} +spec: + podSelector: + matchLabels: + tier: airflow + component: triggerer + release: {{ .Release.Name }} + policyTypes: + - Ingress + ingress: + - from: + - podSelector: + matchLabels: + tier: airflow + release: {{ .Release.Name }} + component: webserver + ports: + - protocol: TCP + port: {{ .Values.ports.triggererLogs }} +{{- end }} +{{- end }} diff --git a/chart/templates/triggerer/triggerer-service.yaml b/chart/templates/triggerer/triggerer-service.yaml new file mode 100644 index 0000000000000..c470232a74bcc --- /dev/null +++ b/chart/templates/triggerer/triggerer-service.yaml @@ -0,0 +1,48 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +################################ +## Airflow triggerer Service +################################# +{{- if semverCompare ">=2.2.0" .Values.airflowVersion }} +{{- if .Values.triggerer.enabled }} +kind: Service +apiVersion: v1 +metadata: + name: {{ .Release.Name }}-triggerer + labels: + tier: airflow + component: triggerer + release: {{ .Release.Name }} + chart: "{{ .Chart.Name }}-{{ .Chart.Version }}" + heritage: {{ .Release.Service }} +{{- if or (.Values.labels) (.Values.triggerer.labels) }} +{{- mustMerge .Values.triggerer.labels .Values.labels | toYaml | nindent 4 }} +{{- end }} +spec: + clusterIP: None + selector: + tier: airflow + component: triggerer + release: {{ .Release.Name }} + ports: + - name: triggerer-logs + protocol: TCP + port: {{ .Values.ports.triggererLogs }} + targetPort: {{ .Values.ports.triggererLogs }} +{{- end }} +{{- end }} diff --git a/chart/values.schema.json b/chart/values.schema.json index 31044ddbc2949..42de3159196e9 100644 --- a/chart/values.schema.json +++ b/chart/values.schema.json @@ -2200,6 +2200,14 @@ "exec airflow triggerer" ] }, + "updateStrategy": { + "description": "Specifies the strategy used to replace old Pods by new ones when deployed as a StatefulSet.", + "type": [ + "null", + "object" + ], + "default": null + }, "strategy": { "description": "Specifies the strategy used to replace old Pods by new ones when deployed as a Deployment.", "type": [ @@ -2241,6 +2249,44 @@ } } }, + "persistence": { + "description": "Persistence configuration.", + "type": "object", + "additionalProperties": false, + "properties": { + "enabled": { + "description": "Enable persistent volumes.", + "type": "boolean", + "default": true + }, + "size": { + "description": "Volume size for triggerer StatefulSet.", + "type": "string", + "default": "100Gi" + }, + "storageClassName": { + "description": "If using a custom StorageClass, pass name ref to all StatefulSets here.", + "type": [ + "string", + "null" + ], + "default": null + }, + "fixPermissions": { + "description": "Execute init container to chown log directory. This is currently only needed in kind, due to usage of local-path provisioner.", + "type": "boolean", + "default": false + }, + "annotations": { + "description": "Annotations to add to triggerer volumes.", + "type": "object", + "default": {}, + "additionalProperties": { + "type": "string" + } + } + } + }, "resources": { "description": "Resources for triggerer pods.", "type": "object", @@ -4949,6 +4995,11 @@ "type": "integer", "default": 8793 }, + "triggererLogs": { + "description": "Triggerer logs port.", + "type": "integer", + "default": 8794 + }, "redisDB": { "description": "Redis port.", "type": "integer", diff --git a/chart/values.yaml b/chart/values.yaml index 17231711ee342..a95c32a0afb0d 100644 --- a/chart/values.yaml +++ b/chart/values.yaml @@ -1064,7 +1064,9 @@ triggerer: # Args to use when running Airflow triggerer (templated). args: ["bash", "-c", "exec airflow triggerer"] - # Update Strategy for triggerers + # Update Strategy when triggerer is deployed as a StatefulSet + updateStrategy: ~ + # Update Strategy when triggerer is deployed as a Deployment strategy: rollingUpdate: maxSurge: "100%" @@ -1096,6 +1098,20 @@ triggerer: # fsGroup: 0 # runAsGroup: 0 + persistence: + # Enable persistent volumes + enabled: true + # Volume size for triggerer StatefulSet + size: 100Gi + # If using a custom storageClass, pass name ref to all statefulSets here + storageClassName: + # Execute init container to chown log directory. + # This is currently only needed in kind, due to usage + # of local-path provisioner. + fixPermissions: false + # Annotations to add to triggerer volumes + annotations: {} + resources: {} # limits: # cpu: 100m @@ -1630,6 +1646,7 @@ ports: flowerUI: 5555 airflowUI: 8080 workerLogs: 8793 + triggererLogs: 8794 redisDB: 6379 statsdIngest: 9125 statsdScrape: 9102 diff --git a/tests/charts/test_basic_helm_chart.py b/tests/charts/test_basic_helm_chart.py index a80b95aa35009..3b01e98832ff4 100644 --- a/tests/charts/test_basic_helm_chart.py +++ b/tests/charts/test_basic_helm_chart.py @@ -26,7 +26,7 @@ from tests.charts.helm_template_generator import render_chart -OBJECT_COUNT_IN_BASIC_DEPLOYMENT = 35 +OBJECT_COUNT_IN_BASIC_DEPLOYMENT = 36 class TestBaseChartTest: @@ -87,14 +87,15 @@ def test_basic_deployments(self, version): ("Service", "test-basic-postgresql"), ("Service", "test-basic-redis"), ("Service", "test-basic-statsd"), + ("Service", "test-basic-triggerer"), ("Service", "test-basic-webserver"), ("Service", "test-basic-worker"), ("Deployment", "test-basic-scheduler"), ("Deployment", "test-basic-statsd"), - ("Deployment", "test-basic-triggerer"), ("Deployment", "test-basic-webserver"), ("StatefulSet", "test-basic-postgresql"), ("StatefulSet", "test-basic-redis"), + ("StatefulSet", "test-basic-triggerer"), ("StatefulSet", "test-basic-worker"), ("Job", "test-basic-create-user"), ("Job", "test-basic-run-airflow-migrations"), @@ -164,15 +165,16 @@ def test_basic_deployment_with_standalone_dag_processor(self, version): ("Service", "test-basic-postgresql"), ("Service", "test-basic-redis"), ("Service", "test-basic-statsd"), + ("Service", "test-basic-triggerer"), ("Service", "test-basic-webserver"), ("Service", "test-basic-worker"), ("Deployment", "test-basic-scheduler"), ("Deployment", "test-basic-statsd"), - ("Deployment", "test-basic-triggerer"), ("Deployment", "test-basic-dag-processor"), ("Deployment", "test-basic-webserver"), ("StatefulSet", "test-basic-postgresql"), ("StatefulSet", "test-basic-redis"), + ("StatefulSet", "test-basic-triggerer"), ("StatefulSet", "test-basic-worker"), ("Job", "test-basic-create-user"), ("Job", "test-basic-run-airflow-migrations"), @@ -338,7 +340,7 @@ def test_labels_are_valid(self): (f"{release_name}-worker", "Service", "worker"), (f"{release_name}-worker", "StatefulSet", "worker"), (f"{release_name}-worker-policy", "NetworkPolicy", "airflow-worker-policy"), - (f"{release_name}-triggerer", "Deployment", "triggerer"), + (f"{release_name}-triggerer", "StatefulSet", "triggerer"), (f"{release_name}-dag-processor", "Deployment", "dag-processor"), (f"{release_name}-logs", "PersistentVolumeClaim", "logs-pvc"), (f"{release_name}-dags", "PersistentVolumeClaim", "dags-pvc"), diff --git a/tests/charts/test_extra_env_env_from.py b/tests/charts/test_extra_env_env_from.py index b32b6fae5ef50..c1de0b882fc95 100644 --- a/tests/charts/test_extra_env_env_from.py +++ b/tests/charts/test_extra_env_env_from.py @@ -59,7 +59,7 @@ ), ), ( - ("Deployment", f"{RELEASE_NAME}-triggerer"), + ("StatefulSet", f"{RELEASE_NAME}-triggerer"), ( "spec.template.spec.initContainers[0]", "spec.template.spec.containers[0]", @@ -114,6 +114,7 @@ def test_extra_env(self, k8s_obj_key, env_paths): name: {RELEASE_NAME}-some-secret """ ).lstrip() + print(k8s_obj_key) k8s_object = self.k8s_objects_by_key[k8s_obj_key] for path in env_paths: env = jmespath.search(f"{path}.env", k8s_object) diff --git a/tests/charts/test_rbac.py b/tests/charts/test_rbac.py index 9a0ad4f559330..84047af9bd387 100644 --- a/tests/charts/test_rbac.py +++ b/tests/charts/test_rbac.py @@ -35,15 +35,16 @@ ("Service", "test-rbac-flower"), ("Service", "test-rbac-pgbouncer"), ("Service", "test-rbac-redis"), + ("Service", "test-rbac-triggerer"), ("Service", "test-rbac-worker"), ("Deployment", "test-rbac-scheduler"), ("Deployment", "test-rbac-statsd"), ("Deployment", "test-rbac-webserver"), ("Deployment", "test-rbac-flower"), ("Deployment", "test-rbac-pgbouncer"), - ("Deployment", "test-rbac-triggerer"), ("StatefulSet", "test-rbac-postgresql"), ("StatefulSet", "test-rbac-redis"), + ("StatefulSet", "test-rbac-triggerer"), ("StatefulSet", "test-rbac-worker"), ("Secret", "test-rbac-broker-url"), ("Secret", "test-rbac-fernet-key"), diff --git a/tests/charts/test_triggerer.py b/tests/charts/test_triggerer.py index 8e2503455012e..e5561d27b2b9b 100644 --- a/tests/charts/test_triggerer.py +++ b/tests/charts/test_triggerer.py @@ -217,7 +217,7 @@ def test_should_create_valid_affinity_tolerations_and_node_selector(self): show_only=["templates/triggerer/triggerer-deployment.yaml"], ) - assert "Deployment" == jmespath.search("kind", docs[0]) + assert "StatefulSet" == jmespath.search("kind", docs[0]) assert "foo" == jmespath.search( "spec.template.spec.affinity.nodeAffinity." "requiredDuringSchedulingIgnoredDuringExecution." @@ -362,7 +362,10 @@ def test_livenessprobe_values_are_configurable(self): ) def test_logs_persistence_changes_volume(self, log_persistence_values, expected_volume): docs = render_chart( - values={"logs": {"persistence": log_persistence_values}}, + values={ + "triggerer": {"persistence": {"enabled": False}}, + "logs": {"persistence": log_persistence_values}, + }, show_only=["templates/triggerer/triggerer-deployment.yaml"], ) @@ -407,20 +410,43 @@ def test_resources_are_not_added_by_default(self): assert jmespath.search("spec.template.spec.containers[0].resources", docs[0]) == {} @pytest.mark.parametrize( - "strategy, expected_strategy", + "persistence, update_strategy, expected_update_strategy", + [ + (False, None, None), + (True, {"rollingUpdate": {"partition": 0}}, {"rollingUpdate": {"partition": 0}}), + (True, None, None), + ], + ) + def test_update_strategy(self, persistence, update_strategy, expected_update_strategy): + docs = render_chart( + values={ + "executor": "CeleryExecutor", + "triggerer": { + "persistence": {"enabled": persistence}, + "updateStrategy": update_strategy, + }, + }, + show_only=["templates/triggerer/triggerer-deployment.yaml"], + ) + + assert expected_update_strategy == jmespath.search("spec.updateStrategy", docs[0]) + + @pytest.mark.parametrize( + "persistence, strategy, expected_strategy", [ - (None, None), + (True, None, None), ( + False, {"rollingUpdate": {"maxSurge": "100%", "maxUnavailable": "50%"}}, {"rollingUpdate": {"maxSurge": "100%", "maxUnavailable": "50%"}}, ), + (False, None, None), ], ) - def test_strategy(self, strategy, expected_strategy): - """strategy should be used when we aren't using both LocalExecutor and workers.persistence""" + def test_strategy(self, persistence, strategy, expected_strategy): docs = render_chart( values={ - "triggerer": {"strategy": strategy}, + "triggerer": {"persistence": {"enabled": persistence}, "strategy": strategy}, }, show_only=["templates/triggerer/triggerer-deployment.yaml"], ) diff --git a/tests/jobs/test_triggerer_job.py b/tests/jobs/test_triggerer_job.py index 5fa64c9c47993..61c7556f3a3d5 100644 --- a/tests/jobs/test_triggerer_job.py +++ b/tests/jobs/test_triggerer_job.py @@ -19,12 +19,16 @@ import asyncio import datetime +import importlib import time from threading import Thread +from unittest.mock import patch import pytest -from airflow.jobs.triggerer_job import TriggererJob, TriggerRunner +from airflow.config_templates import airflow_local_settings +from airflow.jobs.triggerer_job import TriggererJob, TriggerRunner, setup_queue_listener +from airflow.logging_config import configure_logging from airflow.models import DagModel, DagRun, TaskInstance, Trigger from airflow.operators.empty import EmptyOperator from airflow.operators.python import PythonOperator @@ -32,8 +36,11 @@ from airflow.triggers.temporal import TimeDeltaTrigger from airflow.triggers.testing import FailureTrigger, SuccessTrigger from airflow.utils import timezone +from airflow.utils.log.logging_mixin import RedirectStdHandler +from airflow.utils.log.trigger_handler import LocalQueueHandler from airflow.utils.session import create_session from airflow.utils.state import State, TaskInstanceState +from tests.core.test_logging_config import reset_logging from tests.test_utils.db import clear_db_dags, clear_db_runs @@ -453,3 +460,50 @@ def test_invalid_trigger(session, dag_maker): assert task_instance.next_method == "__fail__" assert task_instance.next_kwargs["error"] == "Trigger failure" assert task_instance.next_kwargs["traceback"][-1] == "ModuleNotFoundError: No module named 'fake'\n" + + +@pytest.mark.parametrize("modify", (True, False)) +@patch("airflow.jobs.triggerer_job.configure_trigger_log_handler") +def test_handler_config_respects_donot_modify(mock_configure, modify): + from airflow.jobs import triggerer_job + + triggerer_job.DONOT_MODIFY_HANDLERS = not modify + TriggererJob() + if modify: + mock_configure.assert_called() + else: + mock_configure.assert_not_called() + + +@patch("airflow.jobs.triggerer_job.setup_queue_listener") +def test_triggerer_job_always_creates_listener(mock_setup): + mock_setup.assert_not_called() + TriggererJob() + mock_setup.assert_called() + + +def test_queue_listener(): + """ + When listener func called, root handlers should be moved to queue listener + and replaced with queuehandler. + """ + reset_logging() + importlib.reload(airflow_local_settings) + configure_logging() + + def non_pytest_handlers(val): + return [h for h in val if "pytest" not in h.__module__] + + import logging + + log = logging.getLogger() + handlers = non_pytest_handlers(log.handlers) + assert len(handlers) == 1 + handler = handlers[0] + assert handler.__class__ == RedirectStdHandler + listener = setup_queue_listener() + assert handler not in non_pytest_handlers(log.handlers) + qh = log.handlers[-1] + assert qh.__class__ == LocalQueueHandler + assert qh.queue == listener.queue + listener.stop() diff --git a/tests/jobs/test_triggerer_job_logging.py b/tests/jobs/test_triggerer_job_logging.py new file mode 100644 index 0000000000000..731fa77d7252b --- /dev/null +++ b/tests/jobs/test_triggerer_job_logging.py @@ -0,0 +1,445 @@ +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. + +from __future__ import annotations + +import importlib +import logging +import warnings + +import pytest + +from airflow.config_templates import airflow_local_settings +from airflow.jobs import triggerer_job +from airflow.logging_config import configure_logging +from airflow.providers.amazon.aws.log.s3_task_handler import S3TaskHandler +from airflow.utils.log.file_task_handler import FileTaskHandler +from airflow.utils.log.logging_mixin import RedirectStdHandler +from airflow.utils.log.trigger_handler import DropTriggerLogsFilter, TriggererHandlerWrapper +from tests.test_utils.config import conf_vars + + +def non_pytest_handlers(val): + return [h for h in val if "pytest" not in h.__module__] + + +def assert_handlers(logger, *classes): + handlers = non_pytest_handlers(logger.handlers) + assert [x.__class__ for x in handlers] == list(classes or []) + return handlers + + +def clear_logger_handlers(log): + for h in log.handlers[:]: + if "pytest" not in h.__module__: + log.removeHandler(h) + + +@pytest.fixture(autouse=True) +def reload_triggerer_job(): + importlib.reload(triggerer_job) + + +def test_configure_trigger_log_handler_file(): + """ + root logger: RedirectStdHandler + task: FTH + result: wrap + + """ + # reset logging + root_logger = logging.getLogger() + clear_logger_handlers(root_logger) + configure_logging() + + # before config + assert_handlers(root_logger, RedirectStdHandler) + + # default task logger + task_logger = logging.getLogger("airflow.task") + task_handlers = assert_handlers(task_logger, FileTaskHandler) + + # not yet configured to use wrapper + assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is False + + triggerer_job.configure_trigger_log_handler() + # after config + assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is True + root_handlers = assert_handlers(root_logger, RedirectStdHandler, TriggererHandlerWrapper) + assert root_handlers[1].base_handler == task_handlers[0] + # other handlers have DropTriggerLogsFilter + assert root_handlers[0].filters[1].__class__ == DropTriggerLogsFilter + # no filters on wrapper handler + assert root_handlers[1].filters == [] + # wrapper handler uses handler from airflow.task + assert root_handlers[1].base_handler.__class__ == FileTaskHandler + + +def test_configure_trigger_log_handler_s3(): + """ + root logger: RedirectStdHandler + task: S3TH + result: wrap + """ + with conf_vars( + { + ("logging", "remote_logging"): "True", + ("logging", "remote_log_conn_id"): "some_aws", + ("logging", "remote_base_log_folder"): "s3://some-folder", + } + ): + importlib.reload(airflow_local_settings) + configure_logging() + + # before config + root_logger = logging.getLogger() + assert_handlers(root_logger, RedirectStdHandler) + # default task logger + task_logger = logging.getLogger("airflow.task") + task_handlers = assert_handlers(task_logger, S3TaskHandler) + # not yet configured to use wrapper + assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is False + + triggerer_job.configure_trigger_log_handler() + # after config + assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is True + handlers = assert_handlers(root_logger, RedirectStdHandler, TriggererHandlerWrapper) + assert handlers[1].base_handler == task_handlers[0] + # other handlers have DropTriggerLogsFilter + assert handlers[0].filters[1].__class__ == DropTriggerLogsFilter + # no filters on wrapper handler + assert handlers[1].filters == [] + # wrapper handler uses handler from airflow.task + assert handlers[1].base_handler.__class__ == S3TaskHandler + + +class OldFileTaskHandler(FileTaskHandler): + """Handler that hasn't been updated to support triggerer""" + + def _read(self, ti, try_number, metadata=None): + super()._read(self, ti, try_number, metadata) + + +non_file_task_handler = { + "version": 1, + "handlers": {"task": {"class": "logging.Handler"}}, + "loggers": {"airflow.task": {"handlers": ["task"]}}, +} + +old_file_task_handler = { + "version": 1, + "handlers": { + "task": { + "class": "tests.jobs.test_triggerer_job_logging.OldFileTaskHandler", + "base_log_folder": "hello", + } + }, + "loggers": {"airflow.task": {"handlers": ["task"]}}, +} + +not_supported_message = [ + "Handler OldFileTaskHandler does not support individual trigger logging. " + "Please check the release notes for your provider to see if a newer version " + "supports individual trigger logging.", + "Could not find log handler suitable for individual trigger logging.", +] +not_found_message = ["Could not find log handler suitable for individual trigger logging."] + + +@pytest.mark.parametrize( + "cfg, cls, msg", + [ + ("old_file_task_handler", OldFileTaskHandler, not_supported_message), + ("non_file_task_handler", logging.Handler, not_found_message), + ], +) +def test_configure_trigger_log_handler_not_file_task_handler(cfg, cls, msg): + """ + No root handler configured. + When non FileTaskHandler is configured, don't modify. + When when an incompatible subclass of FileTaskHandler is configured, don't modify. + """ + # reset handlers + root_logger = logging.getLogger() + clear_logger_handlers(root_logger) + + with conf_vars( + { + ( + "logging", + "logging_config_class", + ): f"tests.jobs.test_triggerer_job_logging.{cfg}", + } + ): + importlib.reload(airflow_local_settings) + configure_logging() + + # no root handlers + assert_handlers(root_logger) + + # default task logger + task_logger = logging.getLogger("airflow.task") + assert_handlers(task_logger, cls) + + # not yet configured to use wrapper + assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is False + + with warnings.catch_warnings(record=True) as captured: + triggerer_job.configure_trigger_log_handler() + + assert [x.message.args[0] for x in captured] == msg + + # after config + # doesn't use TriggererHandlerWrapper, no change in handler + assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is False + + # still no root handlers + assert_handlers(root_logger) + + +fallback_task = { + "version": 1, + "handlers": { + "task": { + "class": "airflow.providers.amazon.aws.log.s3_task_handler.S3TaskHandler", + "base_log_folder": "~/abc", + "s3_log_folder": "s3://abc", + "filename_template": "blah", + }, + }, + "loggers": {"airflow.task": {"handlers": ["task"]}}, +} + + +def test_configure_trigger_log_handler_fallback_task(): + """ + root: no handler + task: FTH + result: wrap + """ + with conf_vars( + { + ("logging", "logging_config_class"): "tests.jobs.test_triggerer_job_logging.fallback_task", + } + ): + importlib.reload(airflow_local_settings) + configure_logging() + + # check custom config used + task_logger = logging.getLogger("airflow.task") + assert_handlers(task_logger, S3TaskHandler) + + # before config + root_logger = logging.getLogger() + assert_handlers(root_logger) + assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is False + + triggerer_job.configure_trigger_log_handler() + + # after config + assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is True + + handlers = assert_handlers(root_logger, TriggererHandlerWrapper) + assert handlers[0].base_handler == task_logger.handlers[0] + # no filters on wrapper handler + assert handlers[0].filters == [] + + +root_has_task_handler = { + "version": 1, + "handlers": { + "task": {"class": "logging.Handler"}, + "trigger": { + "class": "airflow.utils.log.file_task_handler.FileTaskHandler", + "base_log_folder": "blah", + }, + }, + "loggers": { + "airflow.task": {"handlers": ["task"]}, + "": {"handlers": ["trigger"]}, + }, +} + + +def test_configure_trigger_log_handler_root_has_task_handler(): + """ + root logger: single handler that supports triggerer + result: wrap + """ + with conf_vars( + { + ( + "logging", + "logging_config_class", + ): "tests.jobs.test_triggerer_job_logging.root_has_task_handler", + } + ): + configure_logging() + + # check custom config used + task_logger = logging.getLogger("airflow.task") + assert_handlers(task_logger, logging.Handler) + + # before config + root_logger = logging.getLogger() + assert_handlers(root_logger, FileTaskHandler) + assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is False + + # configure + triggerer_job.configure_trigger_log_handler() + + # after config + assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is True + handlers = assert_handlers(root_logger, TriggererHandlerWrapper) + # no filters on wrapper handler + assert handlers[0].filters == [] + # wrapper handler uses handler from airflow.task + assert handlers[0].base_handler.__class__ == FileTaskHandler + + +root_not_file_task = { + "version": 1, + "handlers": { + "task": { + "class": "airflow.providers.amazon.aws.log.s3_task_handler.S3TaskHandler", + "base_log_folder": "~/abc", + "s3_log_folder": "s3://abc", + "filename_template": "blah", + }, + "trigger": {"class": "logging.Handler"}, + }, + "loggers": { + "airflow.task": {"handlers": ["task"]}, + "": {"handlers": ["trigger"]}, + }, +} + + +def test_configure_trigger_log_handler_root_not_file_task(): + """ + root: A handler that doesn't support trigger or inherit FileTaskHandler + task: Supports triggerer + Result: + * wrap and use the task logger handler + * other root handlers filter trigger logging + """ + with conf_vars( + { + ( + "logging", + "logging_config_class", + ): "tests.jobs.test_triggerer_job_logging.root_not_file_task", + } + ): + configure_logging() + + # check custom config used + task_logger = logging.getLogger("airflow.task") + assert_handlers(task_logger, S3TaskHandler) + + # before config + root_logger = logging.getLogger() + assert_handlers(root_logger, logging.Handler) + assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is False + + # configure + with warnings.catch_warnings(record=True) as captured: + triggerer_job.configure_trigger_log_handler() + assert captured == [] + + # after config + assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is True + handlers = assert_handlers(root_logger, logging.Handler, TriggererHandlerWrapper) + # other handlers have DropTriggerLogsFilter + assert handlers[0].filters[0].__class__ == DropTriggerLogsFilter + # no filters on wrapper handler + assert handlers[1].filters == [] + # wrapper handler uses handler from airflow.task + assert handlers[1].base_handler.__class__ == S3TaskHandler + + +root_logger_old_file_task = { + "version": 1, + "handlers": { + "task": { + "class": "airflow.providers.amazon.aws.log.s3_task_handler.S3TaskHandler", + "base_log_folder": "~/abc", + "s3_log_folder": "s3://abc", + "filename_template": "blah", + }, + "trigger": { + "class": "tests.jobs.test_triggerer_job_logging.OldFileTaskHandler", + "base_log_folder": "abc", + }, + }, + "loggers": { + "airflow.task": {"handlers": ["task"]}, + "": {"handlers": ["trigger"]}, + }, +} + + +def test_configure_trigger_log_handler_root_old_file_task(): + """ + Root logger handler: An older subclass of FileTaskHandler that doesn't support triggerer + Task logger handler: Supports triggerer + Result: + * wrap and use the task logger handler + * other root handlers filter trigger logging + """ + + with conf_vars( + { + ( + "logging", + "logging_config_class", + ): "tests.jobs.test_triggerer_job_logging.root_logger_old_file_task", + } + ): + + configure_logging() + + # check custom config used + assert_handlers(logging.getLogger("airflow.task"), S3TaskHandler) + + # before config + root_logger = logging.getLogger() + assert_handlers(root_logger, OldFileTaskHandler) + + assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is False + + with warnings.catch_warnings(record=True) as captured: + triggerer_job.configure_trigger_log_handler() + + # since a root logger is explicitly configured with an old FileTaskHandler which doesn't + # work properly with individual trigger logging, warn + # todo: we should probably just remove the handler in this case it's basically misconfiguration + assert [x.message.args[0] for x in captured] == [ + "Handler OldFileTaskHandler does not support individual trigger logging. " + "Please check the release notes for your provider to see if a newer version " + "supports individual trigger logging.", + ] + + # after config + assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is True + handlers = assert_handlers(root_logger, OldFileTaskHandler, TriggererHandlerWrapper) + # other handlers have DropTriggerLogsFilter + assert handlers[0].filters[0].__class__ == DropTriggerLogsFilter + # no filters on wrapper handler + assert handlers[1].filters == [] + # wrapper handler uses handler from airflow.task + assert handlers[1].base_handler.__class__ == S3TaskHandler diff --git a/tests/providers/amazon/aws/log/test_s3_task_handler.py b/tests/providers/amazon/aws/log/test_s3_task_handler.py index 7c3119049899a..ab416d8a3a1cd 100644 --- a/tests/providers/amazon/aws/log/test_s3_task_handler.py +++ b/tests/providers/amazon/aws/log/test_s3_task_handler.py @@ -20,6 +20,7 @@ import contextlib import os from unittest import mock +from unittest.mock import MagicMock, patch import boto3 import moto @@ -30,6 +31,7 @@ from airflow.operators.empty import EmptyOperator from airflow.providers.amazon.aws.hooks.s3 import S3Hook from airflow.providers.amazon.aws.log.s3_task_handler import S3TaskHandler +from airflow.utils.log.file_task_handler import LogType from airflow.utils.session import create_session from airflow.utils.state import State from airflow.utils.timezone import datetime @@ -138,7 +140,7 @@ def test_read_when_s3_log_missing(self): assert 1 == len(log) assert len(log) == len(metadata) - assert "*** Log file does not exist:" in log[0][0][-1] + assert "*** Logs not found locally" in log[0][0][-1] assert {"end_of_log": True} == metadata[0] def test_s3_read_when_log_missing(self): @@ -204,3 +206,31 @@ def test_close_no_upload(self): with pytest.raises(ClientError): boto3.resource("s3").Object("bucket", self.remote_log_key).get() + + +@pytest.mark.parametrize("log_type", [LogType.TRIGGER, None]) +@patch("airflow.utils.log.file_task_handler.FileTaskHandler._read") +@patch("airflow.utils.log.file_task_handler.FileTaskHandler._render_filename") +def test_read_backcompat(_render_mock, _read_mock_parent, log_type): + """ + Don't don't make trigger-related calls / use triggerer-related params unless + asked. This ensures backcompat with pre-2.6 airflow versions. + todo: after min airflow version == 2.6, we can just forward log_type + """ + # just forces exit of function after this call. otherwise, have to mock more. + _render_mock.return_value = "tmp/any" + + _read_mock_parent.return_value = ["ret", None] + client = boto3.client("s3") + client.create_bucket(Bucket="abc") + h = S3TaskHandler("/tmp", "s3://abc") + h.handler = MagicMock() + ti = MagicMock() + if log_type == "trigger": + ti.triggerer_job.id = 123 + h._read(ti, 1, log_type=log_type) + _read_mock_parent.assert_called_with(ti, 1, None, log_type=log_type) + else: + h._read(ti, 1) + # unless triggerer log type asked for, arg is not forwarded + _read_mock_parent.assert_called_with(ti, 1, None) diff --git a/tests/utils/log/test_log_reader.py b/tests/utils/log/test_log_reader.py index c6e0a6286d563..c382b1fa28945 100644 --- a/tests/utils/log/test_log_reader.py +++ b/tests/utils/log/test_log_reader.py @@ -74,11 +74,11 @@ def configure_loggers(self, log_dir, settings_folder): logging_config = copy.deepcopy(DEFAULT_LOGGING_CONFIG) logging_config["handlers"]["task"]["base_log_folder"] = log_dir logging_config["handlers"]["task"]["filename_template"] = self.FILENAME_TEMPLATE - settings_file = os.path.join(settings_folder, "airflow_local_settings.py") + settings_file = os.path.join(settings_folder, "airflow_local_settings_test.py") with open(settings_file, "w") as handle: new_logging_file = f"LOGGING_CONFIG = {logging_config}" handle.writelines(new_logging_file) - with conf_vars({("logging", "logging_config_class"): "airflow_local_settings.LOGGING_CONFIG"}): + with conf_vars({("logging", "logging_config_class"): "airflow_local_settings_test.LOGGING_CONFIG"}): settings.configure_logging() yield logging.config.dictConfig(DEFAULT_LOGGING_CONFIG) @@ -206,9 +206,9 @@ def test_read_log_stream_should_support_multiple_chunks(self, mock_read): mock_read.assert_has_calls( [ - mock.call(self.ti, 1, metadata={}), - mock.call(self.ti, 1, metadata={}), - mock.call(self.ti, 1, metadata={"end_of_log": False}), + mock.call(self.ti, 1, metadata={}, log_type=None), + mock.call(self.ti, 1, metadata={}, log_type=None), + mock.call(self.ti, 1, metadata={"end_of_log": False}, log_type=None), ], any_order=False, ) @@ -227,9 +227,9 @@ def test_read_log_stream_should_read_each_try_in_turn(self, mock_read): mock_read.assert_has_calls( [ - mock.call(self.ti, 1, metadata={}), - mock.call(self.ti, 2, metadata={}), - mock.call(self.ti, 3, metadata={}), + mock.call(self.ti, 1, metadata={}, log_type=None), + mock.call(self.ti, 2, metadata={}, log_type=None), + mock.call(self.ti, 3, metadata={}, log_type=None), ], any_order=False, ) diff --git a/tests/utils/test_log_handlers.py b/tests/utils/test_log_handlers.py index 3a49a47f6bcd4..dc5ad55f2b779 100644 --- a/tests/utils/test_log_handlers.py +++ b/tests/utils/test_log_handlers.py @@ -21,13 +21,16 @@ import logging.config import os import re +import tempfile from unittest.mock import patch +import pendulum import pytest from kubernetes.client import models as k8s from airflow.config_templates.airflow_local_settings import DEFAULT_LOGGING_CONFIG -from airflow.models import DAG, DagRun, TaskInstance +from airflow.jobs.triggerer_job import TriggererJob +from airflow.models import DAG, DagRun, TaskInstance, Trigger from airflow.operators.python import PythonOperator from airflow.utils.log.file_task_handler import FileTaskHandler from airflow.utils.log.logging_mixin import set_context @@ -294,6 +297,34 @@ def task_callable(ti): _preload_content=False, ) + def test_add_triggerer_suffix(self): + sample = "any/path/to/thing.txt" + assert FileTaskHandler.add_triggerer_suffix(sample) == sample + ".trigger" + assert FileTaskHandler.add_triggerer_suffix(sample, job_id=None) == sample + ".trigger" + assert FileTaskHandler.add_triggerer_suffix(sample, job_id=123) == sample + ".trigger.123.log" + assert FileTaskHandler.add_triggerer_suffix(sample, job_id="123") == sample + ".trigger.123.log" + + @pytest.mark.parametrize("is_a_trigger", [True, False]) + def test_set_context_trigger(self, create_dummy_dag, dag_maker, is_a_trigger, session): + create_dummy_dag(dag_id="test_fth", task_id="dummy") + (ti,) = dag_maker.create_dagrun(execution_date=pendulum.datetime(2023, 1, 1, tz="UTC")).task_instances + assert isinstance(ti, TaskInstance) + if is_a_trigger: + ti.is_trigger_log_context = True + job = TriggererJob() + t = Trigger("", {}) + t.triggerer_job = job + ti.triggerer = t + t.task_instance = ti + with tempfile.TemporaryDirectory() as td: + h = FileTaskHandler(base_log_folder=td) + h.set_context(ti) + expected = "dag_id=test_fth/run_id=test/task_id=dummy/attempt=1.log" + if is_a_trigger: + expected += f".trigger.{job.id}.log" + actual = h.handler.baseFilename + assert actual.replace(td + "/", "") == expected + class TestFilenameRendering: def test_python_formatting(self, create_log_template, create_task_instance): @@ -340,8 +371,8 @@ def test_log_retrieval_valid(self, create_task_instance): execution_date=DEFAULT_DATE, ) log_url_ti.hostname = "hostname" - url = FileTaskHandler._get_log_retrieval_url(log_url_ti, "DYNAMIC_PATH") - assert url == "http://hostname:8793/log/DYNAMIC_PATH" + ret = FileTaskHandler("")._get_log_retrieval_url(log_url_ti, "DYNAMIC_PATH") + assert ret == ("http://hostname:8793/log/DYNAMIC_PATH", "DYNAMIC_PATH") @pytest.mark.parametrize( diff --git a/tests/www/views/test_views_grid.py b/tests/www/views/test_views_grid.py index 79e9401f684f0..9a2a4d0ac6da8 100644 --- a/tests/www/views/test_views_grid.py +++ b/tests/www/views/test_views_grid.py @@ -98,6 +98,7 @@ def test_no_runs(admin_client, dag_without_runs): assert resp.status_code == 200, resp.json assert resp.json == { "dag_runs": [], + "reader_supports_triggerer": True, "groups": { "children": [ { @@ -226,6 +227,7 @@ def test_one_run(admin_client, dag_with_runs: list[DagRun], session): "state": "running", }, ], + "reader_supports_triggerer": True, "groups": { "children": [ { @@ -241,6 +243,7 @@ def test_one_run(admin_client, dag_with_runs: list[DagRun], session): "state": "success", "task_id": "task1", "try_number": 0, + "has_deferred": False, }, { "run_id": "run_2", @@ -250,6 +253,7 @@ def test_one_run(admin_client, dag_with_runs: list[DagRun], session): "state": "success", "task_id": "task1", "try_number": 0, + "has_deferred": False, }, ], "is_mapped": False, @@ -403,6 +407,7 @@ def _expected_task_details(task_id, has_outlet_datasets): assert resp.status_code == 200, resp.json assert resp.json == { "dag_runs": [], + "reader_supports_triggerer": True, "groups": { "children": [ _expected_task_details("task1", False), diff --git a/tests/www/views/test_views_tasks.py b/tests/www/views/test_views_tasks.py index c0160bb2baa67..cc8366dee25f3 100644 --- a/tests/www/views/test_views_tasks.py +++ b/tests/www/views/test_views_tasks.py @@ -22,6 +22,7 @@ import re import unittest.mock import urllib.parse +from getpass import getuser import pytest import time_machine @@ -780,10 +781,10 @@ def _get_appbuilder_pk_string(model_view_cls, instance) -> str: Example usage:: - >>> from airflow.www.views import TaskInstanceModelView - >>> ti = session.Query(TaskInstance).filter(...).one() - >>> pk = _get_appbuilder_pk_string(TaskInstanceModelView, ti) - >>> client.post("...", data={"action": "...", "rowid": pk}) + from airflow.www.views import TaskInstanceModelView + ti = session.Query(TaskInstance).filter(...).one() + pk = _get_appbuilder_pk_string(TaskInstanceModelView, ti) + client.post("...", data={"action": "...", "rowid": pk}) """ pk_value = model_view_cls.datamodel.get_pk_value(instance) return model_view_cls._serialize_pk_if_composite(model_view_cls, pk_value) @@ -1059,7 +1060,7 @@ def test_task_instances(admin_client): "trigger_id": None, "trigger_timeout": None, "try_number": 1, - "unixname": "root", + "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), }, "run_after_loop": { @@ -1089,7 +1090,7 @@ def test_task_instances(admin_client): "trigger_id": None, "trigger_timeout": None, "try_number": 1, - "unixname": "root", + "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), }, "run_this_last": { @@ -1119,7 +1120,7 @@ def test_task_instances(admin_client): "trigger_id": None, "trigger_timeout": None, "try_number": 1, - "unixname": "root", + "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), }, "runme_0": { @@ -1149,7 +1150,7 @@ def test_task_instances(admin_client): "trigger_id": None, "trigger_timeout": None, "try_number": 1, - "unixname": "root", + "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), }, "runme_1": { @@ -1179,7 +1180,7 @@ def test_task_instances(admin_client): "trigger_id": None, "trigger_timeout": None, "try_number": 1, - "unixname": "root", + "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), }, "runme_2": { @@ -1209,7 +1210,7 @@ def test_task_instances(admin_client): "trigger_id": None, "trigger_timeout": None, "try_number": 1, - "unixname": "root", + "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), }, "this_will_skip": { @@ -1239,7 +1240,7 @@ def test_task_instances(admin_client): "trigger_id": None, "trigger_timeout": None, "try_number": 1, - "unixname": "root", + "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), }, } From 666cc42a58015b8143748888ad270714820afd74 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 11 Jan 2023 00:22:26 -0800 Subject: [PATCH 02/96] Don't emit end_of_log for task deferral When the task run ends in a deferral, we should neither emit the "end of log" marker nor run the follow-on schedule check. --- airflow/cli/commands/task_command.py | 16 +++++++++------- airflow/jobs/local_task_job.py | 19 ++++++++++++++++++- airflow/models/taskinstance.py | 11 +++++++++-- .../elasticsearch/log/es_task_handler.py | 4 +++- .../task/task_runner/standard_task_runner.py | 7 ++++++- airflow/utils/log/file_task_handler.py | 7 +++++++ airflow/utils/log/logging_mixin.py | 1 - 7 files changed, 52 insertions(+), 13 deletions(-) diff --git a/airflow/cli/commands/task_command.py b/airflow/cli/commands/task_command.py index 0f3f5a8bba6ca..b786610e4c225 100644 --- a/airflow/cli/commands/task_command.py +++ b/airflow/cli/commands/task_command.py @@ -44,6 +44,7 @@ from airflow.models.dag import DAG from airflow.models.dagrun import DagRun from airflow.models.operator import needs_expansion +from airflow.models.taskinstance import ArgDeferred from airflow.settings import IS_K8S_EXECUTOR_POD from airflow.ti_deps.dep_context import DepContext from airflow.ti_deps.dependencies_deps import SCHEDULER_QUEUED_DEPS @@ -182,7 +183,7 @@ def _get_ti( return ti, dr_created -def _run_task_by_selected_method(args, dag: DAG, ti: TaskInstance) -> None: +def _run_task_by_selected_method(args, dag: DAG, ti: TaskInstance) -> None | ArgDeferred: """ Runs the task based on a mode. @@ -195,7 +196,7 @@ def _run_task_by_selected_method(args, dag: DAG, ti: TaskInstance) -> None: if args.local: _run_task_by_local_task_job(args, ti) elif args.raw: - _run_raw_task(args, ti) + return _run_raw_task(args, ti) else: _run_task_by_executor(args, dag, ti) @@ -269,9 +270,9 @@ def _run_task_by_local_task_job(args, ti): ] -def _run_raw_task(args, ti: TaskInstance) -> None: +def _run_raw_task(args, ti: TaskInstance) -> None | ArgDeferred: """Runs the main task handling code.""" - ti._run_raw_task( + return ti._run_raw_task( mark_success=args.mark_success, job_id=args.job_id, pool=args.pool, @@ -407,18 +408,19 @@ def task_run(args, dag=None): # this should be last thing before running, to reduce likelihood of an open session # which can cause trouble if running process in a fork. settings.reconfigure_orm(disable_connection_pool=True) - + ret = None try: if args.interactive: - _run_task_by_selected_method(args, dag, ti) + ret = _run_task_by_selected_method(args, dag, ti) else: with _move_task_handlers_to_root(ti), _redirect_stdout_to_ti_log(ti): - _run_task_by_selected_method(args, dag, ti) + ret = _run_task_by_selected_method(args, dag, ti) finally: try: get_listener_manager().hook.before_stopping(component=TaskCommandMarker()) except Exception: pass + return ret @cli_utils.action_cli(check_db=False) diff --git a/airflow/jobs/local_task_job.py b/airflow/jobs/local_task_job.py index 991900a4bf994..148ca8845d68c 100644 --- a/airflow/jobs/local_task_job.py +++ b/airflow/jobs/local_task_job.py @@ -17,6 +17,7 @@ # under the License. from __future__ import annotations +import logging import signal import psutil @@ -29,6 +30,7 @@ from airflow.models.taskinstance import TaskInstance from airflow.stats import Stats from airflow.task.task_runner import get_task_runner +from airflow.task.task_runner.standard_task_runner import DEFERRAL_EXIT_CODE from airflow.utils import timezone from airflow.utils.net import get_hostname from airflow.utils.session import provide_session @@ -65,6 +67,18 @@ ********************************************************************************************************""" +def set_task_deferred_context_var(): + """ + Tell task log handler that task exited with deferral. + + :meta private: + """ + logger = logging.getLogger() + h = next((h for h in logger.handlers if hasattr(h, "ctx_task_deferred")), None) + if h: + h.ctx_task_deferred = True + + class LocalTaskJob(BaseJob): """LocalTaskJob runs a single task instance.""" @@ -210,8 +224,11 @@ def handle_task_exit(self, return_code: int) -> None: self.terminating = True self.log.info("Task exited with return code %s", return_code) self._log_return_code_metric(return_code) + is_deferral = return_code == DEFERRAL_EXIT_CODE + if is_deferral: + set_task_deferred_context_var() - if not self.task_instance.test_mode: + if not self.task_instance.test_mode and not is_deferral: if conf.getboolean("scheduler", "schedule_after_task_execution", fallback=True): self.task_instance.schedule_downstream_tasks() diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index b3031d0610d53..4678e1f6c4cc4 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -139,6 +139,13 @@ PAST_DEPENDS_MET = "past_depends_met" +class ArgDeferred: + """sentinel.""" + + +DEFERRED = ArgDeferred() + + @contextlib.contextmanager def set_current_context(context: Context) -> Generator[Context, None, None]: """ @@ -1370,7 +1377,7 @@ def _run_raw_task( job_id: str | None = None, pool: str | None = None, session: Session = NEW_SESSION, - ) -> None: + ) -> ArgDeferred | None: """ Immediately runs the task (without checking or changing db state before execution) and then sets the appropriate final state after @@ -1420,7 +1427,7 @@ def _run_raw_task( session.add(Log(self.state, self)) session.merge(self) session.commit() - return + return DEFERRED except AirflowSkipException as e: # Recording SKIP # log only if exception has any arguments to prevent log flooding diff --git a/airflow/providers/elasticsearch/log/es_task_handler.py b/airflow/providers/elasticsearch/log/es_task_handler.py index fce2f1881497f..0573758b3ae09 100644 --- a/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/airflow/providers/elasticsearch/log/es_task_handler.py @@ -360,7 +360,9 @@ def close(self) -> None: if self.closed: return - if not self.mark_end_on_close: + # todo: remove `getattr` when min airflow version >= 2.6 + if not self.mark_end_on_close or getattr(self, "ctx_task_deferred", None): + # when we're closing due to task deferral, don't mark end of log self.closed = True return diff --git a/airflow/task/task_runner/standard_task_runner.py b/airflow/task/task_runner/standard_task_runner.py index 4d2d55e9276d6..a5dccf93f91c2 100644 --- a/airflow/task/task_runner/standard_task_runner.py +++ b/airflow/task/task_runner/standard_task_runner.py @@ -24,11 +24,14 @@ import psutil from setproctitle import setproctitle +from airflow.models.taskinstance import ArgDeferred from airflow.settings import CAN_FORK from airflow.task.task_runner.base_task_runner import BaseTaskRunner from airflow.utils.dag_parsing_context import _airflow_parsing_context_manager from airflow.utils.process_utils import reap_process_group, set_new_process_group +DEFERRAL_EXIT_CODE = 100 + class StandardTaskRunner(BaseTaskRunner): """Standard runner for all tasks.""" @@ -92,8 +95,10 @@ def _start_by_fork(self): dag_id=self._task_instance.dag_id, task_id=self._task_instance.task_id, ): - args.func(args, dag=self.dag) + ret = args.func(args, dag=self.dag) return_code = 0 + if isinstance(ret, ArgDeferred): + return_code = DEFERRAL_EXIT_CODE except Exception as exc: return_code = 1 diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index a1c5638adb97b..5c01e02d4cfc2 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -103,6 +103,13 @@ def __init__(self, base_log_folder: str, filename_template: str | None = None): :meta private: """ + self.ctx_task_deferred = False + """ + If true, task exited with deferral to trigger. + + Some handlers emit "end of log" markers, and may not wish to do so when task defers. + """ + def set_context(self, ti: TaskInstance) -> None | SetContextPropagate: """ Provide task_instance context to airflow task handler. diff --git a/airflow/utils/log/logging_mixin.py b/airflow/utils/log/logging_mixin.py index 85ff71a94f1bf..ada57e3971833 100644 --- a/airflow/utils/log/logging_mixin.py +++ b/airflow/utils/log/logging_mixin.py @@ -31,7 +31,6 @@ # 7-bit C1 ANSI escape sequences ANSI_ESCAPE = re.compile(r"\x1B[@-_][0-?]*[ -/]*[@-~]") - # Private: A sentinel objects class SetContextPropagate(enum.Enum): """:meta private:""" From d36667fb5ad664b6d2a21c86d4cad657814758d5 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 11 Jan 2023 10:56:19 -0800 Subject: [PATCH 03/96] add support for ES --- airflow/providers/elasticsearch/log/es_task_handler.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/airflow/providers/elasticsearch/log/es_task_handler.py b/airflow/providers/elasticsearch/log/es_task_handler.py index 0573758b3ae09..96b8478978f8e 100644 --- a/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/airflow/providers/elasticsearch/log/es_task_handler.py @@ -188,15 +188,15 @@ def _read_grouped_logs(self): return True def _read( - self, ti: TaskInstance, try_number: int, metadata: dict | None = None + self, ti: TaskInstance, try_number: int, metadata: dict | None = None, *, log_type=None ) -> tuple[EsLogMsgType, dict]: """ Endpoint for streaming log. :param ti: task instance object :param try_number: try_number of the task instance - :param metadata: log metadata, - can be used for steaming log reading and auto-tailing. + :param metadata: log metadata, can be used for steaming log reading and auto-tailing. + :param log_type: not used :return: a list of tuple with host and log documents, metadata. """ if not metadata: @@ -324,7 +324,8 @@ def set_context(self, ti: TaskInstance) -> None: :param ti: task instance object """ - self.mark_end_on_close = not ti.raw + is_trigger_log_context = getattr(ti, "is_trigger_log_context", None) + self.mark_end_on_close = not ti.raw and not is_trigger_log_context if self.json_format: self.formatter = ElasticsearchJSONFormatter( From c15d88a0c6d982c8ef90470d24415238845ae7fd Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 11 Jan 2023 21:31:17 -0800 Subject: [PATCH 04/96] get ES working again --- .../example_time_delta_sensor_async.py | 2 +- .../elasticsearch/log/es_task_handler.py | 29 ++++++++++++------- 2 files changed, 19 insertions(+), 12 deletions(-) diff --git a/airflow/example_dags/example_time_delta_sensor_async.py b/airflow/example_dags/example_time_delta_sensor_async.py index d1562c5751d7d..8361009ff73ca 100644 --- a/airflow/example_dags/example_time_delta_sensor_async.py +++ b/airflow/example_dags/example_time_delta_sensor_async.py @@ -36,6 +36,6 @@ catchup=False, tags=["example"], ) as dag: - wait = TimeDeltaSensorAsync(task_id="wait", delta=datetime.timedelta(seconds=10)) + wait = TimeDeltaSensorAsync(task_id="wait", delta=datetime.timedelta(seconds=30)) finish = EmptyOperator(task_id="finish") wait >> finish diff --git a/airflow/providers/elasticsearch/log/es_task_handler.py b/airflow/providers/elasticsearch/log/es_task_handler.py index 96b8478978f8e..25099bdd888ca 100644 --- a/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/airflow/providers/elasticsearch/log/es_task_handler.py @@ -95,7 +95,10 @@ def __init__( :param host: Elasticsearch host name """ es_kwargs = es_kwargs or {} - super().__init__(base_log_folder, filename_template) + super_args = [] + if filename_template: + super_args.append(filename_template) + super().__init__(base_log_folder, *super_args) self.closed = False self.client = elasticsearch.Elasticsearch(host.split(";"), **es_kwargs) # type: ignore[attr-defined] @@ -120,6 +123,7 @@ def __init__( self.formatter: logging.Formatter self.handler: logging.FileHandler | logging.StreamHandler # type: ignore[assignment] + self.is_trigger_log_context: bool | None = None def _render_log_id(self, ti: TaskInstance, try_number: int) -> str: with create_session() as session: @@ -324,8 +328,8 @@ def set_context(self, ti: TaskInstance) -> None: :param ti: task instance object """ - is_trigger_log_context = getattr(ti, "is_trigger_log_context", None) - self.mark_end_on_close = not ti.raw and not is_trigger_log_context + self.is_trigger_log_context = getattr(ti, "is_trigger_log_context", None) + self.mark_end_on_close = not getattr(ti, "raw", None) if self.json_format: self.formatter = ElasticsearchJSONFormatter( @@ -362,7 +366,9 @@ def close(self) -> None: return # todo: remove `getattr` when min airflow version >= 2.6 - if not self.mark_end_on_close or getattr(self, "ctx_task_deferred", None): + ctx_task_deferred = getattr(self, "ctx_task_deferred", None) + + if not self.mark_end_on_close: # when we're closing due to task deferral, don't mark end of log self.closed = True return @@ -372,14 +378,15 @@ def close(self) -> None: self.closed = True return - # Reopen the file stream, because FileHandler.close() would be called - # first in logging.shutdown() and the stream in it would be set to None. - if self.handler.stream is None or self.handler.stream.closed: # type: ignore[attr-defined] - self.handler.stream = self.handler._open() # type: ignore[union-attr] + if not (self.is_trigger_log_context or ctx_task_deferred): + # Reopen the file stream, because FileHandler.close() would be called + # first in logging.shutdown() and the stream in it would be set to None. + if self.handler.stream is None or self.handler.stream.closed: # type: ignore[attr-defined] + self.handler.stream = self.handler._open() # type: ignore[union-attr] - # Mark the end of file using end of log mark, - # so we know where to stop while auto-tailing. - self.emit(logging.makeLogRecord({"msg": self.end_of_log_mark})) + # Mark the end of file using end of log mark, + # so we know where to stop while auto-tailing. + self.emit(logging.makeLogRecord({"msg": self.end_of_log_mark})) if self.write_stdout: self.handler.close() From db346d0b10200757481133fa1fc059feeae759cd Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 11 Jan 2023 23:04:55 -0800 Subject: [PATCH 05/96] don't say 'starting' when resuming from deferral --- airflow/models/taskinstance.py | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index 4678e1f6c4cc4..59f90256a3a9f 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -1315,7 +1315,10 @@ def check_and_change_state_before_execution( session.commit() return False - self.log.info("Starting attempt %s of %s", self.try_number, self.max_tries + 1) + if self.next_kwargs is not None: + self.log.info("resuming after deferral") + else: + self.log.info("Starting attempt %s of %s", self.try_number, self.max_tries + 1) self._try_number += 1 if not test_mode: From 52ece7e834db4923b7938ba7609585e6610d961d Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 12 Jan 2023 00:02:32 -0800 Subject: [PATCH 06/96] fixup! get ES working again --- .../elasticsearch/log/es_task_handler.py | 30 ++++++++----------- 1 file changed, 12 insertions(+), 18 deletions(-) diff --git a/airflow/providers/elasticsearch/log/es_task_handler.py b/airflow/providers/elasticsearch/log/es_task_handler.py index 25099bdd888ca..6250b62a831eb 100644 --- a/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/airflow/providers/elasticsearch/log/es_task_handler.py @@ -95,10 +95,7 @@ def __init__( :param host: Elasticsearch host name """ es_kwargs = es_kwargs or {} - super_args = [] - if filename_template: - super_args.append(filename_template) - super().__init__(base_log_folder, *super_args) + super().__init__(base_log_folder, filename_template) self.closed = False self.client = elasticsearch.Elasticsearch(host.split(";"), **es_kwargs) # type: ignore[attr-defined] @@ -123,7 +120,6 @@ def __init__( self.formatter: logging.Formatter self.handler: logging.FileHandler | logging.StreamHandler # type: ignore[assignment] - self.is_trigger_log_context: bool | None = None def _render_log_id(self, ti: TaskInstance, try_number: int) -> str: with create_session() as session: @@ -328,8 +324,9 @@ def set_context(self, ti: TaskInstance) -> None: :param ti: task instance object """ - self.is_trigger_log_context = getattr(ti, "is_trigger_log_context", None) - self.mark_end_on_close = not getattr(ti, "raw", None) + is_trigger_log_context = getattr(ti, "is_trigger_log_context", None) + is_ti_raw = getattr(ti, "raw", None) + self.mark_end_on_close = not is_ti_raw and not is_trigger_log_context if self.json_format: self.formatter = ElasticsearchJSONFormatter( @@ -366,9 +363,7 @@ def close(self) -> None: return # todo: remove `getattr` when min airflow version >= 2.6 - ctx_task_deferred = getattr(self, "ctx_task_deferred", None) - - if not self.mark_end_on_close: + if not self.mark_end_on_close or getattr(self, "ctx_task_deferred", None): # when we're closing due to task deferral, don't mark end of log self.closed = True return @@ -378,15 +373,14 @@ def close(self) -> None: self.closed = True return - if not (self.is_trigger_log_context or ctx_task_deferred): - # Reopen the file stream, because FileHandler.close() would be called - # first in logging.shutdown() and the stream in it would be set to None. - if self.handler.stream is None or self.handler.stream.closed: # type: ignore[attr-defined] - self.handler.stream = self.handler._open() # type: ignore[union-attr] + # Reopen the file stream, because FileHandler.close() would be called + # first in logging.shutdown() and the stream in it would be set to None. + if self.handler.stream is None or self.handler.stream.closed: # type: ignore[attr-defined] + self.handler.stream = self.handler._open() # type: ignore[union-attr] - # Mark the end of file using end of log mark, - # so we know where to stop while auto-tailing. - self.emit(logging.makeLogRecord({"msg": self.end_of_log_mark})) + # Mark the end of file using end of log mark, + # so we know where to stop while auto-tailing. + self.emit(logging.makeLogRecord({"msg": self.end_of_log_mark})) if self.write_stdout: self.handler.close() From be77d4e6e4832fe514bdaf71056145ec5bc48037 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 12 Jan 2023 01:35:08 -0800 Subject: [PATCH 07/96] make it optional to have logs in sep pane --- airflow/api_connexion/endpoints/log_endpoint.py | 2 +- airflow/jobs/triggerer_job.py | 10 +++++----- .../providers/elasticsearch/log/es_task_handler.py | 5 +++++ .../microsoft/azure/log/wasb_task_handler.py | 2 +- airflow/utils/log/file_task_handler.py | 11 ++++++++++- airflow/utils/log/log_reader.py | 8 ++++---- airflow/www/static/js/api/useGridData.ts | 4 ++-- .../www/static/js/dag/details/taskInstance/index.tsx | 8 ++++---- airflow/www/static/js/dag/grid/index.test.tsx | 2 +- airflow/www/views.py | 2 +- tests/www/views/test_views_grid.py | 6 +++--- 11 files changed, 37 insertions(+), 23 deletions(-) diff --git a/airflow/api_connexion/endpoints/log_endpoint.py b/airflow/api_connexion/endpoints/log_endpoint.py index 54ed462fe0168..3621fd47bffa7 100644 --- a/airflow/api_connexion/endpoints/log_endpoint.py +++ b/airflow/api_connexion/endpoints/log_endpoint.py @@ -76,7 +76,7 @@ def get_log( metadata["download_logs"] = False task_log_reader = TaskLogReader() - if log_type == LogType.TRIGGER and not task_log_reader.supports_triggerer: + if log_type == LogType.TRIGGER and not task_log_reader.triggerer_logs_separate: raise BadRequest("Task log handler does not support trigger logging.") if not task_log_reader.supports_read: diff --git a/airflow/jobs/triggerer_job.py b/airflow/jobs/triggerer_job.py index 2d7c2ffff4526..52d2207947809 100644 --- a/airflow/jobs/triggerer_job.py +++ b/airflow/jobs/triggerer_job.py @@ -80,24 +80,24 @@ def configure_trigger_log_handler(): """ global USING_TRIGGERER_HANDLER_WRAPPER - def supports_triggerer(handler): - return getattr(handler, "supports_triggerer", False) + def should_wrap_for_triggerer(handler): + return getattr(handler, "wrap_for_triggerer", False) def get_task_handler_from_logger(logger_): for h in logger_.handlers: - if isinstance(h, FileTaskHandler) and not supports_triggerer(h): + if isinstance(h, FileTaskHandler) and not should_wrap_for_triggerer(h): warnings.warn( f"Handler {h.__class__.__name__} does not support " "individual trigger logging. Please check the release notes " "for your provider to see if a newer version supports " "individual trigger logging." ) - if supports_triggerer(h): + if should_wrap_for_triggerer(h): return h def find_suitable_task_handler(): # check root logger then check airflow.task to see if a handler - # suitable for use with TriggerHandlerWrapper (has supports_triggerer + # suitable for use with TriggerHandlerWrapper (has wrap_for_triggerer # attr, likely inherits from FileTaskHandler) h = get_task_handler_from_logger(root_logger) if not h: diff --git a/airflow/providers/elasticsearch/log/es_task_handler.py b/airflow/providers/elasticsearch/log/es_task_handler.py index 6250b62a831eb..7c467ef8dfabe 100644 --- a/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/airflow/providers/elasticsearch/log/es_task_handler.py @@ -32,6 +32,7 @@ import pendulum from elasticsearch_dsl import Search +from airflow.compat.functools import cached_property from airflow.configuration import conf from airflow.models.dagrun import DagRun from airflow.models.taskinstance import TaskInstance @@ -121,6 +122,10 @@ def __init__( self.formatter: logging.Formatter self.handler: logging.FileHandler | logging.StreamHandler # type: ignore[assignment] + @cached_property + def triggerer_logs_separate(self): + return False + def _render_log_id(self, ti: TaskInstance, try_number: int) -> str: with create_session() as session: dag_run = ti.get_dagrun(session=session) diff --git a/airflow/providers/microsoft/azure/log/wasb_task_handler.py b/airflow/providers/microsoft/azure/log/wasb_task_handler.py index b8cce38edaf5d..5d2c2880250be 100644 --- a/airflow/providers/microsoft/azure/log/wasb_task_handler.py +++ b/airflow/providers/microsoft/azure/log/wasb_task_handler.py @@ -71,7 +71,7 @@ def hook(self): return None @cached_property - def supports_triggerer(self): + def triggerer_logs_separate(self): """Not implemented yet.""" return False diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index 5c01e02d4cfc2..9657a27eacf66 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -146,7 +146,16 @@ def add_triggerer_suffix(full_path, job_id=None): return full_path @cached_property - def supports_triggerer(self): + def triggerer_logs_separate(self): + """ + If true, webserver should render trigger logs in distinct tab + + :meta private: + """ + return "log_type" in inspect.signature(self._read).parameters.keys() + + @cached_property + def wrap_for_triggerer(self): """ If true, this handler has been updated to support individual logging as implemented in triggerer_job. diff --git a/airflow/utils/log/log_reader.py b/airflow/utils/log/log_reader.py index 9b95bd5e1e010..448ac4600dc60 100644 --- a/airflow/utils/log/log_reader.py +++ b/airflow/utils/log/log_reader.py @@ -60,7 +60,7 @@ def read_log_chunks( contain information about the task log which can enable you read logs to the end. """ - kwargs = {"log_type": log_type} if self.supports_triggerer else {} + kwargs = {"log_type": log_type} if self.triggerer_logs_separate else {} logs, metadatas = self.log_handler.read(ti, try_number, metadata=metadata, **kwargs) metadata = metadatas[0] return logs, metadata @@ -86,7 +86,7 @@ def read_log_stream( metadata.pop("offset", None) metadata.pop("log_pos", None) while True: - kwargs = {"log_type": log_type} if self.supports_triggerer else {} + kwargs = {"log_type": log_type} if self.triggerer_logs_separate else {} logs, metadata = self.read_log_chunks(ti, current_try_number, metadata, **kwargs) for host, log in logs[0]: yield "\n".join([host or "", log]) + "\n" @@ -114,8 +114,8 @@ def supports_read(self): return hasattr(self.log_handler, "read") @property - def supports_triggerer(self): - return getattr(self.log_handler, "supports_triggerer", False) + def triggerer_logs_separate(self): + return getattr(self.log_handler, "triggerer_logs_separate", False) @property def supports_external_link(self) -> bool: diff --git a/airflow/www/static/js/api/useGridData.ts b/airflow/www/static/js/api/useGridData.ts index 6dd7649189f81..26aa2db09a57f 100644 --- a/airflow/www/static/js/api/useGridData.ts +++ b/airflow/www/static/js/api/useGridData.ts @@ -40,7 +40,7 @@ export interface GridData { dagRuns: DagRun[]; groups: Task; ordering: RunOrdering; - readerSupportsTriggerer: boolean; + readerTriggererLogsSeparate: boolean; } export const emptyGridData: GridData = { @@ -51,7 +51,7 @@ export const emptyGridData: GridData = { instances: [], }, ordering: [], - readerSupportsTriggerer: false, + readerTriggererLogsSeparate: false, }; const formatOrdering = (data: GridData) => ({ diff --git a/airflow/www/static/js/dag/details/taskInstance/index.tsx b/airflow/www/static/js/dag/details/taskInstance/index.tsx index cf8e71e7fefe6..6ad586ca12e90 100644 --- a/airflow/www/static/js/dag/details/taskInstance/index.tsx +++ b/airflow/www/static/js/dag/details/taskInstance/index.tsx @@ -64,7 +64,7 @@ const TaskInstance = ({ }: Props) => { const isMapIndexDefined = !(mapIndex === undefined); const actionsMapIndexes = isMapIndexDefined ? [mapIndex] : []; - const { data: { dagRuns, groups, readerSupportsTriggerer } } = useGridData(); + const { data: { dagRuns, groups, readerTriggererLogsSeparate } } = useGridData(); const containerRef = useContainerRef(); const detailsRef = useRef(null); const offsetHeight = useOffsetHeight(detailsRef); @@ -97,7 +97,7 @@ const TaskInstance = ({ }; let triggerLogsTooltipProps; - if (!readerSupportsTriggerer) { + if (!readerTriggererLogsSeparate) { triggerLogsTooltipProps = { label: 'The configured log handler does not support reading trigger logs.', isDisabled: false, @@ -113,7 +113,7 @@ const TaskInstance = ({ isDisabled: true, }; } - const disableTriggerLogs = !(readerSupportsTriggerer && instance?.hasDeferred === true); + const disableTriggerLogs = !(readerTriggererLogsSeparate && instance?.hasDeferred === true); useEffect(() => { // Reset preferred tab if it is disabled @@ -183,7 +183,7 @@ const TaskInstance = ({ )} - {!isGroupOrMappedTaskSummary && ( + {!isGroupOrMappedTaskSummary && !disableTriggerLogs && ( Date: Thu, 12 Jan 2023 13:12:32 -0800 Subject: [PATCH 08/96] less joinedload --- airflow/api_connexion/endpoints/log_endpoint.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/airflow/api_connexion/endpoints/log_endpoint.py b/airflow/api_connexion/endpoints/log_endpoint.py index 3621fd47bffa7..a53cff4f017be 100644 --- a/airflow/api_connexion/endpoints/log_endpoint.py +++ b/airflow/api_connexion/endpoints/log_endpoint.py @@ -81,7 +81,7 @@ def get_log( if not task_log_reader.supports_read: raise BadRequest("Task log handler does not support read logs.") - ti = ( + query = ( session.query(TaskInstance) .filter( TaskInstance.task_id == task_id, @@ -90,10 +90,10 @@ def get_log( TaskInstance.map_index == map_index, ) .join(TaskInstance.dag_run) - .options(joinedload("trigger")) - .options(joinedload("trigger.triggerer_job")) - .one_or_none() ) + if log_type == LogType.TRIGGER: + query = query.options(joinedload("trigger")).options(joinedload("trigger.triggerer_job")) + ti = query.one_or_none() if ti is None: metadata["end_of_log"] = True raise NotFound(title="TaskInstance not found") From 66e9a989099b24ad626613a8b8cd43207145c517 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 12 Jan 2023 13:29:41 -0800 Subject: [PATCH 09/96] docstring --- airflow/jobs/triggerer_job.py | 2 +- airflow/utils/log/trigger_handler.py | 5 ++++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/airflow/jobs/triggerer_job.py b/airflow/jobs/triggerer_job.py index 52d2207947809..47d7af4b62dc4 100644 --- a/airflow/jobs/triggerer_job.py +++ b/airflow/jobs/triggerer_job.py @@ -59,7 +59,7 @@ """ If this value is true, trigger logging is configured to use TriggerHandlerWrapper -:meta :private +:meta private: """ logger = logging.getLogger(__name__) diff --git a/airflow/utils/log/trigger_handler.py b/airflow/utils/log/trigger_handler.py index f9f3f9cde1a05..3410747d38172 100644 --- a/airflow/utils/log/trigger_handler.py +++ b/airflow/utils/log/trigger_handler.py @@ -53,7 +53,10 @@ def filter(self, record): class DropTriggerLogsFilter(logging.Filter): """ - If record has non-empty attr trigger_id, filter the record. + If record has attr with name ctx_indiv_trigger, filter the record. + + The purpose here is to prevent trigger logs from going to stdout + in the trigger service. :meta private: """ From 7aea4627a53c4eaac9230fc729c9e75ac8a0ab42 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 12 Jan 2023 22:42:14 -0800 Subject: [PATCH 10/96] use enum for reader mode --- .../api_connexion/endpoints/log_endpoint.py | 11 +++++-- .../elasticsearch/log/es_task_handler.py | 6 ++-- .../microsoft/azure/log/wasb_task_handler.py | 6 ++-- airflow/utils/log/file_task_handler.py | 27 ++++++++++++++--- airflow/utils/log/log_reader.py | 15 +++++++--- airflow/www/static/js/api/useGridData.ts | 10 +++++-- .../js/dag/details/taskInstance/index.tsx | 29 ++++++++++--------- airflow/www/static/js/dag/grid/index.test.tsx | 5 ++-- airflow/www/views.py | 2 +- tests/www/views/test_views_grid.py | 6 ++-- 10 files changed, 79 insertions(+), 38 deletions(-) diff --git a/airflow/api_connexion/endpoints/log_endpoint.py b/airflow/api_connexion/endpoints/log_endpoint.py index a53cff4f017be..582a525a5933b 100644 --- a/airflow/api_connexion/endpoints/log_endpoint.py +++ b/airflow/api_connexion/endpoints/log_endpoint.py @@ -32,7 +32,7 @@ from airflow.models import TaskInstance from airflow.security import permissions from airflow.utils.airflow_flask_app import get_airflow_app -from airflow.utils.log.file_task_handler import LogType +from airflow.utils.log.file_task_handler import LogType, TriggerLogsPresentationMode from airflow.utils.log.log_reader import TaskLogReader from airflow.utils.session import NEW_SESSION, provide_session @@ -76,8 +76,13 @@ def get_log( metadata["download_logs"] = False task_log_reader = TaskLogReader() - if log_type == LogType.TRIGGER and not task_log_reader.triggerer_logs_separate: - raise BadRequest("Task log handler does not support trigger logging.") + if log_type == LogType.TRIGGER: + not_supported = TriggerLogsPresentationMode.NOT_SUPPORTED + interleaved = TriggerLogsPresentationMode.INTERLEAVED + if task_log_reader.trigger_logs_presentation_mode == not_supported: + raise BadRequest("Task log handler does not support trigger logging.") + if task_log_reader.trigger_logs_presentation_mode == interleaved: + raise BadRequest("Trigger logs requested but handler does not split trigger logs.") if not task_log_reader.supports_read: raise BadRequest("Task log handler does not support read logs.") diff --git a/airflow/providers/elasticsearch/log/es_task_handler.py b/airflow/providers/elasticsearch/log/es_task_handler.py index 7c467ef8dfabe..a76154750a144 100644 --- a/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/airflow/providers/elasticsearch/log/es_task_handler.py @@ -38,7 +38,7 @@ from airflow.models.taskinstance import TaskInstance from airflow.providers.elasticsearch.log.es_json_formatter import ElasticsearchJSONFormatter from airflow.utils import timezone -from airflow.utils.log.file_task_handler import FileTaskHandler +from airflow.utils.log.file_task_handler import FileTaskHandler, TriggerLogsPresentationMode from airflow.utils.log.logging_mixin import ExternalLoggingMixin, LoggingMixin from airflow.utils.session import create_session @@ -123,8 +123,8 @@ def __init__( self.handler: logging.FileHandler | logging.StreamHandler # type: ignore[assignment] @cached_property - def triggerer_logs_separate(self): - return False + def trigger_logs_presentation_mode(self): + return TriggerLogsPresentationMode.INTERLEAVED def _render_log_id(self, ti: TaskInstance, try_number: int) -> str: with create_session() as session: diff --git a/airflow/providers/microsoft/azure/log/wasb_task_handler.py b/airflow/providers/microsoft/azure/log/wasb_task_handler.py index 5d2c2880250be..b4c7323c0f459 100644 --- a/airflow/providers/microsoft/azure/log/wasb_task_handler.py +++ b/airflow/providers/microsoft/azure/log/wasb_task_handler.py @@ -23,7 +23,7 @@ from airflow.compat.functools import cached_property from airflow.configuration import conf -from airflow.utils.log.file_task_handler import FileTaskHandler +from airflow.utils.log.file_task_handler import FileTaskHandler, TriggerLogsPresentationMode from airflow.utils.log.logging_mixin import LoggingMixin @@ -71,9 +71,9 @@ def hook(self): return None @cached_property - def triggerer_logs_separate(self): + def trigger_logs_presentation_mode(self): """Not implemented yet.""" - return False + return TriggerLogsPresentationMode.NOT_SUPPORTED def set_context(self, ti) -> None: super().set_context(ti) diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index 9657a27eacf66..e4b041d770404 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -73,6 +73,19 @@ def _fetch_logs_from_service(url, log_relative_path): return response +class TriggerLogsPresentationMode(str, Enum): + """ + Controls how trigger logs are presented in webserver. + + Handlers may be implemented either to have trigger logs interleaved with task logs + or shown in a separate tab. + """ + + SPLIT = "split" + INTERLEAVED = "interleaved" + NOT_SUPPORTED = "not_supported" + + class FileTaskHandler(logging.Handler): """ FileTaskHandler is a python log handler that handles and reads @@ -146,13 +159,16 @@ def add_triggerer_suffix(full_path, job_id=None): return full_path @cached_property - def triggerer_logs_separate(self): + def trigger_logs_presentation_mode(self): """ - If true, webserver should render trigger logs in distinct tab + Tells webserver whether to use separate tab for triggerer logs. :meta private: """ - return "log_type" in inspect.signature(self._read).parameters.keys() + if "log_type" in inspect.signature(self._read).parameters.keys(): + return TriggerLogsPresentationMode.SPLIT # split is default, most common + else: + return TriggerLogsPresentationMode.NOT_SUPPORTED @cached_property def wrap_for_triggerer(self): @@ -162,7 +178,10 @@ def wrap_for_triggerer(self): :meta private: """ - return "log_type" in inspect.signature(self._read).parameters.keys() + return self.trigger_logs_presentation_mode in ( + TriggerLogsPresentationMode.SPLIT, + TriggerLogsPresentationMode.INTERLEAVED, + ) def emit(self, record): if self.handler: diff --git a/airflow/utils/log/log_reader.py b/airflow/utils/log/log_reader.py index 448ac4600dc60..eb533505b26da 100644 --- a/airflow/utils/log/log_reader.py +++ b/airflow/utils/log/log_reader.py @@ -26,6 +26,7 @@ from airflow.configuration import conf from airflow.models.taskinstance import TaskInstance from airflow.utils.helpers import render_log_filename +from airflow.utils.log.file_task_handler import TriggerLogsPresentationMode from airflow.utils.log.logging_mixin import ExternalLoggingMixin from airflow.utils.session import NEW_SESSION, provide_session from airflow.utils.state import State @@ -60,7 +61,8 @@ def read_log_chunks( contain information about the task log which can enable you read logs to the end. """ - kwargs = {"log_type": log_type} if self.triggerer_logs_separate else {} + split_mode = self.trigger_logs_presentation_mode == TriggerLogsPresentationMode.SPLIT + kwargs = {"log_type": log_type} if split_mode else {} logs, metadatas = self.log_handler.read(ti, try_number, metadata=metadata, **kwargs) metadata = metadatas[0] return logs, metadata @@ -86,7 +88,8 @@ def read_log_stream( metadata.pop("offset", None) metadata.pop("log_pos", None) while True: - kwargs = {"log_type": log_type} if self.triggerer_logs_separate else {} + split_mode = self.trigger_logs_presentation_mode == TriggerLogsPresentationMode.SPLIT + kwargs = {"log_type": log_type} if split_mode else {} logs, metadata = self.read_log_chunks(ti, current_try_number, metadata, **kwargs) for host, log in logs[0]: yield "\n".join([host or "", log]) + "\n" @@ -114,8 +117,12 @@ def supports_read(self): return hasattr(self.log_handler, "read") @property - def triggerer_logs_separate(self): - return getattr(self.log_handler, "triggerer_logs_separate", False) + def trigger_logs_presentation_mode(self): + return getattr( + self.log_handler, + "trigger_logs_presentation_mode", + TriggerLogsPresentationMode.NOT_SUPPORTED, + ) @property def supports_external_link(self) -> bool: diff --git a/airflow/www/static/js/api/useGridData.ts b/airflow/www/static/js/api/useGridData.ts index 26aa2db09a57f..321d23a54ec82 100644 --- a/airflow/www/static/js/api/useGridData.ts +++ b/airflow/www/static/js/api/useGridData.ts @@ -36,11 +36,17 @@ const dagId = getMetaValue(DAG_ID_PARAM); const gridDataUrl = getMetaValue('grid_data_url'); const urlRoot = getMetaValue('root'); +export enum TriggerLogsPresentationMode { + SPLIT = 'split', + NOT_SUPPORTED = 'not_supported', + INTERLEAVED = 'interleaved', +} + export interface GridData { dagRuns: DagRun[]; groups: Task; ordering: RunOrdering; - readerTriggererLogsSeparate: boolean; + triggerLogsPresentationMode: TriggerLogsPresentationMode; } export const emptyGridData: GridData = { @@ -51,7 +57,7 @@ export const emptyGridData: GridData = { instances: [], }, ordering: [], - readerTriggererLogsSeparate: false, + triggerLogsPresentationMode: TriggerLogsPresentationMode.NOT_SUPPORTED, }; const formatOrdering = (data: GridData) => ({ diff --git a/airflow/www/static/js/dag/details/taskInstance/index.tsx b/airflow/www/static/js/dag/details/taskInstance/index.tsx index 6ad586ca12e90..8bf2c527e643e 100644 --- a/airflow/www/static/js/dag/details/taskInstance/index.tsx +++ b/airflow/www/static/js/dag/details/taskInstance/index.tsx @@ -40,6 +40,7 @@ import NotesAccordion from 'src/dag/details/NotesAccordion'; import { useContainerRef } from 'src/context/containerRef'; import TabWithTooltip from 'src/components/TabWithTooltip'; +import { TriggerLogsPresentationMode } from 'src/api/useGridData'; import ExtraLinks from './ExtraLinks'; import Logs from './Logs'; import TaskNav from './Nav'; @@ -64,7 +65,7 @@ const TaskInstance = ({ }: Props) => { const isMapIndexDefined = !(mapIndex === undefined); const actionsMapIndexes = isMapIndexDefined ? [mapIndex] : []; - const { data: { dagRuns, groups, readerTriggererLogsSeparate } } = useGridData(); + const { data: { dagRuns, groups, triggerLogsPresentationMode } } = useGridData(); const containerRef = useContainerRef(); const detailsRef = useRef(null); const offsetHeight = useOffsetHeight(detailsRef); @@ -95,14 +96,18 @@ const TaskInstance = ({ localStorage.setItem(detailsPanelActiveTabIndex, index.toString()); setPreferedTabIndex(index); }; - + // eslint-disable-next-line max-len + const triggerLogsNotSupported = triggerLogsPresentationMode === TriggerLogsPresentationMode.NOT_SUPPORTED; + const triggerLogsSplit = triggerLogsPresentationMode === TriggerLogsPresentationMode.SPLIT; + // eslint-disable-next-line max-len + const triggerLogsInterleaved = triggerLogsPresentationMode === TriggerLogsPresentationMode.INTERLEAVED; let triggerLogsTooltipProps; - if (!readerTriggererLogsSeparate) { + if (triggerLogsNotSupported) { triggerLogsTooltipProps = { label: 'The configured log handler does not support reading trigger logs.', isDisabled: false, }; - } else if (!instance?.hasDeferred) { + } else if (triggerLogsSplit && !instance?.hasDeferred) { triggerLogsTooltipProps = { label: 'This task has no deferrals.', isDisabled: false, @@ -113,14 +118,13 @@ const TaskInstance = ({ isDisabled: true, }; } - const disableTriggerLogs = !(readerTriggererLogsSeparate && instance?.hasDeferred === true); - + const skipTriggerLogsRender = !triggerLogsSplit || !instance?.hasDeferred; useEffect(() => { // Reset preferred tab if it is disabled - if (disableTriggerLogs && preferedTabIndex === 2) { + if (skipTriggerLogsRender && preferedTabIndex === 2) { setPreferedTabIndex(1); } - }, [disableTriggerLogs, preferedTabIndex]); + }, [skipTriggerLogsRender, preferedTabIndex]); if (!group || !run || !instance) return null; @@ -179,18 +183,17 @@ const TaskInstance = ({ )} {!isGroupOrMappedTaskSummary && ( - Logs + Logs )} - - {!isGroupOrMappedTaskSummary && !disableTriggerLogs && ( + {!isGroupOrMappedTaskSummary && !triggerLogsInterleaved && ( - + Triggerer logs @@ -265,7 +268,7 @@ const TaskInstance = ({ {/* Triggerer Logs Tab */} {!isGroupOrMappedTaskSummary && ( - {instance?.hasDeferred === true && ( + {!skipTriggerLogsRender && ( Date: Thu, 12 Jan 2023 23:47:56 -0800 Subject: [PATCH 11/96] use-enum --- airflow/cli/commands/task_command.py | 6 +++--- airflow/jobs/local_task_job.py | 4 +++- airflow/models/taskinstance.py | 15 ++++++++++----- airflow/task/task_runner/standard_task_runner.py | 4 ++-- 4 files changed, 18 insertions(+), 11 deletions(-) diff --git a/airflow/cli/commands/task_command.py b/airflow/cli/commands/task_command.py index b786610e4c225..8b46a44dca6b3 100644 --- a/airflow/cli/commands/task_command.py +++ b/airflow/cli/commands/task_command.py @@ -44,7 +44,7 @@ from airflow.models.dag import DAG from airflow.models.dagrun import DagRun from airflow.models.operator import needs_expansion -from airflow.models.taskinstance import ArgDeferred +from airflow.models.taskinstance import TaskReturnCode from airflow.settings import IS_K8S_EXECUTOR_POD from airflow.ti_deps.dep_context import DepContext from airflow.ti_deps.dependencies_deps import SCHEDULER_QUEUED_DEPS @@ -183,7 +183,7 @@ def _get_ti( return ti, dr_created -def _run_task_by_selected_method(args, dag: DAG, ti: TaskInstance) -> None | ArgDeferred: +def _run_task_by_selected_method(args, dag: DAG, ti: TaskInstance) -> None | TaskReturnCode: """ Runs the task based on a mode. @@ -270,7 +270,7 @@ def _run_task_by_local_task_job(args, ti): ] -def _run_raw_task(args, ti: TaskInstance) -> None | ArgDeferred: +def _run_raw_task(args, ti: TaskInstance) -> None | TaskReturnCode: """Runs the main task handling code.""" return ti._run_raw_task( mark_success=args.mark_success, diff --git a/airflow/jobs/local_task_job.py b/airflow/jobs/local_task_job.py index 148ca8845d68c..859ad24509d3f 100644 --- a/airflow/jobs/local_task_job.py +++ b/airflow/jobs/local_task_job.py @@ -222,11 +222,13 @@ def handle_task_exit(self, return_code: int) -> None: """ # Without setting this, heartbeat may get us self.terminating = True - self.log.info("Task exited with return code %s", return_code) self._log_return_code_metric(return_code) is_deferral = return_code == DEFERRAL_EXIT_CODE if is_deferral: + self.log.info("Task exited with return code %s (task deferral)", return_code) set_task_deferred_context_var() + else: + self.log.info("Task exited with return code %s", return_code) if not self.task_instance.test_mode and not is_deferral: if conf.getboolean("scheduler", "schedule_after_task_execution", fallback=True): diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index 59f90256a3a9f..f58731494b2aa 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -28,6 +28,7 @@ import warnings from collections import defaultdict from datetime import datetime, timedelta +from enum import Enum from functools import partial from types import TracebackType from typing import TYPE_CHECKING, Any, Callable, Collection, Generator, Iterable, NamedTuple, Tuple @@ -139,11 +140,15 @@ PAST_DEPENDS_MET = "past_depends_met" -class ArgDeferred: - """sentinel.""" +class TaskReturnCode(Enum): + """ + Enum to signal manner of exit for task run command. + :meta private: + """ -DEFERRED = ArgDeferred() + DEFERRED = 101 + """When task exits with deferral to trigger.""" @contextlib.contextmanager @@ -1380,7 +1385,7 @@ def _run_raw_task( job_id: str | None = None, pool: str | None = None, session: Session = NEW_SESSION, - ) -> ArgDeferred | None: + ) -> TaskReturnCode | None: """ Immediately runs the task (without checking or changing db state before execution) and then sets the appropriate final state after @@ -1430,7 +1435,7 @@ def _run_raw_task( session.add(Log(self.state, self)) session.merge(self) session.commit() - return DEFERRED + return TaskReturnCode.DEFERRED except AirflowSkipException as e: # Recording SKIP # log only if exception has any arguments to prevent log flooding diff --git a/airflow/task/task_runner/standard_task_runner.py b/airflow/task/task_runner/standard_task_runner.py index a5dccf93f91c2..10d02f79cfd98 100644 --- a/airflow/task/task_runner/standard_task_runner.py +++ b/airflow/task/task_runner/standard_task_runner.py @@ -24,7 +24,7 @@ import psutil from setproctitle import setproctitle -from airflow.models.taskinstance import ArgDeferred +from airflow.models.taskinstance import TaskReturnCode from airflow.settings import CAN_FORK from airflow.task.task_runner.base_task_runner import BaseTaskRunner from airflow.utils.dag_parsing_context import _airflow_parsing_context_manager @@ -97,7 +97,7 @@ def _start_by_fork(self): ): ret = args.func(args, dag=self.dag) return_code = 0 - if isinstance(ret, ArgDeferred): + if isinstance(ret, TaskReturnCode): return_code = DEFERRAL_EXIT_CODE except Exception as exc: return_code = 1 From 301907dff0d094668571de0b3648b58ae2930ce3 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 12 Jan 2023 23:49:05 -0800 Subject: [PATCH 12/96] handle odd error --- .../www/static/js/dag/details/taskInstance/Logs/utils.ts | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/airflow/www/static/js/dag/details/taskInstance/Logs/utils.ts b/airflow/www/static/js/dag/details/taskInstance/Logs/utils.ts index 4a5b8b8664b70..1ca51cd3efe64 100644 --- a/airflow/www/static/js/dag/details/taskInstance/Logs/utils.ts +++ b/airflow/www/static/js/dag/details/taskInstance/Logs/utils.ts @@ -46,8 +46,13 @@ export const parseLogs = ( if (!data) { return {}; } - - const lines = data.split('\n'); + let lines; + try { + lines = data.split('\n'); + } catch (err) { + // debugger + return {}; + } const parsedLines: Array = []; const fileSources: Set = new Set(); From e63ebb517720f35de9a28198c8bda73f9ebfed13 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 12 Jan 2023 23:53:47 -0800 Subject: [PATCH 13/96] enum --- airflow/utils/log/file_task_handler.py | 2 +- airflow/www/views.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index e4b041d770404..efe282ecb9d61 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -73,7 +73,7 @@ def _fetch_logs_from_service(url, log_relative_path): return response -class TriggerLogsPresentationMode(str, Enum): +class TriggerLogsPresentationMode(Enum): """ Controls how trigger logs are presented in webserver. diff --git a/airflow/www/views.py b/airflow/www/views.py index dc90d2d6644a1..45313a96803a5 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -3621,7 +3621,7 @@ def grid_data(self): "groups": dag_to_grid(dag, dag_runs, session), "dag_runs": encoded_runs, "ordering": dag.timetable.run_ordering, - "trigger_logs_presentation_mode": task_log_reader.trigger_logs_presentation_mode, + "trigger_logs_presentation_mode": task_log_reader.trigger_logs_presentation_mode.value, } # avoid spaces to reduce payload size return ( From 6409cda40b96fbf2abdce2021978b149b356e09d Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 13 Jan 2023 00:07:35 -0800 Subject: [PATCH 14/96] no extra code --- airflow/jobs/local_task_job.py | 5 ++--- airflow/models/taskinstance.py | 2 +- airflow/task/task_runner/standard_task_runner.py | 4 +--- 3 files changed, 4 insertions(+), 7 deletions(-) diff --git a/airflow/jobs/local_task_job.py b/airflow/jobs/local_task_job.py index 859ad24509d3f..1b3e7b9c705a9 100644 --- a/airflow/jobs/local_task_job.py +++ b/airflow/jobs/local_task_job.py @@ -27,10 +27,9 @@ from airflow.jobs.base_job import BaseJob from airflow.listeners.events import register_task_instance_state_events from airflow.listeners.listener import get_listener_manager -from airflow.models.taskinstance import TaskInstance +from airflow.models.taskinstance import TaskInstance, TaskReturnCode from airflow.stats import Stats from airflow.task.task_runner import get_task_runner -from airflow.task.task_runner.standard_task_runner import DEFERRAL_EXIT_CODE from airflow.utils import timezone from airflow.utils.net import get_hostname from airflow.utils.session import provide_session @@ -223,7 +222,7 @@ def handle_task_exit(self, return_code: int) -> None: # Without setting this, heartbeat may get us self.terminating = True self._log_return_code_metric(return_code) - is_deferral = return_code == DEFERRAL_EXIT_CODE + is_deferral = return_code == TaskReturnCode.DEFERRED.value if is_deferral: self.log.info("Task exited with return code %s (task deferral)", return_code) set_task_deferred_context_var() diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index f58731494b2aa..09ed479ec3f51 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -147,7 +147,7 @@ class TaskReturnCode(Enum): :meta private: """ - DEFERRED = 101 + DEFERRED = 100 """When task exits with deferral to trigger.""" diff --git a/airflow/task/task_runner/standard_task_runner.py b/airflow/task/task_runner/standard_task_runner.py index 10d02f79cfd98..a06f2eb82de92 100644 --- a/airflow/task/task_runner/standard_task_runner.py +++ b/airflow/task/task_runner/standard_task_runner.py @@ -30,8 +30,6 @@ from airflow.utils.dag_parsing_context import _airflow_parsing_context_manager from airflow.utils.process_utils import reap_process_group, set_new_process_group -DEFERRAL_EXIT_CODE = 100 - class StandardTaskRunner(BaseTaskRunner): """Standard runner for all tasks.""" @@ -98,7 +96,7 @@ def _start_by_fork(self): ret = args.func(args, dag=self.dag) return_code = 0 if isinstance(ret, TaskReturnCode): - return_code = DEFERRAL_EXIT_CODE + return_code = ret.value except Exception as exc: return_code = 1 From eeb66a6f21c66860bbffe28dd67cd858e91ba683 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 13 Jan 2023 01:35:11 -0800 Subject: [PATCH 15/96] add support for cloudwatch --- .../providers/amazon/aws/log/cloudwatch_task_handler.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py b/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py index e50a6d4d74e2e..a9ab9feb29c42 100644 --- a/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py +++ b/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py @@ -24,7 +24,7 @@ from airflow.compat.functools import cached_property from airflow.configuration import conf from airflow.providers.amazon.aws.hooks.logs import AwsLogsHook -from airflow.utils.log.file_task_handler import FileTaskHandler +from airflow.utils.log.file_task_handler import FileTaskHandler, TriggerLogsPresentationMode from airflow.utils.log.logging_mixin import LoggingMixin @@ -68,6 +68,10 @@ def set_context(self, ti): boto3_client=self.hook.get_conn(), ) + @cached_property + def trigger_logs_presentation_mode(self): + return TriggerLogsPresentationMode.INTERLEAVED + def close(self): """Close the handler responsible for the upload of the local log file to Cloudwatch.""" # When application exit, system shuts down all handlers by @@ -82,7 +86,7 @@ def close(self): # Mark closed so we don't double write if close is called twice self.closed = True - def _read(self, task_instance, try_number, metadata=None): + def _read(self, task_instance, try_number, metadata=None, log_type=None): stream_name = self._render_filename(task_instance, try_number) try: return ( From 4efce54d94e21c1d9d1c0a38d114b685a134de67 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 13 Jan 2023 13:12:18 -0800 Subject: [PATCH 16/96] add support for gcp --- .../google/cloud/log/gcs_task_handler.py | 78 ++++++++++++++----- 1 file changed, 58 insertions(+), 20 deletions(-) diff --git a/airflow/providers/google/cloud/log/gcs_task_handler.py b/airflow/providers/google/cloud/log/gcs_task_handler.py index 5fbba80798498..e4e75dd636c76 100644 --- a/airflow/providers/google/cloud/log/gcs_task_handler.py +++ b/airflow/providers/google/cloud/log/gcs_task_handler.py @@ -17,16 +17,19 @@ # under the License. from __future__ import annotations +import json import os +from pathlib import Path from typing import Collection # not sure why but mypy complains on missing `storage` but it is clearly there and is importable from google.cloud import storage # type: ignore[attr-defined] from airflow.compat.functools import cached_property +from airflow.providers.google.cloud.hooks.gcs import _parse_gcs_url from airflow.providers.google.cloud.utils.credentials_provider import get_credentials_and_project_id from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.utils.log.file_task_handler import FileTaskHandler +from airflow.utils.log.file_task_handler import FileTaskHandler, LogType from airflow.utils.log.logging_mixin import LoggingMixin _DEFAULT_SCOPESS = frozenset( @@ -100,8 +103,10 @@ def set_context(self, ti): # Log relative path is used to construct local and remote # log path to upload log files into GCS and read from the # remote location. - self.log_relative_path = self._render_filename(ti, ti.try_number) - self.upload_on_close = not ti.raw + full_path = self.handler.baseFilename + self.log_relative_path = Path(full_path).relative_to(self.local_base).as_posix() + is_trigger_log_context = getattr(ti, "is_trigger_log_context", False) + self.upload_on_close = is_trigger_log_context or not ti.raw def close(self): """Close and upload local log file to remote storage GCS.""" @@ -128,7 +133,7 @@ def close(self): # Mark closed so we don't double write if close is called twice self.closed = True - def _read(self, ti, try_number, metadata=None): + def _read(self, ti, try_number, metadata=None, log_type=None): """ Read logs of given task instance and try_number from GCS. If failed, read the log from task instance host machine. @@ -138,21 +143,40 @@ def _read(self, ti, try_number, metadata=None): :param metadata: log metadata, can be used for steaming log reading and auto-tailing. """ - # Explicitly getting log relative path is necessary as the given - # task instance might be different than task instance passed in - # in set_context method. - log_relative_path = self._render_filename(ti, try_number) - remote_loc = os.path.join(self.remote_base, log_relative_path) - + self.log.warning("called with metadata=%s", json.dumps(metadata)) + + # Explicitly getting log relative path is necessary because this method + # is called from webserver from TaskLogReader, where we don't call set_context + # and can read logs for different TIs in each request + log = "" + worker_log_relative_path = self._render_filename(ti, try_number) + remote_loc = os.path.join(self.remote_base, worker_log_relative_path) + uris = [] + if log_type == LogType.TRIGGER: + if ti.triggerer_job: + # triggerer currently running; skip s3 read and try to read from triggerer log server + return super()._read(ti, try_number, metadata, log_type=log_type) + bucket, prefix = _parse_gcs_url(remote_loc) + blobs = list(self.client.list_blobs(bucket_or_name=bucket, prefix=prefix + ".")) + if blobs: + uris = [f"gs://{bucket}/{b.name}" for b in blobs] + else: + log += f"*** No logs found for triggerer; ti=%s {ti}\n" + else: + uris = [remote_loc] try: - blob = storage.Blob.from_string(remote_loc, self.client) - remote_log = blob.download_as_bytes().decode() - log = f"*** Reading remote log from {remote_loc}.\n{remote_log}\n" + for key in uris: + blob = storage.Blob.from_string(key, self.client) + remote_log = blob.download_as_bytes().decode() + log += f"*** Reading remote log from {remote_loc}.\n{remote_log}\n" + return log, {"end_of_log": True} except Exception as e: - log = f"*** Unable to read remote log from {remote_loc}\n*** {str(e)}\n\n" - self.log.error(log) - local_log, metadata = super()._read(ti, try_number, metadata) + log = f"*** Unable to read remote log {e}\n\n" + kwargs = {} + if log_type: + kwargs.update({"log_type": log_type}) + local_log, metadata = super()._read(ti, try_number, metadata, **kwargs) log += local_log return log, metadata @@ -169,12 +193,26 @@ def gcs_write(self, log, remote_log_location): old_log = blob.download_as_bytes().decode() log = "\n".join([old_log, log]) if old_log else log except Exception as e: - if not hasattr(e, "resp") or e.resp.get("status") != "404": - log = f"*** Previous log discarded: {str(e)}\n\n" + log - self.log.info("Previous log discarded: %s", e) - + if self.no_log_found(e): + pass + else: + log += f"*** Error checking for previous log; if exists, may be overwritten: {str(e)}\n\n" + self.log.warning("Error checking for previous log: %s", e) try: blob = storage.Blob.from_string(remote_log_location, self.client) blob.upload_from_string(log, content_type="text/plain") except Exception as e: self.log.error("Could not write logs to %s: %s", remote_log_location, e) + + @staticmethod + def no_log_found(exc): + """ + Given exception, determine whether it is result of log not found. + + :meta private: + """ + if exc.args and isinstance(exc.args[0], str) and "No such object" in exc.args[0]: + return True + elif getattr(exc, "resp", {}).get("status") == "404": + return True + return False From 933add03e9e9895d7be138cd9d41580cc915bc3d Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 13 Jan 2023 20:57:07 -0800 Subject: [PATCH 17/96] --wip-- [skip ci] --- airflow/providers/google/cloud/log/gcs_task_handler.py | 2 +- airflow/utils/log/file_task_handler.py | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/airflow/providers/google/cloud/log/gcs_task_handler.py b/airflow/providers/google/cloud/log/gcs_task_handler.py index e4e75dd636c76..47e161984e13e 100644 --- a/airflow/providers/google/cloud/log/gcs_task_handler.py +++ b/airflow/providers/google/cloud/log/gcs_task_handler.py @@ -154,7 +154,7 @@ def _read(self, ti, try_number, metadata=None, log_type=None): uris = [] if log_type == LogType.TRIGGER: if ti.triggerer_job: - # triggerer currently running; skip s3 read and try to read from triggerer log server + # triggerer currently running; skip remote read and try to read from triggerer log server return super()._read(ti, try_number, metadata, log_type=log_type) bucket, prefix = _parse_gcs_url(remote_loc) blobs = list(self.client.list_blobs(bucket_or_name=bucket, prefix=prefix + ".")) diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index efe282ecb9d61..b0f163b888329 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -267,6 +267,7 @@ def _read( metadata: dict[str, Any] | None = None, *, log_type: LogType | None = LogType.WORKER, + base_content=None, ): """ Template method that contains custom logic of reading From 3bf09e4aced80f7fe26b8a90fbb3399fde84a5c0 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Sun, 15 Jan 2023 20:56:43 -0800 Subject: [PATCH 18/96] forward shut down logs arg to task runner --- airflow/cli/commands/task_command.py | 1 + airflow/jobs/local_task_job.py | 2 ++ airflow/models/taskinstance.py | 6 ++++++ airflow/task/task_runner/base_task_runner.py | 1 + airflow/task/task_runner/standard_task_runner.py | 3 ++- 5 files changed, 12 insertions(+), 1 deletion(-) diff --git a/airflow/cli/commands/task_command.py b/airflow/cli/commands/task_command.py index 8b46a44dca6b3..ad3e1deb68b1b 100644 --- a/airflow/cli/commands/task_command.py +++ b/airflow/cli/commands/task_command.py @@ -253,6 +253,7 @@ def _run_task_by_local_task_job(args, ti): ignore_ti_state=args.force, pool=args.pool, external_executor_id=_extract_external_executor_id(args), + shut_down_logging=args.shut_down_logging, ) try: run_job.run() diff --git a/airflow/jobs/local_task_job.py b/airflow/jobs/local_task_job.py index 1b3e7b9c705a9..f932574eefed2 100644 --- a/airflow/jobs/local_task_job.py +++ b/airflow/jobs/local_task_job.py @@ -95,6 +95,7 @@ def __init__( pickle_id: str | None = None, pool: str | None = None, external_executor_id: str | None = None, + shut_down_logging: bool | None = None, *args, **kwargs, ): @@ -109,6 +110,7 @@ def __init__( self.pickle_id = pickle_id self.mark_success = mark_success self.external_executor_id = external_executor_id + self.shut_down_logging = shut_down_logging # terminating state is used so that a job don't try to # terminate multiple times diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index 09ed479ec3f51..cefc49eabf8b9 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -622,6 +622,7 @@ def command_as_list( job_id=None, pool=None, cfg_path=None, + shut_down_logging: bool | None = None, ): """ Returns a command that can be executed anywhere where airflow is @@ -666,6 +667,7 @@ def command_as_list( pool=pool, cfg_path=cfg_path, map_index=self.map_index, + shut_down_logging=shut_down_logging, ) @staticmethod @@ -687,6 +689,7 @@ def generate_command( pool: str | None = None, cfg_path: str | None = None, map_index: int = -1, + shut_down_logging: bool | None = None, ) -> list[str]: """ Generates the shell command required to execute this task instance. @@ -711,6 +714,7 @@ def generate_command( :param job_id: job ID (needs more details) :param pool: the Airflow pool that the task should run in :param cfg_path: the Path to the configuration file + :param shut_down_logging: whether logging.shutdown() should be called :return: shell command that can be used to run the task instance """ cmd = ["airflow", "tasks", "run", dag_id, task_id, run_id] @@ -742,6 +746,8 @@ def generate_command( cmd.extend(["--cfg-path", cfg_path]) if map_index != -1: cmd.extend(["--map-index", str(map_index)]) + if shut_down_logging is False: + cmd.extend(["--no-shut-down-logging"]) return cmd @property diff --git a/airflow/task/task_runner/base_task_runner.py b/airflow/task/task_runner/base_task_runner.py index 095c9b7300ff8..64aec85105c5a 100644 --- a/airflow/task/task_runner/base_task_runner.py +++ b/airflow/task/task_runner/base_task_runner.py @@ -99,6 +99,7 @@ def __init__(self, local_task_job): job_id=local_task_job.id, pool=local_task_job.pool, cfg_path=cfg_path, + shut_down_logging=local_task_job.shut_down_logging, ) self.process = None diff --git a/airflow/task/task_runner/standard_task_runner.py b/airflow/task/task_runner/standard_task_runner.py index a06f2eb82de92..d56778962e825 100644 --- a/airflow/task/task_runner/standard_task_runner.py +++ b/airflow/task/task_runner/standard_task_runner.py @@ -121,7 +121,8 @@ def _start_by_fork(self): try: # Explicitly flush any pending exception to Sentry and logging if enabled Sentry.flush() - logging.shutdown() + if args.shut_down_logging: + logging.shutdown() except BaseException: # also make sure to silently ignore ALL POSSIBLE exceptions thrown in the flush/shutdown, # otherwise os._exit() might never be called. We could have used `except:` but From 4f92a0916bdd023d94f452f1d746a001f1284961 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Sun, 15 Jan 2023 20:59:41 -0800 Subject: [PATCH 19/96] ensure that deferral return code propagates to local task job --- airflow/cli/commands/task_command.py | 32 +++++++++++++++++++++------- airflow/jobs/base_job.py | 4 +++- airflow/jobs/local_task_job.py | 20 ++++------------- 3 files changed, 31 insertions(+), 25 deletions(-) diff --git a/airflow/cli/commands/task_command.py b/airflow/cli/commands/task_command.py index ad3e1deb68b1b..2b3eb481138b7 100644 --- a/airflow/cli/commands/task_command.py +++ b/airflow/cli/commands/task_command.py @@ -70,6 +70,18 @@ CreateIfNecessary = Union[Literal[False], Literal["db"], Literal["memory"]] +def _set_task_deferred_context_var(): + """ + Tell task log handler that task exited with deferral. + + :meta private: + """ + logger = logging.getLogger() + h = next((h for h in logger.handlers if hasattr(h, "ctx_task_deferred")), None) + if h: + h.ctx_task_deferred = True + + def _generate_temporary_run_id() -> str: """Generate a ``run_id`` for a DAG run that will be created temporarily. @@ -194,11 +206,11 @@ def _run_task_by_selected_method(args, dag: DAG, ti: TaskInstance) -> None | Tas - by executor """ if args.local: - _run_task_by_local_task_job(args, ti) + return _run_task_by_local_task_job(args, ti) elif args.raw: return _run_raw_task(args, ti) else: - _run_task_by_executor(args, dag, ti) + return _run_task_by_executor(args, dag, ti) def _run_task_by_executor(args, dag, ti): @@ -240,7 +252,7 @@ def _run_task_by_executor(args, dag, ti): executor.end() -def _run_task_by_local_task_job(args, ti): +def _run_task_by_local_task_job(args, ti) -> TaskReturnCode | None: """Run LocalTaskJob, which monitors the raw task execution process.""" run_job = LocalTaskJob( task_instance=ti, @@ -256,11 +268,13 @@ def _run_task_by_local_task_job(args, ti): shut_down_logging=args.shut_down_logging, ) try: - run_job.run() + ret = run_job.run() finally: if args.shut_down_logging: logging.shutdown() + with suppress(ValueError): + return TaskReturnCode(ret) RAW_TASK_UNSUPPORTED_OPTION = [ @@ -409,19 +423,21 @@ def task_run(args, dag=None): # this should be last thing before running, to reduce likelihood of an open session # which can cause trouble if running process in a fork. settings.reconfigure_orm(disable_connection_pool=True) - ret = None + task_return_code = None try: if args.interactive: - ret = _run_task_by_selected_method(args, dag, ti) + task_return_code = _run_task_by_selected_method(args, dag, ti) else: with _move_task_handlers_to_root(ti), _redirect_stdout_to_ti_log(ti): - ret = _run_task_by_selected_method(args, dag, ti) + task_return_code = _run_task_by_selected_method(args, dag, ti) + if task_return_code == TaskReturnCode.DEFERRED: + _set_task_deferred_context_var() finally: try: get_listener_manager().hook.before_stopping(component=TaskCommandMarker()) except Exception: pass - return ret + return task_return_code @cli_utils.action_cli(check_db=False) diff --git a/airflow/jobs/base_job.py b/airflow/jobs/base_job.py index 580b5a36edecb..919e1a7fbca80 100644 --- a/airflow/jobs/base_job.py +++ b/airflow/jobs/base_job.py @@ -249,6 +249,7 @@ def run(self): """Starts the job.""" Stats.incr(self.__class__.__name__.lower() + "_start", 1, 1) # Adding an entry in the DB + ret = None with create_session() as session: self.state = State.RUNNING session.add(self) @@ -256,7 +257,7 @@ def run(self): make_transient(self) try: - self._execute() + ret = self._execute() # In case of max runs or max duration self.state = State.SUCCESS except SystemExit: @@ -272,6 +273,7 @@ def run(self): session.commit() Stats.incr(self.__class__.__name__.lower() + "_end", 1, 1) + return ret def _execute(self): raise NotImplementedError("This method needs to be overridden") diff --git a/airflow/jobs/local_task_job.py b/airflow/jobs/local_task_job.py index f932574eefed2..80f39f0f308e5 100644 --- a/airflow/jobs/local_task_job.py +++ b/airflow/jobs/local_task_job.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import logging import signal import psutil @@ -66,18 +65,6 @@ ********************************************************************************************************""" -def set_task_deferred_context_var(): - """ - Tell task log handler that task exited with deferral. - - :meta private: - """ - logger = logging.getLogger() - h = next((h for h in logger.handlers if hasattr(h, "ctx_task_deferred")), None) - if h: - h.ctx_task_deferred = True - - class LocalTaskJob(BaseJob): """LocalTaskJob runs a single task instance.""" @@ -120,7 +107,7 @@ def __init__( super().__init__(*args, **kwargs) - def _execute(self): + def _execute(self) -> int | None: self._enable_task_listeners() self.task_runner = get_task_runner(self) @@ -165,6 +152,7 @@ def sigusr2_debug_handler(signum, frame): self.log.info("Task is not able to be run") return + return_code = None try: self.task_runner.start() @@ -197,7 +185,7 @@ def sigusr2_debug_handler(signum, frame): return_code = self.task_runner.return_code(timeout=max_wait_time) if return_code is not None: self.handle_task_exit(return_code) - return + return return_code self.heartbeat() @@ -214,6 +202,7 @@ def sigusr2_debug_handler(signum, frame): ) finally: self.on_kill() + return return_code def handle_task_exit(self, return_code: int) -> None: """ @@ -227,7 +216,6 @@ def handle_task_exit(self, return_code: int) -> None: is_deferral = return_code == TaskReturnCode.DEFERRED.value if is_deferral: self.log.info("Task exited with return code %s (task deferral)", return_code) - set_task_deferred_context_var() else: self.log.info("Task exited with return code %s", return_code) From 81202bcbe7c928da88ac10ac863480e7f5659d90 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 16 Jan 2023 00:16:33 -0800 Subject: [PATCH 20/96] start the process of switching to single pane -- WIP! --- .../api_connexion/endpoints/log_endpoint.py | 4 +- .../google/cloud/log/gcs_task_handler.py | 80 +++--- airflow/utils/log/file_task_handler.py | 239 ++++++++++-------- tests/utils/test_log_handlers.py | 171 ++++++++++++- 4 files changed, 357 insertions(+), 137 deletions(-) diff --git a/airflow/api_connexion/endpoints/log_endpoint.py b/airflow/api_connexion/endpoints/log_endpoint.py index 582a525a5933b..ba035f5676bf6 100644 --- a/airflow/api_connexion/endpoints/log_endpoint.py +++ b/airflow/api_connexion/endpoints/log_endpoint.py @@ -95,9 +95,9 @@ def get_log( TaskInstance.map_index == map_index, ) .join(TaskInstance.dag_run) + .options(joinedload("trigger")) + .options(joinedload("trigger.triggerer_job")) ) - if log_type == LogType.TRIGGER: - query = query.options(joinedload("trigger")).options(joinedload("trigger.triggerer_job")) ti = query.one_or_none() if ti is None: metadata["end_of_log"] = True diff --git a/airflow/providers/google/cloud/log/gcs_task_handler.py b/airflow/providers/google/cloud/log/gcs_task_handler.py index 47e161984e13e..431eaeaa221e9 100644 --- a/airflow/providers/google/cloud/log/gcs_task_handler.py +++ b/airflow/providers/google/cloud/log/gcs_task_handler.py @@ -17,7 +17,7 @@ # under the License. from __future__ import annotations -import json +import logging import os from pathlib import Path from typing import Collection @@ -29,7 +29,7 @@ from airflow.providers.google.cloud.hooks.gcs import _parse_gcs_url from airflow.providers.google.cloud.utils.credentials_provider import get_credentials_and_project_id from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.utils.log.file_task_handler import FileTaskHandler, LogType +from airflow.utils.log.file_task_handler import FileTaskHandler from airflow.utils.log.logging_mixin import LoggingMixin _DEFAULT_SCOPESS = frozenset( @@ -38,6 +38,8 @@ ] ) +logger = logging.getLogger(__name__) + class GCSTaskHandler(FileTaskHandler, LoggingMixin): """ @@ -133,52 +135,58 @@ def close(self): # Mark closed so we don't double write if close is called twice self.closed = True - def _read(self, ti, try_number, metadata=None, log_type=None): - """ - Read logs of given task instance and try_number from GCS. - If failed, read the log from task instance host machine. - - :param ti: task instance object - :param try_number: task instance try_number to read logs from - :param metadata: log metadata, - can be used for steaming log reading and auto-tailing. - """ - self.log.warning("called with metadata=%s", json.dumps(metadata)) + def _add_message(self, msg): + filename, lineno, func, stackinfo = logger.findCaller() + record = logging.LogRecord("", logging.INFO, filename, lineno, msg + "\n", None, None, func=func) + return self.format(record) + def _read_remote_logs(self, ti, try_number, metadata=None): # Explicitly getting log relative path is necessary because this method # is called from webserver from TaskLogReader, where we don't call set_context # and can read logs for different TIs in each request + messages = [] log = "" worker_log_relative_path = self._render_filename(ti, try_number) remote_loc = os.path.join(self.remote_base, worker_log_relative_path) uris = [] - if log_type == LogType.TRIGGER: - if ti.triggerer_job: - # triggerer currently running; skip remote read and try to read from triggerer log server - return super()._read(ti, try_number, metadata, log_type=log_type) - bucket, prefix = _parse_gcs_url(remote_loc) - blobs = list(self.client.list_blobs(bucket_or_name=bucket, prefix=prefix + ".")) - if blobs: - uris = [f"gs://{bucket}/{b.name}" for b in blobs] - else: - log += f"*** No logs found for triggerer; ti=%s {ti}\n" + bucket, prefix = _parse_gcs_url(remote_loc) + blobs = list(self.client.list_blobs(bucket_or_name=bucket, prefix=prefix)) + + if blobs: + uris = [f"gs://{bucket}/{b.name}" for b in blobs] + messages.extend(["Found remote logs:", *[f" * {x}" for x in uris]]) else: - uris = [remote_loc] + messages.append(f"No logs found in GCS; ti=%s {ti}") try: for key in uris: blob = storage.Blob.from_string(key, self.client) remote_log = blob.download_as_bytes().decode() - log += f"*** Reading remote log from {remote_loc}.\n{remote_log}\n" - - return log, {"end_of_log": True} + if remote_log: + log += remote_log except Exception as e: - log = f"*** Unable to read remote log {e}\n\n" - kwargs = {} - if log_type: - kwargs.update({"log_type": log_type}) - local_log, metadata = super()._read(ti, try_number, metadata, **kwargs) - log += local_log - return log, metadata + messages.append(f"Unable to read remote log {e}") + return messages, log + + def _read(self, ti, try_number, metadata=None): + """ + Read logs of given task instance and try_number from GCS. + If failed, read the log from task instance host machine. + + :param ti: task instance object + :param try_number: task instance try_number to read logs from + :param metadata: log metadata, + can be used for steaming log reading and auto-tailing. + """ + if hasattr(super(), "_read_remote_logs"): + # from Airflow 2.6, we don't implement the `_read` method. + # if parent has _read_remote_logs, we're >= 2.6 + return super()._read(ti, try_number, metadata) + + messages, logs = self._read_remote_logs(ti, try_number, metadata) + if not logs: + return super()._read(ti, try_number, metadata) + + return "".join([f"*** {x}\n" for x in logs]), {"end_of_log": True} def gcs_write(self, log, remote_log_location): """ @@ -196,7 +204,9 @@ def gcs_write(self, log, remote_log_location): if self.no_log_found(e): pass else: - log += f"*** Error checking for previous log; if exists, may be overwritten: {str(e)}\n\n" + log += self._add_message( + f"Error checking for previous log; if exists, may be overwritten: {str(e)}" + ) self.log.warning("Error checking for previous log: %s", e) try: blob = storage.Blob.from_string(remote_log_location, self.client) diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index b0f163b888329..611f8ebe2e17c 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -25,9 +25,11 @@ from contextlib import suppress from enum import Enum from pathlib import Path -from typing import TYPE_CHECKING, Any +from typing import TYPE_CHECKING, Any, Iterable from urllib.parse import urljoin +import pendulum + from airflow.compat.functools import cached_property from airflow.configuration import conf from airflow.exceptions import RemovedInAirflow3Warning @@ -36,11 +38,13 @@ from airflow.utils.log.logging_mixin import SetContextPropagate from airflow.utils.log.non_caching_file_handler import NonCachingFileHandler from airflow.utils.session import create_session -from airflow.utils.state import State +from airflow.utils.state import State, TaskInstanceState if TYPE_CHECKING: from airflow.models import TaskInstance +logger = logging.getLogger(__name__) + class LogType(str, Enum): """ @@ -86,6 +90,31 @@ class TriggerLogsPresentationMode(Enum): NOT_SUPPORTED = "not_supported" +def _parse_timestamps_in_log_file(lines: Iterable[str]): + timestamp = None + next_timestamp = None + for line in lines: + if not line: + continue + with suppress(Exception): + timestamp_str, _ = line.split(" ", 1) + next_timestamp = pendulum.parse(timestamp_str.strip("[]")) + if next_timestamp: + timestamp = next_timestamp + yield timestamp, line + + +def _interleave_logs(*logs): + records = [] + for log in logs: + records.extend(_parse_timestamps_in_log_file(log.splitlines())) + last = None + for _, v in sorted(records, key=lambda x: x[0] or pendulum.datetime(2000, 1, 1)): + if v != last: # dedupe + yield v + last = v + + class FileTaskHandler(logging.Handler): """ FileTaskHandler is a python log handler that handles and reads @@ -178,10 +207,9 @@ def wrap_for_triggerer(self): :meta private: """ - return self.trigger_logs_presentation_mode in ( - TriggerLogsPresentationMode.SPLIT, - TriggerLogsPresentationMode.INTERLEAVED, - ) + # this is just the default inference since we added _read_remote_logs when implementing + # trigger logging in all handlers + return "_read_remote_logs" in self.__dict__ def emit(self, record): if self.handler: @@ -265,9 +293,6 @@ def _read( ti: TaskInstance, try_number: int, metadata: dict[str, Any] | None = None, - *, - log_type: LogType | None = LogType.WORKER, - base_content=None, ): """ Template method that contains custom logic of reading @@ -293,102 +318,38 @@ def _read( # Task instance here might be different from task instance when # initializing the handler. Thus explicitly getting log location # is needed to get correct log path. + messages_list = [] worker_log_rel_path = self._render_filename(ti, try_number) worker_log_full_path = Path(self.local_base, worker_log_rel_path) - local_log_files = self._get_local_log_files(worker_log_path=worker_log_full_path, log_type=log_type) - log = "" - if local_log_files: - try: - for file in local_log_files: - log += f"*** Reading local file: {file}\n" - log += Path(file).read_text() - except Exception as e: - log = f"*** Failed to load local log files: {local_log_files}\n" - log += f"*** {str(e)}\n" - return log, {"end_of_log": True} - elif self._should_check_k8s(ti.queue) and log_type != LogType.TRIGGER: - try: - from airflow.kubernetes.kube_client import get_kube_client - from airflow.kubernetes.pod_generator import PodGenerator - - client = get_kube_client() - - log += f"*** Trying to get logs (last 100 lines) from worker pod {ti.hostname} ***\n\n" - selector = PodGenerator.build_selector_for_k8s_executor_pod( - dag_id=ti.dag_id, - task_id=ti.task_id, - try_number=ti.try_number, - map_index=ti.map_index, - run_id=ti.run_id, - airflow_worker=ti.queued_by_job_id, - ) - namespace = self._get_pod_namespace(ti) - pod_list = client.list_namespaced_pod( - namespace=namespace, - label_selector=selector, - ).items - if not pod_list: - raise RuntimeError("Cannot find pod for ti %s", ti) - elif len(pod_list) > 1: - raise RuntimeError("Found multiple pods for ti %s: %s", ti, pod_list) - res = client.read_namespaced_pod_log( - name=pod_list[0].metadata.name, - namespace=namespace, - container="base", - follow=False, - tail_lines=100, - _preload_content=False, - ) - - for line in res: - log += line.decode() - - except Exception as f: - log += f"*** Unable to fetch logs from worker pod {ti.hostname} ***\n{str(f)}\n\n" - return log, {"end_of_log": True} - else: - log += "*** Logs not found locally\n" - try: - url, rel_path = self._get_log_retrieval_url(ti, worker_log_rel_path, log_type=log_type) - log += f"*** Fetching from: {url}\n" - response = _fetch_logs_from_service(url, rel_path) - if response.status_code == 403: - log += ( - "*** !!!! Please make sure that all your Airflow components (e.g. " - "schedulers, webservers, workers and triggerer) have " - "the same 'secret_key' configured in 'webserver' section and " - "time is synchronized on all your machines (for example with ntpd) !!!!!\n***" - "*** See more at https://airflow.apache.org/docs/apache-airflow/" - "stable/configurations-ref.html#secret-key\n***" - ) - # Check if the resource was properly fetched - response.raise_for_status() - log += "\n" + response.text - except Exception as e: - log += f"*** Failed to fetch log file from {log_type}. {str(e)}\n" - return log, {"end_of_log": True} - - # Process tailing if log is not at it's end + local_messages, local_logs = self._read_from_local(worker_log_full_path) + messages_list.extend(local_messages) + running_logs = "" + running_messages = [] + if ti.state == TaskInstanceState.RUNNING and self._should_check_k8s(ti.queue): + running_messages, running_logs = self._read_from_k8s_worker(ti) + elif ti.state in (TaskInstanceState.RUNNING, TaskInstanceState.DEFERRED): + running_messages, running_logs = self._read_from_logs_server(ti, worker_log_rel_path) + messages_list.extend(running_messages) + remote_logs = "" + remote_messages = [] + with suppress(NotImplementedError): + remote_messages, remote_logs = self._read_remote_logs(ti, try_number, metadata) + messages_list.extend(remote_messages) + logs = "\n".join( + _interleave_logs( + local_logs, + running_logs, + remote_logs, + ) + ) + log_pos = len(logs) + messages = "".join([f"*** {x}\n" for x in messages_list]) end_of_log = ti.try_number != try_number or ti.state not in [State.RUNNING, State.DEFERRED] - log_pos = len(log) if metadata and "log_pos" in metadata: previous_chars = metadata["log_pos"] - log = log[previous_chars:] # Cut off previously passed log test as new tail - return log, {"end_of_log": end_of_log, "log_pos": log_pos} - - def _get_local_log_files(self, worker_log_path: Path, log_type: str | None = None): - """ - If trigger log type, get sublogs. Otherwise, get requested path. - - We take worker log path because the trigger logs are always named the same as - the worker logs but with an additional suffix appended. - """ - if log_type == LogType.TRIGGER: - prefix = self.add_triggerer_suffix(worker_log_path.name, job_id=None) - return list(worker_log_path.parent.rglob(prefix + "*.*")) - elif worker_log_path.exists(): - return [worker_log_path] + logs = logs[previous_chars:] # Cut off previously passed log test as new tail + return messages + logs, {"end_of_log": end_of_log, "log_pos": log_pos} @staticmethod def _get_pod_namespace(ti: TaskInstance): @@ -509,3 +470,83 @@ def _init_file(self, ti): logging.warning("OSError while change ownership of the log file") return full_path + + @staticmethod + def _read_from_local(worker_log_path: Path): + messages = [] + log = "" + for file in worker_log_path.rglob("*"): + messages.append(f"Read from local file: {file}") + log += Path(file).read_text() + return messages, log + + def _read_from_k8s_worker(self, ti: TaskInstance): + messages = [] + log = "" + try: + from airflow.kubernetes.kube_client import get_kube_client + from airflow.kubernetes.pod_generator import PodGenerator + + client = get_kube_client() + + messages.append(f"Trying to get logs (last 100 lines) from worker pod {ti.hostname}") + selector = PodGenerator.build_selector_for_k8s_executor_pod( + dag_id=ti.dag_id, + task_id=ti.task_id, + try_number=ti.try_number, + map_index=ti.map_index, + run_id=ti.run_id, + airflow_worker=ti.queued_by_job_id, + ) + namespace = self._get_pod_namespace(ti) + pod_list = client.list_namespaced_pod( + namespace=namespace, + label_selector=selector, + ).items + if not pod_list: + raise RuntimeError("Cannot find pod for ti %s", ti) + elif len(pod_list) > 1: + raise RuntimeError("Found multiple pods for ti %s: %s", ti, pod_list) + res = client.read_namespaced_pod_log( + name=pod_list[0].metadata.name, + namespace=namespace, + container="base", + follow=False, + tail_lines=100, + _preload_content=False, + ) + + for line in res: + log += line.decode() + except Exception as e: + messages.append(f"Reading from k8s pod logs failed: {str(e)}") + return messages, log + + def _read_from_logs_server(self, ti, worker_log_rel_path): + messages = [] + log = "" + try: + log_type = LogType.TRIGGER if ti.triggerer_job else LogType.WORKER + url, rel_path = self._get_log_retrieval_url(ti, worker_log_rel_path, log_type=log_type) + messages.append(f"Fetching from: {url}") + response = _fetch_logs_from_service(url, rel_path) + if response.status_code == 403: + messages.append( + "!!!! Please make sure that all your Airflow components (e.g. " + "schedulers, webservers, workers and triggerer) have " + "the same 'secret_key' configured in 'webserver' section and " + "time is synchronized on all your machines (for example with ntpd)\n" + "See more at https://airflow.apache.org/docs/apache-airflow/" + "stable/configurations-ref.html#secret-key" + ) + # Check if the resource was properly fetched + response.raise_for_status() + log += "\n" + response.text + except Exception as e: + log += str(e) + logger.exception(msg="error") + return messages, log + + def _read_remote_logs(self, ti, try_number, metadata=None): + """Implement in subclasses to read from the remote service""" + raise NotImplementedError diff --git a/tests/utils/test_log_handlers.py b/tests/utils/test_log_handlers.py index dc5ad55f2b779..af9e0d764c9ba 100644 --- a/tests/utils/test_log_handlers.py +++ b/tests/utils/test_log_handlers.py @@ -32,7 +32,11 @@ from airflow.jobs.triggerer_job import TriggererJob from airflow.models import DAG, DagRun, TaskInstance, Trigger from airflow.operators.python import PythonOperator -from airflow.utils.log.file_task_handler import FileTaskHandler +from airflow.utils.log.file_task_handler import ( + FileTaskHandler, + _interleave_logs, + _parse_timestamps_in_log_file, +) from airflow.utils.log.logging_mixin import set_context from airflow.utils.session import create_session from airflow.utils.state import State @@ -406,3 +410,168 @@ def test_log_retrieval_valid(self, create_task_instance): def test__should_check_k8s(config, queue, expected): with patch.dict("os.environ", **config): assert FileTaskHandler._should_check_k8s(queue) == expected + + +log_sample = """[2022-11-16T00:05:54.278-0800] {taskinstance.py:1257} INFO - +-------------------------------------------------------------------------------- +[2022-11-16T00:05:54.278-0800] {taskinstance.py:1258} INFO - Starting attempt 1 of 1 +[2022-11-16T00:05:54.279-0800] {taskinstance.py:1259} INFO - +-------------------------------------------------------------------------------- +[2022-11-16T00:05:54.295-0800] {taskinstance.py:1278} INFO - Executing on 2022-11-16 08:05:52.324532+00:00 +[2022-11-16T00:05:54.300-0800] {standard_task_runner.py:55} INFO - Started process 52536 to run task +[2022-11-16T00:05:54.300-0800] {standard_task_runner.py:55} INFO - Started process 52536 to run task +[2022-11-16T00:05:54.300-0800] {standard_task_runner.py:55} INFO - Started process 52536 to run task +[2022-11-16T00:05:54.306-0800] {standard_task_runner.py:82} INFO - Running: ['airflow', 'tasks', 'run', 'simple_async_timedelta', 'wait', 'manual__2022-11-16T08:05:52.324532+00:00', '--job-id', '33648', '--raw', '--subdir', '/Users/dstandish/code/airflow/airflow/example_dags/example_time_delta_sensor_async.py', '--cfg-path', '/var/folders/7_/1xx0hqcs3txd7kqt0ngfdjth0000gn/T/tmp725r305n'] +[2022-11-16T00:05:54.309-0800] {standard_task_runner.py:83} INFO - Job 33648: Subtask wait +[2022-11-16T00:05:54.457-0800] {task_command.py:376} INFO - Running on host daniels-mbp-2.lan +[2022-11-16T00:05:54.592-0800] {taskinstance.py:1485} INFO - Exporting the following env vars: +AIRFLOW_CTX_DAG_OWNER=airflow +AIRFLOW_CTX_DAG_ID=simple_async_timedelta +AIRFLOW_CTX_TASK_ID=wait +AIRFLOW_CTX_EXECUTION_DATE=2022-11-16T08:05:52.324532+00:00 +AIRFLOW_CTX_TRY_NUMBER=1 +AIRFLOW_CTX_DAG_RUN_ID=manual__2022-11-16T08:05:52.324532+00:00 +[2022-11-16T00:05:54.604-0800] {taskinstance.py:1360} INFO - Pausing task as DEFERRED. dag_id=simple_async_timedelta, task_id=wait, execution_date=20221116T080552, start_date=20221116T080554 +""" + + +def test_parse_timestamps(): + actual = [] + for timestamp, _ in _parse_timestamps_in_log_file(log_sample.splitlines()): + actual.append(timestamp) + assert actual == [ + pendulum.parse("2022-11-16T00:05:54.278000-08:00"), + pendulum.parse("2022-11-16T00:05:54.278000-08:00"), + pendulum.parse("2022-11-16T00:05:54.278000-08:00"), + pendulum.parse("2022-11-16T00:05:54.279000-08:00"), + pendulum.parse("2022-11-16T00:05:54.279000-08:00"), + pendulum.parse("2022-11-16T00:05:54.295000-08:00"), + pendulum.parse("2022-11-16T00:05:54.300000-08:00"), + pendulum.parse("2022-11-16T00:05:54.300000-08:00"), # duplicate + pendulum.parse("2022-11-16T00:05:54.300000-08:00"), # duplicate + pendulum.parse("2022-11-16T00:05:54.306000-08:00"), + pendulum.parse("2022-11-16T00:05:54.309000-08:00"), + pendulum.parse("2022-11-16T00:05:54.457000-08:00"), + pendulum.parse("2022-11-16T00:05:54.592000-08:00"), + pendulum.parse("2022-11-16T00:05:54.592000-08:00"), + pendulum.parse("2022-11-16T00:05:54.592000-08:00"), + pendulum.parse("2022-11-16T00:05:54.592000-08:00"), + pendulum.parse("2022-11-16T00:05:54.592000-08:00"), + pendulum.parse("2022-11-16T00:05:54.592000-08:00"), + pendulum.parse("2022-11-16T00:05:54.592000-08:00"), + pendulum.parse("2022-11-16T00:05:54.604000-08:00"), + ] + + +def test_interleave_interleaves(): + log_sample1 = "\n".join( + [ + "[2022-11-16T00:05:54.278-0800] {taskinstance.py:1258} INFO - Starting attempt 1 of 1", + ] + ) + log_sample2 = "\n".join( + [ + "[2022-11-16T00:05:54.295-0800] {taskinstance.py:1278} INFO - Executing on 2022-11-16 08:05:52.324532+00:00", + "[2022-11-16T00:05:54.300-0800] {standard_task_runner.py:55} INFO - Started process 52536 to run task", + "[2022-11-16T00:05:54.300-0800] {standard_task_runner.py:55} INFO - Started process 52536 to run task", + "[2022-11-16T00:05:54.300-0800] {standard_task_runner.py:55} INFO - Started process 52536 to run task", + "[2022-11-16T00:05:54.306-0800] {standard_task_runner.py:82} INFO - Running: ['airflow', 'tasks', 'run', 'simple_async_timedelta', 'wait', 'manual__2022-11-16T08:05:52.324532+00:00', '--job-id', '33648', '--raw', '--subdir', '/Users/dstandish/code/airflow/airflow/example_dags/example_time_delta_sensor_async.py', '--cfg-path', '/var/folders/7_/1xx0hqcs3txd7kqt0ngfdjth0000gn/T/tmp725r305n']", + "[2022-11-16T00:05:54.309-0800] {standard_task_runner.py:83} INFO - Job 33648: Subtask wait", + ] + ) + log_sample3 = "\n".join( + [ + "[2022-11-16T00:05:54.457-0800] {task_command.py:376} INFO - Running on host daniels-mbp-2.lan", + "[2022-11-16T00:05:54.592-0800] {taskinstance.py:1485} INFO - Exporting env vars: AIRFLOW_CTX_DAG_OWNER=airflow", + "AIRFLOW_CTX_DAG_ID=simple_async_timedelta", + "AIRFLOW_CTX_TASK_ID=wait", + "AIRFLOW_CTX_EXECUTION_DATE=2022-11-16T08:05:52.324532+00:00", + "AIRFLOW_CTX_TRY_NUMBER=1", + "AIRFLOW_CTX_DAG_RUN_ID=manual__2022-11-16T08:05:52.324532+00:00", + "[2022-11-16T00:05:54.604-0800] {taskinstance.py:1360} INFO - Pausing task as DEFERRED. dag_id=simple_async_timedelta, task_id=wait, execution_date=20221116T080552, start_date=20221116T080554", + ] + ) + expected = "\n".join( + [ + "[2022-11-16T00:05:54.278-0800] {taskinstance.py:1258} INFO - Starting attempt 1 of 1", + "[2022-11-16T00:05:54.295-0800] {taskinstance.py:1278} INFO - Executing on 2022-11-16 08:05:52.324532+00:00", + "[2022-11-16T00:05:54.300-0800] {standard_task_runner.py:55} INFO - Started process 52536 to run task", + "[2022-11-16T00:05:54.306-0800] {standard_task_runner.py:82} INFO - Running: ['airflow', 'tasks', 'run', 'simple_async_timedelta', 'wait', 'manual__2022-11-16T08:05:52.324532+00:00', '--job-id', '33648', '--raw', '--subdir', '/Users/dstandish/code/airflow/airflow/example_dags/example_time_delta_sensor_async.py', '--cfg-path', '/var/folders/7_/1xx0hqcs3txd7kqt0ngfdjth0000gn/T/tmp725r305n']", + "[2022-11-16T00:05:54.309-0800] {standard_task_runner.py:83} INFO - Job 33648: Subtask wait", + "[2022-11-16T00:05:54.457-0800] {task_command.py:376} INFO - Running on host daniels-mbp-2.lan", + "[2022-11-16T00:05:54.592-0800] {taskinstance.py:1485} INFO - Exporting env vars: AIRFLOW_CTX_DAG_OWNER=airflow", + "AIRFLOW_CTX_DAG_ID=simple_async_timedelta", + "AIRFLOW_CTX_TASK_ID=wait", + "AIRFLOW_CTX_EXECUTION_DATE=2022-11-16T08:05:52.324532+00:00", + "AIRFLOW_CTX_TRY_NUMBER=1", + "AIRFLOW_CTX_DAG_RUN_ID=manual__2022-11-16T08:05:52.324532+00:00", + "[2022-11-16T00:05:54.604-0800] {taskinstance.py:1360} INFO - Pausing task as DEFERRED. dag_id=simple_async_timedelta, task_id=wait, execution_date=20221116T080552, start_date=20221116T080554", + ] + ) + assert "\n".join(_interleave_logs(log_sample2, log_sample1, log_sample3)) == expected + + +long_sample = """ +*** yoyoyoyo +[2023-01-15T22:36:46.474-0800] {taskinstance.py:1131} INFO - Dependencies all met for dep_context=non-requeueable deps ti= +[2023-01-15T22:36:46.482-0800] {taskinstance.py:1131} INFO - Dependencies all met for dep_context=requeueable deps ti= +[2023-01-15T22:36:46.483-0800] {taskinstance.py:1332} INFO - Starting attempt 1 of 1 +[2023-01-15T22:36:46.516-0800] {taskinstance.py:1351} INFO - Executing on 2023-01-16 06:36:43.044492+00:00 +[2023-01-15T22:36:46.522-0800] {standard_task_runner.py:56} INFO - Started process 38807 to run task +[2023-01-15T22:36:46.530-0800] {standard_task_runner.py:83} INFO - Running: ['airflow', 'tasks', 'run', 'example_time_delta_sensor_async', 'wait', 'manual__2023-01-16T06:36:43.044492+00:00', '--job-id', '487', '--raw', '--subdir', '/Users/dstandish/code/airflow/airflow/example_dags/example_time_delta_sensor_async.py', '--cfg-path', '/var/folders/7_/1xx0hqcs3txd7kqt0ngfdjth0000gn/T/tmpiwyl54bn', '--no-shut-down-logging'] +[2023-01-15T22:36:46.536-0800] {standard_task_runner.py:84} INFO - Job 487: Subtask wait +[2023-01-15T22:36:46.624-0800] {task_command.py:417} INFO - Running on host daniels-mbp-2.lan +[2023-01-15T22:36:46.918-0800] {taskinstance.py:1558} INFO - Exporting env vars: AIRFLOW_CTX_DAG_OWNER='airflow' AIRFLOW_CTX_DAG_ID='example_time_delta_sensor_async' AIRFLOW_CTX_TASK_ID='wait' AIRFLOW_CTX_EXECUTION_DATE='2023-01-16T06:36:43.044492+00:00' AIRFLOW_CTX_TRY_NUMBER='1' AIRFLOW_CTX_DAG_RUN_ID='manual__2023-01-16T06:36:43.044492+00:00' +[2023-01-15T22:36:46.929-0800] {taskinstance.py:1433} INFO - Pausing task as DEFERRED. dag_id=example_time_delta_sensor_async, task_id=wait, execution_date=20230116T063643, start_date=20230116T063646 +[2023-01-15T22:36:46.981-0800] {local_task_job.py:218} INFO - Task exited with return code 100 (task deferral) + +[2023-01-15T22:36:46.474-0800] {taskinstance.py:1131} INFO - Dependencies all met for dep_context=non-requeueable deps ti= +[2023-01-15T22:36:46.482-0800] {taskinstance.py:1131} INFO - Dependencies all met for dep_context=requeueable deps ti= +[2023-01-15T22:36:46.483-0800] {taskinstance.py:1332} INFO - Starting attempt 1 of 1 +[2023-01-15T22:36:46.516-0800] {taskinstance.py:1351} INFO - Executing on 2023-01-16 06:36:43.044492+00:00 +[2023-01-15T22:36:46.522-0800] {standard_task_runner.py:56} INFO - Started process 38807 to run task +[2023-01-15T22:36:46.530-0800] {standard_task_runner.py:83} INFO - Running: ['airflow', 'tasks', 'run', 'example_time_delta_sensor_async', 'wait', 'manual__2023-01-16T06:36:43.044492+00:00', '--job-id', '487', '--raw', '--subdir', '/Users/dstandish/code/airflow/airflow/example_dags/example_time_delta_sensor_async.py', '--cfg-path', '/var/folders/7_/1xx0hqcs3txd7kqt0ngfdjth0000gn/T/tmpiwyl54bn', '--no-shut-down-logging'] +[2023-01-15T22:36:46.536-0800] {standard_task_runner.py:84} INFO - Job 487: Subtask wait +[2023-01-15T22:36:46.624-0800] {task_command.py:417} INFO - Running on host daniels-mbp-2.lan +[2023-01-15T22:36:46.918-0800] {taskinstance.py:1558} INFO - Exporting env vars: AIRFLOW_CTX_DAG_OWNER='airflow' AIRFLOW_CTX_DAG_ID='example_time_delta_sensor_async' AIRFLOW_CTX_TASK_ID='wait' AIRFLOW_CTX_EXECUTION_DATE='2023-01-16T06:36:43.044492+00:00' AIRFLOW_CTX_TRY_NUMBER='1' AIRFLOW_CTX_DAG_RUN_ID='manual__2023-01-16T06:36:43.044492+00:00' +[2023-01-15T22:36:46.929-0800] {taskinstance.py:1433} INFO - Pausing task as DEFERRED. dag_id=example_time_delta_sensor_async, task_id=wait, execution_date=20230116T063643, start_date=20230116T063646 +[2023-01-15T22:36:46.981-0800] {local_task_job.py:218} INFO - Task exited with return code 100 (task deferral) +[2023-01-15T22:37:17.673-0800] {taskinstance.py:1131} INFO - Dependencies all met for dep_context=non-requeueable deps ti= +[2023-01-15T22:37:17.681-0800] {taskinstance.py:1131} INFO - Dependencies all met for dep_context=requeueable deps ti= +[2023-01-15T22:37:17.682-0800] {taskinstance.py:1330} INFO - resuming after deferral +[2023-01-15T22:37:17.693-0800] {taskinstance.py:1351} INFO - Executing on 2023-01-16 06:36:43.044492+00:00 +[2023-01-15T22:37:17.697-0800] {standard_task_runner.py:56} INFO - Started process 39090 to run task +[2023-01-15T22:37:17.703-0800] {standard_task_runner.py:83} INFO - Running: ['airflow', 'tasks', 'run', 'example_time_delta_sensor_async', 'wait', 'manual__2023-01-16T06:36:43.044492+00:00', '--job-id', '488', '--raw', '--subdir', '/Users/dstandish/code/airflow/airflow/example_dags/example_time_delta_sensor_async.py', '--cfg-path', '/var/folders/7_/1xx0hqcs3txd7kqt0ngfdjth0000gn/T/tmp_sa9sau4', '--no-shut-down-logging'] +[2023-01-15T22:37:17.707-0800] {standard_task_runner.py:84} INFO - Job 488: Subtask wait +[2023-01-15T22:37:17.771-0800] {task_command.py:417} INFO - Running on host daniels-mbp-2.lan +[2023-01-15T22:37:18.043-0800] {taskinstance.py:1369} INFO - Marking task as SUCCESS. dag_id=example_time_delta_sensor_async, task_id=wait, execution_date=20230116T063643, start_date=20230116T063646, end_date=20230116T063718 +[2023-01-15T22:37:18.117-0800] {local_task_job.py:220} INFO - Task exited with return code 0 +[2023-01-15T22:37:18.147-0800] {taskinstance.py:2648} INFO - 0 downstream tasks scheduled from follow-on schedule check +[2023-01-15T22:37:18.173-0800] {:0} Level None - end_of_log + +*** hihihi! +[2023-01-15T22:36:48.348-0800] {temporal.py:62} INFO - trigger starting +[2023-01-15T22:36:48.348-0800] {temporal.py:66} INFO - 24 seconds remaining; sleeping 10 seconds +[2023-01-15T22:36:58.349-0800] {temporal.py:71} INFO - sleeping 1 second... +[2023-01-15T22:36:59.349-0800] {temporal.py:71} INFO - sleeping 1 second... +[2023-01-15T22:37:00.349-0800] {temporal.py:71} INFO - sleeping 1 second... +[2023-01-15T22:37:01.350-0800] {temporal.py:71} INFO - sleeping 1 second... +[2023-01-15T22:37:02.350-0800] {temporal.py:71} INFO - sleeping 1 second... +[2023-01-15T22:37:03.351-0800] {temporal.py:71} INFO - sleeping 1 second... +[2023-01-15T22:37:04.351-0800] {temporal.py:71} INFO - sleeping 1 second... +[2023-01-15T22:37:05.353-0800] {temporal.py:71} INFO - sleeping 1 second... +[2023-01-15T22:37:06.354-0800] {temporal.py:71} INFO - sleeping 1 second... +[2023-01-15T22:37:07.355-0800] {temporal.py:71} INFO - sleeping 1 second... +[2023-01-15T22:37:08.356-0800] {temporal.py:71} INFO - sleeping 1 second... +[2023-01-15T22:37:09.357-0800] {temporal.py:71} INFO - sleeping 1 second... +[2023-01-15T22:37:10.358-0800] {temporal.py:71} INFO - sleeping 1 second... +[2023-01-15T22:37:11.359-0800] {temporal.py:71} INFO - sleeping 1 second... +[2023-01-15T22:37:12.359-0800] {temporal.py:71} INFO - sleeping 1 second... +[2023-01-15T22:37:13.360-0800] {temporal.py:74} INFO - yielding event with payload DateTime(2023, 1, 16, 6, 37, 13, 44492, tzinfo=Timezone('UTC')) +[2023-01-15T22:37:13.361-0800] {triggerer_job.py:540} INFO - Trigger (ID 106) fired: TriggerEvent +""" + +for line in _parse_timestamps_in_log_file(long_sample.splitlines()): + print(line) +for line in _interleave_logs(long_sample): + print(line) From 4fd4f6dbdec6dcb2ac27be8979e4ee3c9da40307 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 16 Jan 2023 12:23:37 -0800 Subject: [PATCH 21/96] revert changes to UI --- .../api_connexion/endpoints/log_endpoint.py | 13 +-- airflow/api_connexion/openapi/v1.yaml | 6 -- .../amazon/aws/log/cloudwatch_task_handler.py | 8 +- .../microsoft/azure/log/wasb_task_handler.py | 14 +-- airflow/utils/log/log_reader.py | 24 +---- airflow/utils/log/logging_mixin.py | 1 + airflow/www/static/js/api/useGridData.ts | 8 -- airflow/www/static/js/api/useTaskLog.ts | 6 +- .../dag/details/taskInstance/Logs/index.tsx | 3 - .../js/dag/details/taskInstance/Logs/utils.ts | 9 +- .../js/dag/details/taskInstance/index.tsx | 96 +++---------------- airflow/www/static/js/dag/grid/index.test.tsx | 4 +- airflow/www/static/js/types/api-generated.ts | 2 - airflow/www/static/js/types/index.ts | 1 - airflow/www/views.py | 5 - tests/www/views/test_views_grid.py | 5 - 16 files changed, 34 insertions(+), 171 deletions(-) diff --git a/airflow/api_connexion/endpoints/log_endpoint.py b/airflow/api_connexion/endpoints/log_endpoint.py index ba035f5676bf6..8e3555d409014 100644 --- a/airflow/api_connexion/endpoints/log_endpoint.py +++ b/airflow/api_connexion/endpoints/log_endpoint.py @@ -32,7 +32,6 @@ from airflow.models import TaskInstance from airflow.security import permissions from airflow.utils.airflow_flask_app import get_airflow_app -from airflow.utils.log.file_task_handler import LogType, TriggerLogsPresentationMode from airflow.utils.log.log_reader import TaskLogReader from airflow.utils.session import NEW_SESSION, provide_session @@ -54,7 +53,6 @@ def get_log( full_content: bool = False, map_index: int = -1, token: str | None = None, - log_type: LogType | None = None, session: Session = NEW_SESSION, ) -> APIResponse: """Get logs for specific task instance.""" @@ -76,13 +74,6 @@ def get_log( metadata["download_logs"] = False task_log_reader = TaskLogReader() - if log_type == LogType.TRIGGER: - not_supported = TriggerLogsPresentationMode.NOT_SUPPORTED - interleaved = TriggerLogsPresentationMode.INTERLEAVED - if task_log_reader.trigger_logs_presentation_mode == not_supported: - raise BadRequest("Task log handler does not support trigger logging.") - if task_log_reader.trigger_logs_presentation_mode == interleaved: - raise BadRequest("Trigger logs requested but handler does not split trigger logs.") if not task_log_reader.supports_read: raise BadRequest("Task log handler does not support read logs.") @@ -115,12 +106,12 @@ def get_log( # return_type would be either the above two or None logs: Any if return_type == "application/json" or return_type is None: # default - logs, metadata = task_log_reader.read_log_chunks(ti, task_try_number, metadata, log_type=log_type) + logs, metadata = task_log_reader.read_log_chunks(ti, task_try_number, metadata) logs = logs[0] if task_try_number is not None else logs # we must have token here, so we can safely ignore it token = URLSafeSerializer(key).dumps(metadata) # type: ignore[assignment] return logs_schema.dump(LogResponseObject(continuation_token=token, content=logs)) # text/plain. Stream - logs = task_log_reader.read_log_stream(ti, task_try_number, metadata, log_type=log_type) + logs = task_log_reader.read_log_stream(ti, task_try_number, metadata) return Response(logs, headers={"Content-Type": return_type}) diff --git a/airflow/api_connexion/openapi/v1.yaml b/airflow/api_connexion/openapi/v1.yaml index 1f62eb0d6241a..f2cc9b3af20d8 100644 --- a/airflow/api_connexion/openapi/v1.yaml +++ b/airflow/api_connexion/openapi/v1.yaml @@ -1656,12 +1656,6 @@ paths: - $ref: '#/components/parameters/FullContent' - $ref: '#/components/parameters/FilterMapIndex' - $ref: '#/components/parameters/ContinuationToken' - - name: log_type - in: query - schema: - type: string - required: false - get: summary: Get logs diff --git a/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py b/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py index a9ab9feb29c42..e50a6d4d74e2e 100644 --- a/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py +++ b/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py @@ -24,7 +24,7 @@ from airflow.compat.functools import cached_property from airflow.configuration import conf from airflow.providers.amazon.aws.hooks.logs import AwsLogsHook -from airflow.utils.log.file_task_handler import FileTaskHandler, TriggerLogsPresentationMode +from airflow.utils.log.file_task_handler import FileTaskHandler from airflow.utils.log.logging_mixin import LoggingMixin @@ -68,10 +68,6 @@ def set_context(self, ti): boto3_client=self.hook.get_conn(), ) - @cached_property - def trigger_logs_presentation_mode(self): - return TriggerLogsPresentationMode.INTERLEAVED - def close(self): """Close the handler responsible for the upload of the local log file to Cloudwatch.""" # When application exit, system shuts down all handlers by @@ -86,7 +82,7 @@ def close(self): # Mark closed so we don't double write if close is called twice self.closed = True - def _read(self, task_instance, try_number, metadata=None, log_type=None): + def _read(self, task_instance, try_number, metadata=None): stream_name = self._render_filename(task_instance, try_number) try: return ( diff --git a/airflow/providers/microsoft/azure/log/wasb_task_handler.py b/airflow/providers/microsoft/azure/log/wasb_task_handler.py index b4c7323c0f459..8c0fe220830ef 100644 --- a/airflow/providers/microsoft/azure/log/wasb_task_handler.py +++ b/airflow/providers/microsoft/azure/log/wasb_task_handler.py @@ -23,7 +23,7 @@ from airflow.compat.functools import cached_property from airflow.configuration import conf -from airflow.utils.log.file_task_handler import FileTaskHandler, TriggerLogsPresentationMode +from airflow.utils.log.file_task_handler import FileTaskHandler from airflow.utils.log.logging_mixin import LoggingMixin @@ -70,11 +70,6 @@ def hook(self): ) return None - @cached_property - def trigger_logs_presentation_mode(self): - """Not implemented yet.""" - return TriggerLogsPresentationMode.NOT_SUPPORTED - def set_context(self, ti) -> None: super().set_context(ti) # Local location and remote location is needed to open and @@ -110,7 +105,7 @@ def close(self) -> None: self.closed = True def _read( - self, ti, try_number: int, metadata: dict[str, Any] | None = None, *, log_type=None + self, ti, try_number: int, metadata: dict[str, Any] | None = None ) -> tuple[str, dict[str, bool]]: """ Read logs of given task instance and try_number from Wasb remote storage. @@ -118,9 +113,8 @@ def _read( :param ti: task instance object :param try_number: task instance try_number to read logs from - :param metadata: log metadata, can be used for steaming log reading and auto-tailing. - :param log_type: Currently unused. In future will control whether to retrieve - logs from triggerer or worker. + :param metadata: log metadata, + can be used for steaming log reading and auto-tailing. """ # Explicitly getting log relative path is necessary as the given # task instance might be different than task instance passed in diff --git a/airflow/utils/log/log_reader.py b/airflow/utils/log/log_reader.py index eb533505b26da..5cc8b9377e186 100644 --- a/airflow/utils/log/log_reader.py +++ b/airflow/utils/log/log_reader.py @@ -26,7 +26,6 @@ from airflow.configuration import conf from airflow.models.taskinstance import TaskInstance from airflow.utils.helpers import render_log_filename -from airflow.utils.log.file_task_handler import TriggerLogsPresentationMode from airflow.utils.log.logging_mixin import ExternalLoggingMixin from airflow.utils.session import NEW_SESSION, provide_session from airflow.utils.state import State @@ -39,7 +38,7 @@ class TaskLogReader: """Time to sleep between loops while waiting for more logs""" def read_log_chunks( - self, ti: TaskInstance, try_number: int | None, metadata, *, log_type=None + self, ti: TaskInstance, try_number: int | None, metadata ) -> tuple[list[tuple[tuple[str, str]]], dict[str, str]]: """ Reads chunks of Task Instance logs. @@ -48,7 +47,6 @@ def read_log_chunks( :param try_number: If provided, logs for the given try will be returned. Otherwise, logs from all attempts are returned. :param metadata: A dictionary containing information about how to read the task log - :param log_type: Either trigger or worker, to determine where to read logs from. The following is an example of how to use this method to read log: @@ -61,15 +59,11 @@ def read_log_chunks( contain information about the task log which can enable you read logs to the end. """ - split_mode = self.trigger_logs_presentation_mode == TriggerLogsPresentationMode.SPLIT - kwargs = {"log_type": log_type} if split_mode else {} - logs, metadatas = self.log_handler.read(ti, try_number, metadata=metadata, **kwargs) + logs, metadatas = self.log_handler.read(ti, try_number, metadata=metadata) metadata = metadatas[0] return logs, metadata - def read_log_stream( - self, ti: TaskInstance, try_number: int | None, metadata: dict, *, log_type=None - ) -> Iterator[str]: + def read_log_stream(self, ti: TaskInstance, try_number: int | None, metadata: dict) -> Iterator[str]: """ Used to continuously read log to the end @@ -88,9 +82,7 @@ def read_log_stream( metadata.pop("offset", None) metadata.pop("log_pos", None) while True: - split_mode = self.trigger_logs_presentation_mode == TriggerLogsPresentationMode.SPLIT - kwargs = {"log_type": log_type} if split_mode else {} - logs, metadata = self.read_log_chunks(ti, current_try_number, metadata, **kwargs) + logs, metadata = self.read_log_chunks(ti, current_try_number, metadata) for host, log in logs[0]: yield "\n".join([host or "", log]) + "\n" if "end_of_log" not in metadata or ( @@ -116,14 +108,6 @@ def supports_read(self): """Checks if a read operation is supported by a current log handler.""" return hasattr(self.log_handler, "read") - @property - def trigger_logs_presentation_mode(self): - return getattr( - self.log_handler, - "trigger_logs_presentation_mode", - TriggerLogsPresentationMode.NOT_SUPPORTED, - ) - @property def supports_external_link(self) -> bool: """Check if the logging handler supports external links (e.g. to Elasticsearch, Stackdriver, etc).""" diff --git a/airflow/utils/log/logging_mixin.py b/airflow/utils/log/logging_mixin.py index ada57e3971833..85ff71a94f1bf 100644 --- a/airflow/utils/log/logging_mixin.py +++ b/airflow/utils/log/logging_mixin.py @@ -31,6 +31,7 @@ # 7-bit C1 ANSI escape sequences ANSI_ESCAPE = re.compile(r"\x1B[@-_][0-?]*[ -/]*[@-~]") + # Private: A sentinel objects class SetContextPropagate(enum.Enum): """:meta private:""" diff --git a/airflow/www/static/js/api/useGridData.ts b/airflow/www/static/js/api/useGridData.ts index 321d23a54ec82..091127fee2401 100644 --- a/airflow/www/static/js/api/useGridData.ts +++ b/airflow/www/static/js/api/useGridData.ts @@ -36,17 +36,10 @@ const dagId = getMetaValue(DAG_ID_PARAM); const gridDataUrl = getMetaValue('grid_data_url'); const urlRoot = getMetaValue('root'); -export enum TriggerLogsPresentationMode { - SPLIT = 'split', - NOT_SUPPORTED = 'not_supported', - INTERLEAVED = 'interleaved', -} - export interface GridData { dagRuns: DagRun[]; groups: Task; ordering: RunOrdering; - triggerLogsPresentationMode: TriggerLogsPresentationMode; } export const emptyGridData: GridData = { @@ -57,7 +50,6 @@ export const emptyGridData: GridData = { instances: [], }, ordering: [], - triggerLogsPresentationMode: TriggerLogsPresentationMode.NOT_SUPPORTED, }; const formatOrdering = (data: GridData) => ({ diff --git a/airflow/www/static/js/api/useTaskLog.ts b/airflow/www/static/js/api/useTaskLog.ts index 25366015b8419..bbb6395878b4f 100644 --- a/airflow/www/static/js/api/useTaskLog.ts +++ b/airflow/www/static/js/api/useTaskLog.ts @@ -32,7 +32,7 @@ interface Props extends API.GetLogVariables { } const useTaskLog = ({ - dagId, dagRunId, taskId, taskTryNumber, mapIndex, fullContent, state, logType, + dagId, dagRunId, taskId, taskTryNumber, mapIndex, fullContent, state, }: Props) => { let url: string = ''; const [isPreviousStatePending, setPrevState] = useState(true); @@ -55,14 +55,14 @@ const useTaskLog = ({ const expectingLogs = isStatePending || isPreviousStatePending; return useQuery( - ['taskLogs', dagId, dagRunId, taskId, mapIndex, taskTryNumber, fullContent, logType], + ['taskLogs', dagId, dagRunId, taskId, mapIndex, taskTryNumber, fullContent], () => { setPrevState(isStatePending); return axios.get( url, { headers: { Accept: 'text/plain' }, - params: { map_index: mapIndex, full_content: fullContent, log_type: logType }, + params: { map_index: mapIndex, full_content: fullContent }, }, ); }, diff --git a/airflow/www/static/js/dag/details/taskInstance/Logs/index.tsx b/airflow/www/static/js/dag/details/taskInstance/Logs/index.tsx index 6c4323fb532b3..0139d8e22392a 100644 --- a/airflow/www/static/js/dag/details/taskInstance/Logs/index.tsx +++ b/airflow/www/static/js/dag/details/taskInstance/Logs/index.tsx @@ -89,7 +89,6 @@ interface Props { executionDate: DagRun['executionDate']; tryNumber: TaskInstance['tryNumber']; state?: TaskInstance['state']; - logType?: string; } const Logs = ({ @@ -100,7 +99,6 @@ const Logs = ({ executionDate, tryNumber, state, - logType, }: Props) => { const [internalIndexes, externalIndexes] = getLinkIndexes(tryNumber); const [selectedTryNumber, setSelectedTryNumber] = useState(); @@ -119,7 +117,6 @@ const Logs = ({ taskTryNumber, fullContent: shouldRequestFullContent, state, - logType, }); const params = new URLSearchParamsWrapper({ diff --git a/airflow/www/static/js/dag/details/taskInstance/Logs/utils.ts b/airflow/www/static/js/dag/details/taskInstance/Logs/utils.ts index 1ca51cd3efe64..4a5b8b8664b70 100644 --- a/airflow/www/static/js/dag/details/taskInstance/Logs/utils.ts +++ b/airflow/www/static/js/dag/details/taskInstance/Logs/utils.ts @@ -46,13 +46,8 @@ export const parseLogs = ( if (!data) { return {}; } - let lines; - try { - lines = data.split('\n'); - } catch (err) { - // debugger - return {}; - } + + const lines = data.split('\n'); const parsedLines: Array = []; const fileSources: Set = new Set(); diff --git a/airflow/www/static/js/dag/details/taskInstance/index.tsx b/airflow/www/static/js/dag/details/taskInstance/index.tsx index 8bf2c527e643e..894b22a5c8122 100644 --- a/airflow/www/static/js/dag/details/taskInstance/index.tsx +++ b/airflow/www/static/js/dag/details/taskInstance/index.tsx @@ -19,7 +19,7 @@ /* global localStorage */ -import React, { useEffect, useRef, useState } from 'react'; +import React, { useRef, useState } from 'react'; import { Box, Text, @@ -28,7 +28,6 @@ import { Tab, TabPanels, TabPanel, - Tooltip, } from '@chakra-ui/react'; import { useGridData, useTaskInstance } from 'src/api'; @@ -37,10 +36,7 @@ import useOffsetHeight from 'src/utils/useOffsetHeight'; import type { DagRun, TaskInstance as TaskInstanceType } from 'src/types'; import type { SelectionProps } from 'src/dag/useSelection'; import NotesAccordion from 'src/dag/details/NotesAccordion'; -import { useContainerRef } from 'src/context/containerRef'; -import TabWithTooltip from 'src/components/TabWithTooltip'; -import { TriggerLogsPresentationMode } from 'src/api/useGridData'; import ExtraLinks from './ExtraLinks'; import Logs from './Logs'; import TaskNav from './Nav'; @@ -65,8 +61,7 @@ const TaskInstance = ({ }: Props) => { const isMapIndexDefined = !(mapIndex === undefined); const actionsMapIndexes = isMapIndexDefined ? [mapIndex] : []; - const { data: { dagRuns, groups, triggerLogsPresentationMode } } = useGridData(); - const containerRef = useContainerRef(); + const { data: { dagRuns, groups } } = useGridData(); const detailsRef = useRef(null); const offsetHeight = useOffsetHeight(detailsRef); @@ -96,35 +91,6 @@ const TaskInstance = ({ localStorage.setItem(detailsPanelActiveTabIndex, index.toString()); setPreferedTabIndex(index); }; - // eslint-disable-next-line max-len - const triggerLogsNotSupported = triggerLogsPresentationMode === TriggerLogsPresentationMode.NOT_SUPPORTED; - const triggerLogsSplit = triggerLogsPresentationMode === TriggerLogsPresentationMode.SPLIT; - // eslint-disable-next-line max-len - const triggerLogsInterleaved = triggerLogsPresentationMode === TriggerLogsPresentationMode.INTERLEAVED; - let triggerLogsTooltipProps; - if (triggerLogsNotSupported) { - triggerLogsTooltipProps = { - label: 'The configured log handler does not support reading trigger logs.', - isDisabled: false, - }; - } else if (triggerLogsSplit && !instance?.hasDeferred) { - triggerLogsTooltipProps = { - label: 'This task has no deferrals.', - isDisabled: false, - }; - } else { - triggerLogsTooltipProps = { - label: '', - isDisabled: true, - }; - } - const skipTriggerLogsRender = !triggerLogsSplit || !instance?.hasDeferred; - useEffect(() => { - // Reset preferred tab if it is disabled - if (skipTriggerLogsRender && preferedTabIndex === 2) { - setPreferedTabIndex(1); - } - }, [skipTriggerLogsRender, preferedTabIndex]); if (!group || !run || !instance) return null; @@ -137,9 +103,6 @@ const TaskInstance = ({ case 1: isPreferedTabDisplayed = !isGroup || (isGroup && !!isMapped); break; - case 2: // triggerer logs - isPreferedTabDisplayed = !isGroup || (isGroup && !!isMapped); - break; default: isPreferedTabDisplayed = false; } @@ -183,21 +146,9 @@ const TaskInstance = ({ )} {!isGroupOrMappedTaskSummary && ( - Logs + Logs )} - {!isGroupOrMappedTaskSummary && !triggerLogsInterleaved && ( - - - Triggerer logs - - - )} )} - {/* Triggerer Logs Tab */} - {!isGroupOrMappedTaskSummary && ( - - {!skipTriggerLogsRender && ( - - )} - - )} - {/* Mapped Task Instances Tab */} - {isMappedTaskSummary && !isGroup && ( - - onSelect({ runId, taskId, mapIndex: row.values.mapIndex })} - /> - - )} - + { + isMappedTaskSummary && !isGroup && ( + + onSelect({ runId, taskId, mapIndex: row.values.mapIndex })} + /> + + ) + } diff --git a/airflow/www/static/js/dag/grid/index.test.tsx b/airflow/www/static/js/dag/grid/index.test.tsx index 984a88a8a1c8a..4475a75a345a7 100644 --- a/airflow/www/static/js/dag/grid/index.test.tsx +++ b/airflow/www/static/js/dag/grid/index.test.tsx @@ -21,11 +21,10 @@ /* global describe, test, expect, beforeEach, beforeAll, jest, window */ import React from 'react'; -import { fireEvent, render, waitFor } from '@testing-library/react'; +import { render, fireEvent, waitFor } from '@testing-library/react'; import { Wrapper } from 'src/utils/testUtils'; import * as useGridDataModule from 'src/api/useGridData'; -import { TriggerLogsPresentationMode } from 'src/api/useGridData'; import Grid from '.'; @@ -108,7 +107,6 @@ const mockGridData = { }, ], ordering: ['dataIntervalStart'], - triggerLogsPresentationMode: TriggerLogsPresentationMode.NOT_SUPPORTED, } as useGridDataModule.GridData; const EXPAND = 'Expand all task groups'; diff --git a/airflow/www/static/js/types/api-generated.ts b/airflow/www/static/js/types/api-generated.ts index e79dd651ca34f..8841cd7225ddd 100644 --- a/airflow/www/static/js/types/api-generated.ts +++ b/airflow/www/static/js/types/api-generated.ts @@ -493,7 +493,6 @@ export interface paths { * If passed, it will specify the location from which the download should be continued. */ token?: components["parameters"]["ContinuationToken"]; - log_type?: string; }; }; }; @@ -3792,7 +3791,6 @@ export interface operations { * If passed, it will specify the location from which the download should be continued. */ token?: components["parameters"]["ContinuationToken"]; - log_type?: string; }; }; responses: { diff --git a/airflow/www/static/js/types/index.ts b/airflow/www/static/js/types/index.ts index 4a119bc23d07b..524e09a92a3a1 100644 --- a/airflow/www/static/js/types/index.ts +++ b/airflow/www/static/js/types/index.ts @@ -72,7 +72,6 @@ interface TaskInstance { tryNumber?: number; triggererJob?: Job; trigger?: Trigger; - hasDeferred?: boolean | null; // comes from grid_data not API note: string | null; } diff --git a/airflow/www/views.py b/airflow/www/views.py index 45313a96803a5..8618a61fa66ae 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -145,8 +145,6 @@ "} else {xLabel = d3.time.format('%H:%M, %d %b')(new Date(parseInt(d)));} return xLabel;}" ) -task_log_reader = TaskLogReader() - def truncate_task_duration(task_duration): """ @@ -266,7 +264,6 @@ def dag_to_grid(dag: DagModel, dag_runs: Sequence[DagRun], session: Session): TaskInstance.task_id, TaskInstance.run_id, TaskInstance.state, - func.max(case((TaskInstance.next_kwargs.is_not(None), 1), else_=0)).label("has_deferred"), TaskInstance._try_number, func.min(TaskInstanceNote.content).label("note"), func.count(func.coalesce(TaskInstance.state, sqla.literal("no_status"))).label("state_count"), @@ -296,7 +293,6 @@ def _get_summary(task_instance): "end_date": task_instance.end_date, "try_number": wwwutils.get_try_count(task_instance._try_number, task_instance.state), "note": task_instance.note, - "has_deferred": bool(task_instance.has_deferred), } def _mapped_summary(ti_summaries): @@ -3621,7 +3617,6 @@ def grid_data(self): "groups": dag_to_grid(dag, dag_runs, session), "dag_runs": encoded_runs, "ordering": dag.timetable.run_ordering, - "trigger_logs_presentation_mode": task_log_reader.trigger_logs_presentation_mode.value, } # avoid spaces to reduce payload size return ( diff --git a/tests/www/views/test_views_grid.py b/tests/www/views/test_views_grid.py index 2264b86bc1788..79e9401f684f0 100644 --- a/tests/www/views/test_views_grid.py +++ b/tests/www/views/test_views_grid.py @@ -98,7 +98,6 @@ def test_no_runs(admin_client, dag_without_runs): assert resp.status_code == 200, resp.json assert resp.json == { "dag_runs": [], - "trigger_logs_presentation_mode": "not_supported", "groups": { "children": [ { @@ -227,7 +226,6 @@ def test_one_run(admin_client, dag_with_runs: list[DagRun], session): "state": "running", }, ], - "trigger_logs_presentation_mode": "not_supported", "groups": { "children": [ { @@ -243,7 +241,6 @@ def test_one_run(admin_client, dag_with_runs: list[DagRun], session): "state": "success", "task_id": "task1", "try_number": 0, - "has_deferred": False, }, { "run_id": "run_2", @@ -253,7 +250,6 @@ def test_one_run(admin_client, dag_with_runs: list[DagRun], session): "state": "success", "task_id": "task1", "try_number": 0, - "has_deferred": False, }, ], "is_mapped": False, @@ -407,7 +403,6 @@ def _expected_task_details(task_id, has_outlet_datasets): assert resp.status_code == 200, resp.json assert resp.json == { "dag_runs": [], - "trigger_logs_presentation_mode": "not_supported", "groups": { "children": [ _expected_task_details("task1", False), From 32076b513c676b686dbeec91fa70475b94fc2dc2 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 16 Jan 2023 12:51:12 -0800 Subject: [PATCH 22/96] s3 ... untested --- .../amazon/aws/log/s3_task_handler.py | 70 ++++++++----------- airflow/utils/log/file_task_handler.py | 2 +- 2 files changed, 31 insertions(+), 41 deletions(-) diff --git a/airflow/providers/amazon/aws/log/s3_task_handler.py b/airflow/providers/amazon/aws/log/s3_task_handler.py index 9eb6be85c8d14..a5086718b6413 100644 --- a/airflow/providers/amazon/aws/log/s3_task_handler.py +++ b/airflow/providers/amazon/aws/log/s3_task_handler.py @@ -99,57 +99,47 @@ def close(self): # Mark closed so we don't double write if close is called twice self.closed = True - def _read(self, ti, try_number, metadata=None, *, log_type=None): + def _read_remote_logs(self, ti, try_number, metadata=None): + # Explicitly getting log relative path is necessary as the given + # task instance might be different than task instance passed in + # in set_context method. + log = "" + messages = [] + worker_log_rel_path = self._render_filename(ti, try_number) + bucket, prefix = self.hook.parse_s3_url(s3url=os.path.join(self.remote_base, worker_log_rel_path)) + keys = self.hook.list_keys(bucket_name=bucket, prefix=prefix) + if keys: + keys = [f"s3://{bucket}/{key}" for key in keys] + messages.extend(["Reading logs from s3:", *[f"\n * {x}" for x in keys]]) + for key in keys: + log += self.s3_read(key, return_error=True) + else: + messages.append(f"No logs found on s3 for ti={ti}") + return messages, log + + def _read(self, ti, try_number, metadata=None): """ Read logs of given task instance and try_number from S3 remote storage. If failed, read the log from task instance host machine. + todo: when min airflow version >= 2.6 then remove this method (``_read``) + :param ti: task instance object :param try_number: task instance try_number to read logs from :param metadata: log metadata, can be used for steaming log reading and auto-tailing. """ - # Explicitly getting log relative path is necessary as the given - # task instance might be different than task instance passed in - # in set_context method. - log = "" - keys = [] - worker_log_rel_path = self._render_filename(ti, try_number) - if log_type == LogType.TRIGGER: - if ti.triggerer_job: - # triggerer currently running; skip s3 read and try to read from triggerer log server - return super()._read(ti, try_number, metadata, log_type=log_type) - trigger_log_rel_prefix = self.add_triggerer_suffix(worker_log_rel_path) - bucket, prefix = self.hook.parse_s3_url( - s3url=os.path.join(self.remote_base, trigger_log_rel_prefix) - ) - keys = self.hook.list_keys(bucket_name=bucket, prefix=prefix) - if keys: - keys = [f"s3://{bucket}/{key}" for key in keys] - else: - log += f"*** No logs found for triggerer; ti=%s {ti}\n" - else: - remote_loc = os.path.join(self.remote_base, worker_log_rel_path) - try: - if self.s3_log_exists(remote_loc): - keys = [remote_loc] - except Exception as error: - self.log.exception("Failed to verify remote log exists %s.", remote_loc) - log += f"*** Failed to verify remote log exists {remote_loc}.\n{error}\n" - - if keys: - # If S3 remote file exists, we do not fetch logs from task instance - # local machine even if there are errors reading remote logs, as - # returned remote_log will contain error messages. - for key in keys: - remote_log = self.s3_read(key, return_error=True) - log += f"*** Reading remote log from {key}.\n{remote_log}\n" - return log, {"end_of_log": True} + # from airflow 2.6 we no longer implement the _read method + if hasattr(super(), "_read_remote_logs"): + return super()._read(ti, try_number, metadata) + + # if we get here, we're on airflow < 2.6 and we use this backcompat logic + messages, log = self._read_remote_logs(ti, try_number, metadata) + if log: + return {"log": log + "".join(f"*** {x}\n" for x in messages), "end_of_log": True} else: log += "*** Falling back to local log\n" - # todo: after min airflow version >= 2.6, just forward log_type to super()._read - kwargs = {"log_type": log_type} if log_type else {} - local_log, metadata = super()._read(ti, try_number, metadata, **kwargs) + local_log, metadata = super()._read(ti, try_number, metadata) return log + local_log, metadata def s3_log_exists(self, remote_log_location: str) -> bool: diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index 611f8ebe2e17c..b5a58c6cc0bb0 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -209,7 +209,7 @@ def wrap_for_triggerer(self): """ # this is just the default inference since we added _read_remote_logs when implementing # trigger logging in all handlers - return "_read_remote_logs" in self.__dict__ + return "_read_remote_logs" in self.__class__.__dict__ def emit(self, record): if self.handler: From e37b4492951797ebb518d6e8d098f2fc9a663003 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 16 Jan 2023 19:29:10 -0800 Subject: [PATCH 23/96] fixups --- airflow/providers/amazon/aws/log/s3_task_handler.py | 5 +++-- airflow/providers/google/cloud/log/gcs_task_handler.py | 2 +- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/airflow/providers/amazon/aws/log/s3_task_handler.py b/airflow/providers/amazon/aws/log/s3_task_handler.py index a5086718b6413..7f47b363008ec 100644 --- a/airflow/providers/amazon/aws/log/s3_task_handler.py +++ b/airflow/providers/amazon/aws/log/s3_task_handler.py @@ -103,14 +103,15 @@ def _read_remote_logs(self, ti, try_number, metadata=None): # Explicitly getting log relative path is necessary as the given # task instance might be different than task instance passed in # in set_context method. + worker_log_rel_path = self._render_filename(ti, try_number) + log = "" messages = [] - worker_log_rel_path = self._render_filename(ti, try_number) bucket, prefix = self.hook.parse_s3_url(s3url=os.path.join(self.remote_base, worker_log_rel_path)) keys = self.hook.list_keys(bucket_name=bucket, prefix=prefix) if keys: keys = [f"s3://{bucket}/{key}" for key in keys] - messages.extend(["Reading logs from s3:", *[f"\n * {x}" for x in keys]]) + messages.extend(["Reading logs from s3:", *[f" * {x}\n" for x in keys]]) for key in keys: log += self.s3_read(key, return_error=True) else: diff --git a/airflow/providers/google/cloud/log/gcs_task_handler.py b/airflow/providers/google/cloud/log/gcs_task_handler.py index 431eaeaa221e9..e095be1ec0ed1 100644 --- a/airflow/providers/google/cloud/log/gcs_task_handler.py +++ b/airflow/providers/google/cloud/log/gcs_task_handler.py @@ -186,7 +186,7 @@ def _read(self, ti, try_number, metadata=None): if not logs: return super()._read(ti, try_number, metadata) - return "".join([f"*** {x}\n" for x in logs]), {"end_of_log": True} + return "".join([f"*** {x}\n" for x in messages]) + logs, {"end_of_log": True} def gcs_write(self, log, remote_log_location): """ From ae387f019af0a68071c679d9506c08fe6d84b98e Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 16 Jan 2023 21:24:40 -0800 Subject: [PATCH 24/96] fixups --- airflow/providers/amazon/aws/log/s3_task_handler.py | 5 ++--- airflow/utils/log/file_task_handler.py | 5 +++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/airflow/providers/amazon/aws/log/s3_task_handler.py b/airflow/providers/amazon/aws/log/s3_task_handler.py index 7f47b363008ec..6d2e2faaf95dd 100644 --- a/airflow/providers/amazon/aws/log/s3_task_handler.py +++ b/airflow/providers/amazon/aws/log/s3_task_handler.py @@ -111,7 +111,7 @@ def _read_remote_logs(self, ti, try_number, metadata=None): keys = self.hook.list_keys(bucket_name=bucket, prefix=prefix) if keys: keys = [f"s3://{bucket}/{key}" for key in keys] - messages.extend(["Reading logs from s3:", *[f" * {x}\n" for x in keys]]) + messages.extend(["Found logs in s3:", *[f" * {x}" for x in keys]]) for key in keys: log += self.s3_read(key, return_error=True) else: @@ -133,11 +133,10 @@ def _read(self, ti, try_number, metadata=None): # from airflow 2.6 we no longer implement the _read method if hasattr(super(), "_read_remote_logs"): return super()._read(ti, try_number, metadata) - # if we get here, we're on airflow < 2.6 and we use this backcompat logic messages, log = self._read_remote_logs(ti, try_number, metadata) if log: - return {"log": log + "".join(f"*** {x}\n" for x in messages), "end_of_log": True} + return "".join(f"*** {x}\n" for x in messages) + log, {"end_of_log": True} else: log += "*** Falling back to local log\n" local_log, metadata = super()._read(ti, try_number, metadata) diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index b5a58c6cc0bb0..e5fa6a0fefc55 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -475,8 +475,9 @@ def _init_file(self, ti): def _read_from_local(worker_log_path: Path): messages = [] log = "" - for file in worker_log_path.rglob("*"): - messages.append(f"Read from local file: {file}") + files = list(worker_log_path.parent.glob(worker_log_path.name + "*")) + messages.extend(["Found local files:", *[f" * {x}" for x in files]]) + for file in files: log += Path(file).read_text() return messages, log From 3c05dbd197bd08cd64e00766f6634fc5a144755c Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 16 Jan 2023 23:52:43 -0800 Subject: [PATCH 25/96] remove some removed stuff --- .../elasticsearch/log/es_task_handler.py | 10 ++----- airflow/utils/log/file_task_handler.py | 26 ------------------- 2 files changed, 2 insertions(+), 34 deletions(-) diff --git a/airflow/providers/elasticsearch/log/es_task_handler.py b/airflow/providers/elasticsearch/log/es_task_handler.py index a76154750a144..34ee1eb76e20d 100644 --- a/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/airflow/providers/elasticsearch/log/es_task_handler.py @@ -32,13 +32,12 @@ import pendulum from elasticsearch_dsl import Search -from airflow.compat.functools import cached_property from airflow.configuration import conf from airflow.models.dagrun import DagRun from airflow.models.taskinstance import TaskInstance from airflow.providers.elasticsearch.log.es_json_formatter import ElasticsearchJSONFormatter from airflow.utils import timezone -from airflow.utils.log.file_task_handler import FileTaskHandler, TriggerLogsPresentationMode +from airflow.utils.log.file_task_handler import FileTaskHandler from airflow.utils.log.logging_mixin import ExternalLoggingMixin, LoggingMixin from airflow.utils.session import create_session @@ -122,10 +121,6 @@ def __init__( self.formatter: logging.Formatter self.handler: logging.FileHandler | logging.StreamHandler # type: ignore[assignment] - @cached_property - def trigger_logs_presentation_mode(self): - return TriggerLogsPresentationMode.INTERLEAVED - def _render_log_id(self, ti: TaskInstance, try_number: int) -> str: with create_session() as session: dag_run = ti.get_dagrun(session=session) @@ -193,7 +188,7 @@ def _read_grouped_logs(self): return True def _read( - self, ti: TaskInstance, try_number: int, metadata: dict | None = None, *, log_type=None + self, ti: TaskInstance, try_number: int, metadata: dict | None = None ) -> tuple[EsLogMsgType, dict]: """ Endpoint for streaming log. @@ -201,7 +196,6 @@ def _read( :param ti: task instance object :param try_number: try_number of the task instance :param metadata: log metadata, can be used for steaming log reading and auto-tailing. - :param log_type: not used :return: a list of tuple with host and log documents, metadata. """ if not metadata: diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index e5fa6a0fefc55..78716c3fb3b04 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -18,7 +18,6 @@ """File logging handler for tasks.""" from __future__ import annotations -import inspect import logging import os import warnings @@ -77,19 +76,6 @@ def _fetch_logs_from_service(url, log_relative_path): return response -class TriggerLogsPresentationMode(Enum): - """ - Controls how trigger logs are presented in webserver. - - Handlers may be implemented either to have trigger logs interleaved with task logs - or shown in a separate tab. - """ - - SPLIT = "split" - INTERLEAVED = "interleaved" - NOT_SUPPORTED = "not_supported" - - def _parse_timestamps_in_log_file(lines: Iterable[str]): timestamp = None next_timestamp = None @@ -187,18 +173,6 @@ def add_triggerer_suffix(full_path, job_id=None): full_path += f".{job_id}.log" return full_path - @cached_property - def trigger_logs_presentation_mode(self): - """ - Tells webserver whether to use separate tab for triggerer logs. - - :meta private: - """ - if "log_type" in inspect.signature(self._read).parameters.keys(): - return TriggerLogsPresentationMode.SPLIT # split is default, most common - else: - return TriggerLogsPresentationMode.NOT_SUPPORTED - @cached_property def wrap_for_triggerer(self): """ From 5227dd382040847fcff917de1d40dfda199bbfe0 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 16 Jan 2023 23:56:43 -0800 Subject: [PATCH 26/96] fixup remove log type --- airflow/utils/log/file_task_handler.py | 7 ++--- .../amazon/aws/log/test_s3_task_handler.py | 30 ------------------- tests/utils/log/test_log_reader.py | 12 ++++---- 3 files changed, 8 insertions(+), 41 deletions(-) diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index 78716c3fb3b04..07e42ea7d2131 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -281,7 +281,6 @@ def _read( which was retrieved in previous calls, this part will be skipped and only following test returned to be added to tail. - :param log_type: controls whether to fetch worker or trigger logs for task :return: log message as a string and metadata. Following attributes are used in metadata: end_of_log: Boolean, True if end of log is reached or False @@ -351,7 +350,7 @@ def _get_log_retrieval_url( log_relative_path, ) - def read(self, task_instance, try_number=None, metadata=None, log_type=None): + def read(self, task_instance, try_number=None, metadata=None): """ Read logs of given task instance from local machine. @@ -359,7 +358,6 @@ def read(self, task_instance, try_number=None, metadata=None, log_type=None): :param try_number: task instance try_number to read logs from. If None it returns all logs separated by try_number :param metadata: log metadata, can be used for steaming log reading and auto-tailing. - :param log_type: Whether to retrieve logs for triggerer or worker. :return: a list of listed tuples which order log string by host """ # Task instance increments its try number when it starts to run. @@ -381,9 +379,8 @@ def read(self, task_instance, try_number=None, metadata=None, log_type=None): metadata_array = [{}] * len(try_numbers) # subclasses implement _read and may not have log_type, which was added recently - kwargs = {"log_type": log_type} if log_type == LogType.TRIGGER else {} for i, try_number_element in enumerate(try_numbers): - log, out_metadata = self._read(task_instance, try_number_element, metadata, **kwargs) + log, out_metadata = self._read(task_instance, try_number_element, metadata) # es_task_handler return logs grouped by host. wrap other handler returning log string # with default/ empty host so that UI can render the response in the same way logs[i] = log if self._read_grouped_logs() else [(task_instance.hostname, log)] diff --git a/tests/providers/amazon/aws/log/test_s3_task_handler.py b/tests/providers/amazon/aws/log/test_s3_task_handler.py index ab416d8a3a1cd..89ed5b8fcbaf6 100644 --- a/tests/providers/amazon/aws/log/test_s3_task_handler.py +++ b/tests/providers/amazon/aws/log/test_s3_task_handler.py @@ -20,7 +20,6 @@ import contextlib import os from unittest import mock -from unittest.mock import MagicMock, patch import boto3 import moto @@ -31,7 +30,6 @@ from airflow.operators.empty import EmptyOperator from airflow.providers.amazon.aws.hooks.s3 import S3Hook from airflow.providers.amazon.aws.log.s3_task_handler import S3TaskHandler -from airflow.utils.log.file_task_handler import LogType from airflow.utils.session import create_session from airflow.utils.state import State from airflow.utils.timezone import datetime @@ -206,31 +204,3 @@ def test_close_no_upload(self): with pytest.raises(ClientError): boto3.resource("s3").Object("bucket", self.remote_log_key).get() - - -@pytest.mark.parametrize("log_type", [LogType.TRIGGER, None]) -@patch("airflow.utils.log.file_task_handler.FileTaskHandler._read") -@patch("airflow.utils.log.file_task_handler.FileTaskHandler._render_filename") -def test_read_backcompat(_render_mock, _read_mock_parent, log_type): - """ - Don't don't make trigger-related calls / use triggerer-related params unless - asked. This ensures backcompat with pre-2.6 airflow versions. - todo: after min airflow version == 2.6, we can just forward log_type - """ - # just forces exit of function after this call. otherwise, have to mock more. - _render_mock.return_value = "tmp/any" - - _read_mock_parent.return_value = ["ret", None] - client = boto3.client("s3") - client.create_bucket(Bucket="abc") - h = S3TaskHandler("/tmp", "s3://abc") - h.handler = MagicMock() - ti = MagicMock() - if log_type == "trigger": - ti.triggerer_job.id = 123 - h._read(ti, 1, log_type=log_type) - _read_mock_parent.assert_called_with(ti, 1, None, log_type=log_type) - else: - h._read(ti, 1) - # unless triggerer log type asked for, arg is not forwarded - _read_mock_parent.assert_called_with(ti, 1, None) diff --git a/tests/utils/log/test_log_reader.py b/tests/utils/log/test_log_reader.py index c382b1fa28945..4fab589896846 100644 --- a/tests/utils/log/test_log_reader.py +++ b/tests/utils/log/test_log_reader.py @@ -206,9 +206,9 @@ def test_read_log_stream_should_support_multiple_chunks(self, mock_read): mock_read.assert_has_calls( [ - mock.call(self.ti, 1, metadata={}, log_type=None), - mock.call(self.ti, 1, metadata={}, log_type=None), - mock.call(self.ti, 1, metadata={"end_of_log": False}, log_type=None), + mock.call(self.ti, 1, metadata={}), + mock.call(self.ti, 1, metadata={}), + mock.call(self.ti, 1, metadata={"end_of_log": False}), ], any_order=False, ) @@ -227,9 +227,9 @@ def test_read_log_stream_should_read_each_try_in_turn(self, mock_read): mock_read.assert_has_calls( [ - mock.call(self.ti, 1, metadata={}, log_type=None), - mock.call(self.ti, 2, metadata={}, log_type=None), - mock.call(self.ti, 3, metadata={}, log_type=None), + mock.call(self.ti, 1, metadata={}), + mock.call(self.ti, 2, metadata={}), + mock.call(self.ti, 3, metadata={}), ], any_order=False, ) From abd39fcc12a572223b604cb58651cebf00030c0b Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Tue, 17 Jan 2023 11:34:36 -0800 Subject: [PATCH 27/96] Update airflow/cli/commands/task_command.py Co-authored-by: Tzu-ping Chung --- airflow/cli/commands/task_command.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow/cli/commands/task_command.py b/airflow/cli/commands/task_command.py index 2b3eb481138b7..360ca7aa19a86 100644 --- a/airflow/cli/commands/task_command.py +++ b/airflow/cli/commands/task_command.py @@ -77,8 +77,8 @@ def _set_task_deferred_context_var(): :meta private: """ logger = logging.getLogger() - h = next((h for h in logger.handlers if hasattr(h, "ctx_task_deferred")), None) - if h: + with suppress(StopIteration): + h = next(h for h in logger.handlers if hasattr(h, "ctx_task_deferred")) h.ctx_task_deferred = True From dbe4467c322430f3ba08a935768a9c7944bf6423 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Tue, 17 Jan 2023 12:06:58 -0800 Subject: [PATCH 28/96] top import --- airflow/cli/commands/triggerer_command.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/airflow/cli/commands/triggerer_command.py b/airflow/cli/commands/triggerer_command.py index c4e4921e99d9a..8bf0c2822ddae 100644 --- a/airflow/cli/commands/triggerer_command.py +++ b/airflow/cli/commands/triggerer_command.py @@ -19,6 +19,7 @@ import signal from contextlib import contextmanager +from functools import partial from multiprocessing import Process from typing import Generator @@ -38,8 +39,6 @@ def _serve_logs(skip_serve_logs: bool = False) -> Generator[None, None, None]: """Starts serve_logs sub-process""" sub_proc = None if skip_serve_logs is False: - from functools import partial - port = conf.getint("logging", "trigger_log_server_port", fallback=8794) sub_proc = Process(target=partial(serve_logs, port=port)) sub_proc.start() From 6284dee6288bd5f0ca38713127b981dba0db4350 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Tue, 17 Jan 2023 12:20:32 -0800 Subject: [PATCH 29/96] pluggable timestamp parser --- airflow/config_templates/config.yml | 10 ++++++++++ airflow/config_templates/default_airflow.cfg | 7 +++++++ airflow/utils/log/file_task_handler.py | 12 ++++++++++-- 3 files changed, 27 insertions(+), 2 deletions(-) diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml index 7da4c176bd70e..507837bd416c8 100644 --- a/airflow/config_templates/config.yml +++ b/airflow/config_templates/config.yml @@ -796,6 +796,16 @@ logging: type: string example: ~ default: "8794" + interleave_timestamp_parser: + description: | + We must parse timestamps to interleave logs between trigger and task. To do so, + we need to parse timestamps in log files. In case your log format is non-standard, + you may provide import path to callable which takes a string log line and returns + the timestamp (datetime.datetime compatible). + version_added: 2.6.0 + type: string + example: path.to.my_func + default: ~ metrics: description: | StatsD (https://github.com/etsy/statsd) integration settings. diff --git a/airflow/config_templates/default_airflow.cfg b/airflow/config_templates/default_airflow.cfg index 243bf50a86599..7983ad2c7a611 100644 --- a/airflow/config_templates/default_airflow.cfg +++ b/airflow/config_templates/default_airflow.cfg @@ -438,6 +438,13 @@ worker_log_server_port = 8793 # for more info. trigger_log_server_port = 8794 +# We must parse timestamps to interleave logs between trigger and task. To do so, +# we need to parse timestamps in log files. In case your log format is non-standard, +# you may provide import path to callable which takes a string log line and returns +# the timestamp (datetime.datetime compatible). +# Example: interleave_timestamp_parser = path.to.my_func +# interleave_timestamp_parser = + [metrics] # StatsD (https://github.com/etsy/statsd) integration settings. diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index 07e42ea7d2131..d6c23cc62f6db 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -76,6 +76,15 @@ def _fetch_logs_from_service(url, log_relative_path): return response +_parse_timestamp = conf.getimport("core", "interleave_timestamp_parser", fallback=None) + +if not _parse_timestamp: + + def _parse_timestamp(line: str): + timestamp_str, _ = line.split(" ", 1) + return pendulum.parse(timestamp_str.strip("[]")) + + def _parse_timestamps_in_log_file(lines: Iterable[str]): timestamp = None next_timestamp = None @@ -83,8 +92,7 @@ def _parse_timestamps_in_log_file(lines: Iterable[str]): if not line: continue with suppress(Exception): - timestamp_str, _ = line.split(" ", 1) - next_timestamp = pendulum.parse(timestamp_str.strip("[]")) + next_timestamp = _parse_timestamp(line) if next_timestamp: timestamp = next_timestamp yield timestamp, line From e17420c35fa5b7e05a35769989a7575f0424e064 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Tue, 17 Jan 2023 14:13:45 -0800 Subject: [PATCH 30/96] fix handling when mult messages same timestamp --- .../amazon/aws/log/cloudwatch_task_handler.py | 4 ++ .../amazon/aws/log/s3_task_handler.py | 19 +++++---- .../elasticsearch/log/es_task_handler.py | 5 +++ .../google/cloud/log/gcs_task_handler.py | 14 +++---- airflow/utils/log/file_task_handler.py | 40 ++++++++++--------- 5 files changed, 46 insertions(+), 36 deletions(-) diff --git a/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py b/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py index e50a6d4d74e2e..3b48688ae8703 100644 --- a/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py +++ b/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py @@ -49,6 +49,10 @@ def __init__(self, base_log_folder: str, log_group_arn: str, filename_template: self.region_name = split_arn[3] self.closed = False + @cached_property + def wrap_for_triggerer(self): + return True + @cached_property def hook(self): """Returns AwsLogsHook.""" diff --git a/airflow/providers/amazon/aws/log/s3_task_handler.py b/airflow/providers/amazon/aws/log/s3_task_handler.py index 6d2e2faaf95dd..c8142600f8404 100644 --- a/airflow/providers/amazon/aws/log/s3_task_handler.py +++ b/airflow/providers/amazon/aws/log/s3_task_handler.py @@ -105,18 +105,18 @@ def _read_remote_logs(self, ti, try_number, metadata=None): # in set_context method. worker_log_rel_path = self._render_filename(ti, try_number) - log = "" + logs = [] messages = [] bucket, prefix = self.hook.parse_s3_url(s3url=os.path.join(self.remote_base, worker_log_rel_path)) keys = self.hook.list_keys(bucket_name=bucket, prefix=prefix) if keys: keys = [f"s3://{bucket}/{key}" for key in keys] - messages.extend(["Found logs in s3:", *[f" * {x}" for x in keys]]) - for key in keys: - log += self.s3_read(key, return_error=True) + messages.extend(["Found logs in s3:", *[f" * {x}" for x in sorted(keys)]]) + for key in sorted(keys): + logs.append(self.s3_read(key, return_error=True)) else: messages.append(f"No logs found on s3 for ti={ti}") - return messages, log + return messages, logs def _read(self, ti, try_number, metadata=None): """ @@ -134,13 +134,12 @@ def _read(self, ti, try_number, metadata=None): if hasattr(super(), "_read_remote_logs"): return super()._read(ti, try_number, metadata) # if we get here, we're on airflow < 2.6 and we use this backcompat logic - messages, log = self._read_remote_logs(ti, try_number, metadata) - if log: - return "".join(f"*** {x}\n" for x in messages) + log, {"end_of_log": True} + messages, logs = self._read_remote_logs(ti, try_number, metadata) + if logs: + return "".join(f"*** {x}\n" for x in messages) + "\n".join(logs), {"end_of_log": True} else: - log += "*** Falling back to local log\n" local_log, metadata = super()._read(ti, try_number, metadata) - return log + local_log, metadata + return "*** Falling back to local log\n" + local_log, metadata def s3_log_exists(self, remote_log_location: str) -> bool: """ diff --git a/airflow/providers/elasticsearch/log/es_task_handler.py b/airflow/providers/elasticsearch/log/es_task_handler.py index 34ee1eb76e20d..bde8a0f0418bb 100644 --- a/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/airflow/providers/elasticsearch/log/es_task_handler.py @@ -32,6 +32,7 @@ import pendulum from elasticsearch_dsl import Search +from airflow.compat.functools import cached_property from airflow.configuration import conf from airflow.models.dagrun import DagRun from airflow.models.taskinstance import TaskInstance @@ -121,6 +122,10 @@ def __init__( self.formatter: logging.Formatter self.handler: logging.FileHandler | logging.StreamHandler # type: ignore[assignment] + @cached_property + def wrap_for_triggerer(self): + return True + def _render_log_id(self, ti: TaskInstance, try_number: int) -> str: with create_session() as session: dag_run = ti.get_dagrun(session=session) diff --git a/airflow/providers/google/cloud/log/gcs_task_handler.py b/airflow/providers/google/cloud/log/gcs_task_handler.py index e095be1ec0ed1..c61062e2ad215 100644 --- a/airflow/providers/google/cloud/log/gcs_task_handler.py +++ b/airflow/providers/google/cloud/log/gcs_task_handler.py @@ -140,12 +140,12 @@ def _add_message(self, msg): record = logging.LogRecord("", logging.INFO, filename, lineno, msg + "\n", None, None, func=func) return self.format(record) - def _read_remote_logs(self, ti, try_number, metadata=None): + def _read_remote_logs(self, ti, try_number, metadata=None) -> tuple[list[str], list[str]]: # Explicitly getting log relative path is necessary because this method # is called from webserver from TaskLogReader, where we don't call set_context # and can read logs for different TIs in each request messages = [] - log = "" + logs = [] worker_log_relative_path = self._render_filename(ti, try_number) remote_loc = os.path.join(self.remote_base, worker_log_relative_path) uris = [] @@ -154,18 +154,18 @@ def _read_remote_logs(self, ti, try_number, metadata=None): if blobs: uris = [f"gs://{bucket}/{b.name}" for b in blobs] - messages.extend(["Found remote logs:", *[f" * {x}" for x in uris]]) + messages.extend(["Found remote logs:", *[f" * {x}" for x in sorted(uris)]]) else: messages.append(f"No logs found in GCS; ti=%s {ti}") try: - for key in uris: + for key in sorted(uris): blob = storage.Blob.from_string(key, self.client) remote_log = blob.download_as_bytes().decode() if remote_log: - log += remote_log + logs.append(remote_log) except Exception as e: messages.append(f"Unable to read remote log {e}") - return messages, log + return messages, logs def _read(self, ti, try_number, metadata=None): """ @@ -186,7 +186,7 @@ def _read(self, ti, try_number, metadata=None): if not logs: return super()._read(ti, try_number, metadata) - return "".join([f"*** {x}\n" for x in messages]) + logs, {"end_of_log": True} + return "".join([f"*** {x}\n" for x in messages]) + "\n".join(logs), {"end_of_log": True} def gcs_write(self, log, remote_log_location): """ diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index d6c23cc62f6db..600191a5a3e8a 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -88,14 +88,14 @@ def _parse_timestamp(line: str): def _parse_timestamps_in_log_file(lines: Iterable[str]): timestamp = None next_timestamp = None - for line in lines: + for idx, line in enumerate(lines): if not line: continue with suppress(Exception): next_timestamp = _parse_timestamp(line) if next_timestamp: timestamp = next_timestamp - yield timestamp, line + yield timestamp, idx, line def _interleave_logs(*logs): @@ -103,7 +103,9 @@ def _interleave_logs(*logs): for log in logs: records.extend(_parse_timestamps_in_log_file(log.splitlines())) last = None - for _, v in sorted(records, key=lambda x: x[0] or pendulum.datetime(2000, 1, 1)): + for _, _, v in sorted( + records, key=lambda x: (x[0], x[1]) if x[0] else (pendulum.datetime(2000, 1, 1), x[1]) + ): if v != last: # dedupe yield v last = v @@ -319,9 +321,9 @@ def _read( messages_list.extend(remote_messages) logs = "\n".join( _interleave_logs( - local_logs, - running_logs, - remote_logs, + *local_logs, + *running_logs, + *remote_logs, ) ) log_pos = len(logs) @@ -451,14 +453,14 @@ def _init_file(self, ti): return full_path @staticmethod - def _read_from_local(worker_log_path: Path): + def _read_from_local(worker_log_path: Path) -> tuple[list[str], list[str]]: messages = [] - log = "" + logs = [] files = list(worker_log_path.parent.glob(worker_log_path.name + "*")) - messages.extend(["Found local files:", *[f" * {x}" for x in files]]) - for file in files: - log += Path(file).read_text() - return messages, log + messages.extend(["Found local files:", *[f" * {x}" for x in sorted(files)]]) + for file in sorted(files): + logs.append(Path(file).read_text()) + return messages, logs def _read_from_k8s_worker(self, ti: TaskInstance): messages = [] @@ -502,13 +504,12 @@ def _read_from_k8s_worker(self, ti: TaskInstance): messages.append(f"Reading from k8s pod logs failed: {str(e)}") return messages, log - def _read_from_logs_server(self, ti, worker_log_rel_path): + def _read_from_logs_server(self, ti, worker_log_rel_path) -> tuple[list[str], list[str]]: messages = [] - log = "" + logs = [] try: log_type = LogType.TRIGGER if ti.triggerer_job else LogType.WORKER url, rel_path = self._get_log_retrieval_url(ti, worker_log_rel_path, log_type=log_type) - messages.append(f"Fetching from: {url}") response = _fetch_logs_from_service(url, rel_path) if response.status_code == 403: messages.append( @@ -521,11 +522,12 @@ def _read_from_logs_server(self, ti, worker_log_rel_path): ) # Check if the resource was properly fetched response.raise_for_status() - log += "\n" + response.text + if response.text: + logs.append(response.text) except Exception as e: - log += str(e) - logger.exception(msg="error") - return messages, log + messages.append(f"Could not read served logs: {str(e)}") + logger.exception("Could not read served logs") + return messages, logs def _read_remote_logs(self, ti, try_number, metadata=None): """Implement in subclasses to read from the remote service""" From a597a5c029b5868c58481222b0c84fc33c2a58a4 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Tue, 17 Jan 2023 14:22:32 -0800 Subject: [PATCH 31/96] comment --- airflow/utils/log/file_task_handler.py | 1 + 1 file changed, 1 insertion(+) diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index 600191a5a3e8a..852eebbc35e01 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -92,6 +92,7 @@ def _parse_timestamps_in_log_file(lines: Iterable[str]): if not line: continue with suppress(Exception): + # next_timestamp unchanged if line can't be parsed next_timestamp = _parse_timestamp(line) if next_timestamp: timestamp = next_timestamp From f034542bd8be438cb58ed6d7feb14e8f2a52e161 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Tue, 17 Jan 2023 14:46:33 -0800 Subject: [PATCH 32/96] no read local if fin --- airflow/utils/log/file_task_handler.py | 27 +++++++++++++------------- 1 file changed, 14 insertions(+), 13 deletions(-) diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index 852eebbc35e01..8a4f032fb8f38 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -304,22 +304,22 @@ def _read( # is needed to get correct log path. messages_list = [] worker_log_rel_path = self._render_filename(ti, try_number) - worker_log_full_path = Path(self.local_base, worker_log_rel_path) - - local_messages, local_logs = self._read_from_local(worker_log_full_path) - messages_list.extend(local_messages) - running_logs = "" - running_messages = [] + remote_logs = [] + running_logs = [] + local_logs = [] + with suppress(NotImplementedError): + remote_messages, remote_logs = self._read_remote_logs(ti, try_number, metadata) + messages_list.extend(remote_messages) if ti.state == TaskInstanceState.RUNNING and self._should_check_k8s(ti.queue): running_messages, running_logs = self._read_from_k8s_worker(ti) elif ti.state in (TaskInstanceState.RUNNING, TaskInstanceState.DEFERRED): running_messages, running_logs = self._read_from_logs_server(ti, worker_log_rel_path) - messages_list.extend(running_messages) - remote_logs = "" - remote_messages = [] - with suppress(NotImplementedError): - remote_messages, remote_logs = self._read_remote_logs(ti, try_number, metadata) - messages_list.extend(remote_messages) + messages_list.extend(running_messages) + if not (remote_logs and ti.state not in State.unfinished): + # when finished, if we have remote logs, no need to check local + worker_log_full_path = Path(self.local_base, worker_log_rel_path) + local_messages, local_logs = self._read_from_local(worker_log_full_path) + messages_list.extend(local_messages) logs = "\n".join( _interleave_logs( *local_logs, @@ -458,7 +458,8 @@ def _read_from_local(worker_log_path: Path) -> tuple[list[str], list[str]]: messages = [] logs = [] files = list(worker_log_path.parent.glob(worker_log_path.name + "*")) - messages.extend(["Found local files:", *[f" * {x}" for x in sorted(files)]]) + if files: + messages.extend(["Found local files:", *[f" * {x}" for x in sorted(files)]]) for file in sorted(files): logs.append(Path(file).read_text()) return messages, logs From f99e1ded4dadb621204cad29f40cc1b30e8da40f Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Tue, 17 Jan 2023 14:58:35 -0800 Subject: [PATCH 33/96] no needed --- airflow/providers/amazon/aws/log/s3_task_handler.py | 9 --------- 1 file changed, 9 deletions(-) diff --git a/airflow/providers/amazon/aws/log/s3_task_handler.py b/airflow/providers/amazon/aws/log/s3_task_handler.py index c8142600f8404..c3da24e98a5ac 100644 --- a/airflow/providers/amazon/aws/log/s3_task_handler.py +++ b/airflow/providers/amazon/aws/log/s3_task_handler.py @@ -26,15 +26,6 @@ from airflow.utils.log.file_task_handler import FileTaskHandler from airflow.utils.log.logging_mixin import LoggingMixin -try: - from airflow.utils.log.file_task_handler import LogType -except ImportError: - # todo: remove when min airflow version >= 2.6 - class LogType: # type: ignore[no-redef] - """For backcompat""" - - TRIGGER = object() - class S3TaskHandler(FileTaskHandler, LoggingMixin): """ From fc5fc3997011a536d2ed185ac1b4f19a88c191b8 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 18 Jan 2023 14:59:16 -0800 Subject: [PATCH 34/96] not needed --- airflow/providers/elasticsearch/log/es_task_handler.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/airflow/providers/elasticsearch/log/es_task_handler.py b/airflow/providers/elasticsearch/log/es_task_handler.py index bde8a0f0418bb..04743b93c94a1 100644 --- a/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/airflow/providers/elasticsearch/log/es_task_handler.py @@ -200,7 +200,8 @@ def _read( :param ti: task instance object :param try_number: try_number of the task instance - :param metadata: log metadata, can be used for steaming log reading and auto-tailing. + :param metadata: log metadata, + can be used for steaming log reading and auto-tailing. :return: a list of tuple with host and log documents, metadata. """ if not metadata: From d4f78c157adf8dbd6f63142db47a8cca5fc06d8a Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 18 Jan 2023 15:35:49 -0800 Subject: [PATCH 35/96] docstring --- .../providers/amazon/aws/log/cloudwatch_task_handler.py | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py b/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py index 3b48688ae8703..ae250921c3d82 100644 --- a/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py +++ b/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py @@ -51,6 +51,14 @@ def __init__(self, base_log_folder: str, log_group_arn: str, filename_template: @cached_property def wrap_for_triggerer(self): + """ + Signals to triggerer_job that this handler supports individual triggerer logging. + + Overriding this property is necessary when a handler does not implement _read_remote_logs. + Handlers which stream to the log sink, such as Cloudwatch and ElasticSearch, do not need + to lean on the behavior of FileTaskHandler which reads from all possible sources, so + they short-circuit this behavior by implementing _read directly. + """ return True @cached_property From 41d55d6039500bc34600715201824441213fd8fd Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 18 Jan 2023 15:36:53 -0800 Subject: [PATCH 36/96] docstring --- airflow/providers/amazon/aws/log/cloudwatch_task_handler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py b/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py index ae250921c3d82..d2d0df3d59033 100644 --- a/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py +++ b/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py @@ -52,7 +52,7 @@ def __init__(self, base_log_folder: str, log_group_arn: str, filename_template: @cached_property def wrap_for_triggerer(self): """ - Signals to triggerer_job that this handler supports individual triggerer logging. + Tells triggerer_job that this handler supports individual triggerer logging. Overriding this property is necessary when a handler does not implement _read_remote_logs. Handlers which stream to the log sink, such as Cloudwatch and ElasticSearch, do not need From 524f1bb18dae6cf50ee5a8c525eb9071ef85cfde Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 19 Jan 2023 01:03:06 -0800 Subject: [PATCH 37/96] support wasb --- .../microsoft/azure/log/wasb_task_handler.py | 71 ++++++++++++++----- 1 file changed, 55 insertions(+), 16 deletions(-) diff --git a/airflow/providers/microsoft/azure/log/wasb_task_handler.py b/airflow/providers/microsoft/azure/log/wasb_task_handler.py index 8c0fe220830ef..af8d21174f90c 100644 --- a/airflow/providers/microsoft/azure/log/wasb_task_handler.py +++ b/airflow/providers/microsoft/azure/log/wasb_task_handler.py @@ -19,8 +19,11 @@ import os import shutil +from pathlib import Path from typing import Any +from azure.core.exceptions import HttpResponseError + from airflow.compat.functools import cached_property from airflow.configuration import conf from airflow.utils.log.file_task_handler import FileTaskHandler @@ -74,8 +77,10 @@ def set_context(self, ti) -> None: super().set_context(ti) # Local location and remote location is needed to open and # upload local log file to Wasb remote storage. - self.log_relative_path = self._render_filename(ti, ti.try_number) - self.upload_on_close = not ti.raw + full_path = self.handler.baseFilename + self.log_relative_path = Path(full_path).relative_to(self.local_base).as_posix() + is_trigger_log_context = getattr(ti, "is_trigger_log_context", False) + self.upload_on_close = is_trigger_log_context or not ti.raw def close(self) -> None: """Close and upload local log file to remote storage Wasb.""" @@ -104,6 +109,43 @@ def close(self) -> None: # Mark closed so we don't double write if close is called twice self.closed = True + def _read_remote_logs(self, ti, try_number, metadata=None): + messages = [] + logs = [] + worker_log_relative_path = self._render_filename(ti, try_number) + # todo: fix this + # for some reason this handler was designed such that (1) container name is not configurable + # (i.e. it's hardcoded in airflow_local_settings.py) and (2) the "relative path" is actually... + # whatever you put in REMOTE_BASE_LOG_FOLDER i.e. it includes the "wasb://" in the blob + # name. it's very screwed up but to change it we have to be careful not to break backcompat. + prefix = os.path.join(self.remote_base, worker_log_relative_path) + blob_names = [] + try: + blob_names = self.hook.get_blobs_list(container_name=self.wasb_container, prefix=prefix) + except HttpResponseError as e: + messages.append(f"tried listing blobs with prefix={prefix} and container={self.wasb_container}") + messages.append("could not list blobs " + str(e)) + self.log.exception("can't list blobs") + + if blob_names: + uris = [f"wasb://{self.wasb_container}/{b}" for b in blob_names] + messages.extend(["Found remote logs:", *[f" * {x}" for x in sorted(uris)]]) + else: + messages.append(f"No logs found in WASB; ti=%s {ti}") + + for name in sorted(blob_names): + remote_log = "" + try: + remote_log = self.hook.read_file(self.wasb_container, name) + if remote_log: + logs.append(remote_log) + except Exception as e: + messages.append( + f"Unable to read remote blob '{name}' in container '{self.wasb_container}'\n{e}" + ) + self.log.exception("Could not read blob") + return messages, logs + def _read( self, ti, try_number: int, metadata: dict[str, Any] | None = None ) -> tuple[str, dict[str, bool]]: @@ -111,26 +153,23 @@ def _read( Read logs of given task instance and try_number from Wasb remote storage. If failed, read the log from task instance host machine. + todo: when min airflow version >= 2.6, remove this method + :param ti: task instance object :param try_number: task instance try_number to read logs from :param metadata: log metadata, can be used for steaming log reading and auto-tailing. """ - # Explicitly getting log relative path is necessary as the given - # task instance might be different than task instance passed in - # in set_context method. - log_relative_path = self._render_filename(ti, try_number) - remote_loc = os.path.join(self.remote_base, log_relative_path) - - if self.wasb_log_exists(remote_loc): - # If Wasb remote file exists, we do not fetch logs from task instance - # local machine even if there are errors reading remote logs, as - # returned remote_log will contain error messages. - remote_log = self.wasb_read(remote_loc, return_error=True) - log = f"*** Reading remote log from {remote_loc}.\n{remote_log}\n" - return log, {"end_of_log": True} - else: + if hasattr(super(), "_read_remote_logs"): + # from Airflow 2.6, we don't implement the `_read` method. + # if parent has _read_remote_logs, we're >= 2.6 + return super()._read(ti, try_number, metadata) + + # below is backcompat, for airflow < 2.6 + messages, logs = self._read_remote_logs(ti, try_number, metadata) + if not logs: return super()._read(ti, try_number, metadata) + return "".join([f"*** {x}\n" for x in messages]) + "\n".join(logs), {"end_of_log": True} def wasb_log_exists(self, remote_log_location: str) -> bool: """ From 837c8db198346e0e8e982138716c16194629dba2 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 19 Jan 2023 01:17:44 -0800 Subject: [PATCH 38/96] todo --- airflow/providers/google/cloud/log/gcs_task_handler.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/airflow/providers/google/cloud/log/gcs_task_handler.py b/airflow/providers/google/cloud/log/gcs_task_handler.py index c61062e2ad215..ef7aaa29cf5b8 100644 --- a/airflow/providers/google/cloud/log/gcs_task_handler.py +++ b/airflow/providers/google/cloud/log/gcs_task_handler.py @@ -172,6 +172,8 @@ def _read(self, ti, try_number, metadata=None): Read logs of given task instance and try_number from GCS. If failed, read the log from task instance host machine. + todo: when min airflow version >= 2.6, remove this method + :param ti: task instance object :param try_number: task instance try_number to read logs from :param metadata: log metadata, From d0b980d969db3b4a3a32d56d3e60a7d8662c9bee Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 19 Jan 2023 12:05:15 -0800 Subject: [PATCH 39/96] fix es eol --- airflow/cli/commands/task_command.py | 13 +------------ airflow/jobs/local_task_job.py | 2 ++ .../elasticsearch/log/es_task_handler.py | 8 ++++++++ airflow/utils/log/file_task_handler.py | 19 +++++++++++++++++++ 4 files changed, 30 insertions(+), 12 deletions(-) diff --git a/airflow/cli/commands/task_command.py b/airflow/cli/commands/task_command.py index 360ca7aa19a86..d5d07f98c5abd 100644 --- a/airflow/cli/commands/task_command.py +++ b/airflow/cli/commands/task_command.py @@ -59,6 +59,7 @@ suppress_logs_and_warning, ) from airflow.utils.dates import timezone +from airflow.utils.log.file_task_handler import _set_task_deferred_context_var from airflow.utils.log.logging_mixin import StreamLogWriter from airflow.utils.log.secrets_masker import RedactedIO from airflow.utils.net import get_hostname @@ -70,18 +71,6 @@ CreateIfNecessary = Union[Literal[False], Literal["db"], Literal["memory"]] -def _set_task_deferred_context_var(): - """ - Tell task log handler that task exited with deferral. - - :meta private: - """ - logger = logging.getLogger() - with suppress(StopIteration): - h = next(h for h in logger.handlers if hasattr(h, "ctx_task_deferred")) - h.ctx_task_deferred = True - - def _generate_temporary_run_id() -> str: """Generate a ``run_id`` for a DAG run that will be created temporarily. diff --git a/airflow/jobs/local_task_job.py b/airflow/jobs/local_task_job.py index 80f39f0f308e5..a40a296df7b09 100644 --- a/airflow/jobs/local_task_job.py +++ b/airflow/jobs/local_task_job.py @@ -30,6 +30,7 @@ from airflow.stats import Stats from airflow.task.task_runner import get_task_runner from airflow.utils import timezone +from airflow.utils.log.file_task_handler import _set_task_deferred_context_var from airflow.utils.net import get_hostname from airflow.utils.session import provide_session from airflow.utils.state import State @@ -216,6 +217,7 @@ def handle_task_exit(self, return_code: int) -> None: is_deferral = return_code == TaskReturnCode.DEFERRED.value if is_deferral: self.log.info("Task exited with return code %s (task deferral)", return_code) + _set_task_deferred_context_var() else: self.log.info("Task exited with return code %s", return_code) diff --git a/airflow/providers/elasticsearch/log/es_task_handler.py b/airflow/providers/elasticsearch/log/es_task_handler.py index 04743b93c94a1..738bb071de56f 100644 --- a/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/airflow/providers/elasticsearch/log/es_task_handler.py @@ -124,6 +124,14 @@ def __init__( @cached_property def wrap_for_triggerer(self): + """ + Tells triggerer_job that this handler supports individual triggerer logging. + + Overriding this property is necessary when a handler does not implement _read_remote_logs. + Handlers which stream to the log sink, such as Cloudwatch and ElasticSearch, do not need + to lean on the behavior of FileTaskHandler which reads from all possible sources, so + they short-circuit this behavior by implementing _read directly. + """ return True def _render_log_id(self, ti: TaskInstance, try_number: int) -> str: diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index 8a4f032fb8f38..02cf7d6152a01 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -56,6 +56,25 @@ class LogType(str, Enum): WORKER = "worker" +def _set_task_deferred_context_var(): + """ + Tell task log handler that task exited with deferral. + + This exists for the sole purpose of telling elasticsearch handler not to + emit end_of_log mark after task deferral. + + Depending on how the task is run, we may need to set this in task command or in local task job. + Kubernetes executor requires the local task job invocation; local executor requires the task + command invocation. + + :meta private: + """ + logger = logging.getLogger() + with suppress(StopIteration): + h = next(h for h in logger.handlers if hasattr(h, "ctx_task_deferred")) + h.ctx_task_deferred = True + + def _fetch_logs_from_service(url, log_relative_path): import httpx From 85bb4180a9297318e02946f72cfb7fb0b0df1391 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 19 Jan 2023 12:06:17 -0800 Subject: [PATCH 40/96] fix ui logs load bug --- .../www/static/js/dag/details/taskInstance/Logs/utils.ts | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/airflow/www/static/js/dag/details/taskInstance/Logs/utils.ts b/airflow/www/static/js/dag/details/taskInstance/Logs/utils.ts index 4a5b8b8664b70..c340e5253ec7a 100644 --- a/airflow/www/static/js/dag/details/taskInstance/Logs/utils.ts +++ b/airflow/www/static/js/dag/details/taskInstance/Logs/utils.ts @@ -46,8 +46,13 @@ export const parseLogs = ( if (!data) { return {}; } - - const lines = data.split('\n'); + let lines; + try { + lines = data.split('\n'); + } catch (err) { + console.error(err); + return {}; + } const parsedLines: Array = []; const fileSources: Set = new Set(); From a3f0e0526db561371e43d4c17d05f3b91da15ee8 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 19 Jan 2023 12:08:45 -0800 Subject: [PATCH 41/96] simplif --- airflow/providers/amazon/aws/log/s3_task_handler.py | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/airflow/providers/amazon/aws/log/s3_task_handler.py b/airflow/providers/amazon/aws/log/s3_task_handler.py index c3da24e98a5ac..0e25c6a59c29c 100644 --- a/airflow/providers/amazon/aws/log/s3_task_handler.py +++ b/airflow/providers/amazon/aws/log/s3_task_handler.py @@ -53,12 +53,9 @@ def set_context(self, ti): super().set_context(ti) # Local location and remote location is needed to open and # upload local log file to S3 remote storage. - log_relative_path = self._render_filename(ti, ti.try_number) - # use getattr for compat with airflow < 2.6 + full_path = self.handler.baseFilename + self.log_relative_path = pathlib.Path(full_path).relative_to(self.local_base).as_posix() is_trigger_log_context = getattr(ti, "is_trigger_log_context", False) - if is_trigger_log_context: - log_relative_path = self.add_triggerer_suffix(log_relative_path, ti.triggerer_job.id) - self.log_relative_path = log_relative_path self.upload_on_close = is_trigger_log_context or not ti.raw # Clear the file first so that duplicate data is not uploaded # when re-using the same path (e.g. with rescheduled sensors) From 5d691dda225aea60524e43fbf37e738e6e9fe065 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Sat, 21 Jan 2023 01:11:41 -0800 Subject: [PATCH 42/96] can disable trigger logging --- airflow/jobs/triggerer_job.py | 22 ++++++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/airflow/jobs/triggerer_job.py b/airflow/jobs/triggerer_job.py index 47d7af4b62dc4..2cc87c4b54d5a 100644 --- a/airflow/jobs/triggerer_job.py +++ b/airflow/jobs/triggerer_job.py @@ -33,7 +33,6 @@ from airflow.configuration import conf from airflow.jobs.base_job import BaseJob from airflow.models.trigger import Trigger -from airflow.settings import DONOT_MODIFY_HANDLERS from airflow.stats import Stats from airflow.triggers.base import BaseTrigger, TriggerEvent from airflow.typing_compat import TypedDict @@ -65,6 +64,10 @@ logger = logging.getLogger(__name__) +DISABLE_WRAPPER = conf.getboolean("logging", "disable_trigger_handler_wrapper", fallback=False) +DISABLE_LISTENER = conf.getboolean("logging", "disable_trigger_handler_queue_listener", fallback=False) + + def configure_trigger_log_handler(): """ Configure logging such that each trigger logs to its own file and @@ -212,10 +215,21 @@ def __init__(self, capacity=None, *args, **kwargs): else: raise ValueError(f"Capacity number {capacity} is invalid") - if not DONOT_MODIFY_HANDLERS: + if DISABLE_WRAPPER: + self.log.warning( + "Skipping trigger log configuration; disabled by param " + "`disable_trigger_handler_wrapper=True`." + ) + else: configure_trigger_log_handler() - self.listener = setup_queue_listener() - + self.listener = None + if DISABLE_LISTENER: + self.log.warning( + "Skipping trigger logger queue listener; disabled by param " + "`disable_trigger_handler_queue_listener=True`." + ) + else: + self.listener = setup_queue_listener() # Set up runner async thread self.runner = TriggerRunner() From b13f833849e4d4c5e1f51c74b8421201fff51d74 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 23 Jan 2023 14:36:15 -0800 Subject: [PATCH 43/96] no queues for cloudwatch -- avoids many threads --- airflow/providers/amazon/aws/log/cloudwatch_task_handler.py | 1 + 1 file changed, 1 insertion(+) diff --git a/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py b/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py index d2d0df3d59033..f75dd26199dba 100644 --- a/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py +++ b/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py @@ -80,6 +80,7 @@ def set_context(self, ti): boto3_client=self.hook.get_conn(), ) + use_queues=not getattr(ti, "is_trigger_log_context", False), def close(self): """Close the handler responsible for the upload of the local log file to Cloudwatch.""" # When application exit, system shuts down all handlers by From 3306bf170428261189d52a3c13cbbd341185fc03 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Tue, 24 Jan 2023 17:51:45 -0800 Subject: [PATCH 44/96] wip -- got "native" trigger handlers working with stackdriver --- airflow/jobs/triggerer_job.py | 84 ++++++++++++++----- .../amazon/aws/log/cloudwatch_task_handler.py | 16 +--- .../amazon/aws/log/s3_task_handler.py | 2 + .../elasticsearch/log/es_task_handler.py | 2 +- .../google/cloud/log/gcs_task_handler.py | 2 + .../cloud/log/stackdriver_task_handler.py | 42 +++++++--- .../microsoft/azure/log/wasb_task_handler.py | 3 + airflow/utils/log/file_task_handler.py | 14 +--- airflow/utils/log/trigger_handler.py | 8 +- tests/jobs/test_triggerer_job_logging.py | 28 +++---- tests/utils/test_log_handlers.py | 30 ++++++- 11 files changed, 153 insertions(+), 78 deletions(-) diff --git a/airflow/jobs/triggerer_job.py b/airflow/jobs/triggerer_job.py index 2cc87c4b54d5a..d62aaa3ff51ce 100644 --- a/airflow/jobs/triggerer_job.py +++ b/airflow/jobs/triggerer_job.py @@ -25,6 +25,7 @@ import time import warnings from collections import deque +from copy import copy from queue import SimpleQueue from typing import TYPE_CHECKING, Deque @@ -43,9 +44,9 @@ LocalQueueHandler, TriggererHandlerWrapper, TriggerMetadataFilter, - ctx_close_handler, ctx_indiv_trigger, ctx_task_instance, + ctx_trigger_end, ctx_trigger_id, ) from airflow.utils.module_loading import import_string @@ -54,9 +55,17 @@ if TYPE_CHECKING: from airflow.models import TaskInstance -USING_TRIGGERER_HANDLER_WRAPPER = False +HANDLER_SUPPORTS_TRIGGERER = False """ -If this value is true, trigger logging is configured to use TriggerHandlerWrapper +If this value is true, root handler is configured to log individual trigger messages +visible in task logs. + +:meta private: +""" + +SEND_TRIGGER_END_MARKER = True +""" +If handler natively supports triggers, may want to disable sending trigger end marker. :meta private: """ @@ -81,26 +90,50 @@ def configure_trigger_log_handler(): :meta private: """ - global USING_TRIGGERER_HANDLER_WRAPPER + global HANDLER_SUPPORTS_TRIGGERER - def should_wrap_for_triggerer(handler): - return getattr(handler, "wrap_for_triggerer", False) + def should_wrap(handler): + return handler.__dict__.get("trigger_should_wrap", False) or handler.__class__.__dict__.get( + "trigger_should_wrap", False + ) + + def should_queue(handler): + return handler.__dict__.get("trigger_should_queue", True) or handler.__class__.__dict__.get( + "trigger_should_queue", True + ) + + def send_trigger_end_marker(handler): + val = handler.__dict__.get("trigger_send_end_marker", None) + if val is not None: + return val + + val = handler.__class__.__dict__.get("trigger_send_end_marker", None) + if val is not None: + return val + return True + + def supports_triggerer(handler): + return ( + should_wrap(handler) + or handler.__dict__.get("trigger_supported", False) + or handler.__class__.__dict__.get("trigger_supported", False) + ) def get_task_handler_from_logger(logger_): for h in logger_.handlers: - if isinstance(h, FileTaskHandler) and not should_wrap_for_triggerer(h): + if isinstance(h, FileTaskHandler) and not supports_triggerer(h): warnings.warn( f"Handler {h.__class__.__name__} does not support " "individual trigger logging. Please check the release notes " "for your provider to see if a newer version supports " "individual trigger logging." ) - if should_wrap_for_triggerer(h): + if supports_triggerer(h): return h def find_suitable_task_handler(): # check root logger then check airflow.task to see if a handler - # suitable for use with TriggerHandlerWrapper (has wrap_for_triggerer + # suitable for use with TriggerHandlerWrapper (has trigger_should_wrap # attr, likely inherits from FileTaskHandler) h = get_task_handler_from_logger(root_logger) if not h: @@ -146,10 +179,17 @@ def add_handler_wrapper_to_root(base_handler): return None if TYPE_CHECKING: assert isinstance(task_handler, FileTaskHandler) - wrapper_handler = add_handler_wrapper_to_root(task_handler) - filter_trigger_logs_from_other_root_handlers(wrapper_handler) - USING_TRIGGERER_HANDLER_WRAPPER = True - return None + if should_wrap(task_handler): + trigger_handler = add_handler_wrapper_to_root(task_handler) + else: + trigger_handler = copy(task_handler) + root_logger.addHandler(trigger_handler) + filter_trigger_logs_from_other_root_handlers(trigger_handler) + if send_trigger_end_marker(trigger_handler) is False: + global SEND_TRIGGER_END_MARKER + SEND_TRIGGER_END_MARKER = False + HANDLER_SUPPORTS_TRIGGERER = True + return should_queue(trigger_handler) def setup_queue_listener(): @@ -215,19 +255,22 @@ def __init__(self, capacity=None, *args, **kwargs): else: raise ValueError(f"Capacity number {capacity} is invalid") + should_queue = True if DISABLE_WRAPPER: self.log.warning( "Skipping trigger log configuration; disabled by param " "`disable_trigger_handler_wrapper=True`." ) else: - configure_trigger_log_handler() + should_queue = configure_trigger_log_handler() self.listener = None if DISABLE_LISTENER: self.log.warning( "Skipping trigger logger queue listener; disabled by param " "`disable_trigger_handler_queue_listener=True`." ) + elif should_queue is False: + self.log.warning("Skipping trigger logger queue listener; disabled by handler setting.") else: self.listener = setup_queue_listener() # Set up runner async thread @@ -536,7 +579,7 @@ def set_individual_trigger_logging(trigger): # set logging context vars for routing to appropriate handler ctx_task_instance.set(trigger.task_instance) ctx_trigger_id.set(trigger.trigger_id) - ctx_close_handler.set(False) + ctx_trigger_end.set(False) # mark that we're in the context of an individual trigger so log records can be filtered ctx_indiv_trigger.set(True) @@ -560,18 +603,19 @@ async def run_trigger(self, trigger_id, trigger): # allow triggers a chance to cleanup, either in that case or if # they exit cleanly. trigger.cleanup() - self.close_handler(trigger) + if SEND_TRIGGER_END_MARKER: + self.mark_trigger_end(trigger) # unsetting ctx_indiv_trigger var restores stdout logging ctx_indiv_trigger.set(None) self.log.info("trigger %s completed", name) @staticmethod - def close_handler(trigger): - if not USING_TRIGGERER_HANDLER_WRAPPER: + def mark_trigger_end(trigger): + if not HANDLER_SUPPORTS_TRIGGERER: return - ctx_close_handler.set(True) - trigger.log.log(level=100, msg="close handler") + ctx_trigger_end.set(True) + trigger.log.log(level=100, msg="trigger end") def update_triggers(self, requested_trigger_ids: set[int]): """ diff --git a/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py b/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py index f75dd26199dba..d2f54cc780da3 100644 --- a/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py +++ b/airflow/providers/amazon/aws/log/cloudwatch_task_handler.py @@ -40,6 +40,8 @@ class CloudwatchTaskHandler(FileTaskHandler, LoggingMixin): :param filename_template: template for file name (local storage) or log stream name (remote) """ + trigger_should_wrap = True + def __init__(self, base_log_folder: str, log_group_arn: str, filename_template: str | None = None): super().__init__(base_log_folder, filename_template) split_arn = log_group_arn.split(":") @@ -49,18 +51,6 @@ def __init__(self, base_log_folder: str, log_group_arn: str, filename_template: self.region_name = split_arn[3] self.closed = False - @cached_property - def wrap_for_triggerer(self): - """ - Tells triggerer_job that this handler supports individual triggerer logging. - - Overriding this property is necessary when a handler does not implement _read_remote_logs. - Handlers which stream to the log sink, such as Cloudwatch and ElasticSearch, do not need - to lean on the behavior of FileTaskHandler which reads from all possible sources, so - they short-circuit this behavior by implementing _read directly. - """ - return True - @cached_property def hook(self): """Returns AwsLogsHook.""" @@ -77,10 +67,10 @@ def set_context(self, ti): self.handler = watchtower.CloudWatchLogHandler( log_group_name=self.log_group, log_stream_name=self._render_filename(ti, ti.try_number), + use_queues=not getattr(ti, "is_trigger_log_context", False), boto3_client=self.hook.get_conn(), ) - use_queues=not getattr(ti, "is_trigger_log_context", False), def close(self): """Close the handler responsible for the upload of the local log file to Cloudwatch.""" # When application exit, system shuts down all handlers by diff --git a/airflow/providers/amazon/aws/log/s3_task_handler.py b/airflow/providers/amazon/aws/log/s3_task_handler.py index 0e25c6a59c29c..831c86417185f 100644 --- a/airflow/providers/amazon/aws/log/s3_task_handler.py +++ b/airflow/providers/amazon/aws/log/s3_task_handler.py @@ -34,6 +34,8 @@ class S3TaskHandler(FileTaskHandler, LoggingMixin): uploads to and reads from S3 remote storage. """ + trigger_should_wrap = True + def __init__(self, base_log_folder: str, s3_log_folder: str, filename_template: str | None = None): super().__init__(base_log_folder, filename_template) self.remote_base = s3_log_folder diff --git a/airflow/providers/elasticsearch/log/es_task_handler.py b/airflow/providers/elasticsearch/log/es_task_handler.py index 738bb071de56f..94e7acad485f3 100644 --- a/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/airflow/providers/elasticsearch/log/es_task_handler.py @@ -123,7 +123,7 @@ def __init__( self.handler: logging.FileHandler | logging.StreamHandler # type: ignore[assignment] @cached_property - def wrap_for_triggerer(self): + def trigger_should_wrap(self): """ Tells triggerer_job that this handler supports individual triggerer logging. diff --git a/airflow/providers/google/cloud/log/gcs_task_handler.py b/airflow/providers/google/cloud/log/gcs_task_handler.py index ef7aaa29cf5b8..19078d96cc10c 100644 --- a/airflow/providers/google/cloud/log/gcs_task_handler.py +++ b/airflow/providers/google/cloud/log/gcs_task_handler.py @@ -62,6 +62,8 @@ class GCSTaskHandler(FileTaskHandler, LoggingMixin): :param project_id: Project ID to read the secrets from. If not passed, the project ID from credentials will be used. """ + trigger_should_wrap = True + def __init__( self, diff --git a/airflow/providers/google/cloud/log/stackdriver_task_handler.py b/airflow/providers/google/cloud/log/stackdriver_task_handler.py index 0478693761e11..0374a2dedcfe9 100644 --- a/airflow/providers/google/cloud/log/stackdriver_task_handler.py +++ b/airflow/providers/google/cloud/log/stackdriver_task_handler.py @@ -32,6 +32,7 @@ from airflow.models import TaskInstance from airflow.providers.google.cloud.utils.credentials_provider import get_credentials_and_project_id from airflow.providers.google.common.consts import CLIENT_INFO +from airflow.utils.log.trigger_handler import ctx_indiv_trigger DEFAULT_LOGGER_NAME = "airflow" _GLOBAL_RESOURCE = Resource(type="global", labels={}) @@ -78,6 +79,11 @@ class StackdriverTaskHandler(logging.Handler): LOG_VIEWER_BASE_URL = "https://console.cloud.google.com/logs/viewer" LOG_NAME = "Google Stackdriver" + trigger_supported = True + trigger_should_queue = False + trigger_should_wrap = False + trigger_send_end_marker = False + def __init__( self, gcp_key_path: str | None = None, @@ -132,23 +138,35 @@ def _transport(self) -> Transport: # arguments are a requirement for any class that derives from Transport class, hence ignore: return self.transport_type(self._client, self.name) # type: ignore[call-arg] - def emit(self, record: logging.LogRecord) -> None: - """Actually log the specified logging record. - - :param record: The record to be logged. - """ - message = self.format(record) + def _get_labels(self, task_instance=None): + """When""" + if task_instance: + ti_labels = self._task_instance_to_labels(task_instance) + else: + ti_labels = self.task_instance_labels labels: dict[str, str] | None - if self.labels and self.task_instance_labels: + if self.labels and ti_labels: labels = {} labels.update(self.labels) - labels.update(self.task_instance_labels) + labels.update(ti_labels) elif self.labels: labels = self.labels - elif self.task_instance_labels: - labels = self.task_instance_labels + elif ti_labels: + labels = ti_labels else: labels = None + return labels or {} + + def emit(self, record: logging.LogRecord) -> None: + """Actually log the specified logging record. + + :param record: The record to be logged. + """ + message = self.format(record) + ti = None + if getattr(record, ctx_indiv_trigger.name, None): + ti = getattr(record, "task_instance", None) # trigger context + labels = self._get_labels(ti) self._transport.send(record, message, resource=self.resource, labels=labels) def set_context(self, task_instance: TaskInstance) -> None: @@ -291,7 +309,9 @@ def _read_single_logs_page(self, log_filter: str, page_token: str | None = None) page: ListLogEntriesResponse = next(response.pages) messages = [] for entry in page.entries: - if "message" in entry.json_payload: + if entry.text_payload: + messages.append(entry.text_payload) + elif "message" in entry.json_payload or {}: messages.append(entry.json_payload["message"]) return "\n".join(messages), page.next_page_token diff --git a/airflow/providers/microsoft/azure/log/wasb_task_handler.py b/airflow/providers/microsoft/azure/log/wasb_task_handler.py index af8d21174f90c..db44a85d9d41a 100644 --- a/airflow/providers/microsoft/azure/log/wasb_task_handler.py +++ b/airflow/providers/microsoft/azure/log/wasb_task_handler.py @@ -37,6 +37,9 @@ class WasbTaskHandler(FileTaskHandler, LoggingMixin): uploads to and reads from Wasb remote storage. """ + trigger_should_wrap = True + trigger_should_queue = True + def __init__( self, base_log_folder: str, diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index 02cf7d6152a01..a17b61eab4e15 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -142,6 +142,8 @@ class FileTaskHandler(logging.Handler): :param filename_template: template filename string """ + trigger_should_wrap = True + def __init__(self, base_log_folder: str, filename_template: str | None = None): super().__init__() self.handler: logging.FileHandler | None = None @@ -203,18 +205,6 @@ def add_triggerer_suffix(full_path, job_id=None): full_path += f".{job_id}.log" return full_path - @cached_property - def wrap_for_triggerer(self): - """ - If true, this handler has been updated to support individual logging as implemented - in triggerer_job. - - :meta private: - """ - # this is just the default inference since we added _read_remote_logs when implementing - # trigger logging in all handlers - return "_read_remote_logs" in self.__class__.__dict__ - def emit(self, record): if self.handler: self.handler.emit(record) diff --git a/airflow/utils/log/trigger_handler.py b/airflow/utils/log/trigger_handler.py index 3410747d38172..50756cb048375 100644 --- a/airflow/utils/log/trigger_handler.py +++ b/airflow/utils/log/trigger_handler.py @@ -27,7 +27,7 @@ ctx_task_instance: ContextVar = ContextVar("task_instance") ctx_trigger_id: ContextVar = ContextVar("trigger_id") -ctx_close_handler: ContextVar = ContextVar("close_handler") +ctx_trigger_end: ContextVar = ContextVar("trigger_end") ctx_indiv_trigger: ContextVar = ContextVar("__individual_trigger") @@ -42,7 +42,7 @@ def filter(self, record): for var in ( ctx_task_instance, ctx_trigger_id, - ctx_close_handler, + ctx_trigger_end, ctx_indiv_trigger, ): val = var.get(None) @@ -73,6 +73,8 @@ class TriggererHandlerWrapper(logging.Handler): :meta private: """ + trigger_should_queue = True + def __init__(self, base_handler: FileTaskHandler, level=logging.NOTSET): super().__init__(level=level) self.base_handler: FileTaskHandler = base_handler @@ -95,7 +97,7 @@ def emit(self, record): def handle(self, record): if not getattr(record, ctx_indiv_trigger.name, None): return False - if record.close_handler: + if record.trigger_end: self.close_one(record.trigger_id) return False emit = self.filter(record) diff --git a/tests/jobs/test_triggerer_job_logging.py b/tests/jobs/test_triggerer_job_logging.py index 731fa77d7252b..27a641d8e22df 100644 --- a/tests/jobs/test_triggerer_job_logging.py +++ b/tests/jobs/test_triggerer_job_logging.py @@ -74,11 +74,11 @@ def test_configure_trigger_log_handler_file(): task_handlers = assert_handlers(task_logger, FileTaskHandler) # not yet configured to use wrapper - assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is False + assert triggerer_job.HANDLER_SUPPORTS_TRIGGERER is False triggerer_job.configure_trigger_log_handler() # after config - assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is True + assert triggerer_job.HANDLER_SUPPORTS_TRIGGERER is True root_handlers = assert_handlers(root_logger, RedirectStdHandler, TriggererHandlerWrapper) assert root_handlers[1].base_handler == task_handlers[0] # other handlers have DropTriggerLogsFilter @@ -112,11 +112,11 @@ def test_configure_trigger_log_handler_s3(): task_logger = logging.getLogger("airflow.task") task_handlers = assert_handlers(task_logger, S3TaskHandler) # not yet configured to use wrapper - assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is False + assert triggerer_job.HANDLER_SUPPORTS_TRIGGERER is False triggerer_job.configure_trigger_log_handler() # after config - assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is True + assert triggerer_job.HANDLER_SUPPORTS_TRIGGERER is True handlers = assert_handlers(root_logger, RedirectStdHandler, TriggererHandlerWrapper) assert handlers[1].base_handler == task_handlers[0] # other handlers have DropTriggerLogsFilter @@ -196,7 +196,7 @@ def test_configure_trigger_log_handler_not_file_task_handler(cfg, cls, msg): assert_handlers(task_logger, cls) # not yet configured to use wrapper - assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is False + assert triggerer_job.HANDLER_SUPPORTS_TRIGGERER is False with warnings.catch_warnings(record=True) as captured: triggerer_job.configure_trigger_log_handler() @@ -205,7 +205,7 @@ def test_configure_trigger_log_handler_not_file_task_handler(cfg, cls, msg): # after config # doesn't use TriggererHandlerWrapper, no change in handler - assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is False + assert triggerer_job.HANDLER_SUPPORTS_TRIGGERER is False # still no root handlers assert_handlers(root_logger) @@ -246,12 +246,12 @@ def test_configure_trigger_log_handler_fallback_task(): # before config root_logger = logging.getLogger() assert_handlers(root_logger) - assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is False + assert triggerer_job.HANDLER_SUPPORTS_TRIGGERER is False triggerer_job.configure_trigger_log_handler() # after config - assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is True + assert triggerer_job.HANDLER_SUPPORTS_TRIGGERER is True handlers = assert_handlers(root_logger, TriggererHandlerWrapper) assert handlers[0].base_handler == task_logger.handlers[0] @@ -297,13 +297,13 @@ def test_configure_trigger_log_handler_root_has_task_handler(): # before config root_logger = logging.getLogger() assert_handlers(root_logger, FileTaskHandler) - assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is False + assert triggerer_job.HANDLER_SUPPORTS_TRIGGERER is False # configure triggerer_job.configure_trigger_log_handler() # after config - assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is True + assert triggerer_job.HANDLER_SUPPORTS_TRIGGERER is True handlers = assert_handlers(root_logger, TriggererHandlerWrapper) # no filters on wrapper handler assert handlers[0].filters == [] @@ -354,7 +354,7 @@ def test_configure_trigger_log_handler_root_not_file_task(): # before config root_logger = logging.getLogger() assert_handlers(root_logger, logging.Handler) - assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is False + assert triggerer_job.HANDLER_SUPPORTS_TRIGGERER is False # configure with warnings.catch_warnings(record=True) as captured: @@ -362,7 +362,7 @@ def test_configure_trigger_log_handler_root_not_file_task(): assert captured == [] # after config - assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is True + assert triggerer_job.HANDLER_SUPPORTS_TRIGGERER is True handlers = assert_handlers(root_logger, logging.Handler, TriggererHandlerWrapper) # other handlers have DropTriggerLogsFilter assert handlers[0].filters[0].__class__ == DropTriggerLogsFilter @@ -420,7 +420,7 @@ def test_configure_trigger_log_handler_root_old_file_task(): root_logger = logging.getLogger() assert_handlers(root_logger, OldFileTaskHandler) - assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is False + assert triggerer_job.HANDLER_SUPPORTS_TRIGGERER is False with warnings.catch_warnings(record=True) as captured: triggerer_job.configure_trigger_log_handler() @@ -435,7 +435,7 @@ def test_configure_trigger_log_handler_root_old_file_task(): ] # after config - assert triggerer_job.USING_TRIGGERER_HANDLER_WRAPPER is True + assert triggerer_job.HANDLER_SUPPORTS_TRIGGERER is True handlers = assert_handlers(root_logger, OldFileTaskHandler, TriggererHandlerWrapper) # other handlers have DropTriggerLogsFilter assert handlers[0].filters[0].__class__ == DropTriggerLogsFilter diff --git a/tests/utils/test_log_handlers.py b/tests/utils/test_log_handlers.py index af9e0d764c9ba..1d163975aa9c6 100644 --- a/tests/utils/test_log_handlers.py +++ b/tests/utils/test_log_handlers.py @@ -571,7 +571,29 @@ def test_interleave_interleaves(): [2023-01-15T22:37:13.361-0800] {triggerer_job.py:540} INFO - Trigger (ID 106) fired: TriggerEvent """ -for line in _parse_timestamps_in_log_file(long_sample.splitlines()): - print(line) -for line in _interleave_logs(long_sample): - print(line) +# for line in _parse_timestamps_in_log_file(long_sample.splitlines()): +# print(line) +# for line in _interleave_logs(long_sample): +# print(line) + + +def test_this(): + """ + Notice there are two messages with timestamp `2023-01-17T12:47:11.883-0800`. + In this case, these should appear in correct order and be deduped in result. + """ + sample_with_dupe = """[2023-01-17T12:46:55.868-0800] {temporal.py:62} INFO - trigger starting + [2023-01-17T12:46:55.868-0800] {temporal.py:71} INFO - sleeping 1 second... + [2023-01-17T12:47:09.882-0800] {temporal.py:71} INFO - sleeping 1 second... + [2023-01-17T12:47:10.882-0800] {temporal.py:71} INFO - sleeping 1 second... + [2023-01-17T12:47:11.883-0800] {temporal.py:74} INFO - yielding event with payload DateTime(2023, 1, 17, 20, 47, 11, 254388, tzinfo=Timezone('UTC')) + [2023-01-17T12:47:11.883-0800] {triggerer_job.py:540} INFO - Trigger (ID 1) fired: TriggerEvent + """ + + assert sample_with_dupe == "\n".join(_interleave_logs(sample_with_dupe, "", sample_with_dupe)) + + +def test_ttthis(): + local_logs = "[2023-01-17T13:41:53.228-0800] {temporal.py:62} INFO - trigger starting\n[2023-01-17T13:41:53.228-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:41:54.228-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:41:55.229-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:41:56.229-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:41:57.229-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:41:58.230-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:41:59.231-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:42:00.231-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:42:01.232-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:42:02.233-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:42:03.233-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:42:04.233-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:42:05.233-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:42:06.234-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:42:07.235-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:42:08.237-0800] {temporal.py:74} INFO - yielding event with payload DateTime(2023, 1, 17, 21, 42, 8, 150776, tzinfo=Timezone('UTC'))\n[2023-01-17T13:42:08.237-0800] {triggerer_job.py:540} INFO - Trigger (ID 4) fired: TriggerEvent\n[2023-01-17T13:42:19.050-0800] {taskinstance.py:1131} INFO - Dependencies all met for dep_context=non-requeueable deps ti=\n[2023-01-17T13:42:19.074-0800] {taskinstance.py:1131} INFO - Dependencies all met for dep_context=requeueable deps ti=\n[2023-01-17T13:42:19.074-0800] {taskinstance.py:1330} INFO - resuming after deferral\n[2023-01-17T13:42:19.087-0800] {taskinstance.py:1351} INFO - Executing on 2023-01-17 21:41:38.150776+00:00\n[2023-01-17T13:42:19.091-0800] {standard_task_runner.py:56} INFO - Started process 98604 to run task\n[2023-01-17T13:42:19.097-0800] {standard_task_runner.py:83} INFO - Running: ['airflow', 'tasks', 'run', 'example_time_delta_sensor_async', 'wait', 'manual__2023-01-17T21:41:38.150776+00:00', '--job-id', '518', '--raw', '--subdir', '/Users/dstandish/code/airflow/airflow/example_dags/example_time_delta_sensor_async.py', '--cfg-path', '/var/folders/7_/1xx0hqcs3txd7kqt0ngfdjth0000gn/T/tmpx76emkve', '--no-shut-down-logging']\n[2023-01-17T13:42:19.101-0800] {standard_task_runner.py:84} INFO - Job 518: Subtask wait\n[2023-01-17T13:42:19.185-0800] {task_command.py:417} INFO - Running on host Daniels-MacBook-Pro-2.local\n[2023-01-17T13:42:19.274-0800] {taskinstance.py:1369} INFO - Marking task as SUCCESS. dag_id=example_time_delta_sensor_async, task_id=wait, execution_date=20230117T214138, start_date=20230117T214151, end_date=20230117T214219\n[2023-01-17T13:42:19.309-0800] {local_task_job.py:220} INFO - Task exited with return code 0\n[2023-01-17T13:42:19.339-0800] {taskinstance.py:2648} INFO - 1 downstream tasks scheduled from follow-on schedule check\n" + remote_logs = "[2023-01-17T13:41:51.446-0800] {taskinstance.py:1131} INFO - Dependencies all met for dep_context=non-requeueable deps ti=\n[2023-01-17T13:41:51.454-0800] {taskinstance.py:1131} INFO - Dependencies all met for dep_context=requeueable deps ti=\n[2023-01-17T13:41:51.454-0800] {taskinstance.py:1332} INFO - Starting attempt 1 of 1\n[2023-01-17T13:41:51.478-0800] {taskinstance.py:1351} INFO - Executing on 2023-01-17 21:41:38.150776+00:00\n[2023-01-17T13:41:51.482-0800] {standard_task_runner.py:56} INFO - Started process 98256 to run task\n[2023-01-17T13:41:51.487-0800] {standard_task_runner.py:83} INFO - Running: ['airflow', 'tasks', 'run', 'example_time_delta_sensor_async', 'wait', 'manual__2023-01-17T21:41:38.150776+00:00', '--job-id', '517', '--raw', '--subdir', '/Users/dstandish/code/airflow/airflow/example_dags/example_time_delta_sensor_async.py', '--cfg-path', '/var/folders/7_/1xx0hqcs3txd7kqt0ngfdjth0000gn/T/tmpszoemoy8', '--no-shut-down-logging']\n[2023-01-17T13:41:51.490-0800] {standard_task_runner.py:84} INFO - Job 517: Subtask wait\n[2023-01-17T13:41:51.575-0800] {task_command.py:417} INFO - Running on host Daniels-MacBook-Pro-2.local\n[2023-01-17T13:41:51.651-0800] {taskinstance.py:1558} INFO - Exporting env vars: AIRFLOW_CTX_DAG_OWNER='airflow' AIRFLOW_CTX_DAG_ID='example_time_delta_sensor_async' AIRFLOW_CTX_TASK_ID='wait' AIRFLOW_CTX_EXECUTION_DATE='2023-01-17T21:41:38.150776+00:00' AIRFLOW_CTX_TRY_NUMBER='1' AIRFLOW_CTX_DAG_RUN_ID='manual__2023-01-17T21:41:38.150776+00:00'\n[2023-01-17T13:41:51.672-0800] {taskinstance.py:1433} INFO - Pausing task as DEFERRED. dag_id=example_time_delta_sensor_async, task_id=wait, execution_date=20230117T214138, start_date=20230117T214151\n[2023-01-17T13:41:51.700-0800] {local_task_job.py:218} INFO - Task exited with return code 100 (task deferral)\n\n[2023-01-17T13:42:19.050-0800] {taskinstance.py:1131} INFO - Dependencies all met for dep_context=non-requeueable deps ti=\n[2023-01-17T13:42:19.074-0800] {taskinstance.py:1131} INFO - Dependencies all met for dep_context=requeueable deps ti=\n[2023-01-17T13:42:19.074-0800] {taskinstance.py:1330} INFO - resuming after deferral\n[2023-01-17T13:42:19.087-0800] {taskinstance.py:1351} INFO - Executing on 2023-01-17 21:41:38.150776+00:00\n[2023-01-17T13:42:19.091-0800] {standard_task_runner.py:56} INFO - Started process 98604 to run task\n[2023-01-17T13:42:19.097-0800] {standard_task_runner.py:83} INFO - Running: ['airflow', 'tasks', 'run', 'example_time_delta_sensor_async', 'wait', 'manual__2023-01-17T21:41:38.150776+00:00', '--job-id', '518', '--raw', '--subdir', '/Users/dstandish/code/airflow/airflow/example_dags/example_time_delta_sensor_async.py', '--cfg-path', '/var/folders/7_/1xx0hqcs3txd7kqt0ngfdjth0000gn/T/tmpx76emkve', '--no-shut-down-logging']\n[2023-01-17T13:42:19.101-0800] {standard_task_runner.py:84} INFO - Job 518: Subtask wait\n[2023-01-17T13:42:19.185-0800] {task_command.py:417} INFO - Running on host Daniels-MacBook-Pro-2.local\n[2023-01-17T13:42:19.274-0800] {taskinstance.py:1369} INFO - Marking task as SUCCESS. dag_id=example_time_delta_sensor_async, task_id=wait, execution_date=20230117T214138, start_date=20230117T214151, end_date=20230117T214219\n[2023-01-17T13:42:19.309-0800] {local_task_job.py:220} INFO - Task exited with return code 0\n[2023-01-17T13:42:19.339-0800] {taskinstance.py:2648} INFO - 1 downstream tasks scheduled from follow-on schedule check\n[2023-01-17T13:41:53.228-0800] {temporal.py:62} INFO - trigger starting\n[2023-01-17T13:41:53.228-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:41:54.228-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:41:55.229-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:41:56.229-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:41:57.229-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:41:58.230-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:41:59.231-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:42:00.231-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:42:01.232-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:42:02.233-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:42:03.233-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:42:04.233-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:42:05.233-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:42:06.234-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:42:07.235-0800] {temporal.py:71} INFO - sleeping 1 second...\n[2023-01-17T13:42:08.237-0800] {temporal.py:74} INFO - yielding event with payload DateTime(2023, 1, 17, 21, 42, 8, 150776, tzinfo=Timezone('UTC'))\n[2023-01-17T13:42:08.237-0800] {triggerer_job.py:540} INFO - Trigger (ID 4) fired: TriggerEvent\n" + print("\n".join(_interleave_logs(local_logs, remote_logs))) From 3fc9ab89c0719b4abb22d7170d7dfbf423768ee5 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 25 Jan 2023 08:28:15 -0800 Subject: [PATCH 45/96] fixup --- .../elasticsearch/log/es_task_handler.py | 15 ++------------- .../google/cloud/log/gcs_task_handler.py | 2 +- airflow/utils/log/file_task_handler.py | 1 - 3 files changed, 3 insertions(+), 15 deletions(-) diff --git a/airflow/providers/elasticsearch/log/es_task_handler.py b/airflow/providers/elasticsearch/log/es_task_handler.py index 94e7acad485f3..0c521e2c50bb9 100644 --- a/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/airflow/providers/elasticsearch/log/es_task_handler.py @@ -32,7 +32,6 @@ import pendulum from elasticsearch_dsl import Search -from airflow.compat.functools import cached_property from airflow.configuration import conf from airflow.models.dagrun import DagRun from airflow.models.taskinstance import TaskInstance @@ -73,6 +72,8 @@ class ElasticsearchTaskHandler(FileTaskHandler, ExternalLoggingMixin, LoggingMix MAX_LINE_PER_PAGE = 1000 LOG_NAME = "Elasticsearch" + trigger_should_wrap = True + def __init__( self, base_log_folder: str, @@ -122,18 +123,6 @@ def __init__( self.formatter: logging.Formatter self.handler: logging.FileHandler | logging.StreamHandler # type: ignore[assignment] - @cached_property - def trigger_should_wrap(self): - """ - Tells triggerer_job that this handler supports individual triggerer logging. - - Overriding this property is necessary when a handler does not implement _read_remote_logs. - Handlers which stream to the log sink, such as Cloudwatch and ElasticSearch, do not need - to lean on the behavior of FileTaskHandler which reads from all possible sources, so - they short-circuit this behavior by implementing _read directly. - """ - return True - def _render_log_id(self, ti: TaskInstance, try_number: int) -> str: with create_session() as session: dag_run = ti.get_dagrun(session=session) diff --git a/airflow/providers/google/cloud/log/gcs_task_handler.py b/airflow/providers/google/cloud/log/gcs_task_handler.py index 19078d96cc10c..53255d9496d33 100644 --- a/airflow/providers/google/cloud/log/gcs_task_handler.py +++ b/airflow/providers/google/cloud/log/gcs_task_handler.py @@ -62,8 +62,8 @@ class GCSTaskHandler(FileTaskHandler, LoggingMixin): :param project_id: Project ID to read the secrets from. If not passed, the project ID from credentials will be used. """ - trigger_should_wrap = True + trigger_should_wrap = True def __init__( self, diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index a17b61eab4e15..91a2d394e96c6 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -29,7 +29,6 @@ import pendulum -from airflow.compat.functools import cached_property from airflow.configuration import conf from airflow.exceptions import RemovedInAirflow3Warning from airflow.utils.context import Context From a5fac73b56fa2ad811db411a6623919eb923d901 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 25 Jan 2023 08:51:14 -0800 Subject: [PATCH 46/96] fixup --- airflow/providers/microsoft/azure/log/wasb_task_handler.py | 1 - 1 file changed, 1 deletion(-) diff --git a/airflow/providers/microsoft/azure/log/wasb_task_handler.py b/airflow/providers/microsoft/azure/log/wasb_task_handler.py index db44a85d9d41a..560236dbebe96 100644 --- a/airflow/providers/microsoft/azure/log/wasb_task_handler.py +++ b/airflow/providers/microsoft/azure/log/wasb_task_handler.py @@ -38,7 +38,6 @@ class WasbTaskHandler(FileTaskHandler, LoggingMixin): """ trigger_should_wrap = True - trigger_should_queue = True def __init__( self, From a3c8ecff6748a7b32f9c2922a83a92a78956fd6a Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 26 Jan 2023 11:11:41 -0800 Subject: [PATCH 47/96] documentation --- .../logging-monitoring/logging-tasks.rst | 32 ++++++++++--------- 1 file changed, 17 insertions(+), 15 deletions(-) diff --git a/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst b/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst index e5b033565140e..743a14063d31b 100644 --- a/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst +++ b/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst @@ -21,17 +21,14 @@ Logging for Tasks ================= Airflow writes logs for tasks in a way that allows you to see the logs for each task separately in the Airflow UI. -Core Airflow implements writing and serving logs locally. However, you can also write logs to remote -services via community providers, or write your own loggers. - -Below we describe the local task logging, the Apache Airflow Community also releases providers for many +Core Airflow provides an interface FileTaskHandler, which writes task logs to file, and includes a mechanism to serve them from workers while tasks are running. The Apache Airflow Community also releases providers for many services (:doc:`apache-airflow-providers:index`) and some of them provide handlers that extend the logging capability of Apache Airflow. You can see all of these providers in :doc:`apache-airflow-providers:core-extensions/logging`. -Writing logs Locally --------------------- +Configuring logging +------------------- -You can specify the directory to place log files in ``airflow.cfg`` using +For the default handler, FileTaskHandler, you can specify the directory to place log files in ``airflow.cfg`` using ``base_log_folder``. By default, logs are placed in the ``AIRFLOW_HOME`` directory. @@ -47,11 +44,14 @@ These patterns can be adjusted by :ref:`config:logging__log_filename_template`. In addition, you can supply a remote location to store current logs and backups. -In the Airflow UI, remote logs take precedence over local logs when remote logging is enabled. If remote logs -can not be found or accessed, local logs will be displayed. Note that logs -are only sent to remote storage once a task is complete (including failure). In other words, remote logs for -running tasks are unavailable (but local logs are available). +Interleaving of logs +-------------------- + +Airflow's remote task logging handlers can broadly be separated into two categories: streaming handlers (such as ElasticSearch, AWS Cloudwatch, and GCP operations logging, formerly stackdriver) and blob storage handlers (e.g. S3, GCS, WASB). + +For blob storage handlers, depending on the state of the task, logs could be in a lot of different places and in multiple different files. For this reason, we need to check all locations and interleave what we find. To do this we need to be able to parse the timestamp for each line. If you are using a custom formatter you may need to override the default parser by providing a callable name at Airflow setting ``logging > interleave_timestamp_parser``. +For streaming handlers, no matter the task phase or location of execution, all log messages can be sent to the logging service with the same identifier so generally speaking there isn't a need to check multiple sources and interleave. Troubleshooting --------------- @@ -122,15 +122,17 @@ When using remote logging, you can configure Airflow to show a link to an extern Some external systems require specific configuration in Airflow for redirection to work but others do not. -Serving logs from workers -------------------------- +Serving logs from workers and triggerer +--------------------------------------- -Most task handlers send logs upon completion of a task. In order to view logs in real time, Airflow automatically starts an HTTP server to serve the logs in the following cases: +Most task handlers send logs upon completion of a task. In order to view logs in real time, Airflow starts an HTTP server to serve the logs in the following cases: - If ``SequentialExecutor`` or ``LocalExecutor`` is used, then when ``airflow scheduler`` is running. - If ``CeleryExecutor`` is used, then when ``airflow worker`` is running. -The server is running on the port specified by ``worker_log_server_port`` option in ``[logging]`` section. By default, it is ``8793``. +In triggerer, logs are served unless the service is started with option ``--skip-serve-logs``. + +The server is running on the port specified by ``worker_log_server_port`` option in ``[logging]`` section, and option ``triggerer_log_server_port`` for triggerer. Defaults are 8793 and 8794, respectively. Communication between the webserver and the worker is signed with the key specified by ``secret_key`` option in ``[webserver]`` section. You must ensure that the key matches so that communication can take place without problems. We are using `Gunicorn `__ as a WSGI server. Its configuration options can be overridden with the ``GUNICORN_CMD_ARGS`` env variable. For details, see `Gunicorn settings `__. From 53021395ee87cd11fbcd184a5a9954e8c045c6e4 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 26 Jan 2023 11:38:12 -0800 Subject: [PATCH 48/96] docs --- .../logging-monitoring/logging-tasks.rst | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst b/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst index 743a14063d31b..18e57a4119617 100644 --- a/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst +++ b/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst @@ -136,3 +136,19 @@ The server is running on the port specified by ``worker_log_server_port`` option Communication between the webserver and the worker is signed with the key specified by ``secret_key`` option in ``[webserver]`` section. You must ensure that the key matches so that communication can take place without problems. We are using `Gunicorn `__ as a WSGI server. Its configuration options can be overridden with the ``GUNICORN_CMD_ARGS`` env variable. For details, see `Gunicorn settings `__. + +Implementing a custom file task handler +--------------------------------------- + +.. note:: This is an advanced topic and most users should be able to just use an existing handler from :doc:`apache-airflow-providers:core-extensions/logging`. + +In our providers we have a healthy variety of options with all the major cloud providers. But should you need to implement logging with a different service, and should you then decide to implement a custom FileTaskHandler, there are a few settings to be aware of, particularly in the context of trigger logging. + +Triggers require a shift in the way that logging is set up. In contrast with tasks, many triggers run in the same process, and with triggers, since they run in asyncio, we have to be mindful of not introducing blocking calls through the logging handler. And because of the variation in handler behavior (some write to file, some upload to blob storage, some send messages over network as they arrive, some do so in thread), we need to have some way to let triggerer know how to use them. + +To accomplish this we have a few attributes that may be set either on the handler, either instance or the class. Inheritance is not respected for these parameters, because subclasses of FileTaskHandler may differ from it in the relevant characteristics. These params are described below: + +- ``trigger_should_wrap``: controls whether this handler should be wrapped by TriggerHandlerWrapper. This is necessary when each instance of handler creates a file handler that it writes all messages to. +- ``trigger_should_queue``: controls whether the triggerer should put a QueueListener between the event loop and the handler, to ensure blocking IO in the handler does not disrupt the event loop. +- ``trigger_send_end_marker``: whether an END signal should be sent to the logger when trigger completes. It is used to tell the wrapper to close and remove the individual file handler specific to the trigger that just completed. +- ``trigger_supported``: If ``trigger_should_wrap`` and ``trigger_should_queue`` are not True, we generally assume that the handler does not support triggers. But if in this case the handler has ``trigger_supported`` set to True, then we'll still move the handler to root at triggerer start so that it will process trigger messages. Essentially, this should be true for handlers that "natively" support triggers. And one such example of this is the StackdriverTaskHandler. From bf981125c2de6c8a83d72da8e1c4bf9647b93549 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 26 Jan 2023 12:27:59 -0800 Subject: [PATCH 49/96] fix some tests --- .../aws/log/test_cloudwatch_task_handler.py | 5 ++--- .../amazon/aws/log/test_s3_task_handler.py | 15 ++++++++------- .../google/cloud/log/test_gcs_task_handler.py | 9 ++++++--- 3 files changed, 16 insertions(+), 13 deletions(-) diff --git a/tests/providers/amazon/aws/log/test_cloudwatch_task_handler.py b/tests/providers/amazon/aws/log/test_cloudwatch_task_handler.py index 954eaef5ba9ec..43d63eb2d7bdd 100644 --- a/tests/providers/amazon/aws/log/test_cloudwatch_task_handler.py +++ b/tests/providers/amazon/aws/log/test_cloudwatch_task_handler.py @@ -164,11 +164,10 @@ def test_should_read_from_local_on_failure_to_fetch_remote_logs(self): f"*** Unable to read remote logs from Cloudwatch (log_group: {self.remote_log_group}, " f"log_stream: {self.remote_log_stream})\n*** Failed to connect\n\n" # The value of "log_pos" is equal to the length of this next line - f"*** Reading local file: {self.local_log_location}/{self.remote_log_stream}\n" + f"*** Found local files:\n*** * {self.local_log_location}/{self.remote_log_stream}\n" ) assert log == expected_log - expected_log_pos = 26 + len(self.local_log_location) + len(self.remote_log_stream) - assert metadata == {"end_of_log": False, "log_pos": expected_log_pos} + assert metadata == {"end_of_log": False, "log_pos": 0} mock_get_logs.assert_called_once_with(stream_name=self.remote_log_stream) def test_close_prevents_duplicate_calls(self): diff --git a/tests/providers/amazon/aws/log/test_s3_task_handler.py b/tests/providers/amazon/aws/log/test_s3_task_handler.py index b49e0dd7a27d2..8ccbf2a54edde 100644 --- a/tests/providers/amazon/aws/log/test_s3_task_handler.py +++ b/tests/providers/amazon/aws/log/test_s3_task_handler.py @@ -127,19 +127,20 @@ def test_set_context_not_raw(self): def test_read(self): self.conn.put_object(Bucket="bucket", Key=self.remote_log_key, Body=b"Log line\n") log, metadata = self.s3_task_handler.read(self.ti) - assert ( - log[0][0][-1] - == "*** Reading remote log from s3://bucket/remote/log/location/1.log.\nLog line\n\n" - ) - assert metadata == [{"end_of_log": True}] + actual = log[0][0][-1] + expected = "*** Found logs in s3:\n*** * s3://bucket/remote/log/location/1.log\nLog line" + assert actual == expected + assert metadata == [{"end_of_log": False, "log_pos": 8}] def test_read_when_s3_log_missing(self): log, metadata = self.s3_task_handler.read(self.ti) assert 1 == len(log) assert len(log) == len(metadata) - assert "*** Logs not found locally" in log[0][0][-1] - assert {"end_of_log": True} == metadata[0] + actual = log[0][0][-1] + expected = "*** No logs found on s3 for ti=\n" # noqa: E501 + assert actual == expected + assert {"end_of_log": False, "log_pos": 0} == metadata[0] def test_s3_read_when_log_missing(self): handler = self.s3_task_handler diff --git a/tests/providers/google/cloud/log/test_gcs_task_handler.py b/tests/providers/google/cloud/log/test_gcs_task_handler.py index b801d1fc0544b..8205fea1fbe34 100644 --- a/tests/providers/google/cloud/log/test_gcs_task_handler.py +++ b/tests/providers/google/cloud/log/test_gcs_task_handler.py @@ -19,6 +19,7 @@ import logging import tempfile from unittest import mock +from unittest.mock import MagicMock import pytest from pytest import param @@ -90,15 +91,17 @@ def test_client_conn_id_behavior(self, mock_get_cred, mock_client, mock_hook, co @mock.patch("google.cloud.storage.Client") @mock.patch("google.cloud.storage.Blob") def test_should_read_logs_from_remote(self, mock_blob, mock_client, mock_creds): + mock_obj = MagicMock() + mock_obj.name = "remote/log/location/1.log" + mock_client.return_value.list_blobs.return_value = [mock_obj] mock_blob.from_string.return_value.download_as_bytes.return_value = b"CONTENT" logs, metadata = self.gcs_task_handler._read(self.ti, self.ti.try_number) mock_blob.from_string.assert_called_once_with( "gs://bucket/remote/log/location/1.log", mock_client.return_value ) - - assert "*** Reading remote log from gs://bucket/remote/log/location/1.log.\nCONTENT\n" == logs - assert {"end_of_log": True} == metadata + assert "*** Found remote logs:\n*** * gs://bucket/remote/log/location/1.log\nCONTENT" == logs + assert {"end_of_log": False, "log_pos": 7} == metadata @mock.patch( "airflow.providers.google.cloud.log.gcs_task_handler.get_credentials_and_project_id", From 1f3cca229bcf7de0d378b8cb93bffd38d9a78520 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 27 Jan 2023 10:16:19 -0800 Subject: [PATCH 50/96] fix tests --- airflow/models/taskinstance.py | 2 +- .../cloud/log/stackdriver_task_handler.py | 6 +++--- airflow/utils/log/file_task_handler.py | 16 +++++++------- .../aws/log/test_cloudwatch_task_handler.py | 8 +++---- .../google/cloud/log/test_gcs_task_handler.py | 21 ++++++++++++------- .../azure/log/test_wasb_task_handler.py | 19 ++++++++++------- 6 files changed, 42 insertions(+), 30 deletions(-) diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index f3a44ee1817d7..7180b5540e67c 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -629,7 +629,7 @@ def command_as_list( installed. This command is part of the message sent to executors by the orchestrator. """ - dag: DAG | DagModel + dag: DAG | DagModel | None # Use the dag if we have it, else fallback to the ORM dag_model, which might not be loaded if hasattr(self, "task") and hasattr(self.task, "dag"): dag = self.task.dag diff --git a/airflow/providers/google/cloud/log/stackdriver_task_handler.py b/airflow/providers/google/cloud/log/stackdriver_task_handler.py index 0374a2dedcfe9..5b0b5d53b735b 100644 --- a/airflow/providers/google/cloud/log/stackdriver_task_handler.py +++ b/airflow/providers/google/cloud/log/stackdriver_task_handler.py @@ -309,10 +309,10 @@ def _read_single_logs_page(self, log_filter: str, page_token: str | None = None) page: ListLogEntriesResponse = next(response.pages) messages = [] for entry in page.entries: - if entry.text_payload: - messages.append(entry.text_payload) - elif "message" in entry.json_payload or {}: + if "message" in entry.json_payload or {}: messages.append(entry.json_payload["message"]) + elif entry.text_payload: + messages.append(entry.text_payload) return "\n".join(messages), page.next_page_token @classmethod diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index 62e7e56c85f41..ac4e9ff5cffe8 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -147,7 +147,7 @@ class FileTaskHandler(logging.Handler): def __init__(self, base_log_folder: str, filename_template: str | None = None): super().__init__() self.handler: logging.FileHandler | None = None - self.local_base = base_log_folder + self.local_base = Path(base_log_folder).resolve() if filename_template is not None: warnings.warn( "Passing filename_template to a log handler is deprecated and has no effect", @@ -293,14 +293,14 @@ def _read( # Task instance here might be different from task instance when # initializing the handler. Thus explicitly getting log location # is needed to get correct log path. - messages_list = [] worker_log_rel_path = self._render_filename(ti, try_number) - remote_logs = [] - running_logs = [] - local_logs = [] - executor_messages = [] - executor_logs = [] - served_logs = [] + messages_list: list[str] = [] + remote_logs: list[str] = [] + running_logs: list[str] = [] + local_logs: list[str] = [] + executor_messages: list[str] = [] + executor_logs: list[str] = [] + served_logs: list[str] = [] with suppress(NotImplementedError): remote_messages, remote_logs = self._read_remote_logs(ti, try_number, metadata) messages_list.extend(remote_messages) diff --git a/tests/providers/amazon/aws/log/test_cloudwatch_task_handler.py b/tests/providers/amazon/aws/log/test_cloudwatch_task_handler.py index 43d63eb2d7bdd..b962691d024f1 100644 --- a/tests/providers/amazon/aws/log/test_cloudwatch_task_handler.py +++ b/tests/providers/amazon/aws/log/test_cloudwatch_task_handler.py @@ -161,10 +161,10 @@ def test_should_read_from_local_on_failure_to_fetch_remote_logs(self): mock_get_logs.side_effect = Exception("Failed to connect") log, metadata = self.cloudwatch_task_handler._read(self.ti, self.ti.try_number) expected_log = ( - f"*** Unable to read remote logs from Cloudwatch (log_group: {self.remote_log_group}, " - f"log_stream: {self.remote_log_stream})\n*** Failed to connect\n\n" - # The value of "log_pos" is equal to the length of this next line - f"*** Found local files:\n*** * {self.local_log_location}/{self.remote_log_stream}\n" + f"*** Unable to read remote logs from Cloudwatch (log_group: {self.remote_log_group}, log_stream: {self.remote_log_stream})\n" # noqa: E501 + "*** Failed to connect\n\n" + "*** Found local files:\n" + f"*** * {self.local_log_location}/{self.remote_log_stream}\n" ) assert log == expected_log assert metadata == {"end_of_log": False, "log_pos": 0} diff --git a/tests/providers/google/cloud/log/test_gcs_task_handler.py b/tests/providers/google/cloud/log/test_gcs_task_handler.py index 8205fea1fbe34..d011387491f3b 100644 --- a/tests/providers/google/cloud/log/test_gcs_task_handler.py +++ b/tests/providers/google/cloud/log/test_gcs_task_handler.py @@ -100,7 +100,7 @@ def test_should_read_logs_from_remote(self, mock_blob, mock_client, mock_creds): mock_blob.from_string.assert_called_once_with( "gs://bucket/remote/log/location/1.log", mock_client.return_value ) - assert "*** Found remote logs:\n*** * gs://bucket/remote/log/location/1.log\nCONTENT" == logs + assert logs == "*** Found remote logs:\n*** * gs://bucket/remote/log/location/1.log\nCONTENT" assert {"end_of_log": False, "log_pos": 7} == metadata @mock.patch( @@ -109,17 +109,23 @@ def test_should_read_logs_from_remote(self, mock_blob, mock_client, mock_creds): ) @mock.patch("google.cloud.storage.Client") @mock.patch("google.cloud.storage.Blob") - def test_should_read_from_local(self, mock_blob, mock_client, mock_creds): + def test_should_read_from_local_on_logs_read_error(self, mock_blob, mock_client, mock_creds): + mock_obj = MagicMock() + mock_obj.name = "remote/log/location/1.log" + mock_client.return_value.list_blobs.return_value = [mock_obj] mock_blob.from_string.return_value.download_as_bytes.side_effect = Exception("Failed to connect") self.gcs_task_handler.set_context(self.ti) log, metadata = self.gcs_task_handler._read(self.ti, self.ti.try_number) - assert ( - log == "*** Unable to read remote log from gs://bucket/remote/log/location/1.log\n*** " - f"Failed to connect\n\n*** Reading local file: {self.local_log_location}/1.log\n" + assert log == ( + "*** Found remote logs:\n" + "*** * gs://bucket/remote/log/location/1.log\n" + "*** Unable to read remote log Failed to connect\n" + "*** Found local files:\n" + f"*** * {self.local_log_location}/1.log\n" ) - assert metadata == {"end_of_log": False, "log_pos": 31 + len(self.local_log_location)} + assert metadata == {"end_of_log": False, "log_pos": 0} mock_blob.from_string.assert_called_once_with( "gs://bucket/remote/log/location/1.log", mock_client.return_value ) @@ -230,7 +236,8 @@ def test_write_to_remote_on_close_failed_read_old_logs(self, mock_blob, mock_cli mock.call.from_string().download_as_bytes(), mock.call.from_string("gs://bucket/remote/log/location/1.log", mock_client.return_value), mock.call.from_string().upload_from_string( - "*** Previous log discarded: Fail to download\n\nMESSAGE\n", content_type="text/plain" + "MESSAGE\nError checking for previous log; if exists, may be overwritten: Fail to download\n", # noqa: E501 + content_type="text/plain", ), ], any_order=False, diff --git a/tests/providers/microsoft/azure/log/test_wasb_task_handler.py b/tests/providers/microsoft/azure/log/test_wasb_task_handler.py index 3f29ea0c86d8b..ef2c7d5df77e4 100644 --- a/tests/providers/microsoft/azure/log/test_wasb_task_handler.py +++ b/tests/providers/microsoft/azure/log/test_wasb_task_handler.py @@ -17,6 +17,7 @@ from __future__ import annotations from unittest import mock +from unittest.mock import MagicMock import pytest from azure.common import AzureHttpError @@ -81,11 +82,11 @@ def test_hook_warns(self): def test_set_context_raw(self, ti): ti.raw = True self.wasb_task_handler.set_context(ti) - assert not self.wasb_task_handler.upload_on_close + assert self.wasb_task_handler.upload_on_close is False def test_set_context_not_raw(self, ti): self.wasb_task_handler.set_context(ti) - assert self.wasb_task_handler.upload_on_close + assert self.wasb_task_handler.upload_on_close is True @mock.patch("airflow.providers.microsoft.azure.hooks.wasb.WasbHook") def test_wasb_log_exists(self, mock_hook): @@ -97,20 +98,24 @@ def test_wasb_log_exists(self, mock_hook): ) @mock.patch("airflow.providers.microsoft.azure.hooks.wasb.WasbHook") - def test_wasb_read(self, mock_hook, ti): - mock_hook.return_value.read_file.return_value = "Log line" + def test_wasb_read(self, mock_hook_cls, ti): + mock_hook = mock_hook_cls.return_value + m = MagicMock(name="hello") + mock_hook.get_blobs_list.return_value = [m] + mock_hook.read_file.return_value = "Log line" assert self.wasb_task_handler.wasb_read(self.remote_log_location) == "Log line" assert self.wasb_task_handler.read(ti) == ( [ [ ( "localhost", - "*** Reading remote log from wasb://container/remote/log/location/1.log.\n" - "Log line\n", + "*** Found remote logs:\n" + "*** * wasb://wasb-container/\n" + "Log line", ) ] ], - [{"end_of_log": True}], + [{"end_of_log": False, "log_pos": 8}], ) @mock.patch( From 47ce1717aab2b2303523762c0d1f100f724f81ee Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 27 Jan 2023 11:03:16 -0800 Subject: [PATCH 51/96] fix static checks --- airflow/models/taskinstance.py | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index 7180b5540e67c..4aafb75054df1 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -30,6 +30,7 @@ from datetime import datetime, timedelta from enum import Enum from functools import partial +from pathlib import PurePath from types import TracebackType from typing import TYPE_CHECKING, Any, Callable, Collection, Generator, Iterable, NamedTuple, Tuple from urllib.parse import quote @@ -629,17 +630,19 @@ def command_as_list( installed. This command is part of the message sent to executors by the orchestrator. """ - dag: DAG | DagModel | None + dag: DAG | DagModel # Use the dag if we have it, else fallback to the ORM dag_model, which might not be loaded - if hasattr(self, "task") and hasattr(self.task, "dag"): + if hasattr(self, "task") and hasattr(self.task, "dag") and self.task.dag is not None: dag = self.task.dag else: dag = self.dag_model should_pass_filepath = not pickle_id and dag - path = None + path: PurePath | None = None if should_pass_filepath: if dag.is_subdag: + if TYPE_CHECKING: + assert dag.parent_dag is not None path = dag.parent_dag.relative_fileloc else: path = dag.relative_fileloc @@ -647,7 +650,6 @@ def command_as_list( if path: if not path.is_absolute(): path = "DAGS_FOLDER" / path - path = str(path) return TaskInstance.generate_command( self.dag_id, @@ -661,7 +663,7 @@ def command_as_list( ignore_ti_state=ignore_ti_state, local=local, pickle_id=pickle_id, - file_path=path, + file_path=str(path) if path else None, raw=raw, job_id=job_id, pool=pool, @@ -1458,7 +1460,7 @@ def _run_raw_task( except AirflowRescheduleException as reschedule_exception: self._handle_reschedule(actual_start_date, reschedule_exception, test_mode, session=session) session.commit() - return + return None except (AirflowFailException, AirflowSensorTimeout) as e: # If AirflowFailException is raised, task should not retry. # If a sensor in reschedule mode reaches timeout, task should not retry. @@ -1475,7 +1477,7 @@ def _run_raw_task( self.clear_next_method_args() session.merge(self) session.commit() - return + return None else: self.handle_failure(e, test_mode, context, session=session) session.commit() @@ -1504,6 +1506,7 @@ def _run_raw_task( if self.state == TaskInstanceState.SUCCESS: self._register_dataset_changes(session=session) session.commit() + return None def _register_dataset_changes(self, *, session: Session) -> None: for obj in self.task.outlets or []: From 8598ade074113bcab1cef6ce8997a45d6b43b6f4 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 27 Jan 2023 12:04:41 -0800 Subject: [PATCH 52/96] fix static checks --- airflow/cli/commands/task_command.py | 1 + airflow/executors/celery_kubernetes_executor.py | 6 +++--- airflow/executors/local_kubernetes_executor.py | 7 +++---- airflow/jobs/local_task_job.py | 2 +- airflow/providers/microsoft/azure/log/wasb_task_handler.py | 5 ++++- 5 files changed, 12 insertions(+), 9 deletions(-) diff --git a/airflow/cli/commands/task_command.py b/airflow/cli/commands/task_command.py index 151e45987e828..d63e772176e45 100644 --- a/airflow/cli/commands/task_command.py +++ b/airflow/cli/commands/task_command.py @@ -264,6 +264,7 @@ def _run_task_by_local_task_job(args, ti) -> TaskReturnCode | None: logging.shutdown() with suppress(ValueError): return TaskReturnCode(ret) + return None RAW_TASK_UNSUPPORTED_OPTION = [ diff --git a/airflow/executors/celery_kubernetes_executor.py b/airflow/executors/celery_kubernetes_executor.py index 8426fb526fe35..25c9cff1787c5 100644 --- a/airflow/executors/celery_kubernetes_executor.py +++ b/airflow/executors/celery_kubernetes_executor.py @@ -141,11 +141,11 @@ def queue_task_instance( cfg_path=cfg_path, ) - def get_task_log(self, ti: TaskInstance, log: str = "") -> None | str | tuple[str, dict[str, bool]]: + def get_task_log(self, ti: TaskInstance) -> tuple[list[str], list[str]]: """Fetch task log from Kubernetes executor""" if ti.queue == self.kubernetes_executor.kubernetes_queue: - return self.kubernetes_executor.get_task_log(ti=ti, log=log) - return None + return self.kubernetes_executor.get_task_log(ti=ti) + return [], [] def has_task(self, task_instance: TaskInstance) -> bool: """ diff --git a/airflow/executors/local_kubernetes_executor.py b/airflow/executors/local_kubernetes_executor.py index 258135f31c46b..8de95ef29b254 100644 --- a/airflow/executors/local_kubernetes_executor.py +++ b/airflow/executors/local_kubernetes_executor.py @@ -142,12 +142,11 @@ def queue_task_instance( cfg_path=cfg_path, ) - def get_task_log(self, ti: TaskInstance, log: str = "") -> None | str | tuple[str, dict[str, bool]]: + def get_task_log(self, ti: TaskInstance) -> tuple[list[str], list[str]]: """Fetch task log from kubernetes executor""" if ti.queue == self.kubernetes_executor.kubernetes_queue: - return self.kubernetes_executor.get_task_log(ti=ti, log=log) - - return None + return self.kubernetes_executor.get_task_log(ti=ti) + return [], [] def has_task(self, task_instance: TaskInstance) -> bool: """ diff --git a/airflow/jobs/local_task_job.py b/airflow/jobs/local_task_job.py index c7107a0dadca3..15606f71aa73b 100644 --- a/airflow/jobs/local_task_job.py +++ b/airflow/jobs/local_task_job.py @@ -155,7 +155,7 @@ def sigusr2_debug_handler(signum, frame): external_executor_id=self.external_executor_id, ): self.log.info("Task is not able to be run") - return + return None return_code = None try: diff --git a/airflow/providers/microsoft/azure/log/wasb_task_handler.py b/airflow/providers/microsoft/azure/log/wasb_task_handler.py index 560236dbebe96..52af3171c2d6c 100644 --- a/airflow/providers/microsoft/azure/log/wasb_task_handler.py +++ b/airflow/providers/microsoft/azure/log/wasb_task_handler.py @@ -20,7 +20,7 @@ import os import shutil from pathlib import Path -from typing import Any +from typing import TYPE_CHECKING, Any from azure.core.exceptions import HttpResponseError @@ -79,6 +79,9 @@ def set_context(self, ti) -> None: super().set_context(ti) # Local location and remote location is needed to open and # upload local log file to Wasb remote storage. + if TYPE_CHECKING: + assert self.handler is not None + full_path = self.handler.baseFilename self.log_relative_path = Path(full_path).relative_to(self.local_base).as_posix() is_trigger_log_context = getattr(ti, "is_trigger_log_context", False) From 66f4deff11dba0e2bd2d377f5b1e0a65569eb147 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 27 Jan 2023 12:18:41 -0800 Subject: [PATCH 53/96] provider future compat --- .../google/cloud/log/stackdriver_task_handler.py | 12 ++++++++++-- .../microsoft/azure/log/test_wasb_task_handler.py | 8 ++------ 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/airflow/providers/google/cloud/log/stackdriver_task_handler.py b/airflow/providers/google/cloud/log/stackdriver_task_handler.py index 5b0b5d53b735b..8e72f20826c38 100644 --- a/airflow/providers/google/cloud/log/stackdriver_task_handler.py +++ b/airflow/providers/google/cloud/log/stackdriver_task_handler.py @@ -18,6 +18,7 @@ from __future__ import annotations import logging +from contextvars import ContextVar from typing import Collection from urllib.parse import urlencode @@ -32,7 +33,13 @@ from airflow.models import TaskInstance from airflow.providers.google.cloud.utils.credentials_provider import get_credentials_and_project_id from airflow.providers.google.common.consts import CLIENT_INFO -from airflow.utils.log.trigger_handler import ctx_indiv_trigger + +try: + # todo: remove this conditional import when min airflow version >= 2.6 + ctx_indiv_trigger: ContextVar | None + from airflow.utils.log.trigger_handler import ctx_indiv_trigger +except ImportError: + ctx_indiv_trigger = None DEFAULT_LOGGER_NAME = "airflow" _GLOBAL_RESOURCE = Resource(type="global", labels={}) @@ -164,7 +171,8 @@ def emit(self, record: logging.LogRecord) -> None: """ message = self.format(record) ti = None - if getattr(record, ctx_indiv_trigger.name, None): + # todo: remove ctx_indiv_trigger is not None check when min airflow version >= 2.6 + if ctx_indiv_trigger is not None and getattr(record, ctx_indiv_trigger.name, None): ti = getattr(record, "task_instance", None) # trigger context labels = self._get_labels(ti) self._transport.send(record, message, resource=self.resource, labels=labels) diff --git a/tests/providers/microsoft/azure/log/test_wasb_task_handler.py b/tests/providers/microsoft/azure/log/test_wasb_task_handler.py index ef2c7d5df77e4..9f033f3739972 100644 --- a/tests/providers/microsoft/azure/log/test_wasb_task_handler.py +++ b/tests/providers/microsoft/azure/log/test_wasb_task_handler.py @@ -17,7 +17,6 @@ from __future__ import annotations from unittest import mock -from unittest.mock import MagicMock import pytest from azure.common import AzureHttpError @@ -100,8 +99,7 @@ def test_wasb_log_exists(self, mock_hook): @mock.patch("airflow.providers.microsoft.azure.hooks.wasb.WasbHook") def test_wasb_read(self, mock_hook_cls, ti): mock_hook = mock_hook_cls.return_value - m = MagicMock(name="hello") - mock_hook.get_blobs_list.return_value = [m] + mock_hook.get_blobs_list.return_value = ["abc/hello.log"] mock_hook.read_file.return_value = "Log line" assert self.wasb_task_handler.wasb_read(self.remote_log_location) == "Log line" assert self.wasb_task_handler.read(ti) == ( @@ -109,9 +107,7 @@ def test_wasb_read(self, mock_hook_cls, ti): [ ( "localhost", - "*** Found remote logs:\n" - "*** * wasb://wasb-container/\n" - "Log line", + "*** Found remote logs:\n" "*** * wasb://wasb-container/abc/hello.log\n" "Log line", ) ] ], From ce271f08f471b49211a8c78e4024436284fd1d0c Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 27 Jan 2023 22:01:54 -0800 Subject: [PATCH 54/96] fix tests --- airflow/utils/log/file_task_handler.py | 2 +- .../microsoft/azure/log/test_wasb_task_handler.py | 7 ++++++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index ac4e9ff5cffe8..60d66612a212d 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -147,7 +147,7 @@ class FileTaskHandler(logging.Handler): def __init__(self, base_log_folder: str, filename_template: str | None = None): super().__init__() self.handler: logging.FileHandler | None = None - self.local_base = Path(base_log_folder).resolve() + self.local_base = base_log_folder if filename_template is not None: warnings.warn( "Passing filename_template to a log handler is deprecated and has no effect", diff --git a/tests/providers/microsoft/azure/log/test_wasb_task_handler.py b/tests/providers/microsoft/azure/log/test_wasb_task_handler.py index 9f033f3739972..d371f0810160a 100644 --- a/tests/providers/microsoft/azure/log/test_wasb_task_handler.py +++ b/tests/providers/microsoft/azure/log/test_wasb_task_handler.py @@ -16,6 +16,8 @@ # under the License. from __future__ import annotations +import tempfile +from pathlib import Path from unittest import mock import pytest @@ -53,7 +55,7 @@ def ti(self, create_task_instance, create_log_template): def setup_method(self): self.wasb_log_folder = "wasb://container/remote/log/location" self.remote_log_location = "remote/log/location/1.log" - self.local_log_location = "local/log/location" + self.local_log_location = str(Path(tempfile.tempdir) / "local/log/location") self.container_name = "wasb-container" self.wasb_task_handler = WasbTaskHandler( base_log_folder=self.local_log_location, @@ -62,6 +64,9 @@ def setup_method(self): delete_local_copy=True, ) + def teardown_method(self): + self.wasb_task_handler.close() + @conf_vars({("logging", "remote_log_conn_id"): "wasb_default"}) @mock.patch("airflow.providers.microsoft.azure.hooks.wasb.BlobServiceClient") def test_hook(self, mock_service): From c533c40511ca7d31519b4855dca4dd33bbb9a042 Mon Sep 17 00:00:00 2001 From: Tzu-ping Chung Date: Mon, 30 Jan 2023 12:59:08 +0800 Subject: [PATCH 55/96] Merge SQLAlchemy query options --- airflow/models/trigger.py | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/airflow/models/trigger.py b/airflow/models/trigger.py index 0a4e83440a77a..7fe62bc2968e4 100644 --- a/airflow/models/trigger.py +++ b/airflow/models/trigger.py @@ -89,15 +89,16 @@ def bulk_fetch(cls, ids: Iterable[int], session=None) -> dict[int, Trigger]: Fetches all of the Triggers by ID and returns a dict mapping ID -> Trigger instance """ - return { - obj.id: obj - for obj in session.query(cls) + query = ( + session.query(cls) .filter(cls.id.in_(ids)) - .options(joinedload("task_instance")) - .options(joinedload("task_instance.trigger")) - .options(joinedload("task_instance.trigger.triggerer_job")) - .all() - } + .options( + joinedload("task_instance"), + joinedload("task_instance.trigger"), + joinedload("task_instance.trigger.triggerer_job"), + ) + ) + return {obj.id: obj for obj in query} @classmethod @provide_session From ef6568b865eb4e5b06042261b05694451c470e31 Mon Sep 17 00:00:00 2001 From: Tzu-ping Chung Date: Mon, 30 Jan 2023 13:00:55 +0800 Subject: [PATCH 56/96] Simplify is_trigger_log_context declaration --- airflow/models/taskinstance.py | 26 ++++++-------------------- 1 file changed, 6 insertions(+), 20 deletions(-) diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index 4aafb75054df1..091e6486d3327 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -461,6 +461,12 @@ class TaskInstance(Base, LoggingMixin): note = association_proxy("task_instance_note", "content", creator=_creator_note) task: Operator # Not always set... + is_trigger_log_context: bool = False + """Indicate to FileTaskHandler that logging context should be set up for trigger logging. + + :meta private: + """ + def __init__( self, task: Operator, @@ -525,26 +531,6 @@ def __init__( # can be changed when calling 'run' self.test_mode = False - self._is_trigger_log_context: bool = False - """ - Flag to indicate to FileTaskHandler that logging context should be set up for - trigger logging. - """ - - @property - def is_trigger_log_context(self): - """ - Flag to indicate to FileTaskHandler that logging context should be set up for - trigger logging. - - :meta private: - """ - return getattr(self, "_is_trigger_log_context", False) - - @is_trigger_log_context.setter - def is_trigger_log_context(self, value): - self._is_trigger_log_context = value - @staticmethod def insert_mapping(run_id: str, task: Operator, map_index: int) -> dict[str, Any]: """:meta private:""" From 8f5ccb8a6cc8d3c6f920ff0509d0ffeab953437c Mon Sep 17 00:00:00 2001 From: Tzu-ping Chung Date: Mon, 30 Jan 2023 13:03:11 +0800 Subject: [PATCH 57/96] Simplify typing when Path is involved --- airflow/models/taskinstance.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index 091e6486d3327..ec32903e73ab5 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -649,7 +649,7 @@ def command_as_list( ignore_ti_state=ignore_ti_state, local=local, pickle_id=pickle_id, - file_path=str(path) if path else None, + file_path=path, raw=raw, job_id=job_id, pool=pool, @@ -671,7 +671,7 @@ def generate_command( ignore_ti_state: bool = False, local: bool = False, pickle_id: int | None = None, - file_path: str | None = None, + file_path: PurePath | str | None = None, raw: bool = False, job_id: str | None = None, pool: str | None = None, @@ -729,7 +729,7 @@ def generate_command( if raw: cmd.extend(["--raw"]) if file_path: - cmd.extend(["--subdir", file_path]) + cmd.extend(["--subdir", os.fspath(file_path)]) if cfg_path: cmd.extend(["--cfg-path", cfg_path]) if map_index != -1: From 576645429324b105852df5aba80e1947ffde5082 Mon Sep 17 00:00:00 2001 From: Tzu-ping Chung Date: Mon, 30 Jan 2023 13:04:07 +0800 Subject: [PATCH 58/96] Log message format --- airflow/models/taskinstance.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index ec32903e73ab5..73827e38ec6cd 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -1320,7 +1320,7 @@ def check_and_change_state_before_execution( return False if self.next_kwargs is not None: - self.log.info("resuming after deferral") + self.log.info("Resuming after deferral") else: self.log.info("Starting attempt %s of %s", self.try_number, self.max_tries + 1) self._try_number += 1 From 6c5bcbf081913c90eb82afd97c397388314fc26d Mon Sep 17 00:00:00 2001 From: Tzu-ping Chung Date: Mon, 30 Jan 2023 13:16:20 +0800 Subject: [PATCH 59/96] Fix task command tests --- tests/cli/commands/test_task_command.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/cli/commands/test_task_command.py b/tests/cli/commands/test_task_command.py index 7e10ad21a1c4d..fafcdf4fc8851 100644 --- a/tests/cli/commands/test_task_command.py +++ b/tests/cli/commands/test_task_command.py @@ -266,6 +266,7 @@ def test_run_with_existing_dag_run_id(self, mock_local_job): pickle_id=None, pool=None, external_executor_id=None, + shut_down_logging=True, ) @mock.patch("airflow.cli.commands.task_command.LocalTaskJob") @@ -642,6 +643,7 @@ def test_external_executor_id_present_for_fork_run_task(self, mock_local_job): ignore_ti_state=False, pool=None, external_executor_id="ABCD12345", + shut_down_logging=True, ) @mock.patch("airflow.cli.commands.task_command.LocalTaskJob") @@ -662,6 +664,7 @@ def test_external_executor_id_present_for_process_run_task(self, mock_local_job) ignore_ti_state=False, pool=None, external_executor_id="ABCD12345", + shut_down_logging=True, ) @pytest.mark.parametrize("is_k8s", ["true", ""]) From 7bf84c43ce875356b6176c0101c3b066340fe8c8 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 30 Jan 2023 10:30:09 -0800 Subject: [PATCH 60/96] cache executor method --- airflow/utils/log/file_task_handler.py | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index 60d66612a212d..a5b7cc49e3804 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -24,11 +24,12 @@ from contextlib import suppress from enum import Enum from pathlib import Path -from typing import TYPE_CHECKING, Any, Iterable +from typing import TYPE_CHECKING, Any, Callable, Iterable from urllib.parse import urljoin import pendulum +from airflow.compat.functools import cached_property from airflow.configuration import conf from airflow.exceptions import RemovedInAirflow3Warning from airflow.executors.executor_loader import ExecutorLoader @@ -264,6 +265,12 @@ def _render_filename(self, ti: TaskInstance, try_number: int) -> str: def _read_grouped_logs(self): return False + @cached_property + def _executor_get_task_log(self) -> Callable[[TaskInstance], tuple[list[str], list[str]]]: + """This cached property avoids loading executor repeatedly.""" + executor = ExecutorLoader.get_default_executor() + return executor.get_task_log + def _read( self, ti: TaskInstance, @@ -305,8 +312,7 @@ def _read( remote_messages, remote_logs = self._read_remote_logs(ti, try_number, metadata) messages_list.extend(remote_messages) if ti.state == TaskInstanceState.RUNNING: - executor = ExecutorLoader.get_default_executor() - response = executor.get_task_log(ti=ti) + response = self._executor_get_task_log(ti) if response: executor_messages, executor_logs = response if executor_messages: @@ -484,6 +490,7 @@ def _read_from_logs_server(self, ti, worker_log_rel_path) -> tuple[list[str], li # Check if the resource was properly fetched response.raise_for_status() if response.text: + messages.append(f"Found logs served from host {url}") logs.append(response.text) except Exception as e: messages.append(f"Could not read served logs: {str(e)}") From 23385019ff8ddd61f48f605c6481980b2f0fc94d Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 30 Jan 2023 11:28:14 -0800 Subject: [PATCH 61/96] stackdriver fix --- airflow/providers/google/cloud/log/stackdriver_task_handler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/providers/google/cloud/log/stackdriver_task_handler.py b/airflow/providers/google/cloud/log/stackdriver_task_handler.py index 8e72f20826c38..5190fbad760b5 100644 --- a/airflow/providers/google/cloud/log/stackdriver_task_handler.py +++ b/airflow/providers/google/cloud/log/stackdriver_task_handler.py @@ -317,7 +317,7 @@ def _read_single_logs_page(self, log_filter: str, page_token: str | None = None) page: ListLogEntriesResponse = next(response.pages) messages = [] for entry in page.entries: - if "message" in entry.json_payload or {}: + if "message" in (entry.json_payload or {}): messages.append(entry.json_payload["message"]) elif entry.text_payload: messages.append(entry.text_payload) From fdb5d169ca741b63a3179cf02298710158f75b2b Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 30 Jan 2023 12:35:32 -0800 Subject: [PATCH 62/96] --wip-- [skip ci] --- airflow/cli/commands/task_command.py | 6 +++--- airflow/jobs/local_task_job.py | 2 +- airflow/task/task_runner/standard_task_runner.py | 8 ++++++++ airflow/utils/log/logging_mixin.py | 2 +- 4 files changed, 13 insertions(+), 5 deletions(-) diff --git a/airflow/cli/commands/task_command.py b/airflow/cli/commands/task_command.py index d63e772176e45..8f0778897b2b3 100644 --- a/airflow/cli/commands/task_command.py +++ b/airflow/cli/commands/task_command.py @@ -311,7 +311,7 @@ def _move_task_handlers_to_root(ti: TaskInstance) -> Generator[None, None, None] console_handler = next((h for h in root_logger.handlers if h.name == "console"), None) with LoggerMutationHelper(root_logger), LoggerMutationHelper(ti.log) as task_helper: task_helper.move(root_logger) - if IS_K8S_EXECUTOR_POD: + if IS_K8S_EXECUTOR_POD or True: if console_handler and console_handler not in root_logger.handlers: root_logger.addHandler(console_handler) yield @@ -710,8 +710,8 @@ def move(self, logger, replace=True): :param replace: if True, remove all handlers from target first; otherwise add if not present. """ self.apply(logger, replace=replace) - self.source_logger.propagate = True - self.source_logger.handlers[:] = [] + # self.source_logger.propagate = True + # self.source_logger.handlers[:] = [] def reset(self): self.apply(self.source_logger) diff --git a/airflow/jobs/local_task_job.py b/airflow/jobs/local_task_job.py index 15606f71aa73b..d096a6622cc05 100644 --- a/airflow/jobs/local_task_job.py +++ b/airflow/jobs/local_task_job.py @@ -99,7 +99,7 @@ def __init__( self.pickle_id = pickle_id self.mark_success = mark_success self.external_executor_id = external_executor_id - self.shut_down_logging = shut_down_logging + self.shut_down_logging = True # terminating state is used so that a job don't try to # terminate multiple times diff --git a/airflow/task/task_runner/standard_task_runner.py b/airflow/task/task_runner/standard_task_runner.py index d56778962e825..66ce0d7b9a117 100644 --- a/airflow/task/task_runner/standard_task_runner.py +++ b/airflow/task/task_runner/standard_task_runner.py @@ -20,6 +20,7 @@ import logging import os +from pathlib import Path import psutil from setproctitle import setproctitle @@ -80,6 +81,13 @@ def _start_by_fork(self): # We prefer the job_id passed on the command-line because at this time, the # task instance may not have been updated. job_id = getattr(args, "job_id", self._task_instance.job_id) + log_handlers = self.log.handlers + root_handlers = logging.getLogger().handlers + message = "" + message += f"log handlers: {log_handlers}\n" + message += f"root handlers: {root_handlers}\n" + Path("~/airflow/debug_file.txt").expanduser().write_text(message) + self.log.warning(message) self.log.info("Running: %s", self._command) self.log.info("Job %s: Subtask %s", job_id, self._task_instance.task_id) diff --git a/airflow/utils/log/logging_mixin.py b/airflow/utils/log/logging_mixin.py index 79746c8492396..25c58cff68eb9 100644 --- a/airflow/utils/log/logging_mixin.py +++ b/airflow/utils/log/logging_mixin.py @@ -203,7 +203,7 @@ def __init__(self, stream): @property def stream(self): """Returns current stream.""" - if IS_K8S_EXECUTOR_POD: + if IS_K8S_EXECUTOR_POD or True: return self._orig_stream if self._use_stderr: return sys.stderr From 376c3280c8f90faf6748332d2733b2efa06e8318 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 30 Jan 2023 12:35:42 -0800 Subject: [PATCH 63/96] Revert "--wip-- [skip ci]" This reverts commit fdb5d169ca741b63a3179cf02298710158f75b2b. --- airflow/cli/commands/task_command.py | 6 +++--- airflow/jobs/local_task_job.py | 2 +- airflow/task/task_runner/standard_task_runner.py | 8 -------- airflow/utils/log/logging_mixin.py | 2 +- 4 files changed, 5 insertions(+), 13 deletions(-) diff --git a/airflow/cli/commands/task_command.py b/airflow/cli/commands/task_command.py index 8f0778897b2b3..d63e772176e45 100644 --- a/airflow/cli/commands/task_command.py +++ b/airflow/cli/commands/task_command.py @@ -311,7 +311,7 @@ def _move_task_handlers_to_root(ti: TaskInstance) -> Generator[None, None, None] console_handler = next((h for h in root_logger.handlers if h.name == "console"), None) with LoggerMutationHelper(root_logger), LoggerMutationHelper(ti.log) as task_helper: task_helper.move(root_logger) - if IS_K8S_EXECUTOR_POD or True: + if IS_K8S_EXECUTOR_POD: if console_handler and console_handler not in root_logger.handlers: root_logger.addHandler(console_handler) yield @@ -710,8 +710,8 @@ def move(self, logger, replace=True): :param replace: if True, remove all handlers from target first; otherwise add if not present. """ self.apply(logger, replace=replace) - # self.source_logger.propagate = True - # self.source_logger.handlers[:] = [] + self.source_logger.propagate = True + self.source_logger.handlers[:] = [] def reset(self): self.apply(self.source_logger) diff --git a/airflow/jobs/local_task_job.py b/airflow/jobs/local_task_job.py index d096a6622cc05..15606f71aa73b 100644 --- a/airflow/jobs/local_task_job.py +++ b/airflow/jobs/local_task_job.py @@ -99,7 +99,7 @@ def __init__( self.pickle_id = pickle_id self.mark_success = mark_success self.external_executor_id = external_executor_id - self.shut_down_logging = True + self.shut_down_logging = shut_down_logging # terminating state is used so that a job don't try to # terminate multiple times diff --git a/airflow/task/task_runner/standard_task_runner.py b/airflow/task/task_runner/standard_task_runner.py index 66ce0d7b9a117..d56778962e825 100644 --- a/airflow/task/task_runner/standard_task_runner.py +++ b/airflow/task/task_runner/standard_task_runner.py @@ -20,7 +20,6 @@ import logging import os -from pathlib import Path import psutil from setproctitle import setproctitle @@ -81,13 +80,6 @@ def _start_by_fork(self): # We prefer the job_id passed on the command-line because at this time, the # task instance may not have been updated. job_id = getattr(args, "job_id", self._task_instance.job_id) - log_handlers = self.log.handlers - root_handlers = logging.getLogger().handlers - message = "" - message += f"log handlers: {log_handlers}\n" - message += f"root handlers: {root_handlers}\n" - Path("~/airflow/debug_file.txt").expanduser().write_text(message) - self.log.warning(message) self.log.info("Running: %s", self._command) self.log.info("Job %s: Subtask %s", job_id, self._task_instance.task_id) diff --git a/airflow/utils/log/logging_mixin.py b/airflow/utils/log/logging_mixin.py index 25c58cff68eb9..79746c8492396 100644 --- a/airflow/utils/log/logging_mixin.py +++ b/airflow/utils/log/logging_mixin.py @@ -203,7 +203,7 @@ def __init__(self, stream): @property def stream(self): """Returns current stream.""" - if IS_K8S_EXECUTOR_POD or True: + if IS_K8S_EXECUTOR_POD: return self._orig_stream if self._use_stderr: return sys.stderr From c6c3ed96e413206a489d57656a474ed54b9d6539 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 30 Jan 2023 12:35:50 -0800 Subject: [PATCH 64/96] Revert "forward shut down logs arg to task runner" This reverts commit 3bf09e4aced80f7fe26b8a90fbb3399fde84a5c0. --- airflow/cli/commands/task_command.py | 1 - airflow/jobs/local_task_job.py | 2 -- airflow/models/taskinstance.py | 6 ------ airflow/task/task_runner/base_task_runner.py | 1 - airflow/task/task_runner/standard_task_runner.py | 3 +-- 5 files changed, 1 insertion(+), 12 deletions(-) diff --git a/airflow/cli/commands/task_command.py b/airflow/cli/commands/task_command.py index d63e772176e45..4ac228974cb80 100644 --- a/airflow/cli/commands/task_command.py +++ b/airflow/cli/commands/task_command.py @@ -254,7 +254,6 @@ def _run_task_by_local_task_job(args, ti) -> TaskReturnCode | None: ignore_ti_state=args.force, pool=args.pool, external_executor_id=_extract_external_executor_id(args), - shut_down_logging=args.shut_down_logging, ) try: ret = run_job.run() diff --git a/airflow/jobs/local_task_job.py b/airflow/jobs/local_task_job.py index 15606f71aa73b..2d3516f26b78a 100644 --- a/airflow/jobs/local_task_job.py +++ b/airflow/jobs/local_task_job.py @@ -84,7 +84,6 @@ def __init__( pickle_id: str | None = None, pool: str | None = None, external_executor_id: str | None = None, - shut_down_logging: bool | None = None, *args, **kwargs, ): @@ -99,7 +98,6 @@ def __init__( self.pickle_id = pickle_id self.mark_success = mark_success self.external_executor_id = external_executor_id - self.shut_down_logging = shut_down_logging # terminating state is used so that a job don't try to # terminate multiple times diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index 73827e38ec6cd..fbfc4aa9befe4 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -609,7 +609,6 @@ def command_as_list( job_id=None, pool=None, cfg_path=None, - shut_down_logging: bool | None = None, ): """ Returns a command that can be executed anywhere where airflow is @@ -655,7 +654,6 @@ def command_as_list( pool=pool, cfg_path=cfg_path, map_index=self.map_index, - shut_down_logging=shut_down_logging, ) @staticmethod @@ -677,7 +675,6 @@ def generate_command( pool: str | None = None, cfg_path: str | None = None, map_index: int = -1, - shut_down_logging: bool | None = None, ) -> list[str]: """ Generates the shell command required to execute this task instance. @@ -702,7 +699,6 @@ def generate_command( :param job_id: job ID (needs more details) :param pool: the Airflow pool that the task should run in :param cfg_path: the Path to the configuration file - :param shut_down_logging: whether logging.shutdown() should be called :return: shell command that can be used to run the task instance """ cmd = ["airflow", "tasks", "run", dag_id, task_id, run_id] @@ -734,8 +730,6 @@ def generate_command( cmd.extend(["--cfg-path", cfg_path]) if map_index != -1: cmd.extend(["--map-index", str(map_index)]) - if shut_down_logging is False: - cmd.extend(["--no-shut-down-logging"]) return cmd @property diff --git a/airflow/task/task_runner/base_task_runner.py b/airflow/task/task_runner/base_task_runner.py index 64aec85105c5a..095c9b7300ff8 100644 --- a/airflow/task/task_runner/base_task_runner.py +++ b/airflow/task/task_runner/base_task_runner.py @@ -99,7 +99,6 @@ def __init__(self, local_task_job): job_id=local_task_job.id, pool=local_task_job.pool, cfg_path=cfg_path, - shut_down_logging=local_task_job.shut_down_logging, ) self.process = None diff --git a/airflow/task/task_runner/standard_task_runner.py b/airflow/task/task_runner/standard_task_runner.py index d56778962e825..a06f2eb82de92 100644 --- a/airflow/task/task_runner/standard_task_runner.py +++ b/airflow/task/task_runner/standard_task_runner.py @@ -121,8 +121,7 @@ def _start_by_fork(self): try: # Explicitly flush any pending exception to Sentry and logging if enabled Sentry.flush() - if args.shut_down_logging: - logging.shutdown() + logging.shutdown() except BaseException: # also make sure to silently ignore ALL POSSIBLE exceptions thrown in the flush/shutdown, # otherwise os._exit() might never be called. We could have used `except:` but From f931a4f5a0f15a06d24ed42788d7f3b3e46b028e Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Tue, 31 Jan 2023 09:45:33 -0800 Subject: [PATCH 65/96] fix tests --- tests/api_connexion/endpoints/test_log_endpoint.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/api_connexion/endpoints/test_log_endpoint.py b/tests/api_connexion/endpoints/test_log_endpoint.py index 023b62b48415d..422830eb6ed53 100644 --- a/tests/api_connexion/endpoints/test_log_endpoint.py +++ b/tests/api_connexion/endpoints/test_log_endpoint.py @@ -166,10 +166,10 @@ def test_should_respond_200_json(self): ) assert ( response.json["content"] - == f"[('localhost', '*** Reading local file: {expected_filename}\\nLog for testing.')]" + == f"[('localhost', '*** Found local files:\\n*** * {expected_filename}\\nLog for testing.')]" ) info = serializer.loads(response.json["continuation_token"]) - assert info == {"end_of_log": True, "log_pos": 41 + len(expected_filename)} + assert info == {"end_of_log": True, "log_pos": 16} assert 200 == response.status_code @pytest.mark.parametrize( @@ -203,7 +203,7 @@ def test_should_respond_200_text_plain(self, request_url, expected_filename, ext assert 200 == response.status_code assert ( response.data.decode("utf-8") - == f"localhost\n*** Reading local file: {expected_filename}\nLog for testing.\n" + == f"localhost\n*** Found local files:\n*** * {expected_filename}\nLog for testing.\n" ) @pytest.mark.parametrize( @@ -244,7 +244,7 @@ def test_get_logs_of_removed_task(self, request_url, expected_filename, extra_qu assert 200 == response.status_code assert ( response.data.decode("utf-8") - == f"localhost\n*** Reading local file: {expected_filename}\nLog for testing.\n" + == f"localhost\n*** Found local files:\n*** * {expected_filename}\nLog for testing.\n" ) def test_get_logs_response_with_ti_equal_to_none(self): From 7102ace5a6c3ad84cc8bc054512d20eb2ce95313 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Tue, 31 Jan 2023 11:40:22 -0800 Subject: [PATCH 66/96] fix tests --- tests/executors/test_base_executor.py | 2 +- .../test_celery_kubernetes_executor.py | 6 +- tests/executors/test_kubernetes_executor.py | 2 +- tests/jobs/test_triggerer_job.py | 8 +-- tests/utils/test_log_handlers.py | 63 ++++++++++++++----- 5 files changed, 55 insertions(+), 26 deletions(-) diff --git a/tests/executors/test_base_executor.py b/tests/executors/test_base_executor.py index 30ddaaacc4867..dc0bbf7f8213e 100644 --- a/tests/executors/test_base_executor.py +++ b/tests/executors/test_base_executor.py @@ -47,7 +47,7 @@ def test_is_local_default_value(): def test_get_task_log(): executor = BaseExecutor() ti = TaskInstance(task=BaseOperator(task_id="dummy")) - assert executor.get_task_log(ti=ti) is None + assert executor.get_task_log(ti=ti) == ([], []) def test_serve_logs_default_value(): diff --git a/tests/executors/test_celery_kubernetes_executor.py b/tests/executors/test_celery_kubernetes_executor.py index 89ccfada2fa5d..6778bad6eb6ec 100644 --- a/tests/executors/test_celery_kubernetes_executor.py +++ b/tests/executors/test_celery_kubernetes_executor.py @@ -179,15 +179,15 @@ def test_log_is_fetched_from_k8s_executor_only_for_k8s_queue(self): cke = CeleryKubernetesExecutor(celery_executor_mock, k8s_executor_mock) simple_task_instance = mock.MagicMock() simple_task_instance.queue = KUBERNETES_QUEUE - cke.get_task_log(ti=simple_task_instance, log="") + cke.get_task_log(ti=simple_task_instance) k8s_executor_mock.get_task_log.assert_called_once_with(ti=simple_task_instance, log=mock.ANY) k8s_executor_mock.reset_mock() simple_task_instance.queue = "test-queue" - log = cke.get_task_log(ti=simple_task_instance, log="") + log = cke.get_task_log(ti=simple_task_instance) k8s_executor_mock.get_task_log.assert_not_called() - assert log is None + assert log == ([], []) def test_get_event_buffer(self): celery_executor_mock = mock.MagicMock() diff --git a/tests/executors/test_kubernetes_executor.py b/tests/executors/test_kubernetes_executor.py index f040eb8d07e5f..5abc531b5a577 100644 --- a/tests/executors/test_kubernetes_executor.py +++ b/tests/executors/test_kubernetes_executor.py @@ -1231,7 +1231,7 @@ def test_get_task_log(self, mock_get_kube_client, create_task_instance_of_operat ti = create_task_instance_of_operator(EmptyOperator, dag_id="test_k8s_log_dag", task_id="test_task") executor = KubernetesExecutor() - log = executor.get_task_log(ti=ti, log="test_init_log") + log = executor.get_task_log(ti=ti) mock_kube_client.read_namespaced_pod_log.assert_called_once() assert "test_init_log" in log diff --git a/tests/jobs/test_triggerer_job.py b/tests/jobs/test_triggerer_job.py index 61c7556f3a3d5..f560caeed1681 100644 --- a/tests/jobs/test_triggerer_job.py +++ b/tests/jobs/test_triggerer_job.py @@ -462,14 +462,14 @@ def test_invalid_trigger(session, dag_maker): assert task_instance.next_kwargs["traceback"][-1] == "ModuleNotFoundError: No module named 'fake'\n" -@pytest.mark.parametrize("modify", (True, False)) +@pytest.mark.parametrize("should_wrap", (True, False)) @patch("airflow.jobs.triggerer_job.configure_trigger_log_handler") -def test_handler_config_respects_donot_modify(mock_configure, modify): +def test_handler_config_respects_donot_wrap(mock_configure, should_wrap): from airflow.jobs import triggerer_job - triggerer_job.DONOT_MODIFY_HANDLERS = not modify + triggerer_job.DISABLE_WRAPPER = not should_wrap TriggererJob() - if modify: + if should_wrap: mock_configure.assert_called() else: mock_configure.assert_not_called() diff --git a/tests/utils/test_log_handlers.py b/tests/utils/test_log_handlers.py index 7815983a8cec9..2baf6bba19530 100644 --- a/tests/utils/test_log_handlers.py +++ b/tests/utils/test_log_handlers.py @@ -22,8 +22,9 @@ import os import re import tempfile +from pathlib import Path from unittest import mock -from unittest.mock import mock_open, patch +from unittest.mock import patch import pendulum import pytest @@ -40,7 +41,7 @@ ) from airflow.utils.log.logging_mixin import set_context from airflow.utils.session import create_session -from airflow.utils.state import State +from airflow.utils.state import State, TaskInstanceState from airflow.utils.timezone import datetime from airflow.utils.types import DagRunType from tests.test_utils.config import conf_vars @@ -229,25 +230,50 @@ def task_callable(ti): # Remove the generated tmp log file. os.remove(log_filename) - def test__read_from_location(self, create_task_instance): - """Test if local log file exists, then log is read from it""" + @patch("airflow.utils.log.file_task_handler.FileTaskHandler._read_from_local") + def test__read_when_local(self, mock_read_local, create_task_instance): + """ + Test if local log file exists, then values returned from _read_from_local should be incorporated + into returned log. + """ + path = Path( + "dag_id=dag_for_testing_local_log_read/run_id=scheduled__2016-01-01T00:00:00+00:00/task_id=task_for_testing_local_log_read/attempt=1.log" # noqa: E501 + ) + mock_read_local.return_value = (["the messages"], ["the log"]) local_log_file_read = create_task_instance( dag_id="dag_for_testing_local_log_read", task_id="task_for_testing_local_log_read", run_type=DagRunType.SCHEDULED, execution_date=DEFAULT_DATE, ) - with patch("os.path.exists", return_value=True): - opener = mock_open(read_data="dummy test log data") - with patch("airflow.utils.log.file_task_handler.open", opener): - fth = FileTaskHandler("") - log = fth._read(ti=local_log_file_read, try_number=1) - assert len(log) == 2 - assert "dummy test log data" in log[0] + fth = FileTaskHandler("") + actual = fth._read(ti=local_log_file_read, try_number=1) + mock_read_local.assert_called_with(path) + assert actual == ("*** the messages\nthe log", {"end_of_log": True, "log_pos": 7}) + + def test__read_from_local(self): + """Tests the behavior of method _read_from_local""" + + with tempfile.TemporaryDirectory() as td: + file1 = Path(td, "hello1.log") + file2 = Path(td, "hello1.log.suffix.log") + file1.write_text("file1 content") + file2.write_text("file2 content") + fth = FileTaskHandler("") + assert fth._read_from_local(file1) == ( + [ + "Found local files:", + f" * {td}/hello1.log", + f" * {td}/hello1.log.suffix.log", + ], + ["file1 content", "file2 content"], + ) @mock.patch("airflow.executors.kubernetes_executor.KubernetesExecutor.get_task_log") - def test__read_for_k8s_executor(self, mock_k8s_get_task_log, create_task_instance): + @pytest.mark.parametrize("state", [TaskInstanceState.RUNNING, TaskInstanceState.SUCCESS]) + def test__read_for_k8s_executor(self, mock_k8s_get_task_log, create_task_instance, state): """Test for k8s executor, the log is read from get_task_log method""" + mock_k8s_get_task_log.return_value = ([], []) executor_name = "KubernetesExecutor" ti = create_task_instance( dag_id="dag_for_testing_k8s_executor_log_read", @@ -255,12 +281,15 @@ def test__read_for_k8s_executor(self, mock_k8s_get_task_log, create_task_instanc run_type=DagRunType.SCHEDULED, execution_date=DEFAULT_DATE, ) - + ti.state = state + ti.triggerer_job = None with conf_vars({("core", "executor"): executor_name}): - with patch("os.path.exists", return_value=False): - fth = FileTaskHandler("") - fth._read(ti=ti, try_number=1) - mock_k8s_get_task_log.assert_called_once_with(ti=ti, log=mock.ANY) + fth = FileTaskHandler("") + fth._read(ti=ti, try_number=1) + if state == TaskInstanceState.RUNNING: + mock_k8s_get_task_log.assert_called_once_with(ti) + else: + mock_k8s_get_task_log.assert_not_called() def test__read_for_celery_executor_fallbacks_to_worker(self, create_task_instance): """Test for executors which do not have `get_task_log` method, it fallbacks to reading From 1e86a5e8eba77ff0d743fce4fd687aa937fdd962 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Tue, 31 Jan 2023 11:53:18 -0800 Subject: [PATCH 67/96] fix tests --- airflow/utils/log/file_task_handler.py | 2 +- tests/utils/test_log_handlers.py | 18 +++++++----------- 2 files changed, 8 insertions(+), 12 deletions(-) diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index a5b7cc49e3804..c0abc5ed01cdd 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -317,7 +317,7 @@ def _read( executor_messages, executor_logs = response if executor_messages: messages_list.extend(messages_list) - elif ti.state in (TaskInstanceState.RUNNING, TaskInstanceState.DEFERRED): + if ti.state in (TaskInstanceState.RUNNING, TaskInstanceState.DEFERRED) and not executor_messages: served_messages, served_logs = self._read_from_logs_server(ti, worker_log_rel_path) messages_list.extend(served_messages) if not (remote_logs and ti.state not in State.unfinished): diff --git a/tests/utils/test_log_handlers.py b/tests/utils/test_log_handlers.py index 2baf6bba19530..a270ef4e4c275 100644 --- a/tests/utils/test_log_handlers.py +++ b/tests/utils/test_log_handlers.py @@ -302,19 +302,15 @@ def test__read_for_celery_executor_fallbacks_to_worker(self, create_task_instanc run_type=DagRunType.SCHEDULED, execution_date=DEFAULT_DATE, ) - + ti.state = TaskInstanceState.RUNNING with conf_vars({("core", "executor"): executor_name}): - with patch("os.path.exists", return_value=False): - fth = FileTaskHandler("") - - def mock_log_from_worker(ti, log, log_relative_path): - return (log, {"end_of_log": True}) + fth = FileTaskHandler("") - fth._get_task_log_from_worker = mock.Mock(side_effect=mock_log_from_worker) - log = fth._read(ti=ti, try_number=1) - fth._get_task_log_from_worker.assert_called_once() - assert "Local log file does not exist" in log[0] - assert "Failed to fetch log from executor. Falling back to fetching log from worker" in log[0] + fth._read_from_logs_server = mock.Mock() + fth._read_from_logs_server.return_value = ["this message"], ["this\nlog\ncontent"] + actual = fth._read(ti=ti, try_number=1) + fth._read_from_logs_server.assert_called_once() + assert actual == ("*** this message\nthis\nlog\ncontent", {"end_of_log": True, "log_pos": 16}) @pytest.mark.parametrize( "pod_override, namespace_to_call", From 3dfdcbd883b66ae720e4d6f2f3d3edd67ccd864a Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Tue, 31 Jan 2023 12:58:41 -0800 Subject: [PATCH 68/96] fix tests --- tests/utils/log/test_log_reader.py | 81 ++++++++++++++++-------------- tests/utils/test_log_handlers.py | 64 ++++++++++------------- 2 files changed, 69 insertions(+), 76 deletions(-) diff --git a/tests/utils/log/test_log_reader.py b/tests/utils/log/test_log_reader.py index 4fab589896846..da7dbc98da890 100644 --- a/tests/utils/log/test_log_reader.py +++ b/tests/utils/log/test_log_reader.py @@ -117,79 +117,82 @@ def prepare_db(self, create_task_instance): def test_test_read_log_chunks_should_read_one_try(self): task_log_reader = TaskLogReader() - logs, metadatas = task_log_reader.read_log_chunks(ti=self.ti, try_number=1, metadata={}) - - assert [ + ti = copy.copy(self.ti) + ti.state = TaskInstanceState.SUCCESS + logs, metadatas = task_log_reader.read_log_chunks(ti=ti, try_number=1, metadata={}) + assert logs[0] == [ ( "localhost", - f"*** Reading local file: " - f"{self.log_dir}/dag_log_reader/task_log_reader/2017-09-01T00.00.00+00.00/1.log\n" - f"try_number=1.\n", + "*** Found local files:\n" + f"*** * {self.log_dir}/dag_log_reader/task_log_reader/2017-09-01T00.00.00+00.00/1.log\n" + "try_number=1.", ) - ] == logs[0] - assert metadatas == {"end_of_log": True, "log_pos": 102 + len(self.log_dir)} + ] + assert metadatas == {"end_of_log": True, "log_pos": 13} def test_test_read_log_chunks_should_read_all_files(self): task_log_reader = TaskLogReader() - logs, metadatas = task_log_reader.read_log_chunks(ti=self.ti, try_number=None, metadata={}) + ti = copy.copy(self.ti) + ti.state = TaskInstanceState.SUCCESS + logs, metadatas = task_log_reader.read_log_chunks(ti=ti, try_number=None, metadata={}) - assert [ + assert logs == [ [ ( "localhost", - "*** Reading local file: " - f"{self.log_dir}/dag_log_reader/task_log_reader/2017-09-01T00.00.00+00.00/1.log\n" - "try_number=1.\n", + "*** Found local files:\n" + f"*** * {self.log_dir}/dag_log_reader/task_log_reader/2017-09-01T00.00.00+00.00/1.log\n" + "try_number=1.", ) ], [ ( "localhost", - f"*** Reading local file: " - f"{self.log_dir}/dag_log_reader/task_log_reader/2017-09-01T00.00.00+00.00/2.log\n" - f"try_number=2.\n", + "*** Found local files:\n" + f"*** * {self.log_dir}/dag_log_reader/task_log_reader/2017-09-01T00.00.00+00.00/2.log\n" + f"try_number=2.", ) ], [ ( "localhost", - f"*** Reading local file: " - f"{self.log_dir}/dag_log_reader/task_log_reader/2017-09-01T00.00.00+00.00/3.log\n" - f"try_number=3.\n", + "*** Found local files:\n" + f"*** * {self.log_dir}/dag_log_reader/task_log_reader/2017-09-01T00.00.00+00.00/3.log\n" + f"try_number=3.", ) ], - ] == logs - assert {"end_of_log": True, "log_pos": 102 + len(self.log_dir)} == metadatas + ] + assert metadatas == {"end_of_log": True, "log_pos": 13} def test_test_test_read_log_stream_should_read_one_try(self): task_log_reader = TaskLogReader() - stream = task_log_reader.read_log_stream(ti=self.ti, try_number=1, metadata={}) - - assert [ - "localhost\n*** Reading local file: " - f"{self.log_dir}/dag_log_reader/task_log_reader/2017-09-01T00.00.00+00.00/1.log\n" + ti = copy.copy(self.ti) + ti.state = TaskInstanceState.SUCCESS + stream = task_log_reader.read_log_stream(ti=ti, try_number=1, metadata={}) + assert list(stream) == [ + "localhost\n*** Found local files:\n" + f"*** * {self.log_dir}/dag_log_reader/task_log_reader/2017-09-01T00.00.00+00.00/1.log\n" "try_number=1.\n" - "\n" - ] == list(stream) + ] def test_test_test_read_log_stream_should_read_all_logs(self): task_log_reader = TaskLogReader() self.ti.state = TaskInstanceState.SUCCESS # Ensure mocked instance is completed to return stream stream = task_log_reader.read_log_stream(ti=self.ti, try_number=None, metadata={}) - assert [ - "localhost\n*** Reading local file: " - f"{self.log_dir}/dag_log_reader/task_log_reader/2017-09-01T00.00.00+00.00/1.log\n" - "try_number=1.\n" + assert list(stream) == [ + "localhost\n*** Found local files:\n" + f"*** * {self.log_dir}/dag_log_reader/task_log_reader/2017-09-01T00.00.00+00.00/1.log\n" + "try_number=1." "\n", - "localhost\n*** Reading local file: " - f"{self.log_dir}/dag_log_reader/task_log_reader/2017-09-01T00.00.00+00.00/2.log\n" - "try_number=2.\n" + "localhost\n*** Found local files:\n" + f"*** * {self.log_dir}/dag_log_reader/task_log_reader/2017-09-01T00.00.00+00.00/2.log\n" + "try_number=2." "\n", - "localhost\n*** Reading local file: " - f"{self.log_dir}/dag_log_reader/task_log_reader/2017-09-01T00.00.00+00.00/3.log\n" - "try_number=3.\n" + "localhost\n*** Found local files:\n" + f"*** * {self.log_dir}/dag_log_reader/task_log_reader/2017-09-01T00.00.00+00.00/3.log\n" + "try_number=3." "\n", - ] == list(stream) + ] @mock.patch("airflow.utils.log.file_task_handler.FileTaskHandler.read") def test_read_log_stream_should_support_multiple_chunks(self, mock_read): diff --git a/tests/utils/test_log_handlers.py b/tests/utils/test_log_handlers.py index a270ef4e4c275..59d9d3ade5ede 100644 --- a/tests/utils/test_log_handlers.py +++ b/tests/utils/test_log_handlers.py @@ -31,15 +31,18 @@ from kubernetes.client import models as k8s from airflow.config_templates.airflow_local_settings import DEFAULT_LOGGING_CONFIG +from airflow.jobs.base_job import BaseJob from airflow.jobs.triggerer_job import TriggererJob from airflow.models import DAG, DagRun, TaskInstance, Trigger from airflow.operators.python import PythonOperator from airflow.utils.log.file_task_handler import ( FileTaskHandler, + LogType, _interleave_logs, _parse_timestamps_in_log_file, ) from airflow.utils.log.logging_mixin import set_context +from airflow.utils.net import get_hostname from airflow.utils.session import create_session from airflow.utils.state import State, TaskInstanceState from airflow.utils.timezone import datetime @@ -353,7 +356,7 @@ def task_callable(ti): file_handler = next((h for h in logger.handlers if h.name == FILE_TASK_HANDLER), None) set_context(logger, ti) ti.run(ignore_ti_state=True) - + ti.state = TaskInstanceState.RUNNING file_handler.read(ti, 3) # first we find pod name @@ -460,41 +463,28 @@ def test_log_retrieval_valid(self, create_task_instance): execution_date=DEFAULT_DATE, ) log_url_ti.hostname = "hostname" - url = FileTaskHandler._get_log_retrieval_url(log_url_ti, "DYNAMIC_PATH") - assert url == "http://hostname:8793/log/DYNAMIC_PATH" - - -@pytest.mark.parametrize( - "config, queue, expected", - [ - (dict(AIRFLOW__CORE__EXECUTOR="LocalExecutor"), None, False), - (dict(AIRFLOW__CORE__EXECUTOR="LocalExecutor"), "kubernetes", False), - (dict(AIRFLOW__CORE__EXECUTOR="KubernetesExecutor"), None, True), - (dict(AIRFLOW__CORE__EXECUTOR="CeleryKubernetesExecutor"), "any", False), - (dict(AIRFLOW__CORE__EXECUTOR="CeleryKubernetesExecutor"), "kubernetes", True), - ( - dict( - AIRFLOW__CORE__EXECUTOR="CeleryKubernetesExecutor", - AIRFLOW__CELERY_KUBERNETES_EXECUTOR__KUBERNETES_QUEUE="hithere", - ), - "hithere", - True, - ), - (dict(AIRFLOW__CORE__EXECUTOR="LocalKubernetesExecutor"), "any", False), - (dict(AIRFLOW__CORE__EXECUTOR="LocalKubernetesExecutor"), "kubernetes", True), - ( - dict( - AIRFLOW__CORE__EXECUTOR="LocalKubernetesExecutor", - AIRFLOW__LOCAL_KUBERNETES_EXECUTOR__KUBERNETES_QUEUE="hithere", - ), - "hithere", - True, - ), - ], -) -def test__should_check_k8s(config, queue, expected): - with patch.dict("os.environ", **config): - assert FileTaskHandler._should_check_k8s(queue) == expected + actual = FileTaskHandler("")._get_log_retrieval_url(log_url_ti, "DYNAMIC_PATH") + assert actual == ("http://hostname:8793/log/DYNAMIC_PATH", "DYNAMIC_PATH") + + def test_log_retrieval_valid_trigger(self, create_task_instance): + ti = create_task_instance( + dag_id="dag_for_testing_filename_rendering", + task_id="task_for_testing_filename_rendering", + run_type=DagRunType.SCHEDULED, + execution_date=DEFAULT_DATE, + ) + ti.hostname = "hostname" + trigger = Trigger("", {}) + job = BaseJob() + job.id = 123 + trigger.triggerer_job = job + ti.trigger = trigger + actual = FileTaskHandler("")._get_log_retrieval_url(ti, "DYNAMIC_PATH", log_type=LogType.TRIGGER) + hostname = get_hostname() + assert actual == ( + f"http://{hostname}:8794/log/DYNAMIC_PATH.trigger.123.log", + "DYNAMIC_PATH.trigger.123.log", + ) log_sample = """[2022-11-16T00:05:54.278-0800] {taskinstance.py:1257} INFO - @@ -522,7 +512,7 @@ def test__should_check_k8s(config, queue, expected): def test_parse_timestamps(): actual = [] - for timestamp, _ in _parse_timestamps_in_log_file(log_sample.splitlines()): + for timestamp, idx, line in _parse_timestamps_in_log_file(log_sample.splitlines()): actual.append(timestamp) assert actual == [ pendulum.parse("2022-11-16T00:05:54.278000-08:00"), From d974d227ea083ed50d1c592254b186c34ba89b64 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Tue, 31 Jan 2023 14:26:33 -0800 Subject: [PATCH 69/96] fix tests --- .../aws/log/test_cloudwatch_task_handler.py | 16 ---------------- .../amazon/aws/log/test_s3_task_handler.py | 18 +++++++++++------- .../google/cloud/log/test_gcs_task_handler.py | 14 +++++++++----- .../azure/log/test_wasb_task_handler.py | 5 ++++- 4 files changed, 24 insertions(+), 29 deletions(-) diff --git a/tests/providers/amazon/aws/log/test_cloudwatch_task_handler.py b/tests/providers/amazon/aws/log/test_cloudwatch_task_handler.py index b962691d024f1..257aa144407af 100644 --- a/tests/providers/amazon/aws/log/test_cloudwatch_task_handler.py +++ b/tests/providers/amazon/aws/log/test_cloudwatch_task_handler.py @@ -154,22 +154,6 @@ def test_read(self): [{"end_of_log": True}], ) - def test_should_read_from_local_on_failure_to_fetch_remote_logs(self): - """Check that local logs are displayed on failure to fetch remote logs""" - self.cloudwatch_task_handler.set_context(self.ti) - with mock.patch.object(self.cloudwatch_task_handler, "get_cloudwatch_logs") as mock_get_logs: - mock_get_logs.side_effect = Exception("Failed to connect") - log, metadata = self.cloudwatch_task_handler._read(self.ti, self.ti.try_number) - expected_log = ( - f"*** Unable to read remote logs from Cloudwatch (log_group: {self.remote_log_group}, log_stream: {self.remote_log_stream})\n" # noqa: E501 - "*** Failed to connect\n\n" - "*** Found local files:\n" - f"*** * {self.local_log_location}/{self.remote_log_stream}\n" - ) - assert log == expected_log - assert metadata == {"end_of_log": False, "log_pos": 0} - mock_get_logs.assert_called_once_with(stream_name=self.remote_log_stream) - def test_close_prevents_duplicate_calls(self): with mock.patch("watchtower.CloudWatchLogHandler.close") as mock_log_handler_close: with mock.patch("airflow.utils.log.file_task_handler.FileTaskHandler.set_context"): diff --git a/tests/providers/amazon/aws/log/test_s3_task_handler.py b/tests/providers/amazon/aws/log/test_s3_task_handler.py index 8ccbf2a54edde..4fbe453274a9b 100644 --- a/tests/providers/amazon/aws/log/test_s3_task_handler.py +++ b/tests/providers/amazon/aws/log/test_s3_task_handler.py @@ -18,6 +18,7 @@ from __future__ import annotations import contextlib +import copy import os from unittest import mock @@ -31,7 +32,7 @@ from airflow.providers.amazon.aws.hooks.s3 import S3Hook from airflow.providers.amazon.aws.log.s3_task_handler import S3TaskHandler from airflow.utils.session import create_session -from airflow.utils.state import State +from airflow.utils.state import State, TaskInstanceState from airflow.utils.timezone import datetime from tests.test_utils.config import conf_vars @@ -126,21 +127,24 @@ def test_set_context_not_raw(self): def test_read(self): self.conn.put_object(Bucket="bucket", Key=self.remote_log_key, Body=b"Log line\n") - log, metadata = self.s3_task_handler.read(self.ti) + ti = copy.copy(self.ti) + ti.state = TaskInstanceState.SUCCESS + log, metadata = self.s3_task_handler.read(ti) actual = log[0][0][-1] expected = "*** Found logs in s3:\n*** * s3://bucket/remote/log/location/1.log\nLog line" assert actual == expected - assert metadata == [{"end_of_log": False, "log_pos": 8}] + assert metadata == [{"end_of_log": True, "log_pos": 8}] def test_read_when_s3_log_missing(self): - log, metadata = self.s3_task_handler.read(self.ti) - + ti = copy.copy(self.ti) + ti.state = TaskInstanceState.SUCCESS + log, metadata = self.s3_task_handler.read(ti) assert 1 == len(log) assert len(log) == len(metadata) actual = log[0][0][-1] - expected = "*** No logs found on s3 for ti=\n" # noqa: E501 + expected = "*** No logs found on s3 for ti=\n" # noqa: E501 assert actual == expected - assert {"end_of_log": False, "log_pos": 0} == metadata[0] + assert {"end_of_log": True, "log_pos": 0} == metadata[0] def test_s3_read_when_log_missing(self): handler = self.s3_task_handler diff --git a/tests/providers/google/cloud/log/test_gcs_task_handler.py b/tests/providers/google/cloud/log/test_gcs_task_handler.py index d011387491f3b..690ae9dc335e4 100644 --- a/tests/providers/google/cloud/log/test_gcs_task_handler.py +++ b/tests/providers/google/cloud/log/test_gcs_task_handler.py @@ -16,6 +16,7 @@ # under the License. from __future__ import annotations +import copy import logging import tempfile from unittest import mock @@ -95,13 +96,14 @@ def test_should_read_logs_from_remote(self, mock_blob, mock_client, mock_creds): mock_obj.name = "remote/log/location/1.log" mock_client.return_value.list_blobs.return_value = [mock_obj] mock_blob.from_string.return_value.download_as_bytes.return_value = b"CONTENT" - - logs, metadata = self.gcs_task_handler._read(self.ti, self.ti.try_number) + ti = copy.copy(self.ti) + ti.state = TaskInstanceState.SUCCESS + logs, metadata = self.gcs_task_handler._read(ti, self.ti.try_number) mock_blob.from_string.assert_called_once_with( "gs://bucket/remote/log/location/1.log", mock_client.return_value ) assert logs == "*** Found remote logs:\n*** * gs://bucket/remote/log/location/1.log\nCONTENT" - assert {"end_of_log": False, "log_pos": 7} == metadata + assert {"end_of_log": True, "log_pos": 7} == metadata @mock.patch( "airflow.providers.google.cloud.log.gcs_task_handler.get_credentials_and_project_id", @@ -116,7 +118,9 @@ def test_should_read_from_local_on_logs_read_error(self, mock_blob, mock_client, mock_blob.from_string.return_value.download_as_bytes.side_effect = Exception("Failed to connect") self.gcs_task_handler.set_context(self.ti) - log, metadata = self.gcs_task_handler._read(self.ti, self.ti.try_number) + ti = copy.copy(self.ti) + ti.state = TaskInstanceState.SUCCESS + log, metadata = self.gcs_task_handler._read(ti, self.ti.try_number) assert log == ( "*** Found remote logs:\n" @@ -125,7 +129,7 @@ def test_should_read_from_local_on_logs_read_error(self, mock_blob, mock_client, "*** Found local files:\n" f"*** * {self.local_log_location}/1.log\n" ) - assert metadata == {"end_of_log": False, "log_pos": 0} + assert metadata == {"end_of_log": True, "log_pos": 0} mock_blob.from_string.assert_called_once_with( "gs://bucket/remote/log/location/1.log", mock_client.return_value ) diff --git a/tests/providers/microsoft/azure/log/test_wasb_task_handler.py b/tests/providers/microsoft/azure/log/test_wasb_task_handler.py index d371f0810160a..60b6947f619ae 100644 --- a/tests/providers/microsoft/azure/log/test_wasb_task_handler.py +++ b/tests/providers/microsoft/azure/log/test_wasb_task_handler.py @@ -16,6 +16,7 @@ # under the License. from __future__ import annotations +import copy import tempfile from pathlib import Path from unittest import mock @@ -107,6 +108,8 @@ def test_wasb_read(self, mock_hook_cls, ti): mock_hook.get_blobs_list.return_value = ["abc/hello.log"] mock_hook.read_file.return_value = "Log line" assert self.wasb_task_handler.wasb_read(self.remote_log_location) == "Log line" + ti = copy.copy(ti) + ti.state = TaskInstanceState.SUCCESS assert self.wasb_task_handler.read(ti) == ( [ [ @@ -116,7 +119,7 @@ def test_wasb_read(self, mock_hook_cls, ti): ) ] ], - [{"end_of_log": False, "log_pos": 8}], + [{"end_of_log": True, "log_pos": 8}], ) @mock.patch( From 74bff9bcda6788b13f66edfdefcaeadae319c708 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Tue, 31 Jan 2023 15:32:40 -0800 Subject: [PATCH 70/96] fix tests --- airflow/executors/kubernetes_executor.py | 2 +- airflow/utils/log/file_task_handler.py | 7 ++++++- .../test_celery_kubernetes_executor.py | 2 +- tests/executors/test_kubernetes_executor.py | 19 ++++++++++--------- .../test_local_kubernetes_executor.py | 15 ++++++--------- tests/utils/test_log_handlers.py | 2 +- 6 files changed, 25 insertions(+), 22 deletions(-) diff --git a/airflow/executors/kubernetes_executor.py b/airflow/executors/kubernetes_executor.py index 2074701f1d58e..b388a3b386493 100644 --- a/airflow/executors/kubernetes_executor.py +++ b/airflow/executors/kubernetes_executor.py @@ -821,7 +821,7 @@ def get_task_log(self, ti: TaskInstance) -> tuple[list[str], list[str]]: log.append(line.decode()) except Exception as e: messages.append(f"Reading from k8s pod logs failed: {str(e)}") - return messages, log + return messages, ["\n".join(log)] def try_adopt_task_instances(self, tis: Sequence[TaskInstance]) -> Sequence[TaskInstance]: tis_to_flush = [ti for ti in tis if not ti.queued_by_job_id] diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index c0abc5ed01cdd..1496b31fac682 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -358,13 +358,18 @@ def _get_log_retrieval_url( if not ti.triggerer_job: raise RuntimeError("Could not build triggerer log URL; no triggerer job.") config_key = "triggerer_log_server_port" + config_default = 8794 hostname = ti.triggerer_job.hostname log_relative_path = self.add_triggerer_suffix(log_relative_path, job_id=ti.triggerer_job.id) else: hostname = ti.hostname config_key = "worker_log_server_port" + config_default = 8793 return ( - urljoin(f"http://{hostname}:{conf.get('logging', config_key)}/log/", log_relative_path), + urljoin( + f"http://{hostname}:{conf.get('logging', config_key, fallback=config_default)}/log/", + log_relative_path, + ), log_relative_path, ) diff --git a/tests/executors/test_celery_kubernetes_executor.py b/tests/executors/test_celery_kubernetes_executor.py index 6778bad6eb6ec..691d24f12100e 100644 --- a/tests/executors/test_celery_kubernetes_executor.py +++ b/tests/executors/test_celery_kubernetes_executor.py @@ -180,7 +180,7 @@ def test_log_is_fetched_from_k8s_executor_only_for_k8s_queue(self): simple_task_instance = mock.MagicMock() simple_task_instance.queue = KUBERNETES_QUEUE cke.get_task_log(ti=simple_task_instance) - k8s_executor_mock.get_task_log.assert_called_once_with(ti=simple_task_instance, log=mock.ANY) + k8s_executor_mock.get_task_log.assert_called_once_with(ti=simple_task_instance) k8s_executor_mock.reset_mock() diff --git a/tests/executors/test_kubernetes_executor.py b/tests/executors/test_kubernetes_executor.py index 5abc531b5a577..954ba9c7d0e2c 100644 --- a/tests/executors/test_kubernetes_executor.py +++ b/tests/executors/test_kubernetes_executor.py @@ -1219,7 +1219,7 @@ def test_clear_not_launched_queued_tasks_clear_only_by_job_id(self, dag_maker, c assert ti0.state == State.SCHEDULED assert ti1.state == State.QUEUED - @mock.patch("airflow.executors.kubernetes_executor.get_kube_client") + @mock.patch("airflow.kubernetes.kube_client.get_kube_client") def test_get_task_log(self, mock_get_kube_client, create_task_instance_of_operator): """fetch task log from pod""" mock_kube_client = mock_get_kube_client.return_value @@ -1231,20 +1231,21 @@ def test_get_task_log(self, mock_get_kube_client, create_task_instance_of_operat ti = create_task_instance_of_operator(EmptyOperator, dag_id="test_k8s_log_dag", task_id="test_task") executor = KubernetesExecutor() - log = executor.get_task_log(ti=ti) + messages, logs = executor.get_task_log(ti=ti) mock_kube_client.read_namespaced_pod_log.assert_called_once() - assert "test_init_log" in log - assert "Trying to get logs (last 100 lines) from worker pod" in log - assert "a_b_c" in log + assert "Trying to get logs (last 100 lines) from worker pod " in messages + assert logs[0] == "a_\nb_\nc_" mock_kube_client.reset_mock() mock_kube_client.read_namespaced_pod_log.side_effect = Exception("error_fetching_pod_log") - log = executor.get_task_log(ti=ti, log="test_init_log") - assert len(log) == 2 - assert "error_fetching_pod_log" in log[0] - assert log[1]["end_of_log"] + messages, logs = executor.get_task_log(ti=ti) + assert logs == [""] + assert messages == [ + "Trying to get logs (last 100 lines) from worker pod ", + "Reading from k8s pod logs failed: error_fetching_pod_log", + ] def test_supports_pickling(self): assert KubernetesExecutor.supports_pickling diff --git a/tests/executors/test_local_kubernetes_executor.py b/tests/executors/test_local_kubernetes_executor.py index 497d3a5f9b505..1c8fa4d309755 100644 --- a/tests/executors/test_local_kubernetes_executor.py +++ b/tests/executors/test_local_kubernetes_executor.py @@ -86,21 +86,18 @@ def test_kubernetes_executor_knows_its_queue(self): def test_log_is_fetched_from_k8s_executor_only_for_k8s_queue(self): local_executor_mock = mock.MagicMock() k8s_executor_mock = mock.MagicMock() - - KUBERNETES_QUEUE = conf.get("local_kubernetes_executor", "kubernetes_queue") LocalKubernetesExecutor(local_executor_mock, k8s_executor_mock) local_k8s_exec = LocalKubernetesExecutor(local_executor_mock, k8s_executor_mock) simple_task_instance = mock.MagicMock() - simple_task_instance.queue = KUBERNETES_QUEUE - local_k8s_exec.get_task_log(ti=simple_task_instance, log="") - k8s_executor_mock.get_task_log.assert_called_once_with(ti=simple_task_instance, log=mock.ANY) - + simple_task_instance.queue = conf.get("local_kubernetes_executor", "kubernetes_queue") + local_k8s_exec.get_task_log(ti=simple_task_instance) + k8s_executor_mock.get_task_log.assert_called_once_with(ti=simple_task_instance) k8s_executor_mock.reset_mock() - simple_task_instance.queue = "test-queue" - log = local_k8s_exec.get_task_log(ti=simple_task_instance, log="") + messages, logs = local_k8s_exec.get_task_log(ti=simple_task_instance) k8s_executor_mock.get_task_log.assert_not_called() - assert log is None + assert logs == [] + assert messages == [] def test_send_callback(self): local_executor_mock = mock.MagicMock() diff --git a/tests/utils/test_log_handlers.py b/tests/utils/test_log_handlers.py index 59d9d3ade5ede..3f762b9a94b86 100644 --- a/tests/utils/test_log_handlers.py +++ b/tests/utils/test_log_handlers.py @@ -326,7 +326,7 @@ def test__read_for_celery_executor_fallbacks_to_worker(self, create_task_instanc ], ) @patch.dict("os.environ", AIRFLOW__CORE__EXECUTOR="KubernetesExecutor") - @patch("airflow.executors.kubernetes_executor.get_kube_client") + @patch("airflow.kubernetes.kube_client.get_kube_client") def test_read_from_k8s_under_multi_namespace_mode( self, mock_kube_client, pod_override, namespace_to_call ): From 2b626f507ccc6cc641990cea6dc30a68e6ae548c Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 1 Feb 2023 07:56:21 -0800 Subject: [PATCH 71/96] fix tests --- airflow/jobs/local_task_job.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/jobs/local_task_job.py b/airflow/jobs/local_task_job.py index 2d3516f26b78a..22c1c9a441b57 100644 --- a/airflow/jobs/local_task_job.py +++ b/airflow/jobs/local_task_job.py @@ -203,9 +203,9 @@ def sigusr2_debug_handler(signum, frame): f"Time since last heartbeat({time_since_last_heartbeat:.2f}s) exceeded limit " f"({heartbeat_time_limit}s)." ) + return return_code finally: self.on_kill() - return return_code def handle_task_exit(self, return_code: int) -> None: """ From 549542a66fd16312e90e5e2404bafadc70883348 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 1 Feb 2023 10:06:44 -0800 Subject: [PATCH 72/96] fix tests --- tests/cli/commands/test_task_command.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/cli/commands/test_task_command.py b/tests/cli/commands/test_task_command.py index fafcdf4fc8851..7e10ad21a1c4d 100644 --- a/tests/cli/commands/test_task_command.py +++ b/tests/cli/commands/test_task_command.py @@ -266,7 +266,6 @@ def test_run_with_existing_dag_run_id(self, mock_local_job): pickle_id=None, pool=None, external_executor_id=None, - shut_down_logging=True, ) @mock.patch("airflow.cli.commands.task_command.LocalTaskJob") @@ -643,7 +642,6 @@ def test_external_executor_id_present_for_fork_run_task(self, mock_local_job): ignore_ti_state=False, pool=None, external_executor_id="ABCD12345", - shut_down_logging=True, ) @mock.patch("airflow.cli.commands.task_command.LocalTaskJob") @@ -664,7 +662,6 @@ def test_external_executor_id_present_for_process_run_task(self, mock_local_job) ignore_ti_state=False, pool=None, external_executor_id="ABCD12345", - shut_down_logging=True, ) @pytest.mark.parametrize("is_k8s", ["true", ""]) From 93087877712a5249bc1e2fd8842fc9f84ef044dd Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 2 Feb 2023 12:52:45 -0800 Subject: [PATCH 73/96] Update tests/charts/test_extra_env_env_from.py Co-authored-by: Jed Cunningham <66968678+jedcunningham@users.noreply.github.com> --- tests/charts/test_extra_env_env_from.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/charts/test_extra_env_env_from.py b/tests/charts/test_extra_env_env_from.py index c1de0b882fc95..cc356693162b2 100644 --- a/tests/charts/test_extra_env_env_from.py +++ b/tests/charts/test_extra_env_env_from.py @@ -114,7 +114,6 @@ def test_extra_env(self, k8s_obj_key, env_paths): name: {RELEASE_NAME}-some-secret """ ).lstrip() - print(k8s_obj_key) k8s_object = self.k8s_objects_by_key[k8s_obj_key] for path in env_paths: env = jmespath.search(f"{path}.env", k8s_object) From fa869513a5705b5f4302d78293d9d4d407bc7294 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 2 Feb 2023 12:52:59 -0800 Subject: [PATCH 74/96] Update docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst Co-authored-by: Jed Cunningham <66968678+jedcunningham@users.noreply.github.com> --- .../logging-monitoring/logging-tasks.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst b/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst index 18e57a4119617..9cd7aa05b3a06 100644 --- a/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst +++ b/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst @@ -151,4 +151,4 @@ To accomplish this we have a few attributes that may be set either on the handle - ``trigger_should_wrap``: controls whether this handler should be wrapped by TriggerHandlerWrapper. This is necessary when each instance of handler creates a file handler that it writes all messages to. - ``trigger_should_queue``: controls whether the triggerer should put a QueueListener between the event loop and the handler, to ensure blocking IO in the handler does not disrupt the event loop. - ``trigger_send_end_marker``: whether an END signal should be sent to the logger when trigger completes. It is used to tell the wrapper to close and remove the individual file handler specific to the trigger that just completed. -- ``trigger_supported``: If ``trigger_should_wrap`` and ``trigger_should_queue`` are not True, we generally assume that the handler does not support triggers. But if in this case the handler has ``trigger_supported`` set to True, then we'll still move the handler to root at triggerer start so that it will process trigger messages. Essentially, this should be true for handlers that "natively" support triggers. And one such example of this is the StackdriverTaskHandler. +- ``trigger_supported``: If ``trigger_should_wrap`` and ``trigger_should_queue`` are not True, we generally assume that the handler does not support triggers. However, if in this case the handler has ``trigger_supported`` set to True, then we'll still move the handler to root at triggerer start so that it will process trigger messages. Essentially, this should be true for handlers that "natively" support triggers. One such example of this is the StackdriverTaskHandler. From e02c2ba701f4703e8ea90f88c3f434e3ed72905d Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 2 Feb 2023 12:53:14 -0800 Subject: [PATCH 75/96] Update docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst Co-authored-by: Jed Cunningham <66968678+jedcunningham@users.noreply.github.com> --- .../logging-monitoring/logging-tasks.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst b/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst index 9cd7aa05b3a06..819cacdd3fb16 100644 --- a/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst +++ b/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst @@ -150,5 +150,5 @@ To accomplish this we have a few attributes that may be set either on the handle - ``trigger_should_wrap``: controls whether this handler should be wrapped by TriggerHandlerWrapper. This is necessary when each instance of handler creates a file handler that it writes all messages to. - ``trigger_should_queue``: controls whether the triggerer should put a QueueListener between the event loop and the handler, to ensure blocking IO in the handler does not disrupt the event loop. -- ``trigger_send_end_marker``: whether an END signal should be sent to the logger when trigger completes. It is used to tell the wrapper to close and remove the individual file handler specific to the trigger that just completed. +- ``trigger_send_end_marker``: controls whether an END signal should be sent to the logger when trigger completes. It is used to tell the wrapper to close and remove the individual file handler specific to the trigger that just completed. - ``trigger_supported``: If ``trigger_should_wrap`` and ``trigger_should_queue`` are not True, we generally assume that the handler does not support triggers. However, if in this case the handler has ``trigger_supported`` set to True, then we'll still move the handler to root at triggerer start so that it will process trigger messages. Essentially, this should be true for handlers that "natively" support triggers. One such example of this is the StackdriverTaskHandler. From 9b0c177b6bfaab29220e071bf0961a01bf832aac Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 2 Feb 2023 12:54:32 -0800 Subject: [PATCH 76/96] Update docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst Co-authored-by: Jed Cunningham <66968678+jedcunningham@users.noreply.github.com> --- .../logging-monitoring/logging-tasks.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst b/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst index 819cacdd3fb16..bc7b551e496a2 100644 --- a/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst +++ b/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst @@ -146,7 +146,7 @@ In our providers we have a healthy variety of options with all the major cloud p Triggers require a shift in the way that logging is set up. In contrast with tasks, many triggers run in the same process, and with triggers, since they run in asyncio, we have to be mindful of not introducing blocking calls through the logging handler. And because of the variation in handler behavior (some write to file, some upload to blob storage, some send messages over network as they arrive, some do so in thread), we need to have some way to let triggerer know how to use them. -To accomplish this we have a few attributes that may be set either on the handler, either instance or the class. Inheritance is not respected for these parameters, because subclasses of FileTaskHandler may differ from it in the relevant characteristics. These params are described below: +To accomplish this we have a few attributes that may be set on the handler instance or class. Inheritance is not respected for these parameters, because subclasses of FileTaskHandler may differ from it in the relevant characteristics. These params are described below: - ``trigger_should_wrap``: controls whether this handler should be wrapped by TriggerHandlerWrapper. This is necessary when each instance of handler creates a file handler that it writes all messages to. - ``trigger_should_queue``: controls whether the triggerer should put a QueueListener between the event loop and the handler, to ensure blocking IO in the handler does not disrupt the event loop. From 2b54b39cb2c05def447a8e89910be2bd9476c13a Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 2 Feb 2023 12:54:47 -0800 Subject: [PATCH 77/96] Update docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst Co-authored-by: Jed Cunningham <66968678+jedcunningham@users.noreply.github.com> --- .../logging-monitoring/logging-tasks.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst b/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst index bc7b551e496a2..5bd57ae7c1ed6 100644 --- a/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst +++ b/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst @@ -144,7 +144,7 @@ Implementing a custom file task handler In our providers we have a healthy variety of options with all the major cloud providers. But should you need to implement logging with a different service, and should you then decide to implement a custom FileTaskHandler, there are a few settings to be aware of, particularly in the context of trigger logging. -Triggers require a shift in the way that logging is set up. In contrast with tasks, many triggers run in the same process, and with triggers, since they run in asyncio, we have to be mindful of not introducing blocking calls through the logging handler. And because of the variation in handler behavior (some write to file, some upload to blob storage, some send messages over network as they arrive, some do so in thread), we need to have some way to let triggerer know how to use them. +Triggers require a shift in the way that logging is set up. In contrast with tasks, many triggers run in the same process, and with triggers, since they run in asyncio, we have to be mindful of not introducing blocking calls through the logging handler. Due to the variation in handler behavior (some write to file, some upload to blob storage, some send messages over network as they arrive, some do so in thread), we need to have some way to let triggerer know how to use them. To accomplish this we have a few attributes that may be set on the handler instance or class. Inheritance is not respected for these parameters, because subclasses of FileTaskHandler may differ from it in the relevant characteristics. These params are described below: From 64ecb1a908aef165de0ad9d195ef7f7417cdd108 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 2 Feb 2023 12:55:05 -0800 Subject: [PATCH 78/96] Update docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst Co-authored-by: Jed Cunningham <66968678+jedcunningham@users.noreply.github.com> --- .../logging-monitoring/logging-tasks.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst b/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst index 5bd57ae7c1ed6..e91f8f1094915 100644 --- a/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst +++ b/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst @@ -142,7 +142,7 @@ Implementing a custom file task handler .. note:: This is an advanced topic and most users should be able to just use an existing handler from :doc:`apache-airflow-providers:core-extensions/logging`. -In our providers we have a healthy variety of options with all the major cloud providers. But should you need to implement logging with a different service, and should you then decide to implement a custom FileTaskHandler, there are a few settings to be aware of, particularly in the context of trigger logging. +In our providers we have a healthy variety of options with all the major cloud providers. However, should you need to implement logging with a different service, and should you then decide to implement a custom FileTaskHandler, there are a few settings to be aware of, particularly in the context of trigger logging. Triggers require a shift in the way that logging is set up. In contrast with tasks, many triggers run in the same process, and with triggers, since they run in asyncio, we have to be mindful of not introducing blocking calls through the logging handler. Due to the variation in handler behavior (some write to file, some upload to blob storage, some send messages over network as they arrive, some do so in thread), we need to have some way to let triggerer know how to use them. From e657b65cd03dde825408fac38ae66a604ce45b33 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 2 Feb 2023 12:55:20 -0800 Subject: [PATCH 79/96] Update docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst Co-authored-by: Jed Cunningham <66968678+jedcunningham@users.noreply.github.com> --- .../logging-monitoring/logging-tasks.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst b/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst index e91f8f1094915..a1b261d442e64 100644 --- a/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst +++ b/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst @@ -49,7 +49,7 @@ Interleaving of logs Airflow's remote task logging handlers can broadly be separated into two categories: streaming handlers (such as ElasticSearch, AWS Cloudwatch, and GCP operations logging, formerly stackdriver) and blob storage handlers (e.g. S3, GCS, WASB). -For blob storage handlers, depending on the state of the task, logs could be in a lot of different places and in multiple different files. For this reason, we need to check all locations and interleave what we find. To do this we need to be able to parse the timestamp for each line. If you are using a custom formatter you may need to override the default parser by providing a callable name at Airflow setting ``logging > interleave_timestamp_parser``. +For blob storage handlers, depending on the state of the task, logs could be in a lot of different places and in multiple different files. For this reason, we need to check all locations and interleave what we find. To do this we need to be able to parse the timestamp for each line. If you are using a custom formatter you may need to override the default parser by providing a callable name at Airflow setting ``[logging] interleave_timestamp_parser``. For streaming handlers, no matter the task phase or location of execution, all log messages can be sent to the logging service with the same identifier so generally speaking there isn't a need to check multiple sources and interleave. From 0628fd30c478c9fa61465fbdfd125e96cc094448 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 2 Feb 2023 12:59:01 -0800 Subject: [PATCH 80/96] Update chart/templates/triggerer/triggerer-service.yaml --- chart/templates/triggerer/triggerer-service.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/chart/templates/triggerer/triggerer-service.yaml b/chart/templates/triggerer/triggerer-service.yaml index c470232a74bcc..1f96fac70784a 100644 --- a/chart/templates/triggerer/triggerer-service.yaml +++ b/chart/templates/triggerer/triggerer-service.yaml @@ -18,7 +18,7 @@ ################################ ## Airflow triggerer Service ################################# -{{- if semverCompare ">=2.2.0" .Values.airflowVersion }} +{{- if semverCompare ">=2.6.0" .Values.airflowVersion }} {{- if .Values.triggerer.enabled }} kind: Service apiVersion: v1 From 74a12c19528ea2b6c1f1c531c849570210d8ee55 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 2 Feb 2023 14:14:23 -0800 Subject: [PATCH 81/96] =?UTF-8?q?restore=20just=20some=20of=20the=20origin?= =?UTF-8?q?al=20phrasing=20re=20however=20vs=20but=20=F0=9F=98=9B?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../logging-monitoring/logging-tasks.rst | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst b/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst index a1b261d442e64..c7d5819b31b83 100644 --- a/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst +++ b/docs/apache-airflow/administration-and-deployment/logging-monitoring/logging-tasks.rst @@ -142,13 +142,13 @@ Implementing a custom file task handler .. note:: This is an advanced topic and most users should be able to just use an existing handler from :doc:`apache-airflow-providers:core-extensions/logging`. -In our providers we have a healthy variety of options with all the major cloud providers. However, should you need to implement logging with a different service, and should you then decide to implement a custom FileTaskHandler, there are a few settings to be aware of, particularly in the context of trigger logging. +In our providers we have a healthy variety of options with all the major cloud providers. But should you need to implement logging with a different service, and should you then decide to implement a custom FileTaskHandler, there are a few settings to be aware of, particularly in the context of trigger logging. -Triggers require a shift in the way that logging is set up. In contrast with tasks, many triggers run in the same process, and with triggers, since they run in asyncio, we have to be mindful of not introducing blocking calls through the logging handler. Due to the variation in handler behavior (some write to file, some upload to blob storage, some send messages over network as they arrive, some do so in thread), we need to have some way to let triggerer know how to use them. +Triggers require a shift in the way that logging is set up. In contrast with tasks, many triggers run in the same process, and with triggers, since they run in asyncio, we have to be mindful of not introducing blocking calls through the logging handler. And because of the variation in handler behavior (some write to file, some upload to blob storage, some send messages over network as they arrive, some do so in thread), we need to have some way to let triggerer know how to use them. -To accomplish this we have a few attributes that may be set on the handler instance or class. Inheritance is not respected for these parameters, because subclasses of FileTaskHandler may differ from it in the relevant characteristics. These params are described below: +To accomplish this we have a few attributes that may be set on the handler, either the instance or the class. Inheritance is not respected for these parameters, because subclasses of FileTaskHandler may differ from it in the relevant characteristics. These params are described below: -- ``trigger_should_wrap``: controls whether this handler should be wrapped by TriggerHandlerWrapper. This is necessary when each instance of handler creates a file handler that it writes all messages to. -- ``trigger_should_queue``: controls whether the triggerer should put a QueueListener between the event loop and the handler, to ensure blocking IO in the handler does not disrupt the event loop. -- ``trigger_send_end_marker``: controls whether an END signal should be sent to the logger when trigger completes. It is used to tell the wrapper to close and remove the individual file handler specific to the trigger that just completed. -- ``trigger_supported``: If ``trigger_should_wrap`` and ``trigger_should_queue`` are not True, we generally assume that the handler does not support triggers. However, if in this case the handler has ``trigger_supported`` set to True, then we'll still move the handler to root at triggerer start so that it will process trigger messages. Essentially, this should be true for handlers that "natively" support triggers. One such example of this is the StackdriverTaskHandler. +- ``trigger_should_wrap``: Controls whether this handler should be wrapped by TriggerHandlerWrapper. This is necessary when each instance of handler creates a file handler that it writes all messages to. +- ``trigger_should_queue``: Controls whether the triggerer should put a QueueListener between the event loop and the handler, to ensure blocking IO in the handler does not disrupt the event loop. +- ``trigger_send_end_marker``: Controls whether an END signal should be sent to the logger when trigger completes. It is used to tell the wrapper to close and remove the individual file handler specific to the trigger that just completed. +- ``trigger_supported``: If ``trigger_should_wrap`` and ``trigger_should_queue`` are not True, we generally assume that the handler does not support triggers. But if in this case the handler has ``trigger_supported`` set to True, then we'll still move the handler to root at triggerer start so that it will process trigger messages. Essentially, this should be true for handlers that "natively" support triggers. One such example of this is the StackdriverTaskHandler. From 36c7e8c7d4d1bb6a96ab15b6d0d75e4c725c4f82 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 2 Feb 2023 14:17:19 -0800 Subject: [PATCH 82/96] Revert "Update chart/templates/triggerer/triggerer-service.yaml" This reverts commit 0628fd30c478c9fa61465fbdfd125e96cc094448. --- chart/templates/triggerer/triggerer-service.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/chart/templates/triggerer/triggerer-service.yaml b/chart/templates/triggerer/triggerer-service.yaml index 1f96fac70784a..c470232a74bcc 100644 --- a/chart/templates/triggerer/triggerer-service.yaml +++ b/chart/templates/triggerer/triggerer-service.yaml @@ -18,7 +18,7 @@ ################################ ## Airflow triggerer Service ################################# -{{- if semverCompare ">=2.6.0" .Values.airflowVersion }} +{{- if semverCompare ">=2.2.0" .Values.airflowVersion }} {{- if .Values.triggerer.enabled }} kind: Service apiVersion: v1 From 10203404e47f00609cfb30aed7bf7a5f54d4c25d Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 2 Feb 2023 14:20:13 -0800 Subject: [PATCH 83/96] Revert "Revert "Update chart/templates/triggerer/triggerer-service.yaml"" This reverts commit 36c7e8c7d4d1bb6a96ab15b6d0d75e4c725c4f82. --- chart/templates/triggerer/triggerer-service.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/chart/templates/triggerer/triggerer-service.yaml b/chart/templates/triggerer/triggerer-service.yaml index c470232a74bcc..1f96fac70784a 100644 --- a/chart/templates/triggerer/triggerer-service.yaml +++ b/chart/templates/triggerer/triggerer-service.yaml @@ -18,7 +18,7 @@ ################################ ## Airflow triggerer Service ################################# -{{- if semverCompare ">=2.2.0" .Values.airflowVersion }} +{{- if semverCompare ">=2.6.0" .Values.airflowVersion }} {{- if .Values.triggerer.enabled }} kind: Service apiVersion: v1 From 76a75c55db421be8592b64c1bb7802b7eb993721 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 2 Feb 2023 14:25:48 -0800 Subject: [PATCH 84/96] add ver check for service name --- chart/templates/triggerer/triggerer-deployment.yaml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/chart/templates/triggerer/triggerer-deployment.yaml b/chart/templates/triggerer/triggerer-deployment.yaml index 6707b87bdaf7d..68e0362efa8b1 100644 --- a/chart/templates/triggerer/triggerer-deployment.yaml +++ b/chart/templates/triggerer/triggerer-deployment.yaml @@ -46,8 +46,10 @@ metadata: {{- end }} spec: {{- if $persistence }} + {{- if semverCompare ">=2.6.0" .Values.airflowVersion }} serviceName: {{ .Release.Name }}-triggerer {{- end }} + {{- end }} replicas: {{ .Values.triggerer.replicas }} {{- if $revisionHistoryLimit }} revisionHistoryLimit: {{ $revisionHistoryLimit }} From d65fcc165d3b34e9a1d61d8ac51ef6b5f3977bd9 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 2 Feb 2023 14:44:59 -0800 Subject: [PATCH 85/96] version guard networkpolicty --- chart/templates/triggerer/triggerer-networkpolicy.yaml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/chart/templates/triggerer/triggerer-networkpolicy.yaml b/chart/templates/triggerer/triggerer-networkpolicy.yaml index e9d919607b78a..62443d290cfb8 100644 --- a/chart/templates/triggerer/triggerer-networkpolicy.yaml +++ b/chart/templates/triggerer/triggerer-networkpolicy.yaml @@ -18,6 +18,7 @@ ################################ ## Airflow triggerer NetworkPolicy ################################# +{{- if semverCompare ">=2.6.0" .Values.airflowVersion }} {{- if .Values.networkPolicies.enabled }} {{- if .Values.triggerer.enabled }} apiVersion: networking.k8s.io/v1 @@ -53,3 +54,4 @@ spec: port: {{ .Values.ports.triggererLogs }} {{- end }} {{- end }} +{{- end }} From d1f387e1839bb0a5c78ad034418a2b87c659af95 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 2 Feb 2023 14:45:35 -0800 Subject: [PATCH 86/96] format --- chart/templates/triggerer/triggerer-networkpolicy.yaml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/chart/templates/triggerer/triggerer-networkpolicy.yaml b/chart/templates/triggerer/triggerer-networkpolicy.yaml index 62443d290cfb8..c8762d0dae27c 100644 --- a/chart/templates/triggerer/triggerer-networkpolicy.yaml +++ b/chart/templates/triggerer/triggerer-networkpolicy.yaml @@ -15,9 +15,9 @@ # specific language governing permissions and limitations # under the License. -################################ +################################## ## Airflow triggerer NetworkPolicy -################################# +################################## {{- if semverCompare ">=2.6.0" .Values.airflowVersion }} {{- if .Values.networkPolicies.enabled }} {{- if .Values.triggerer.enabled }} From 69c06888c40272bf8f0affe94b202d20b5ad2280 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 2 Feb 2023 15:05:03 -0800 Subject: [PATCH 87/96] no version guard in triggerer sts --- chart/templates/triggerer/triggerer-deployment.yaml | 2 -- 1 file changed, 2 deletions(-) diff --git a/chart/templates/triggerer/triggerer-deployment.yaml b/chart/templates/triggerer/triggerer-deployment.yaml index 68e0362efa8b1..6707b87bdaf7d 100644 --- a/chart/templates/triggerer/triggerer-deployment.yaml +++ b/chart/templates/triggerer/triggerer-deployment.yaml @@ -46,10 +46,8 @@ metadata: {{- end }} spec: {{- if $persistence }} - {{- if semverCompare ">=2.6.0" .Values.airflowVersion }} serviceName: {{ .Release.Name }}-triggerer {{- end }} - {{- end }} replicas: {{ .Values.triggerer.replicas }} {{- if $revisionHistoryLimit }} revisionHistoryLimit: {{ $revisionHistoryLimit }} From 3bd48771b189eba99557f80c96c378db039f086e Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 3 Feb 2023 09:14:22 -0800 Subject: [PATCH 88/96] no STS in triggerer dep if < 2.6 --- chart/templates/triggerer/triggerer-deployment.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/chart/templates/triggerer/triggerer-deployment.yaml b/chart/templates/triggerer/triggerer-deployment.yaml index 6707b87bdaf7d..c9c84d72e0fee 100644 --- a/chart/templates/triggerer/triggerer-deployment.yaml +++ b/chart/templates/triggerer/triggerer-deployment.yaml @@ -20,7 +20,7 @@ ################################# {{- if semverCompare ">=2.2.0" .Values.airflowVersion }} {{- if .Values.triggerer.enabled }} -{{- $persistence := .Values.triggerer.persistence.enabled }} +{{- $persistence := and .Values.triggerer.persistence.enabled (semverCompare ">=2.6.0" .Values.airflowVersion) }} {{- $nodeSelector := or .Values.triggerer.nodeSelector .Values.nodeSelector }} {{- $affinity := or .Values.triggerer.affinity .Values.affinity }} {{- $tolerations := or .Values.triggerer.tolerations .Values.tolerations }} From 31a88cb8fdd26e1e71156ceb1e3076b5638558cc Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 3 Feb 2023 09:21:53 -0800 Subject: [PATCH 89/96] no port if < 2.6 --- chart/templates/triggerer/triggerer-deployment.yaml | 4 +++- chart/templates/triggerer/triggerer-networkpolicy.yaml | 2 +- chart/templates/triggerer/triggerer-service.yaml | 2 +- 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/chart/templates/triggerer/triggerer-deployment.yaml b/chart/templates/triggerer/triggerer-deployment.yaml index c9c84d72e0fee..90fc65c8e421b 100644 --- a/chart/templates/triggerer/triggerer-deployment.yaml +++ b/chart/templates/triggerer/triggerer-deployment.yaml @@ -20,7 +20,7 @@ ################################# {{- if semverCompare ">=2.2.0" .Values.airflowVersion }} {{- if .Values.triggerer.enabled }} -{{- $persistence := and .Values.triggerer.persistence.enabled (semverCompare ">=2.6.0" .Values.airflowVersion) }} +{{- $persistence := and .Values.triggerer.persistence.enabled (semverCompare ">=2.6.0" .Values.airflowVersion) }} # 2.6.0 is when triggerer logs serve introduced {{- $nodeSelector := or .Values.triggerer.nodeSelector .Values.nodeSelector }} {{- $affinity := or .Values.triggerer.affinity .Values.affinity }} {{- $tolerations := or .Values.triggerer.tolerations .Values.tolerations }} @@ -201,9 +201,11 @@ spec: {{- else }} {{- include "triggerer_liveness_check_command" . | nindent 16 }} {{- end }} + {{- if semverCompare ">=2.6.0" .Values.airflowVersion }} # 2.6.0 is when triggerer logs serve introduced ports: - name: triggerer-logs containerPort: {{ .Values.ports.triggererLogs }} + {{- end }} {{- if and (.Values.dags.gitSync.enabled) (not .Values.dags.persistence.enabled) }} {{- include "git_sync_container" . | indent 8 }} {{- end }} diff --git a/chart/templates/triggerer/triggerer-networkpolicy.yaml b/chart/templates/triggerer/triggerer-networkpolicy.yaml index c8762d0dae27c..01a6525d4768c 100644 --- a/chart/templates/triggerer/triggerer-networkpolicy.yaml +++ b/chart/templates/triggerer/triggerer-networkpolicy.yaml @@ -18,7 +18,7 @@ ################################## ## Airflow triggerer NetworkPolicy ################################## -{{- if semverCompare ">=2.6.0" .Values.airflowVersion }} +{{- if semverCompare ">=2.6.0" .Values.airflowVersion }} # 2.6.0 is when triggerer logs serve introduced {{- if .Values.networkPolicies.enabled }} {{- if .Values.triggerer.enabled }} apiVersion: networking.k8s.io/v1 diff --git a/chart/templates/triggerer/triggerer-service.yaml b/chart/templates/triggerer/triggerer-service.yaml index 1f96fac70784a..51463d4717240 100644 --- a/chart/templates/triggerer/triggerer-service.yaml +++ b/chart/templates/triggerer/triggerer-service.yaml @@ -18,7 +18,7 @@ ################################ ## Airflow triggerer Service ################################# -{{- if semverCompare ">=2.6.0" .Values.airflowVersion }} +{{- if semverCompare ">=2.6.0" .Values.airflowVersion }} # 2.6.0 is when triggerer logs serve introduced {{- if .Values.triggerer.enabled }} kind: Service apiVersion: v1 From 4799f1ed5a99c894e1c6ed8b14f303ff76743b2d Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 3 Feb 2023 09:23:44 -0800 Subject: [PATCH 90/96] no comment --- chart/templates/triggerer/triggerer-deployment.yaml | 4 ++-- chart/templates/triggerer/triggerer-networkpolicy.yaml | 2 +- chart/templates/triggerer/triggerer-service.yaml | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/chart/templates/triggerer/triggerer-deployment.yaml b/chart/templates/triggerer/triggerer-deployment.yaml index 90fc65c8e421b..bd684001ba58b 100644 --- a/chart/templates/triggerer/triggerer-deployment.yaml +++ b/chart/templates/triggerer/triggerer-deployment.yaml @@ -20,7 +20,7 @@ ################################# {{- if semverCompare ">=2.2.0" .Values.airflowVersion }} {{- if .Values.triggerer.enabled }} -{{- $persistence := and .Values.triggerer.persistence.enabled (semverCompare ">=2.6.0" .Values.airflowVersion) }} # 2.6.0 is when triggerer logs serve introduced +{{- $persistence := and .Values.triggerer.persistence.enabled (semverCompare ">=2.6.0" .Values.airflowVersion) }} {{- $nodeSelector := or .Values.triggerer.nodeSelector .Values.nodeSelector }} {{- $affinity := or .Values.triggerer.affinity .Values.affinity }} {{- $tolerations := or .Values.triggerer.tolerations .Values.tolerations }} @@ -201,7 +201,7 @@ spec: {{- else }} {{- include "triggerer_liveness_check_command" . | nindent 16 }} {{- end }} - {{- if semverCompare ">=2.6.0" .Values.airflowVersion }} # 2.6.0 is when triggerer logs serve introduced + {{- if semverCompare ">=2.6.0" .Values.airflowVersion }} ports: - name: triggerer-logs containerPort: {{ .Values.ports.triggererLogs }} diff --git a/chart/templates/triggerer/triggerer-networkpolicy.yaml b/chart/templates/triggerer/triggerer-networkpolicy.yaml index 01a6525d4768c..c8762d0dae27c 100644 --- a/chart/templates/triggerer/triggerer-networkpolicy.yaml +++ b/chart/templates/triggerer/triggerer-networkpolicy.yaml @@ -18,7 +18,7 @@ ################################## ## Airflow triggerer NetworkPolicy ################################## -{{- if semverCompare ">=2.6.0" .Values.airflowVersion }} # 2.6.0 is when triggerer logs serve introduced +{{- if semverCompare ">=2.6.0" .Values.airflowVersion }} {{- if .Values.networkPolicies.enabled }} {{- if .Values.triggerer.enabled }} apiVersion: networking.k8s.io/v1 diff --git a/chart/templates/triggerer/triggerer-service.yaml b/chart/templates/triggerer/triggerer-service.yaml index 51463d4717240..1f96fac70784a 100644 --- a/chart/templates/triggerer/triggerer-service.yaml +++ b/chart/templates/triggerer/triggerer-service.yaml @@ -18,7 +18,7 @@ ################################ ## Airflow triggerer Service ################################# -{{- if semverCompare ">=2.6.0" .Values.airflowVersion }} # 2.6.0 is when triggerer logs serve introduced +{{- if semverCompare ">=2.6.0" .Values.airflowVersion }} {{- if .Values.triggerer.enabled }} kind: Service apiVersion: v1 From ccd82eaeed69691045a004b294c87335e863e505 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 3 Feb 2023 09:28:02 -0800 Subject: [PATCH 91/96] template comment --- chart/templates/triggerer/triggerer-deployment.yaml | 2 ++ chart/templates/triggerer/triggerer-networkpolicy.yaml | 1 + chart/templates/triggerer/triggerer-service.yaml | 1 + 3 files changed, 4 insertions(+) diff --git a/chart/templates/triggerer/triggerer-deployment.yaml b/chart/templates/triggerer/triggerer-deployment.yaml index bd684001ba58b..0d38ac1d66eb8 100644 --- a/chart/templates/triggerer/triggerer-deployment.yaml +++ b/chart/templates/triggerer/triggerer-deployment.yaml @@ -20,6 +20,7 @@ ################################# {{- if semverCompare ">=2.2.0" .Values.airflowVersion }} {{- if .Values.triggerer.enabled }} +{{- /* Airflow version 2.6.0 is when triggerer logs serve introduced */ -}} {{- $persistence := and .Values.triggerer.persistence.enabled (semverCompare ">=2.6.0" .Values.airflowVersion) }} {{- $nodeSelector := or .Values.triggerer.nodeSelector .Values.nodeSelector }} {{- $affinity := or .Values.triggerer.affinity .Values.affinity }} @@ -201,6 +202,7 @@ spec: {{- else }} {{- include "triggerer_liveness_check_command" . | nindent 16 }} {{- end }} + {{- /* Airflow version 2.6.0 is when triggerer logs serve introduced */ -}} {{- if semverCompare ">=2.6.0" .Values.airflowVersion }} ports: - name: triggerer-logs diff --git a/chart/templates/triggerer/triggerer-networkpolicy.yaml b/chart/templates/triggerer/triggerer-networkpolicy.yaml index c8762d0dae27c..2d51f2f91a0ca 100644 --- a/chart/templates/triggerer/triggerer-networkpolicy.yaml +++ b/chart/templates/triggerer/triggerer-networkpolicy.yaml @@ -18,6 +18,7 @@ ################################## ## Airflow triggerer NetworkPolicy ################################## +{{- /* Airflow version 2.6.0 is when triggerer logs serve introduced */ -}} {{- if semverCompare ">=2.6.0" .Values.airflowVersion }} {{- if .Values.networkPolicies.enabled }} {{- if .Values.triggerer.enabled }} diff --git a/chart/templates/triggerer/triggerer-service.yaml b/chart/templates/triggerer/triggerer-service.yaml index 1f96fac70784a..68536b46ec47f 100644 --- a/chart/templates/triggerer/triggerer-service.yaml +++ b/chart/templates/triggerer/triggerer-service.yaml @@ -18,6 +18,7 @@ ################################ ## Airflow triggerer Service ################################# +{{- /* Airflow version 2.6.0 is when triggerer logs serve introduced */ -}} {{- if semverCompare ">=2.6.0" .Values.airflowVersion }} {{- if .Values.triggerer.enabled }} kind: Service From 611ae28ddfc1e88640ee0a6db11545a144844eea Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 3 Feb 2023 10:49:28 -0800 Subject: [PATCH 92/96] update chart default airflow version --- chart/Chart.yaml | 4 ++-- chart/values.schema.json | 4 ++-- chart/values.yaml | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/chart/Chart.yaml b/chart/Chart.yaml index ba5c27f09b4ad..d079e9d7c5272 100644 --- a/chart/Chart.yaml +++ b/chart/Chart.yaml @@ -19,8 +19,8 @@ --- apiVersion: v2 name: airflow -version: 1.8.0-dev -appVersion: 2.5.1 +version: 1.9.0-dev +appVersion: 2.6.0 description: The official Helm chart to deploy Apache Airflow, a platform to programmatically author, schedule, and monitor workflows home: https://airflow.apache.org/ diff --git a/chart/values.schema.json b/chart/values.schema.json index 803dadda77c91..6928215fc91b2 100644 --- a/chart/values.schema.json +++ b/chart/values.schema.json @@ -77,13 +77,13 @@ "defaultAirflowTag": { "description": "Default airflow tag to deploy.", "type": "string", - "default": "2.5.1", + "default": "2.6.0", "x-docsSection": "Common" }, "airflowVersion": { "description": "Airflow version (Used to make some decisions based on Airflow Version being deployed).", "type": "string", - "default": "2.5.1", + "default": "2.6.0", "x-docsSection": "Common" }, "securityContext": { diff --git a/chart/values.yaml b/chart/values.yaml index 60a957591744b..c716d59cafa07 100644 --- a/chart/values.yaml +++ b/chart/values.yaml @@ -49,10 +49,10 @@ airflowHome: /opt/airflow defaultAirflowRepository: apache/airflow # Default airflow tag to deploy -defaultAirflowTag: "2.5.1" +defaultAirflowTag: "2.6.0" # Airflow version (Used to make some decisions based on Airflow Version being deployed) -airflowVersion: "2.5.1" +airflowVersion: "2.6.0" # Images images: From 566814ddd1f27c0e781520aee47243a7e068481d Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 3 Feb 2023 13:41:08 -0800 Subject: [PATCH 93/96] Revert "update chart default airflow version" This reverts commit 611ae28ddfc1e88640ee0a6db11545a144844eea. --- chart/Chart.yaml | 4 ++-- chart/values.schema.json | 4 ++-- chart/values.yaml | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/chart/Chart.yaml b/chart/Chart.yaml index d079e9d7c5272..ba5c27f09b4ad 100644 --- a/chart/Chart.yaml +++ b/chart/Chart.yaml @@ -19,8 +19,8 @@ --- apiVersion: v2 name: airflow -version: 1.9.0-dev -appVersion: 2.6.0 +version: 1.8.0-dev +appVersion: 2.5.1 description: The official Helm chart to deploy Apache Airflow, a platform to programmatically author, schedule, and monitor workflows home: https://airflow.apache.org/ diff --git a/chart/values.schema.json b/chart/values.schema.json index 6928215fc91b2..803dadda77c91 100644 --- a/chart/values.schema.json +++ b/chart/values.schema.json @@ -77,13 +77,13 @@ "defaultAirflowTag": { "description": "Default airflow tag to deploy.", "type": "string", - "default": "2.6.0", + "default": "2.5.1", "x-docsSection": "Common" }, "airflowVersion": { "description": "Airflow version (Used to make some decisions based on Airflow Version being deployed).", "type": "string", - "default": "2.6.0", + "default": "2.5.1", "x-docsSection": "Common" }, "securityContext": { diff --git a/chart/values.yaml b/chart/values.yaml index c716d59cafa07..60a957591744b 100644 --- a/chart/values.yaml +++ b/chart/values.yaml @@ -49,10 +49,10 @@ airflowHome: /opt/airflow defaultAirflowRepository: apache/airflow # Default airflow tag to deploy -defaultAirflowTag: "2.6.0" +defaultAirflowTag: "2.5.1" # Airflow version (Used to make some decisions based on Airflow Version being deployed) -airflowVersion: "2.6.0" +airflowVersion: "2.5.1" # Images images: From 522524ce1e25ddff3c5c3a17219b339a550895bc Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 3 Feb 2023 13:44:33 -0800 Subject: [PATCH 94/96] revert chart tests --- tests/charts/test_basic_helm_chart.py | 10 +++---- tests/charts/test_extra_env_env_from.py | 2 +- tests/charts/test_rbac.py | 3 +- tests/charts/test_triggerer.py | 40 +++++-------------------- 4 files changed, 13 insertions(+), 42 deletions(-) diff --git a/tests/charts/test_basic_helm_chart.py b/tests/charts/test_basic_helm_chart.py index 8bca84399513e..171a012d4dead 100644 --- a/tests/charts/test_basic_helm_chart.py +++ b/tests/charts/test_basic_helm_chart.py @@ -27,7 +27,7 @@ from tests.charts.helm_template_generator import render_chart -OBJECT_COUNT_IN_BASIC_DEPLOYMENT = 36 +OBJECT_COUNT_IN_BASIC_DEPLOYMENT = 35 class TestBaseChartTest: @@ -88,15 +88,14 @@ def test_basic_deployments(self, version): ("Service", "test-basic-postgresql"), ("Service", "test-basic-redis"), ("Service", "test-basic-statsd"), - ("Service", "test-basic-triggerer"), ("Service", "test-basic-webserver"), ("Service", "test-basic-worker"), ("Deployment", "test-basic-scheduler"), ("Deployment", "test-basic-statsd"), + ("Deployment", "test-basic-triggerer"), ("Deployment", "test-basic-webserver"), ("StatefulSet", "test-basic-postgresql"), ("StatefulSet", "test-basic-redis"), - ("StatefulSet", "test-basic-triggerer"), ("StatefulSet", "test-basic-worker"), ("Job", "test-basic-create-user"), ("Job", "test-basic-run-airflow-migrations"), @@ -166,16 +165,15 @@ def test_basic_deployment_with_standalone_dag_processor(self, version): ("Service", "test-basic-postgresql"), ("Service", "test-basic-redis"), ("Service", "test-basic-statsd"), - ("Service", "test-basic-triggerer"), ("Service", "test-basic-webserver"), ("Service", "test-basic-worker"), ("Deployment", "test-basic-scheduler"), ("Deployment", "test-basic-statsd"), + ("Deployment", "test-basic-triggerer"), ("Deployment", "test-basic-dag-processor"), ("Deployment", "test-basic-webserver"), ("StatefulSet", "test-basic-postgresql"), ("StatefulSet", "test-basic-redis"), - ("StatefulSet", "test-basic-triggerer"), ("StatefulSet", "test-basic-worker"), ("Job", "test-basic-create-user"), ("Job", "test-basic-run-airflow-migrations"), @@ -341,7 +339,7 @@ def test_labels_are_valid(self): (f"{release_name}-worker", "Service", "worker"), (f"{release_name}-worker", "StatefulSet", "worker"), (f"{release_name}-worker-policy", "NetworkPolicy", "airflow-worker-policy"), - (f"{release_name}-triggerer", "StatefulSet", "triggerer"), + (f"{release_name}-triggerer", "Deployment", "triggerer"), (f"{release_name}-dag-processor", "Deployment", "dag-processor"), (f"{release_name}-logs", "PersistentVolumeClaim", "logs-pvc"), (f"{release_name}-dags", "PersistentVolumeClaim", "dags-pvc"), diff --git a/tests/charts/test_extra_env_env_from.py b/tests/charts/test_extra_env_env_from.py index cc356693162b2..b32b6fae5ef50 100644 --- a/tests/charts/test_extra_env_env_from.py +++ b/tests/charts/test_extra_env_env_from.py @@ -59,7 +59,7 @@ ), ), ( - ("StatefulSet", f"{RELEASE_NAME}-triggerer"), + ("Deployment", f"{RELEASE_NAME}-triggerer"), ( "spec.template.spec.initContainers[0]", "spec.template.spec.containers[0]", diff --git a/tests/charts/test_rbac.py b/tests/charts/test_rbac.py index 8ea74bc09a570..ca78b1e27d550 100644 --- a/tests/charts/test_rbac.py +++ b/tests/charts/test_rbac.py @@ -35,16 +35,15 @@ ("Service", "test-rbac-flower"), ("Service", "test-rbac-pgbouncer"), ("Service", "test-rbac-redis"), - ("Service", "test-rbac-triggerer"), ("Service", "test-rbac-worker"), ("Deployment", "test-rbac-scheduler"), ("Deployment", "test-rbac-statsd"), ("Deployment", "test-rbac-webserver"), ("Deployment", "test-rbac-flower"), ("Deployment", "test-rbac-pgbouncer"), + ("Deployment", "test-rbac-triggerer"), ("StatefulSet", "test-rbac-postgresql"), ("StatefulSet", "test-rbac-redis"), - ("StatefulSet", "test-rbac-triggerer"), ("StatefulSet", "test-rbac-worker"), ("Secret", "test-rbac-broker-url"), ("Secret", "test-rbac-fernet-key"), diff --git a/tests/charts/test_triggerer.py b/tests/charts/test_triggerer.py index e5561d27b2b9b..8e2503455012e 100644 --- a/tests/charts/test_triggerer.py +++ b/tests/charts/test_triggerer.py @@ -217,7 +217,7 @@ def test_should_create_valid_affinity_tolerations_and_node_selector(self): show_only=["templates/triggerer/triggerer-deployment.yaml"], ) - assert "StatefulSet" == jmespath.search("kind", docs[0]) + assert "Deployment" == jmespath.search("kind", docs[0]) assert "foo" == jmespath.search( "spec.template.spec.affinity.nodeAffinity." "requiredDuringSchedulingIgnoredDuringExecution." @@ -362,10 +362,7 @@ def test_livenessprobe_values_are_configurable(self): ) def test_logs_persistence_changes_volume(self, log_persistence_values, expected_volume): docs = render_chart( - values={ - "triggerer": {"persistence": {"enabled": False}}, - "logs": {"persistence": log_persistence_values}, - }, + values={"logs": {"persistence": log_persistence_values}}, show_only=["templates/triggerer/triggerer-deployment.yaml"], ) @@ -410,43 +407,20 @@ def test_resources_are_not_added_by_default(self): assert jmespath.search("spec.template.spec.containers[0].resources", docs[0]) == {} @pytest.mark.parametrize( - "persistence, update_strategy, expected_update_strategy", - [ - (False, None, None), - (True, {"rollingUpdate": {"partition": 0}}, {"rollingUpdate": {"partition": 0}}), - (True, None, None), - ], - ) - def test_update_strategy(self, persistence, update_strategy, expected_update_strategy): - docs = render_chart( - values={ - "executor": "CeleryExecutor", - "triggerer": { - "persistence": {"enabled": persistence}, - "updateStrategy": update_strategy, - }, - }, - show_only=["templates/triggerer/triggerer-deployment.yaml"], - ) - - assert expected_update_strategy == jmespath.search("spec.updateStrategy", docs[0]) - - @pytest.mark.parametrize( - "persistence, strategy, expected_strategy", + "strategy, expected_strategy", [ - (True, None, None), + (None, None), ( - False, {"rollingUpdate": {"maxSurge": "100%", "maxUnavailable": "50%"}}, {"rollingUpdate": {"maxSurge": "100%", "maxUnavailable": "50%"}}, ), - (False, None, None), ], ) - def test_strategy(self, persistence, strategy, expected_strategy): + def test_strategy(self, strategy, expected_strategy): + """strategy should be used when we aren't using both LocalExecutor and workers.persistence""" docs = render_chart( values={ - "triggerer": {"persistence": {"enabled": persistence}, "strategy": strategy}, + "triggerer": {"strategy": strategy}, }, show_only=["templates/triggerer/triggerer-deployment.yaml"], ) From 3b3d1bf84678437f0f09dc1003076abf9439a97a Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 3 Feb 2023 14:21:03 -0800 Subject: [PATCH 95/96] fix tests --- tests/charts/test_basic_helm_chart.py | 36 ++++++++++++++++--------- tests/charts/test_triggerer.py | 39 ++++++++++++++++++++++----- 2 files changed, 56 insertions(+), 19 deletions(-) diff --git a/tests/charts/test_basic_helm_chart.py b/tests/charts/test_basic_helm_chart.py index 171a012d4dead..2242ba1efe199 100644 --- a/tests/charts/test_basic_helm_chart.py +++ b/tests/charts/test_basic_helm_chart.py @@ -39,9 +39,11 @@ def _get_values_with_version(self, values, version): def _get_object_count(self, version): if version == "2.3.2": return OBJECT_COUNT_IN_BASIC_DEPLOYMENT + 1 + elif version == "2.6.0": + return OBJECT_COUNT_IN_BASIC_DEPLOYMENT + 1 return OBJECT_COUNT_IN_BASIC_DEPLOYMENT - @pytest.mark.parametrize("version", ["2.3.2", "2.4.0", "default"]) + @pytest.mark.parametrize("version", ["2.3.2", "2.4.0", "2.6.0", "default"]) def test_basic_deployments(self, version): expected_object_count_in_basic_deployment = self._get_object_count(version) k8s_objects = render_chart( @@ -60,10 +62,7 @@ def test_basic_deployments(self, version): list_of_kind_names_tuples = { (k8s_object["kind"], k8s_object["metadata"]["name"]) for k8s_object in k8s_objects } - if version == "2.3.2": - assert ("Secret", "test-basic-airflow-result-backend") in list_of_kind_names_tuples - list_of_kind_names_tuples.remove(("Secret", "test-basic-airflow-result-backend")) - assert list_of_kind_names_tuples == { + expected = { ("ServiceAccount", "test-basic-create-user-job"), ("ServiceAccount", "test-basic-migrate-database-job"), ("ServiceAccount", "test-basic-redis"), @@ -92,7 +91,7 @@ def test_basic_deployments(self, version): ("Service", "test-basic-worker"), ("Deployment", "test-basic-scheduler"), ("Deployment", "test-basic-statsd"), - ("Deployment", "test-basic-triggerer"), + (self.default_trigger_obj(version), "test-basic-triggerer"), ("Deployment", "test-basic-webserver"), ("StatefulSet", "test-basic-postgresql"), ("StatefulSet", "test-basic-redis"), @@ -100,6 +99,11 @@ def test_basic_deployments(self, version): ("Job", "test-basic-create-user"), ("Job", "test-basic-run-airflow-migrations"), } + if version == "2.3.2": + expected.add(("Secret", "test-basic-airflow-result-backend")) + if version == "2.6.0": + expected.add(("Service", "test-basic-triggerer")) + assert list_of_kind_names_tuples == expected assert expected_object_count_in_basic_deployment == len(k8s_objects) for k8s_object in k8s_objects: labels = jmespath.search("metadata.labels", k8s_object) or {} @@ -114,7 +118,7 @@ def test_basic_deployments(self, version): "test-label" ), f"Missing label test-label on {k8s_name}. Current labels: {labels}" - @pytest.mark.parametrize("version", ["2.3.2", "2.4.0", "default"]) + @pytest.mark.parametrize("version", ["2.3.2", "2.4.0", "2.6.0", "default"]) def test_basic_deployment_with_standalone_dag_processor(self, version): # Dag Processor creates two extra objects compared to the basic deployment object_count_in_basic_deployment = self._get_object_count(version) @@ -136,10 +140,7 @@ def test_basic_deployment_with_standalone_dag_processor(self, version): list_of_kind_names_tuples = { (k8s_object["kind"], k8s_object["metadata"]["name"]) for k8s_object in k8s_objects } - if version == "2.3.2": - assert ("Secret", "test-basic-airflow-result-backend") in list_of_kind_names_tuples - list_of_kind_names_tuples.remove(("Secret", "test-basic-airflow-result-backend")) - assert list_of_kind_names_tuples == { + expected = { ("ServiceAccount", "test-basic-create-user-job"), ("ServiceAccount", "test-basic-migrate-database-job"), ("ServiceAccount", "test-basic-redis"), @@ -169,7 +170,7 @@ def test_basic_deployment_with_standalone_dag_processor(self, version): ("Service", "test-basic-worker"), ("Deployment", "test-basic-scheduler"), ("Deployment", "test-basic-statsd"), - ("Deployment", "test-basic-triggerer"), + (self.default_trigger_obj(version), "test-basic-triggerer"), ("Deployment", "test-basic-dag-processor"), ("Deployment", "test-basic-webserver"), ("StatefulSet", "test-basic-postgresql"), @@ -178,6 +179,11 @@ def test_basic_deployment_with_standalone_dag_processor(self, version): ("Job", "test-basic-create-user"), ("Job", "test-basic-run-airflow-migrations"), } + if version == "2.3.2": + expected.add(("Secret", "test-basic-airflow-result-backend")) + if version == "2.6.0": + expected.add(("Service", "test-basic-triggerer")) + assert list_of_kind_names_tuples == expected assert expected_object_count_with_standalone_scheduler == len(k8s_objects) for k8s_object in k8s_objects: labels = jmespath.search("metadata.labels", k8s_object) or {} @@ -339,7 +345,7 @@ def test_labels_are_valid(self): (f"{release_name}-worker", "Service", "worker"), (f"{release_name}-worker", "StatefulSet", "worker"), (f"{release_name}-worker-policy", "NetworkPolicy", "airflow-worker-policy"), - (f"{release_name}-triggerer", "Deployment", "triggerer"), + # (f"{release_name}-triggerer", "StatefulSet", "triggerer"), (f"{release_name}-dag-processor", "Deployment", "dag-processor"), (f"{release_name}-logs", "PersistentVolumeClaim", "logs-pvc"), (f"{release_name}-dags", "PersistentVolumeClaim", "dags-pvc"), @@ -533,3 +539,7 @@ def test_postgres_connection_url_name_override(self): "postgresql://postgres:postgres@overrideName:5432/postgres?sslmode=disable" == base64.b64decode(doc["data"]["connection"]).decode("utf-8") ) + + @staticmethod + def default_trigger_obj(version): + return "StatefulSet" if version == "2.6.0" else "Deployment" diff --git a/tests/charts/test_triggerer.py b/tests/charts/test_triggerer.py index 8e2503455012e..7beb872dcaea7 100644 --- a/tests/charts/test_triggerer.py +++ b/tests/charts/test_triggerer.py @@ -362,7 +362,10 @@ def test_livenessprobe_values_are_configurable(self): ) def test_logs_persistence_changes_volume(self, log_persistence_values, expected_volume): docs = render_chart( - values={"logs": {"persistence": log_persistence_values}}, + values={ + "triggerer": {"persistence": {"enabled": False}}, + "logs": {"persistence": log_persistence_values}, + }, show_only=["templates/triggerer/triggerer-deployment.yaml"], ) @@ -407,20 +410,44 @@ def test_resources_are_not_added_by_default(self): assert jmespath.search("spec.template.spec.containers[0].resources", docs[0]) == {} @pytest.mark.parametrize( - "strategy, expected_strategy", + "persistence, update_strategy, expected_update_strategy", + [ + (False, None, None), + (True, {"rollingUpdate": {"partition": 0}}, {"rollingUpdate": {"partition": 0}}), + (True, None, None), + ], + ) + def test_update_strategy(self, persistence, update_strategy, expected_update_strategy): + docs = render_chart( + values={ + "airflowVersion": "2.6.0", + "executor": "CeleryExecutor", + "triggerer": { + "persistence": {"enabled": persistence}, + "updateStrategy": update_strategy, + }, + }, + show_only=["templates/triggerer/triggerer-deployment.yaml"], + ) + + assert expected_update_strategy == jmespath.search("spec.updateStrategy", docs[0]) + + @pytest.mark.parametrize( + "persistence, strategy, expected_strategy", [ - (None, None), + (True, None, None), ( + False, {"rollingUpdate": {"maxSurge": "100%", "maxUnavailable": "50%"}}, {"rollingUpdate": {"maxSurge": "100%", "maxUnavailable": "50%"}}, ), + (False, None, None), ], ) - def test_strategy(self, strategy, expected_strategy): - """strategy should be used when we aren't using both LocalExecutor and workers.persistence""" + def test_strategy(self, persistence, strategy, expected_strategy): docs = render_chart( values={ - "triggerer": {"strategy": strategy}, + "triggerer": {"persistence": {"enabled": persistence}, "strategy": strategy}, }, show_only=["templates/triggerer/triggerer-deployment.yaml"], ) From d097d5b7dcc92fd634a79a85624fccdaf683fed6 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 3 Feb 2023 14:41:03 -0800 Subject: [PATCH 96/96] fix tests --- tests/charts/test_extra_env_env_from.py | 3 ++- tests/charts/test_rbac.py | 34 +++++++++++++++---------- 2 files changed, 22 insertions(+), 15 deletions(-) diff --git a/tests/charts/test_extra_env_env_from.py b/tests/charts/test_extra_env_env_from.py index b32b6fae5ef50..0dc5ff845a356 100644 --- a/tests/charts/test_extra_env_env_from.py +++ b/tests/charts/test_extra_env_env_from.py @@ -59,7 +59,7 @@ ), ), ( - ("Deployment", f"{RELEASE_NAME}-triggerer"), + ("StatefulSet", f"{RELEASE_NAME}-triggerer"), ( "spec.template.spec.initContainers[0]", "spec.template.spec.containers[0]", @@ -80,6 +80,7 @@ class TestExtraEnvEnvFrom: def setup_class(cls) -> None: values_str = textwrap.dedent( """ + airflowVersion: "2.6.0" flower: enabled: true extraEnvFrom: | diff --git a/tests/charts/test_rbac.py b/tests/charts/test_rbac.py index ca78b1e27d550..1f155d501a17d 100644 --- a/tests/charts/test_rbac.py +++ b/tests/charts/test_rbac.py @@ -16,6 +16,8 @@ # under the License. from __future__ import annotations +from copy import copy + import jmespath import pytest @@ -41,7 +43,6 @@ ("Deployment", "test-rbac-webserver"), ("Deployment", "test-rbac-flower"), ("Deployment", "test-rbac-pgbouncer"), - ("Deployment", "test-rbac-triggerer"), ("StatefulSet", "test-rbac-postgresql"), ("StatefulSet", "test-rbac-redis"), ("StatefulSet", "test-rbac-worker"), @@ -112,14 +113,19 @@ def _get_values_with_version(self, values, version): values["airflowVersion"] = version return values - def _get_object_count(self, version): + @staticmethod + def _get_object_tuples(version): + tuples = copy(DEPLOYMENT_NO_RBAC_NO_SA_KIND_NAME_TUPLES) + if version == "2.6.0": + tuples.append(("Service", "test-rbac-triggerer")) + tuples.append(("StatefulSet", "test-rbac-triggerer")) + else: + tuples.append(("Deployment", "test-rbac-triggerer")) if version == "2.3.2": - return [ - ("Secret", "test-rbac-airflow-result-backend") - ] + DEPLOYMENT_NO_RBAC_NO_SA_KIND_NAME_TUPLES - return DEPLOYMENT_NO_RBAC_NO_SA_KIND_NAME_TUPLES + tuples.append(("Secret", "test-rbac-airflow-result-backend")) + return tuples - @pytest.mark.parametrize("version", ["2.3.2", "2.4.0", "default"]) + @pytest.mark.parametrize("version", ["2.3.2", "2.4.0", "2.6.0", "default"]) def test_deployments_no_rbac_no_sa(self, version): k8s_objects = render_chart( "test-rbac", @@ -155,9 +161,9 @@ def test_deployments_no_rbac_no_sa(self, version): list_of_kind_names_tuples = [ (k8s_object["kind"], k8s_object["metadata"]["name"]) for k8s_object in k8s_objects ] - assert sorted(list_of_kind_names_tuples) == sorted(self._get_object_count(version)) + assert sorted(list_of_kind_names_tuples) == sorted(self._get_object_tuples(version)) - @pytest.mark.parametrize("version", ["2.3.2", "2.4.0", "default"]) + @pytest.mark.parametrize("version", ["2.3.2", "2.4.0", "2.6.0", "default"]) def test_deployments_no_rbac_with_sa(self, version): k8s_objects = render_chart( "test-rbac", @@ -175,10 +181,10 @@ def test_deployments_no_rbac_with_sa(self, version): list_of_kind_names_tuples = [ (k8s_object["kind"], k8s_object["metadata"]["name"]) for k8s_object in k8s_objects ] - real_list_of_kind_names = self._get_object_count(version) + SERVICE_ACCOUNT_NAME_TUPLES + real_list_of_kind_names = self._get_object_tuples(version) + SERVICE_ACCOUNT_NAME_TUPLES assert sorted(list_of_kind_names_tuples) == sorted(real_list_of_kind_names) - @pytest.mark.parametrize("version", ["2.3.2", "2.4.0", "default"]) + @pytest.mark.parametrize("version", ["2.3.2", "2.4.0", "2.6.0", "default"]) def test_deployments_with_rbac_no_sa(self, version): k8s_objects = render_chart( "test-rbac", @@ -213,10 +219,10 @@ def test_deployments_with_rbac_no_sa(self, version): list_of_kind_names_tuples = [ (k8s_object["kind"], k8s_object["metadata"]["name"]) for k8s_object in k8s_objects ] - real_list_of_kind_names = self._get_object_count(version) + RBAC_ENABLED_KIND_NAME_TUPLES + real_list_of_kind_names = self._get_object_tuples(version) + RBAC_ENABLED_KIND_NAME_TUPLES assert sorted(list_of_kind_names_tuples) == sorted(real_list_of_kind_names) - @pytest.mark.parametrize("version", ["2.3.2", "2.4.0", "default"]) + @pytest.mark.parametrize("version", ["2.3.2", "2.4.0", "2.6.0", "default"]) def test_deployments_with_rbac_with_sa(self, version): k8s_objects = render_chart( "test-rbac", @@ -234,7 +240,7 @@ def test_deployments_with_rbac_with_sa(self, version): (k8s_object["kind"], k8s_object["metadata"]["name"]) for k8s_object in k8s_objects ] real_list_of_kind_names = ( - self._get_object_count(version) + SERVICE_ACCOUNT_NAME_TUPLES + RBAC_ENABLED_KIND_NAME_TUPLES + self._get_object_tuples(version) + SERVICE_ACCOUNT_NAME_TUPLES + RBAC_ENABLED_KIND_NAME_TUPLES ) assert sorted(list_of_kind_names_tuples) == sorted(real_list_of_kind_names)