From 2233ddc2f944147db1a655d4be6a277b35fcfd43 Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Mon, 28 Jul 2025 23:05:23 +0800 Subject: [PATCH 01/46] Introduce ElasticsearchRemoteLogIO --- .../airflow_local_settings.py | 14 +++ .../elasticsearch/log/es_task_handler.py | 87 ++++++++++++------- 2 files changed, 70 insertions(+), 31 deletions(-) diff --git a/airflow-core/src/airflow/config_templates/airflow_local_settings.py b/airflow-core/src/airflow/config_templates/airflow_local_settings.py index 7c84ba3f3b133..7b4177ed93c16 100644 --- a/airflow-core/src/airflow/config_templates/airflow_local_settings.py +++ b/airflow-core/src/airflow/config_templates/airflow_local_settings.py @@ -287,6 +287,8 @@ def _default_conn_name_from(mod_path, hook_name): ) remote_task_handler_kwargs = {} elif ELASTICSEARCH_HOST: + from airflow.providers.elasticsearch.log.es_task_handler import ElasticsearchRemoteLogIO + ELASTICSEARCH_END_OF_LOG_MARK: str = conf.get_mandatory_value("elasticsearch", "END_OF_LOG_MARK") ELASTICSEARCH_FRONTEND: str = conf.get_mandatory_value("elasticsearch", "frontend") ELASTICSEARCH_WRITE_STDOUT: bool = conf.getboolean("elasticsearch", "WRITE_STDOUT") @@ -315,6 +317,18 @@ def _default_conn_name_from(mod_path, hook_name): }, } + if ELASTICSEARCH_WRITE_TO_ES: + REMOTE_TASK_LOG = ElasticsearchRemoteLogIO( + **( + { + "host": ELASTICSEARCH_HOST, + "target_index": ELASTICSEARCH_TARGET_INDEX, + "base_log_folder": BASE_LOG_FOLDER, + "delete_local_copy": delete_local_copy, + } + ) + ) + DEFAULT_LOGGING_CONFIG["handlers"].update(ELASTIC_REMOTE_HANDLERS) elif OPENSEARCH_HOST: OPENSEARCH_END_OF_LOG_MARK: str = conf.get_mandatory_value("opensearch", "END_OF_LOG_MARK") diff --git a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py index 4001331e8f72b..738cce4186610 100644 --- a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py @@ -22,10 +22,12 @@ import json import logging import os +from pathlib import Path import pathlib import shutil import sys import time +import attrs from collections import defaultdict from collections.abc import Callable from operator import attrgetter @@ -57,6 +59,7 @@ from airflow.models.taskinstance import TaskInstance, TaskInstanceKey from airflow.utils.log.file_task_handler import LogMetadata + from airflow.sdk.types import RuntimeTaskInstanceProtocol as RuntimeTI LOG_LINE_DEFAULTS = {"exc_text": "", "stack_info": ""} @@ -331,9 +334,10 @@ def _read( # end_of_log_mark may contain characters like '\n' which is needed to # have the log uploaded but will not be stored in elasticsearch. + print(f"self.end_of_log_mark = {self.end_of_log_mark}") metadata["end_of_log"] = False if logs_by_host: - if any(x[-1].message == self.end_of_log_mark for x in logs_by_host.values()): + if any(x[-1].event == self.end_of_log_mark for x in logs_by_host.values()): metadata["end_of_log"] = True cur_ts = pendulum.now() @@ -464,6 +468,7 @@ def set_context(self, ti: TaskInstance, *, identifier: str | None = None) -> Non :param ti: task instance object :param identifier: if set, identifies the Airflow component which is relaying logs from exceptional scenarios related to the task instance + TODO: This API should be removed in airflow 3 """ is_trigger_log_context = getattr(ti, "is_trigger_log_context", None) is_ti_raw = getattr(ti, "raw", None) @@ -504,6 +509,7 @@ def close(self) -> None: # calling close method. Here we check if logger is already # closed to prevent uploading the log to remote storage multiple # times when `logging.shutdown` is called. + # TODO: This API should be simplied since Airflow 3 no longer requires this API for writing log to ES if self.closed: return @@ -530,18 +536,6 @@ def close(self) -> None: self.handler.close() sys.stdout = sys.__stdout__ - if self.write_to_es and not self.write_stdout: - full_path = self.handler.baseFilename # type: ignore[union-attr] - log_relative_path = pathlib.Path(full_path).relative_to(self.local_base).as_posix() - local_loc = os.path.join(self.local_base, log_relative_path) - if os.path.exists(local_loc): - # read log and remove old logs to get just the latest additions - log = pathlib.Path(local_loc).read_text() - log_lines = self._parse_raw_log(log) - success = self._write_to_es(log_lines) - if success and self.delete_local_copy: - shutil.rmtree(os.path.dirname(local_loc)) - super().close() self.closed = True @@ -661,13 +655,56 @@ def _get_result(self, hit: dict[Any, Any], parent_class=None) -> Hit: callback: type[Hit] | Callable[..., Any] = getattr(doc_class, "from_es", doc_class) return callback(hit) - def _parse_raw_log(self, log: str) -> list[dict[str, Any]]: +def getattr_nested(obj, item, default): + """ + Get item from obj but return default if not found. + + E.g. calling ``getattr_nested(a, 'b.c', "NA")`` will return + ``a.b.c`` if such a value exists, and "NA" otherwise. + + :meta private: + """ + try: + return attrgetter(item)(obj) + except AttributeError: + return default + + +@attrs.define(kw_only=True) +class ElasticsearchRemoteLogIO(LoggingMixin): # noqa: D101 + host: str + target_index: str + base_log_folder: Path = attrs.field(converter=Path) + delete_local_copy: bool + + processors = () + + def upload(self, path: os.PathLike | str, ti: RuntimeTI): + """Write the log to ElasticSearch.""" + path = Path(path) + if path.is_absolute(): + local_loc = path + else: + local_loc = self.base_log_folder.joinpath(path) + if local_loc.is_file(): + log_lines = self._parse_raw_log(local_loc.read_text(), ti) + success = self._write_to_es(log_lines) + if success and self.delete_local_copy: + shutil.rmtree(os.path.dirname(local_loc)) + + def _parse_raw_log(self, log: str, ti: RuntimeTI) -> list[dict[str, Any]]: logs = log.split("\n") parsed_logs = [] + offset = 1 for line in logs: # Make sure line is not empty if line.strip(): - parsed_logs.append(json.loads(line)) + # construct log_id which is {dag_id}-{task_id}-{run_id}-{map_index}-{try_number} + log_dict = json.loads(line) + log_id = f"{ti.dag_id}-{ti.task_id}-{ti.run_id}-{ti.map_index}-{ti.try_number}" + log_dict.update({"log_id": log_id, "offset": offset}) + offset += 1 + parsed_logs.append(log_dict) return parsed_logs @@ -677,26 +714,14 @@ def _write_to_es(self, log_lines: list[dict[str, Any]]) -> bool: :param log_lines: the log_lines to write to the ElasticSearch. """ + es_kwargs = get_es_kwargs_from_config() + + client = elasticsearch.Elasticsearch(self.host, **es_kwargs) # Prepare the bulk request for Elasticsearch bulk_actions = [{"_index": self.target_index, "_source": log} for log in log_lines] try: - _ = helpers.bulk(self.client, bulk_actions) + _ = helpers.bulk(client, bulk_actions) return True except Exception as e: self.log.exception("Unable to insert logs into Elasticsearch. Reason: %s", str(e)) return False - - -def getattr_nested(obj, item, default): - """ - Get item from obj but return default if not found. - - E.g. calling ``getattr_nested(a, 'b.c', "NA")`` will return - ``a.b.c`` if such a value exists, and "NA" otherwise. - - :meta private: - """ - try: - return attrgetter(item)(obj) - except AttributeError: - return default From 9bf1cf4ae44fadf143785a8c571594e5c4ee4007 Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Tue, 29 Jul 2025 13:15:59 +0800 Subject: [PATCH 02/46] Fix mypy error. Fix ruff format error. Remove test_write_to_es. Fix build-doc CI. --- .../airflow_local_settings.py | 12 ++++-------- .../elasticsearch/log/es_task_handler.py | 7 ++++++- .../elasticsearch/log/test_es_task_handler.py | 19 ------------------- 3 files changed, 10 insertions(+), 28 deletions(-) diff --git a/airflow-core/src/airflow/config_templates/airflow_local_settings.py b/airflow-core/src/airflow/config_templates/airflow_local_settings.py index 7b4177ed93c16..7458edf7360c6 100644 --- a/airflow-core/src/airflow/config_templates/airflow_local_settings.py +++ b/airflow-core/src/airflow/config_templates/airflow_local_settings.py @@ -319,14 +319,10 @@ def _default_conn_name_from(mod_path, hook_name): if ELASTICSEARCH_WRITE_TO_ES: REMOTE_TASK_LOG = ElasticsearchRemoteLogIO( - **( - { - "host": ELASTICSEARCH_HOST, - "target_index": ELASTICSEARCH_TARGET_INDEX, - "base_log_folder": BASE_LOG_FOLDER, - "delete_local_copy": delete_local_copy, - } - ) + host=ELASTICSEARCH_HOST, + target_index=ELASTICSEARCH_TARGET_INDEX, + base_log_folder=BASE_LOG_FOLDER, + delete_local_copy=delete_local_copy, ) DEFAULT_LOGGING_CONFIG["handlers"].update(ELASTIC_REMOTE_HANDLERS) diff --git a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py index 738cce4186610..653255e3215ff 100644 --- a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py @@ -60,6 +60,7 @@ from airflow.models.taskinstance import TaskInstance, TaskInstanceKey from airflow.utils.log.file_task_handler import LogMetadata from airflow.sdk.types import RuntimeTaskInstanceProtocol as RuntimeTI + from airflow.utils.log.file_task_handler import LogMessages, LogSourceInfo LOG_LINE_DEFAULTS = {"exc_text": "", "stack_info": ""} @@ -463,12 +464,12 @@ def emit(self, record): def set_context(self, ti: TaskInstance, *, identifier: str | None = None) -> None: """ + TODO: This API should be removed in airflow 3 Provide task_instance context to airflow task handler. :param ti: task instance object :param identifier: if set, identifies the Airflow component which is relaying logs from exceptional scenarios related to the task instance - TODO: This API should be removed in airflow 3 """ is_trigger_log_context = getattr(ti, "is_trigger_log_context", None) is_ti_raw = getattr(ti, "raw", None) @@ -655,6 +656,7 @@ def _get_result(self, hit: dict[Any, Any], parent_class=None) -> Hit: callback: type[Hit] | Callable[..., Any] = getattr(doc_class, "from_es", doc_class) return callback(hit) + def getattr_nested(obj, item, default): """ Get item from obj but return default if not found. @@ -725,3 +727,6 @@ def _write_to_es(self, log_lines: list[dict[str, Any]]) -> bool: except Exception as e: self.log.exception("Unable to insert logs into Elasticsearch. Reason: %s", str(e)) return False + + def read(self, relative_path: str, ti: RuntimeTI) -> tuple[LogSourceInfo, LogMessages]: + pass diff --git a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py index 51f00f0d5059c..1e7508788516d 100644 --- a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py +++ b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py @@ -885,25 +885,6 @@ def test_filename_template_for_backward_compatibility(self): filename_template=None, ) - @pytest.mark.db_test - def test_write_to_es(self, ti): - self.es_task_handler.write_to_es = True - self.es_task_handler.json_format = True - self.es_task_handler.write_stdout = False - self.es_task_handler.local_base = Path(os.getcwd()) / "local" / "log" / "location" - formatter = logging.Formatter("%(asctime)s - %(name)s - %(levelname)s - %(message)s") - self.es_task_handler.formatter = formatter - - self.es_task_handler.set_context(ti) - with patch( - "airflow.providers.elasticsearch.log.es_task_handler.ElasticsearchTaskHandler._write_to_es" - ) as mock_write_to_es: - mock_write = Mock(return_value=True) - mock_write_to_es.return_value = mock_write - self.es_task_handler._write_to_es = mock_write_to_es - self.es_task_handler.close() - mock_write_to_es.assert_called_once() - def test_safe_attrgetter(): class A: ... From 68c5546d965da6588c3ad239e09ff21a86d1a446 Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Tue, 29 Jul 2025 17:24:43 +0800 Subject: [PATCH 03/46] Fix mypy error. Fix ruff format error. Fix spelling mistakes --- .../airflow/providers/elasticsearch/log/es_task_handler.py | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py index 653255e3215ff..d670de0da6b85 100644 --- a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py @@ -23,7 +23,6 @@ import logging import os from pathlib import Path -import pathlib import shutil import sys import time @@ -510,7 +509,7 @@ def close(self) -> None: # calling close method. Here we check if logger is already # closed to prevent uploading the log to remote storage multiple # times when `logging.shutdown` is called. - # TODO: This API should be simplied since Airflow 3 no longer requires this API for writing log to ES + # TODO: This API should be simplified since Airflow 3 no longer requires this API for writing log to ES if self.closed: return @@ -728,5 +727,5 @@ def _write_to_es(self, log_lines: list[dict[str, Any]]) -> bool: self.log.exception("Unable to insert logs into Elasticsearch. Reason: %s", str(e)) return False - def read(self, relative_path: str, ti: RuntimeTI) -> tuple[LogSourceInfo, LogMessages]: + def read(self, relative_path: str, ti: RuntimeTI) -> tuple[LogSourceInfo, LogMessages]: # type: ignore[empty-body] pass From b4070751b76d03f76583e90ef50019f2c81a0381 Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Tue, 29 Jul 2025 18:19:39 +0800 Subject: [PATCH 04/46] Fix ruff check --- .../providers/elasticsearch/log/es_task_handler.py | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py index d670de0da6b85..5af7d741505e1 100644 --- a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py @@ -22,17 +22,18 @@ import json import logging import os -from pathlib import Path import shutil import sys import time -import attrs from collections import defaultdict from collections.abc import Callable from operator import attrgetter +from pathlib import Path from typing import TYPE_CHECKING, Any, Literal, cast from urllib.parse import quote, urlparse +import attrs + # Using `from elasticsearch import *` would break elasticsearch mocking used in unit test. import elasticsearch import pendulum @@ -57,9 +58,8 @@ from datetime import datetime from airflow.models.taskinstance import TaskInstance, TaskInstanceKey - from airflow.utils.log.file_task_handler import LogMetadata from airflow.sdk.types import RuntimeTaskInstanceProtocol as RuntimeTI - from airflow.utils.log.file_task_handler import LogMessages, LogSourceInfo + from airflow.utils.log.file_task_handler import LogMessages, LogMetadata, LogSourceInfo LOG_LINE_DEFAULTS = {"exc_text": "", "stack_info": ""} @@ -463,7 +463,8 @@ def emit(self, record): def set_context(self, ti: TaskInstance, *, identifier: str | None = None) -> None: """ - TODO: This API should be removed in airflow 3 + TODO: This API should be removed in airflow 3. + Provide task_instance context to airflow task handler. :param ti: task instance object From 65f78c500619a16f54a89595261c1da0447be260 Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Tue, 29 Jul 2025 18:39:29 +0800 Subject: [PATCH 05/46] Fix ruff check on testEsTaskhandler --- .../tests/unit/elasticsearch/log/test_es_task_handler.py | 1 - 1 file changed, 1 deletion(-) diff --git a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py index 1e7508788516d..c12513b78b153 100644 --- a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py +++ b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py @@ -23,7 +23,6 @@ import re import shutil from io import StringIO -from pathlib import Path from unittest import mock from unittest.mock import Mock, patch from urllib.parse import quote From 3e594746950acdad811be88e4772ea14c7604567 Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Sat, 2 Aug 2025 18:30:20 +0800 Subject: [PATCH 06/46] Add ElasticsearchRemoteIO into TaskHandler. Refactor to handle read/write on RemoteIO class --- .../airflow_local_settings.py | 40 +- .../elasticsearch/log/es_response.py | 28 ++ .../elasticsearch/log/es_task_handler.py | 374 +++++++++--------- 3 files changed, 229 insertions(+), 213 deletions(-) diff --git a/airflow-core/src/airflow/config_templates/airflow_local_settings.py b/airflow-core/src/airflow/config_templates/airflow_local_settings.py index 7458edf7360c6..a4731670454dc 100644 --- a/airflow-core/src/airflow/config_templates/airflow_local_settings.py +++ b/airflow-core/src/airflow/config_templates/airflow_local_settings.py @@ -289,43 +289,23 @@ def _default_conn_name_from(mod_path, hook_name): elif ELASTICSEARCH_HOST: from airflow.providers.elasticsearch.log.es_task_handler import ElasticsearchRemoteLogIO - ELASTICSEARCH_END_OF_LOG_MARK: str = conf.get_mandatory_value("elasticsearch", "END_OF_LOG_MARK") - ELASTICSEARCH_FRONTEND: str = conf.get_mandatory_value("elasticsearch", "frontend") ELASTICSEARCH_WRITE_STDOUT: bool = conf.getboolean("elasticsearch", "WRITE_STDOUT") ELASTICSEARCH_WRITE_TO_ES: bool = conf.getboolean("elasticsearch", "WRITE_TO_ES") - ELASTICSEARCH_JSON_FORMAT: bool = conf.getboolean("elasticsearch", "JSON_FORMAT") - ELASTICSEARCH_JSON_FIELDS: str = conf.get_mandatory_value("elasticsearch", "JSON_FIELDS") ELASTICSEARCH_TARGET_INDEX: str = conf.get_mandatory_value("elasticsearch", "TARGET_INDEX") ELASTICSEARCH_HOST_FIELD: str = conf.get_mandatory_value("elasticsearch", "HOST_FIELD") ELASTICSEARCH_OFFSET_FIELD: str = conf.get_mandatory_value("elasticsearch", "OFFSET_FIELD") - ELASTIC_REMOTE_HANDLERS: dict[str, dict[str, str | bool | None]] = { - "task": { - "class": "airflow.providers.elasticsearch.log.es_task_handler.ElasticsearchTaskHandler", - "formatter": "airflow", - "base_log_folder": BASE_LOG_FOLDER, - "end_of_log_mark": ELASTICSEARCH_END_OF_LOG_MARK, - "host": ELASTICSEARCH_HOST, - "frontend": ELASTICSEARCH_FRONTEND, - "write_stdout": ELASTICSEARCH_WRITE_STDOUT, - "write_to_es": ELASTICSEARCH_WRITE_TO_ES, - "target_index": ELASTICSEARCH_TARGET_INDEX, - "json_format": ELASTICSEARCH_JSON_FORMAT, - "json_fields": ELASTICSEARCH_JSON_FIELDS, - "host_field": ELASTICSEARCH_HOST_FIELD, - "offset_field": ELASTICSEARCH_OFFSET_FIELD, - }, - } - - if ELASTICSEARCH_WRITE_TO_ES: - REMOTE_TASK_LOG = ElasticsearchRemoteLogIO( - host=ELASTICSEARCH_HOST, - target_index=ELASTICSEARCH_TARGET_INDEX, - base_log_folder=BASE_LOG_FOLDER, - delete_local_copy=delete_local_copy, - ) + REMOTE_TASK_LOG = ElasticsearchRemoteLogIO( + host=ELASTICSEARCH_HOST, + target_index=ELASTICSEARCH_TARGET_INDEX, + write_stdout=ELASTICSEARCH_WRITE_STDOUT, + write_to_es=ELASTICSEARCH_WRITE_TO_ES, + offset_field=ELASTICSEARCH_OFFSET_FIELD, + host_field=ELASTICSEARCH_HOST_FIELD, + base_log_folder=BASE_LOG_FOLDER, + delete_local_copy=delete_local_copy, + ) - DEFAULT_LOGGING_CONFIG["handlers"].update(ELASTIC_REMOTE_HANDLERS) elif OPENSEARCH_HOST: OPENSEARCH_END_OF_LOG_MARK: str = conf.get_mandatory_value("opensearch", "END_OF_LOG_MARK") OPENSEARCH_PORT: str = conf.get_mandatory_value("opensearch", "PORT") diff --git a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_response.py b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_response.py index 610b03f96e199..2af39ce736428 100644 --- a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_response.py +++ b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_response.py @@ -17,6 +17,7 @@ from __future__ import annotations from collections.abc import Iterator +from typing import Any def _wrap(val): @@ -25,6 +26,33 @@ def _wrap(val): return val +def resolve_nested(self, hit: dict[Any, Any], parent_class=None) -> type[Hit]: + """ + Resolve nested hits from Elasticsearch by iteratively navigating the `_nested` field. + + The result is used to fetch the appropriate document class to handle the hit. + + This method can be used with nested Elasticsearch fields which are structured + as dictionaries with "field" and "_nested" keys. + """ + doc_class = Hit + + nested_path: list[str] = [] + nesting = hit["_nested"] + while nesting and "field" in nesting: + nested_path.append(nesting["field"]) + nesting = nesting.get("_nested") + nested_path_str = ".".join(nested_path) + + if hasattr(parent_class, "_index"): + nested_field = parent_class._index.resolve_field(nested_path_str) + + if nested_field is not None: + return nested_field._doc_class + + return doc_class + + class AttributeList: """Helper class to provide attribute like access to List objects.""" diff --git a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py index 5af7d741505e1..9b1ae3c62350f 100644 --- a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py @@ -25,6 +25,7 @@ import shutil import sys import time +import pathlib from collections import defaultdict from collections.abc import Callable from operator import attrgetter @@ -46,7 +47,7 @@ from airflow.providers.elasticsearch.log.es_json_formatter import ( ElasticsearchJSONFormatter, ) -from airflow.providers.elasticsearch.log.es_response import ElasticSearchResponse, Hit +from airflow.providers.elasticsearch.log.es_response import ElasticSearchResponse, Hit, resolve_nested from airflow.providers.elasticsearch.version_compat import AIRFLOW_V_3_0_PLUS, EsLogMsgType from airflow.utils import timezone from airflow.utils.log.file_task_handler import FileTaskHandler @@ -113,6 +114,21 @@ def _ensure_ti(ti: TaskInstanceKey | TaskInstance, session) -> TaskInstance: raise AirflowException(f"Could not find TaskInstance for {ti}") +def getattr_nested(obj, item, default): + """ + Get item from obj but return default if not found. + + E.g. calling ``getattr_nested(a, 'b.c', "NA")`` will return + ``a.b.c`` if such a value exists, and "NA" otherwise. + + :meta private: + """ + try: + return attrgetter(item)(obj) + except AttributeError: + return default + + class ElasticsearchTaskHandler(FileTaskHandler, ExternalLoggingMixin, LoggingMixin): """ ElasticsearchTaskHandler is a python log handler that reads logs from Elasticsearch. @@ -192,6 +208,16 @@ def __init__( self.handler: logging.FileHandler | logging.StreamHandler | None = None self._doc_type_map: dict[Any, Any] = {} self._doc_type: list[Any] = [] + self.io = ElasticsearchRemoteLogIO( + host=self.host, + target_index=self.target_index, + write_stdout=self.write_stdout, + write_to_es=self.write_to_es, + offset_field=self.offset_field, + host_field=self.host_field, + base_log_folder=base_log_folder, + delete_local_copy=self.delete_local_copy, + ) @staticmethod def format_url(host: str) -> str: @@ -215,19 +241,6 @@ def format_url(host: str) -> str: return host - def _get_index_patterns(self, ti: TaskInstance | None) -> str: - """ - Get index patterns by calling index_patterns_callable, if provided, or the configured index_patterns. - - :param ti: A TaskInstance object or None. - """ - if self.index_patterns_callable: - self.log.debug("Using index_patterns_callable: %s", self.index_patterns_callable) - index_pattern_callable_obj = import_string(self.index_patterns_callable) - return index_pattern_callable_obj(ti) - self.log.debug("Using index_patterns: %s", self.index_patterns) - return self.index_patterns - def _render_log_id(self, ti: TaskInstance | TaskInstanceKey, try_number: int) -> str: from airflow.models.taskinstance import TaskInstanceKey @@ -288,13 +301,6 @@ def _clean_date(value: datetime | None) -> str: return "" return value.strftime("%Y_%m_%dT%H_%M_%S_%f") - def _group_logs_by_host(self, response: ElasticSearchResponse) -> dict[str, list[Hit]]: - grouped_logs = defaultdict(list) - for hit in response: - key = getattr_nested(hit, self.host_field, None) or self.host - grouped_logs[key].append(hit) - return grouped_logs - def _read_grouped_logs(self): return True @@ -319,9 +325,10 @@ def _read( offset = metadata["offset"] log_id = self._render_log_id(ti, try_number) - response = self._es_read(log_id, offset, ti) + response = self.io._es_read(log_id, offset, ti) + # TODO: Can we skip group logs by host ? if response is not None and response.hits: - logs_by_host = self._group_logs_by_host(response) + logs_by_host = self.io._group_logs_by_host(response) next_offset = attrgetter(self.offset_field)(response[-1]) else: logs_by_host = None @@ -334,10 +341,9 @@ def _read( # end_of_log_mark may contain characters like '\n' which is needed to # have the log uploaded but will not be stored in elasticsearch. - print(f"self.end_of_log_mark = {self.end_of_log_mark}") metadata["end_of_log"] = False if logs_by_host: - if any(x[-1].event == self.end_of_log_mark for x in logs_by_host.values()): + if any(x[-1].message == self.end_of_log_mark for x in logs_by_host.values()): metadata["end_of_log"] = True cur_ts = pendulum.now() @@ -417,45 +423,6 @@ def _format_msg(self, hit: Hit): # Just a safe-guard to preserve backwards-compatibility return hit.message - def _es_read(self, log_id: str, offset: int | str, ti: TaskInstance) -> ElasticSearchResponse | None: - """ - Return the logs matching log_id in Elasticsearch and next offset or ''. - - :param log_id: the log_id of the log to read. - :param offset: the offset start to read log from. - :param ti: the task instance object - - :meta private: - """ - query: dict[Any, Any] = { - "bool": { - "filter": [{"range": {self.offset_field: {"gt": int(offset)}}}], - "must": [{"match_phrase": {"log_id": log_id}}], - } - } - - index_patterns = self._get_index_patterns(ti) - try: - max_log_line = self.client.count(index=index_patterns, query=query)["count"] - except NotFoundError as e: - self.log.exception("The target index pattern %s does not exist", index_patterns) - raise e - - if max_log_line != 0: - try: - res = self.client.search( - index=index_patterns, - query=query, - sort=[self.offset_field], - size=self.MAX_LINE_PER_PAGE, - from_=self.MAX_LINE_PER_PAGE * self.PAGE, - ) - return ElasticSearchResponse(self, res) - except Exception as err: - self.log.exception("Could not read log with log_id: %s. Exception: %s", log_id, err) - - return None - def emit(self, record): if self.handler: setattr(record, self.offset_field, int(time.time() * (10**9))) @@ -563,132 +530,47 @@ def supports_external_link(self) -> bool: """Whether we can support external links.""" return bool(self.frontend) - def _resolve_nested(self, hit: dict[Any, Any], parent_class=None) -> type[Hit]: - """ - Resolve nested hits from Elasticsearch by iteratively navigating the `_nested` field. - - The result is used to fetch the appropriate document class to handle the hit. - - This method can be used with nested Elasticsearch fields which are structured - as dictionaries with "field" and "_nested" keys. - """ - doc_class = Hit - - nested_path: list[str] = [] - nesting = hit["_nested"] - while nesting and "field" in nesting: - nested_path.append(nesting["field"]) - nesting = nesting.get("_nested") - nested_path_str = ".".join(nested_path) - - if hasattr(parent_class, "_index"): - nested_field = parent_class._index.resolve_field(nested_path_str) - - if nested_field is not None: - return nested_field._doc_class - - return doc_class - - def _get_result(self, hit: dict[Any, Any], parent_class=None) -> Hit: - """ - Process a hit (i.e., a result) from an Elasticsearch response and transform it into a class instance. - - The transformation depends on the contents of the hit. If the document in hit contains a nested field, - the '_resolve_nested' method is used to determine the appropriate class (based on the nested path). - If the hit has a document type that is present in the '_doc_type_map', the corresponding class is - used. If not, the method iterates over the '_doc_type' classes and uses the first one whose '_matches' - method returns True for the hit. - - If the hit contains any 'inner_hits', these are also processed into 'ElasticSearchResponse' instances - using the determined class. - - Finally, the transformed hit is returned. If the determined class has a 'from_es' method, this is - used to transform the hit - - An example of the hit argument: - - {'_id': 'jdeZT4kBjAZqZnexVUxk', - '_index': '.ds-filebeat-8.8.2-2023.07.09-000001', - '_score': 2.482621, - '_source': {'@timestamp': '2023-07-13T14:13:15.140Z', - 'asctime': '2023-07-09T07:47:43.907+0000', - 'container': {'id': 'airflow'}, - 'dag_id': 'example_bash_operator', - 'ecs': {'version': '8.0.0'}, - 'logical_date': '2023_07_09T07_47_32_000000', - 'filename': 'taskinstance.py', - 'input': {'type': 'log'}, - 'levelname': 'INFO', - 'lineno': 1144, - 'log': {'file': {'path': "/opt/airflow/Documents/GitHub/airflow/logs/ - dag_id=example_bash_operator'/run_id=owen_run_run/ - task_id=run_after_loop/attempt=1.log"}, - 'offset': 0}, - 'log.offset': 1688888863907337472, - 'log_id': 'example_bash_operator-run_after_loop-owen_run_run--1-1', - 'message': 'Dependencies all met for dep_context=non-requeueable ' - 'deps ti=', - 'task_id': 'run_after_loop', - 'try_number': '1'}, - '_type': '_doc'} - """ - doc_class = Hit - dt = hit.get("_type") - - if "_nested" in hit: - doc_class = self._resolve_nested(hit, parent_class) - - elif dt in self._doc_type_map: - doc_class = self._doc_type_map[dt] - - else: - for doc_type in self._doc_type: - if hasattr(doc_type, "_matches") and doc_type._matches(hit): - doc_class = doc_type - break - - for t in hit.get("inner_hits", ()): - hit["inner_hits"][t] = ElasticSearchResponse(self, hit["inner_hits"][t], doc_class=doc_class) - - # callback should get the Hit class if "from_es" is not defined - callback: type[Hit] | Callable[..., Any] = getattr(doc_class, "from_es", doc_class) - return callback(hit) - - -def getattr_nested(obj, item, default): - """ - Get item from obj but return default if not found. - - E.g. calling ``getattr_nested(a, 'b.c', "NA")`` will return - ``a.b.c`` if such a value exists, and "NA" otherwise. - - :meta private: - """ - try: - return attrgetter(item)(obj) - except AttributeError: - return default - @attrs.define(kw_only=True) class ElasticsearchRemoteLogIO(LoggingMixin): # noqa: D101 - host: str - target_index: str - base_log_folder: Path = attrs.field(converter=Path) + write_stdout: bool delete_local_copy: bool + host: str = "http://localhost:9200" + host_field: str = "host" + target_index: str = "airflow-logs" + offset_field: str = "offset" + write_to_es: bool = False + base_log_folder: Path = attrs.field(converter=Path) processors = () + def __attrs_post_init__(self): + es_kwargs = get_es_kwargs_from_config() + self.client = elasticsearch.Elasticsearch(self.host, **es_kwargs) + self.index_patterns_callable = conf.get("elasticsearch", "index_patterns_callable", fallback="") + self.PAGE = 0 + self.MAX_LINE_PER_PAGE = 1000 + self.index_patterns: str = conf.get("elasticsearch", "index_patterns") + self._doc_type_map: dict[Any, Any] = {} + self._doc_type: list[Any] = [] + def upload(self, path: os.PathLike | str, ti: RuntimeTI): """Write the log to ElasticSearch.""" path = Path(path) + if path.is_absolute(): local_loc = path else: local_loc = self.base_log_folder.joinpath(path) - if local_loc.is_file(): + + if local_loc.is_file() and self.write_stdout: + # Intentionally construct the log_id and offset field + log_lines = self._parse_raw_log(local_loc.read_text(), ti) + for line in log_lines: + sys.stdout.write(json.dumps(line) + "\n") + sys.stdout.flush() + + if local_loc.is_file() and self.write_to_es: log_lines = self._parse_raw_log(local_loc.read_text(), ti) success = self._write_to_es(log_lines) if success and self.delete_local_copy: @@ -702,9 +584,10 @@ def _parse_raw_log(self, log: str, ti: RuntimeTI) -> list[dict[str, Any]]: # Make sure line is not empty if line.strip(): # construct log_id which is {dag_id}-{task_id}-{run_id}-{map_index}-{try_number} + # also construct the offset field (default is 'offset') log_dict = json.loads(line) log_id = f"{ti.dag_id}-{ti.task_id}-{ti.run_id}-{ti.map_index}-{ti.try_number}" - log_dict.update({"log_id": log_id, "offset": offset}) + log_dict.update({"log_id": log_id, self.offset_field: offset}) offset += 1 parsed_logs.append(log_dict) @@ -716,17 +599,142 @@ def _write_to_es(self, log_lines: list[dict[str, Any]]) -> bool: :param log_lines: the log_lines to write to the ElasticSearch. """ - es_kwargs = get_es_kwargs_from_config() - - client = elasticsearch.Elasticsearch(self.host, **es_kwargs) # Prepare the bulk request for Elasticsearch bulk_actions = [{"_index": self.target_index, "_source": log} for log in log_lines] try: - _ = helpers.bulk(client, bulk_actions) + _ = helpers.bulk(self.client, bulk_actions) return True except Exception as e: self.log.exception("Unable to insert logs into Elasticsearch. Reason: %s", str(e)) return False - def read(self, relative_path: str, ti: RuntimeTI) -> tuple[LogSourceInfo, LogMessages]: # type: ignore[empty-body] - pass + def read(self, relative_path: str, ti: RuntimeTI) -> tuple[LogSourceInfo, LogMessages]: + log_id = f"{ti.dag_id}-{ti.task_id}-{ti.run_id}-{ti.map_index}-{ti.try_number}" + self.log.info(f"Reading log {log_id} from Elasticsearch") + offset = 0 + response = self._es_read(log_id, offset, ti) + if response is not None and response.hits: + logs_by_host = self._group_logs_by_host(response) + else: + logs_by_host = None + + if logs_by_host is None: + missing_log_message = ( + f"*** Log {log_id} not found in Elasticsearch. " + "If your task started recently, please wait a moment and reload this page. " + "Otherwise, the logs for this task instance may have been removed." + ) + return ["::group::Log message source details"], [missing_log_message, "::endgroup::"] + + else: + header = [] + # Start log group + header.append("".join([host for host in logs_by_host.keys()])) + + message = [] + # Structured log messages + for hits in logs_by_host.values(): + for hit in hits: + filtered = {k: v for k, v in hit.to_dict().items() if k.lower() in TASK_LOG_FIELDS} + message.append(json.dumps(filtered)) + + message.append(json.dumps({"event": "::endgroup::"})) + + return header, message + + def _es_read(self, log_id: str, offset: int | str, ti: RuntimeTI) -> ElasticSearchResponse | None: + """ + Return the logs matching log_id in Elasticsearch and next offset or ''. + + :param log_id: the log_id of the log to read. + :param offset: the offset start to read log from. + :param ti: the task instance object + + :meta private: + """ + query: dict[Any, Any] = { + "bool": { + "filter": [{"range": {self.offset_field: {"gt": int(offset)}}}], + "must": [{"match_phrase": {"log_id": log_id}}], + } + } + + index_patterns = self._get_index_patterns(ti) + try: + max_log_line = self.client.count(index=index_patterns, query=query)["count"] + except NotFoundError as e: + self.log.exception("The target index pattern %s does not exist", index_patterns) + raise e + + if max_log_line != 0: + try: + res = self.client.search( + index=index_patterns, + query=query, + sort=[self.offset_field], + size=self.MAX_LINE_PER_PAGE, + from_=self.MAX_LINE_PER_PAGE * self.PAGE, + ) + return ElasticSearchResponse(self, res) + except Exception as err: + self.log.exception("Could not read log with log_id: %s. Exception: %s", log_id, err) + + return None + + def _get_index_patterns(self, ti: RuntimeTI | None) -> str: + """ + Get index patterns by calling index_patterns_callable, if provided, or the configured index_patterns. + + :param ti: A TaskInstance object or None. + """ + if self.index_patterns_callable: + self.log.debug("Using index_patterns_callable: %s", self.index_patterns_callable) + index_pattern_callable_obj = import_string(self.index_patterns_callable) + return index_pattern_callable_obj(ti) + self.log.debug("Using index_patterns: %s", self.index_patterns) + return self.index_patterns + + def _group_logs_by_host(self, response: ElasticSearchResponse) -> dict[str, list[Hit]]: + grouped_logs = defaultdict(list) + for hit in response: + key = getattr_nested(hit, self.host_field, None) or self.host + grouped_logs[key].append(hit) + return grouped_logs + + def _get_result(self, hit: dict[Any, Any], parent_class=None) -> Hit: + """ + Process a hit (i.e., a result) from an Elasticsearch response and transform it into a class instance. + + The transformation depends on the contents of the hit. If the document in hit contains a nested field, + the 'resolve_nested' method is used to determine the appropriate class (based on the nested path). + If the hit has a document type that is present in the '_doc_type_map', the corresponding class is + used. If not, the method iterates over the '_doc_type' classes and uses the first one whose '_matches' + method returns True for the hit. + + If the hit contains any 'inner_hits', these are also processed into 'ElasticSearchResponse' instances + using the determined class. + + Finally, the transformed hit is returned. If the determined class has a 'from_es' method, this is + used to transform the hit + """ + doc_class = Hit + dt = hit.get("_type") + + if "_nested" in hit: + doc_class = resolve_nested(hit, parent_class) + + elif dt in self._doc_type_map: + doc_class = self._doc_type_map[dt] + + else: + for doc_type in self._doc_type: + if hasattr(doc_type, "_matches") and doc_type._matches(hit): + doc_class = doc_type + break + + for t in hit.get("inner_hits", ()): + hit["inner_hits"][t] = ElasticSearchResponse(self, hit["inner_hits"][t], doc_class=doc_class) + + # callback should get the Hit class if "from_es" is not defined + callback: type[Hit] | Callable[..., Any] = getattr(doc_class, "from_es", doc_class) + return callback(hit) From a60a3154641dad70af600f239e8ff374b1039020 Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Sun, 3 Aug 2025 21:41:29 +0800 Subject: [PATCH 07/46] Fix existing unit tests --- .../elasticsearch/log/es_task_handler.py | 42 +++++++++++++++++-- .../log/elasticmock/fake_elasticsearch.py | 2 - .../elasticsearch/log/test_es_task_handler.py | 33 +++++++-------- 3 files changed, 53 insertions(+), 24 deletions(-) diff --git a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py index 9b1ae3c62350f..c8f18dfe8b4f4 100644 --- a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py @@ -530,6 +530,44 @@ def supports_external_link(self) -> bool: """Whether we can support external links.""" return bool(self.frontend) + def _get_result(self, hit: dict[Any, Any], parent_class=None) -> Hit: + """ + Process a hit (i.e., a result) from an Elasticsearch response and transform it into a class instance. + + The transformation depends on the contents of the hit. If the document in hit contains a nested field, + the 'resolve_nested' method is used to determine the appropriate class (based on the nested path). + If the hit has a document type that is present in the '_doc_type_map', the corresponding class is + used. If not, the method iterates over the '_doc_type' classes and uses the first one whose '_matches' + method returns True for the hit. + + If the hit contains any 'inner_hits', these are also processed into 'ElasticSearchResponse' instances + using the determined class. + + Finally, the transformed hit is returned. If the determined class has a 'from_es' method, this is + used to transform the hit + """ + doc_class = Hit + dt = hit.get("_type") + + if "_nested" in hit: + doc_class = resolve_nested(hit, parent_class) + + elif dt in self._doc_type_map: + doc_class = self._doc_type_map[dt] + + else: + for doc_type in self._doc_type: + if hasattr(doc_type, "_matches") and doc_type._matches(hit): + doc_class = doc_type + break + + for t in hit.get("inner_hits", ()): + hit["inner_hits"][t] = ElasticSearchResponse(self, hit["inner_hits"][t], doc_class=doc_class) + + # callback should get the Hit class if "from_es" is not defined + callback: type[Hit] | Callable[..., Any] = getattr(doc_class, "from_es", doc_class) + return callback(hit) + @attrs.define(kw_only=True) class ElasticsearchRemoteLogIO(LoggingMixin): # noqa: D101 @@ -624,7 +662,7 @@ def read(self, relative_path: str, ti: RuntimeTI) -> tuple[LogSourceInfo, LogMes "If your task started recently, please wait a moment and reload this page. " "Otherwise, the logs for this task instance may have been removed." ) - return ["::group::Log message source details"], [missing_log_message, "::endgroup::"] + return [], [missing_log_message] else: header = [] @@ -638,8 +676,6 @@ def read(self, relative_path: str, ti: RuntimeTI) -> tuple[LogSourceInfo, LogMes filtered = {k: v for k, v in hit.to_dict().items() if k.lower() in TASK_LOG_FIELDS} message.append(json.dumps(filtered)) - message.append(json.dumps({"event": "::endgroup::"})) - return header, message def _es_read(self, log_id: str, offset: int | str, ti: RuntimeTI) -> ElasticSearchResponse | None: diff --git a/providers/elasticsearch/tests/unit/elasticsearch/log/elasticmock/fake_elasticsearch.py b/providers/elasticsearch/tests/unit/elasticsearch/log/elasticmock/fake_elasticsearch.py index c7746001d6856..92cbad3ca7075 100644 --- a/providers/elasticsearch/tests/unit/elasticsearch/log/elasticmock/fake_elasticsearch.py +++ b/providers/elasticsearch/tests/unit/elasticsearch/log/elasticmock/fake_elasticsearch.py @@ -479,7 +479,6 @@ def _validate_search_targets(self, targets, query): # TODO: support allow_no_indices query parameter matches = set() for target in targets: - print(f"Loop over:::target = {target}") if target in ("_all", ""): matches.update(self.__documents_dict) elif "*" in target: @@ -499,7 +498,6 @@ def _normalize_index_to_list(self, index, query): else: # Is it the correct exception to use ? raise ValueError("Invalid param 'index'") - generator = (target for index in searchable_indexes for target in index.split(",")) return list(self._validate_search_targets(generator, query=query)) diff --git a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py index c12513b78b153..3d1adacd33bb9 100644 --- a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py +++ b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py @@ -130,7 +130,7 @@ def teardown_method(self): def test_es_response(self): sample_response = self.es.sample_log_response() es_response = ElasticSearchResponse(self.es_task_handler, sample_response) - logs_by_host = self.es_task_handler._group_logs_by_host(es_response) + logs_by_host = self.es_task_handler.io._group_logs_by_host(es_response) def concat_logs(lines): log_range = -1 if lines[-1].message == self.es_task_handler.end_of_log_mark else None @@ -262,7 +262,7 @@ def test_read_with_patterns(self, ti): @pytest.mark.db_test def test_read_with_patterns_no_match(self, ti): ts = pendulum.now() - with mock.patch.object(self.es_task_handler, "index_patterns", new="test_other_*,test_another_*"): + with mock.patch.object(self.es_task_handler.io, "index_patterns", new="test_other_*,test_another_*"): logs, metadatas = self.es_task_handler.read( ti, 1, {"offset": 0, "last_log_timestamp": str(ts), "end_of_log": False} ) @@ -286,7 +286,7 @@ def test_read_with_patterns_no_match(self, ti): @pytest.mark.db_test def test_read_with_missing_index(self, ti): ts = pendulum.now() - with mock.patch.object(self.es_task_handler, "index_patterns", new="nonexistent,test_*"): + with mock.patch.object(self.es_task_handler.io, "index_patterns", new="nonexistent,test_*"): with pytest.raises(elasticsearch.exceptions.NotFoundError, match=r"IndexMissingException.*"): self.es_task_handler.read( ti, @@ -551,27 +551,22 @@ def test_read_as_download_logs(self, ti): @pytest.mark.db_test def test_read_raises(self, ti): - with mock.patch.object(self.es_task_handler.log, "exception") as mock_exception: - with mock.patch.object(self.es_task_handler.client, "search") as mock_execute: + with mock.patch.object(self.es_task_handler.io.log, "exception") as mock_exception: + with mock.patch.object(self.es_task_handler.io.client, "search") as mock_execute: mock_execute.side_effect = SearchFailedException("Failed to read") - logs, metadatas = self.es_task_handler.read(ti, 1) + log_sources, log_msgs = self.es_task_handler.io.read("", ti) assert mock_exception.call_count == 1 args, kwargs = mock_exception.call_args assert "Could not read log with log_id:" in args[0] - + print(f"log_msgs: {log_msgs}") if AIRFLOW_V_3_0_PLUS: - assert logs == [] - - metadata = metadatas + assert log_sources == [] else: - assert len(logs) == 1 - assert len(logs) == len(metadatas) - assert logs == [[]] - - metadata = metadatas[0] + assert len(log_sources) == 0 + assert len(log_msgs) == 1 + assert log_sources == [] - assert metadata["offset"] == "0" - assert not metadata["end_of_log"] + assert "not found in Elasticsearch" in log_msgs[0] @pytest.mark.db_test def test_set_context(self, ti): @@ -866,8 +861,8 @@ def test_get_index_patterns_with_callable(self): mock_callable = Mock(return_value="callable_index_pattern") mock_import_string.return_value = mock_callable - self.es_task_handler.index_patterns_callable = "path.to.index_pattern_callable" - result = self.es_task_handler._get_index_patterns({}) + self.es_task_handler.io.index_patterns_callable = "path.to.index_pattern_callable" + result = self.es_task_handler.io._get_index_patterns({}) mock_import_string.assert_called_once_with("path.to.index_pattern_callable") mock_callable.assert_called_once_with({}) From aff9a300ef57b1cb7c1ffd96ddbbaf17545c0222 Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Thu, 7 Aug 2025 11:07:59 +0800 Subject: [PATCH 08/46] Add unit test for ElasticsearchRemoteIO. Add testcontainer as new dependency --- providers/elasticsearch/pyproject.toml | 1 + .../elasticsearch/log/es_task_handler.py | 26 ++-- providers/elasticsearch/tests/conftest.py | 36 +++++ .../elasticsearch/log/test_es_task_handler.py | 141 +++++++++++++++++- 4 files changed, 189 insertions(+), 15 deletions(-) diff --git a/providers/elasticsearch/pyproject.toml b/providers/elasticsearch/pyproject.toml index 7c724ad1262ec..c28435f68fda4 100644 --- a/providers/elasticsearch/pyproject.toml +++ b/providers/elasticsearch/pyproject.toml @@ -70,6 +70,7 @@ dev = [ "apache-airflow-providers-common-sql", # Additional devel dependencies (do not remove this line and add extra development dependencies) "apache-airflow-providers-common-sql[pandas,polars]", + "testcontainers>=4.12.0" ] # To build docs: diff --git a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py index c8f18dfe8b4f4..121005694f010 100644 --- a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py @@ -25,7 +25,6 @@ import shutil import sys import time -import pathlib from collections import defaultdict from collections.abc import Callable from operator import attrgetter @@ -571,8 +570,8 @@ def _get_result(self, hit: dict[Any, Any], parent_class=None) -> Hit: @attrs.define(kw_only=True) class ElasticsearchRemoteLogIO(LoggingMixin): # noqa: D101 - write_stdout: bool - delete_local_copy: bool + write_stdout: bool = False + delete_local_copy: bool = False host: str = "http://localhost:9200" host_field: str = "host" target_index: str = "airflow-logs" @@ -648,7 +647,7 @@ def _write_to_es(self, log_lines: list[dict[str, Any]]) -> bool: def read(self, relative_path: str, ti: RuntimeTI) -> tuple[LogSourceInfo, LogMessages]: log_id = f"{ti.dag_id}-{ti.task_id}-{ti.run_id}-{ti.map_index}-{ti.try_number}" - self.log.info(f"Reading log {log_id} from Elasticsearch") + self.log.info("Reading log %s from Elasticsearch", log_id) offset = 0 response = self._es_read(log_id, offset, ti) if response is not None and response.hits: @@ -664,17 +663,16 @@ def read(self, relative_path: str, ti: RuntimeTI) -> tuple[LogSourceInfo, LogMes ) return [], [missing_log_message] - else: - header = [] - # Start log group - header.append("".join([host for host in logs_by_host.keys()])) + header = [] + # Start log group + header.append("".join([host for host in logs_by_host.keys()])) - message = [] - # Structured log messages - for hits in logs_by_host.values(): - for hit in hits: - filtered = {k: v for k, v in hit.to_dict().items() if k.lower() in TASK_LOG_FIELDS} - message.append(json.dumps(filtered)) + message = [] + # Structured log messages + for hits in logs_by_host.values(): + for hit in hits: + filtered = {k: v for k, v in hit.to_dict().items() if k.lower() in TASK_LOG_FIELDS} + message.append(json.dumps(filtered)) return header, message diff --git a/providers/elasticsearch/tests/conftest.py b/providers/elasticsearch/tests/conftest.py index f56ccce0a3f69..27ecc883783df 100644 --- a/providers/elasticsearch/tests/conftest.py +++ b/providers/elasticsearch/tests/conftest.py @@ -16,4 +16,40 @@ # under the License. from __future__ import annotations +import pytest +from testcontainers.elasticsearch import ElasticSearchContainer + +_container = None + + +class EarlyEnvPlugin: + def pytest_sessionstart(self, session): + global _container + + _container = ElasticSearchContainer("docker.elastic.co/elasticsearch/elasticsearch:8.19.0") + _container.start() + + es_host = _container.get_container_host_ip() + es_port = _container.get_exposed_port(_container.port) + es_url = f"http://{es_host}:{es_port}" + + session.config._es_url = es_url + + def pytest_sessionfinish(self, session, exitstatus): + global _container + if _container: + _container.stop() + + +def pytest_configure(config): + plugin = EarlyEnvPlugin() + config.pluginmanager.register(plugin, name="early_env_plugin") + + +@pytest.fixture(scope="session") +def elasticsearch_8_url(request): + """Provides ES URL and client after early bootstrapping.""" + return request.config._es_url + + pytest_plugins = "tests_common.pytest_plugin" diff --git a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py index 3d1adacd33bb9..47e42a57f1461 100644 --- a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py +++ b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py @@ -22,7 +22,9 @@ import os import re import shutil +import tempfile from io import StringIO +from pathlib import Path from unittest import mock from unittest.mock import Mock, patch from urllib.parse import quote @@ -35,6 +37,7 @@ from airflow.providers.elasticsearch.log.es_response import ElasticSearchResponse from airflow.providers.elasticsearch.log.es_task_handler import ( VALID_ES_CONFIG_KEYS, + ElasticsearchRemoteLogIO, ElasticsearchTaskHandler, get_es_kwargs_from_config, getattr_nested, @@ -558,7 +561,7 @@ def test_read_raises(self, ti): assert mock_exception.call_count == 1 args, kwargs = mock_exception.call_args assert "Could not read log with log_id:" in args[0] - print(f"log_msgs: {log_msgs}") + if AIRFLOW_V_3_0_PLUS: assert log_sources == [] else: @@ -938,3 +941,139 @@ def test_self_not_valid_arg(): Test if self is not a valid argument. """ assert "self" not in VALID_ES_CONFIG_KEYS + + +@pytest.mark.db_test +class TestElasticsearchRemoteLogIO: + DAG_ID = "dag_for_testing_es_log_handler" + TASK_ID = "task_for_testing_es_log_handler" + LOGICAL_DATE = datetime(2016, 1, 1) + FILENAME_TEMPLATE = "{try_number}.log" + + @pytest.fixture(autouse=True) + def setup_tests(self, ti, elasticsearch_8_url): + es_client = elasticsearch.Elasticsearch(elasticsearch_8_url) + self.elasticsearch_8_url = elasticsearch_8_url + self.elasticsearch_io = ElasticsearchRemoteLogIO( + write_to_es=True, + write_stdout=True, + delete_local_copy=True, + host=elasticsearch_8_url, + base_log_folder=Path(""), + client=es_client, + ) + + @pytest.fixture + def tmp_json_file(self): + with tempfile.TemporaryDirectory() as tmpdir: + os.makedirs(tmpdir, exist_ok=True) + + file_path = os.path.join(tmpdir, "1.log") + self.tmp_file = file_path + + sample_logs = [ + {"message": "start"}, + {"message": "processing"}, + {"message": "end"}, + ] + with open(file_path, "w") as f: + for log in sample_logs: + f.write(json.dumps(log) + "\n") + + yield file_path + + del self.tmp_file + + @pytest.fixture + def ti(self, create_task_instance, create_log_template): + create_log_template( + self.FILENAME_TEMPLATE, + ( + "{dag_id}-{task_id}-{logical_date}-{try_number}" + if AIRFLOW_V_3_0_PLUS + else "{dag_id}-{task_id}-{execution_date}-{try_number}" + ), + ) + yield get_ti( + dag_id=self.DAG_ID, + task_id=self.TASK_ID, + logical_date=self.LOGICAL_DATE, + create_task_instance=create_task_instance, + ) + clear_db_runs() + clear_db_dags() + + @pytest.fixture + def write_to_es(self, tmp_json_file, ti): + self.elasticsearch_io.upload(tmp_json_file, ti) + self.elasticsearch_io.client.indices.refresh(index="_all") + + def test_write_to_es(self, tmp_json_file, ti): + self.elasticsearch_io.write_stdout = False + self.elasticsearch_io.upload(tmp_json_file, ti) + self.elasticsearch_io.client.indices.refresh(index="_all") + res = self.elasticsearch_io.client.search(index="_all", query={"match_all": {}}) + + offset = 1 + expected_msg = ["start", "processing", "end"] + expected_log_id = f"{ti.dag_id}-{ti.task_id}-{ti.run_id}-{ti.map_index}-{ti.try_number}" + assert res["hits"]["total"]["value"] == 3 + for msg, hit in zip(expected_msg, res["hits"]["hits"]): + assert hit["_index"] == "airflow-logs" + assert hit["_source"]["message"] == msg + assert hit["_source"]["offset"] == offset + assert hit["_source"]["log_id"] == expected_log_id + offset += 1 + + def test_write_to_stdout(self, tmp_json_file, ti, capsys): + self.elasticsearch_io.write_to_es = False + self.elasticsearch_io.upload(tmp_json_file, ti) + + captured = capsys.readouterr() + stdout_lines = captured.out.strip().splitlines() + + log_entries = [json.loads(line) for line in stdout_lines] + assert log_entries[0]["message"] == "start" + assert log_entries[1]["message"] == "processing" + assert log_entries[2]["message"] == "end" + + def test_invalid_task_log_file_path(self, ti): + with ( + patch.object(self.elasticsearch_io, "_parse_raw_log") as mock_parse, + patch.object(self.elasticsearch_io, "_write_to_es") as mock_write, + ): + self.elasticsearch_io.upload(Path("/invalid/path"), ti) + + # ✅ Assert these methods are NOT called + mock_parse.assert_not_called() + mock_write.assert_not_called() + + def test_raw_log_should_contain_log_id_and_offset(self, tmp_json_file, ti): + with open(self.tmp_file) as f: + raw_log = f.read() + json_log_lines = self.elasticsearch_io._parse_raw_log(raw_log, ti) + assert len(json_log_lines) == 3 + for json_log_line in json_log_lines: + assert "log_id" in json_log_line + assert "offset" in json_log_line + + @patch( + "providers.elasticsearch.src.airflow.providers.elasticsearch.log.es_task_handler.TASK_LOG_FIELDS", + ["message"], + ) + def test_read_es_log(self, write_to_es, ti): + log_source_info, log_messages = self.elasticsearch_io.read("", ti) + assert log_source_info[0] == self.elasticsearch_8_url + assert len(log_messages) == 3 + + expected_msg = ["start", "processing", "end"] + for msg, log_message in zip(expected_msg, log_messages): + json_log = json.loads(log_message) + assert "message" in json_log + assert json_log["message"] == msg + + def test_read_with_missing_log(self, ti): + log_source_info, log_messages = self.elasticsearch_io.read("", ti) + log_id = f"{ti.dag_id}-{ti.task_id}-{ti.run_id}-{ti.map_index}-{ti.try_number}" + assert log_source_info == [] + assert f"*** Log {log_id} not found in Elasticsearch" in log_messages[0] From b1cb76058192fb050ce7b920b248af440d5a098b Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Thu, 7 Aug 2025 17:13:32 +0800 Subject: [PATCH 09/46] Fix provider test --- .../elasticsearch/log/test_es_task_handler.py | 22 ++++++++++++++----- 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py index 47e42a57f1461..fe7db03f20b20 100644 --- a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py +++ b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py @@ -23,6 +23,7 @@ import re import shutil import tempfile +import uuid from io import StringIO from pathlib import Path from unittest import mock @@ -952,7 +953,6 @@ class TestElasticsearchRemoteLogIO: @pytest.fixture(autouse=True) def setup_tests(self, ti, elasticsearch_8_url): - es_client = elasticsearch.Elasticsearch(elasticsearch_8_url) self.elasticsearch_8_url = elasticsearch_8_url self.elasticsearch_io = ElasticsearchRemoteLogIO( write_to_es=True, @@ -960,7 +960,6 @@ def setup_tests(self, ti, elasticsearch_8_url): delete_local_copy=True, host=elasticsearch_8_url, base_log_folder=Path(""), - client=es_client, ) @pytest.fixture @@ -1004,7 +1003,14 @@ def ti(self, create_task_instance, create_log_template): clear_db_dags() @pytest.fixture - def write_to_es(self, tmp_json_file, ti): + def unique_index(self): + """Generate a unique index name for each test.""" + return f"airflow-logs-{uuid.uuid4()}" + + @pytest.fixture + def write_to_es(self, tmp_json_file, ti, unique_index): + self.elasticsearch_io.target_index = unique_index + self.elasticsearch_io.index_pattern = unique_index self.elasticsearch_io.upload(tmp_json_file, ti) self.elasticsearch_io.client.indices.refresh(index="_all") @@ -1024,6 +1030,7 @@ def test_write_to_es(self, tmp_json_file, ti): assert hit["_source"]["offset"] == offset assert hit["_source"]["log_id"] == expected_log_id offset += 1 + self.elasticsearch_io.client.indices.delete(index="airflow-logs") def test_write_to_stdout(self, tmp_json_file, ti, capsys): self.elasticsearch_io.write_to_es = False @@ -1044,7 +1051,6 @@ def test_invalid_task_log_file_path(self, ti): ): self.elasticsearch_io.upload(Path("/invalid/path"), ti) - # ✅ Assert these methods are NOT called mock_parse.assert_not_called() mock_write.assert_not_called() @@ -1058,7 +1064,7 @@ def test_raw_log_should_contain_log_id_and_offset(self, tmp_json_file, ti): assert "offset" in json_log_line @patch( - "providers.elasticsearch.src.airflow.providers.elasticsearch.log.es_task_handler.TASK_LOG_FIELDS", + "airflow.providers.elasticsearch.log.es_task_handler.TASK_LOG_FIELDS", ["message"], ) def test_read_es_log(self, write_to_es, ti): @@ -1072,8 +1078,12 @@ def test_read_es_log(self, write_to_es, ti): assert "message" in json_log assert json_log["message"] == msg - def test_read_with_missing_log(self, ti): + # @patch.object(ElasticsearchRemoteLogIO, "_get_index_patterns", return_value="invalid") + @patch("elasticsearch.Elasticsearch.count", return_value={"count": 0}) + def test_read_with_missing_log(self, mocked_count, mocked_get_index_patterns, ti): log_source_info, log_messages = self.elasticsearch_io.read("", ti) log_id = f"{ti.dag_id}-{ti.task_id}-{ti.run_id}-{ti.map_index}-{ti.try_number}" assert log_source_info == [] assert f"*** Log {log_id} not found in Elasticsearch" in log_messages[0] + mocked_get_index_patterns.assert_called_once_with(ti) + mocked_count.assert_called_once() From f3ce8e7fe84a39fdd30feed27765c4d3e50e478d Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Thu, 7 Aug 2025 17:41:59 +0800 Subject: [PATCH 10/46] Fix typo --- .../tests/unit/elasticsearch/log/test_es_task_handler.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py index fe7db03f20b20..248da20042691 100644 --- a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py +++ b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py @@ -1080,10 +1080,9 @@ def test_read_es_log(self, write_to_es, ti): # @patch.object(ElasticsearchRemoteLogIO, "_get_index_patterns", return_value="invalid") @patch("elasticsearch.Elasticsearch.count", return_value={"count": 0}) - def test_read_with_missing_log(self, mocked_count, mocked_get_index_patterns, ti): + def test_read_with_missing_log(self, mocked_count, ti): log_source_info, log_messages = self.elasticsearch_io.read("", ti) log_id = f"{ti.dag_id}-{ti.task_id}-{ti.run_id}-{ti.map_index}-{ti.try_number}" assert log_source_info == [] assert f"*** Log {log_id} not found in Elasticsearch" in log_messages[0] - mocked_get_index_patterns.assert_called_once_with(ti) mocked_count.assert_called_once() From d28d40a3d92fdf80c1fce7fd565cc725adc2b5e8 Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Thu, 7 Aug 2025 21:35:09 +0800 Subject: [PATCH 11/46] Fix long running provider test --- providers/elasticsearch/tests/conftest.py | 32 ++----------------- .../elasticsearch/log/test_es_task_handler.py | 6 ++-- 2 files changed, 6 insertions(+), 32 deletions(-) diff --git a/providers/elasticsearch/tests/conftest.py b/providers/elasticsearch/tests/conftest.py index 27ecc883783df..f36dd9ad703b1 100644 --- a/providers/elasticsearch/tests/conftest.py +++ b/providers/elasticsearch/tests/conftest.py @@ -19,37 +19,11 @@ import pytest from testcontainers.elasticsearch import ElasticSearchContainer -_container = None - - -class EarlyEnvPlugin: - def pytest_sessionstart(self, session): - global _container - - _container = ElasticSearchContainer("docker.elastic.co/elasticsearch/elasticsearch:8.19.0") - _container.start() - - es_host = _container.get_container_host_ip() - es_port = _container.get_exposed_port(_container.port) - es_url = f"http://{es_host}:{es_port}" - - session.config._es_url = es_url - - def pytest_sessionfinish(self, session, exitstatus): - global _container - if _container: - _container.stop() - - -def pytest_configure(config): - plugin = EarlyEnvPlugin() - config.pluginmanager.register(plugin, name="early_env_plugin") - @pytest.fixture(scope="session") -def elasticsearch_8_url(request): - """Provides ES URL and client after early bootstrapping.""" - return request.config._es_url +def es_8_container_url(): + with ElasticSearchContainer("docker.elastic.co/elasticsearch/elasticsearch:8.19.0") as es: + yield es.get_url() pytest_plugins = "tests_common.pytest_plugin" diff --git a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py index 248da20042691..774527da9160a 100644 --- a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py +++ b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py @@ -952,13 +952,13 @@ class TestElasticsearchRemoteLogIO: FILENAME_TEMPLATE = "{try_number}.log" @pytest.fixture(autouse=True) - def setup_tests(self, ti, elasticsearch_8_url): - self.elasticsearch_8_url = elasticsearch_8_url + def setup_tests(self, ti, es_8_container_url): + self.elasticsearch_8_url = es_8_container_url self.elasticsearch_io = ElasticsearchRemoteLogIO( write_to_es=True, write_stdout=True, delete_local_copy=True, - host=elasticsearch_8_url, + host=es_8_container_url, base_log_folder=Path(""), ) From 998aeeee9583776d42759ad954ec9d54b069080a Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Fri, 8 Aug 2025 14:12:58 +0800 Subject: [PATCH 12/46] Remove leftover comments. Move testcontainer dependency to devel-common --- devel-common/pyproject.toml | 1 + providers/elasticsearch/pyproject.toml | 1 - .../tests/unit/elasticsearch/log/test_es_task_handler.py | 1 - providers/mongo/pyproject.toml | 1 - 4 files changed, 1 insertion(+), 3 deletions(-) diff --git a/devel-common/pyproject.toml b/devel-common/pyproject.toml index 5af7fb079bb06..7cd3c3471bca4 100644 --- a/devel-common/pyproject.toml +++ b/devel-common/pyproject.toml @@ -134,6 +134,7 @@ dependencies = [ "pytest-unordered>=0.6.1", "pytest-xdist>=3.5.0", "pytest>=8.3.3", + "testcontainers>=4.12.0", ] "sentry" = [ "blinker>=1.7.0", diff --git a/providers/elasticsearch/pyproject.toml b/providers/elasticsearch/pyproject.toml index c28435f68fda4..7c724ad1262ec 100644 --- a/providers/elasticsearch/pyproject.toml +++ b/providers/elasticsearch/pyproject.toml @@ -70,7 +70,6 @@ dev = [ "apache-airflow-providers-common-sql", # Additional devel dependencies (do not remove this line and add extra development dependencies) "apache-airflow-providers-common-sql[pandas,polars]", - "testcontainers>=4.12.0" ] # To build docs: diff --git a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py index 774527da9160a..a20370c238d3e 100644 --- a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py +++ b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py @@ -1078,7 +1078,6 @@ def test_read_es_log(self, write_to_es, ti): assert "message" in json_log assert json_log["message"] == msg - # @patch.object(ElasticsearchRemoteLogIO, "_get_index_patterns", return_value="invalid") @patch("elasticsearch.Elasticsearch.count", return_value={"count": 0}) def test_read_with_missing_log(self, mocked_count, ti): log_source_info, log_messages = self.elasticsearch_io.read("", ti) diff --git a/providers/mongo/pyproject.toml b/providers/mongo/pyproject.toml index 89e9b17e47495..21183ac91778e 100644 --- a/providers/mongo/pyproject.toml +++ b/providers/mongo/pyproject.toml @@ -68,7 +68,6 @@ dev = [ "apache-airflow-task-sdk", "apache-airflow-devel-common", # Additional devel dependencies (do not remove this line and add extra development dependencies) - "testcontainers>=4.12.0" ] # To build docs: From c17147dca735aa1f6501711e8897b41c9aa6dab1 Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Tue, 12 Aug 2025 23:54:25 +0800 Subject: [PATCH 13/46] Patch REMOTE_TASK_LOG for airflow 3. Return message / event field depending on airflow version --- .../elasticsearch/log/es_task_handler.py | 49 +++++++++++++++++-- .../log/elasticmock/fake_elasticsearch.py | 23 ++++++--- .../elasticsearch/log/test_es_task_handler.py | 2 +- 3 files changed, 62 insertions(+), 12 deletions(-) diff --git a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py index 121005694f010..d8e866787287a 100644 --- a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py @@ -40,6 +40,7 @@ from elasticsearch import helpers from elasticsearch.exceptions import NotFoundError +import airflow.logging_config as alc from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.models.dagrun import DagRun @@ -217,6 +218,11 @@ def __init__( base_log_folder=base_log_folder, delete_local_copy=self.delete_local_copy, ) + # Airflow 3 introduce REMOTE_TASK_LOG for handling remote logging + # REMOTE_TASK_LOG should be explicitly set in airflow_local_settings.py when trying to use ESTaskHandler + # Before airflow 3.1, REMOTE_TASK_LOG is not set when trying to use ES TaskHandler. + if AIRFLOW_V_3_0_PLUS and alc.REMOTE_TASK_LOG is None: + alc.REMOTE_TASK_LOG = self.io @staticmethod def format_url(host: str) -> str: @@ -342,7 +348,21 @@ def _read( # have the log uploaded but will not be stored in elasticsearch. metadata["end_of_log"] = False if logs_by_host: - if any(x[-1].message == self.end_of_log_mark for x in logs_by_host.values()): + # In Airflow 2.x, the log record JSON has a "message" key, e.g.: + # { + # "message": "Dag name:dataset_consumes_1 queued_at:2025-08-12 15:05:57.703493+00:00", + # "offset": 1755011166339518208, + # "log_id": "dataset_consumes_1-consuming_1-manual__2025-08-12T15:05:57.691303+00:00--1-1" + # } + # + # In Airflow 3.x, the "message" field is renamed to "event". + # We check the correct attribute depending on the Airflow major version. + if AIRFLOW_V_3_0_PLUS: + end_mark_found = any(x[-1].event == self.end_of_log_mark for x in logs_by_host.values()) + else: + end_mark_found = any(x[-1].message == self.end_of_log_mark for x in logs_by_host.values()) + + if end_mark_found: metadata["end_of_log"] = True cur_ts = pendulum.now() @@ -377,7 +397,19 @@ def _read( # If we hit the end of the log, remove the actual end_of_log message # to prevent it from showing in the UI. def concat_logs(hits: list[Hit]) -> str: - log_range = (len(hits) - 1) if hits[-1].message == self.end_of_log_mark else len(hits) + # In Airflow 2.x, the log record JSON has a "message" key, e.g.: + # { + # "message": "Dag name:dataset_consumes_1 queued_at:2025-08-12 15:05:57.703493+00:00", + # "offset": 1755011166339518208, + # "log_id": "dataset_consumes_1-consuming_1-manual__2025-08-12T15:05:57.691303+00:00--1-1" + # } + # + # In Airflow 3.x, the "message" field is renamed to "event". + # We check the correct attribute depending on the Airflow major version. + if AIRFLOW_V_3_0_PLUS: + log_range = (len(hits) - 1) if hits[-1].event == self.end_of_log_mark else len(hits) + else: + log_range = (len(hits) - 1) if hits[-1].message == self.end_of_log_mark else len(hits) return "\n".join(self._format_msg(hits[i]) for i in range(log_range)) if logs_by_host: @@ -420,6 +452,17 @@ def _format_msg(self, hit: Hit): ) # Just a safe-guard to preserve backwards-compatibility + # In Airflow 2.x, the log record JSON has a "message" key, e.g.: + # { + # "message": "Dag name:dataset_consumes_1 queued_at:2025-08-12 15:05:57.703493+00:00", + # "offset": 1755011166339518208, + # "log_id": "dataset_consumes_1-consuming_1-manual__2025-08-12T15:05:57.691303+00:00--1-1" + # } + # + # In Airflow 3.x, the "message" field is renamed to "event". + # We check the correct attribute depending on the Airflow major version. + if AIRFLOW_V_3_0_PLUS: + return hit.event return hit.message def emit(self, record): @@ -499,7 +542,7 @@ def close(self) -> None: # so we know where to stop while auto-tailing. self.emit(logging.makeLogRecord({"msg": self.end_of_log_mark})) - if self.write_stdout: + if self.io.write_stdout: self.handler.close() sys.stdout = sys.__stdout__ diff --git a/providers/elasticsearch/tests/unit/elasticsearch/log/elasticmock/fake_elasticsearch.py b/providers/elasticsearch/tests/unit/elasticsearch/log/elasticmock/fake_elasticsearch.py index 92cbad3ca7075..cf08bd928feae 100644 --- a/providers/elasticsearch/tests/unit/elasticsearch/log/elasticmock/fake_elasticsearch.py +++ b/providers/elasticsearch/tests/unit/elasticsearch/log/elasticmock/fake_elasticsearch.py @@ -104,19 +104,22 @@ def sample_log_response(self, headers=None, params=None): "file": { "path": "/opt/airflow/Documents/GitHub/airflow/logs/" "dag_id=example_bash_operator'" - "/run_id=owen_run_run/task_id=run_after_loop/attempt=1.log" + "/run_id=run_run/task_id=run_after_loop/attempt=1.log" }, "offset": 0, }, "log.offset": 1688888863907337472, - "log_id": "example_bash_operator-run_after_loop-owen_run_run--1-1", + "log_id": "example_bash_operator-run_after_loop-run_run--1-1", "message": "Dependencies all met for " "dep_context=non-requeueable deps " "ti=", + "example_bash_operator.run_after_loop ", "task_id": "run_after_loop", "try_number": "1", + "event": "Dependencies all met for " + "dep_context=non-requeueable deps " + "ti= on 2023-07-09 " "07:47:32+00:00", "task_id": "run_after_loop", "try_number": "1", + "event": "Executing on 2023-07-09 " + "07:47:32+00:00", }, "_type": "_doc", }, diff --git a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py index a20370c238d3e..bd48c3f66a9c8 100644 --- a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py +++ b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py @@ -145,7 +145,7 @@ def concat_logs(lines): assert ( message == "Dependencies all met for dep_context=non-requeueable" - " deps ti=\n" + " deps ti= " "on 2023-07-09 07:47:32+00:00" ) From 9e815bd37304faa4ac6899d1173899616463a2b1 Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Wed, 20 Aug 2025 18:51:18 +0800 Subject: [PATCH 14/46] Use _render_log_id for both reading & writing remote log --- .../airflow_local_settings.py | 2 + .../elasticsearch/log/es_task_handler.py | 142 +++++++++--------- .../elasticsearch/log/test_es_task_handler.py | 15 +- 3 files changed, 82 insertions(+), 77 deletions(-) diff --git a/airflow-core/src/airflow/config_templates/airflow_local_settings.py b/airflow-core/src/airflow/config_templates/airflow_local_settings.py index a4731670454dc..d6b6b124a141e 100644 --- a/airflow-core/src/airflow/config_templates/airflow_local_settings.py +++ b/airflow-core/src/airflow/config_templates/airflow_local_settings.py @@ -291,6 +291,7 @@ def _default_conn_name_from(mod_path, hook_name): ELASTICSEARCH_WRITE_STDOUT: bool = conf.getboolean("elasticsearch", "WRITE_STDOUT") ELASTICSEARCH_WRITE_TO_ES: bool = conf.getboolean("elasticsearch", "WRITE_TO_ES") + ELASTICSEARCH_JSON_FORMAT: bool = conf.getboolean("elasticsearch", "JSON_FORMAT") ELASTICSEARCH_TARGET_INDEX: str = conf.get_mandatory_value("elasticsearch", "TARGET_INDEX") ELASTICSEARCH_HOST_FIELD: str = conf.get_mandatory_value("elasticsearch", "HOST_FIELD") ELASTICSEARCH_OFFSET_FIELD: str = conf.get_mandatory_value("elasticsearch", "OFFSET_FIELD") @@ -304,6 +305,7 @@ def _default_conn_name_from(mod_path, hook_name): host_field=ELASTICSEARCH_HOST_FIELD, base_log_folder=BASE_LOG_FOLDER, delete_local_copy=delete_local_copy, + json_format=ELASTICSEARCH_JSON_FORMAT, ) elif OPENSEARCH_HOST: diff --git a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py index d8e866787287a..d92c10c6fc3b4 100644 --- a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py @@ -44,6 +44,7 @@ from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.models.dagrun import DagRun +from airflow.models.taskinstance import TaskInstance from airflow.providers.elasticsearch.log.es_json_formatter import ( ElasticsearchJSONFormatter, ) @@ -58,7 +59,7 @@ if TYPE_CHECKING: from datetime import datetime - from airflow.models.taskinstance import TaskInstance, TaskInstanceKey + from airflow.models.taskinstance import TaskInstanceKey from airflow.sdk.types import RuntimeTaskInstanceProtocol as RuntimeTI from airflow.utils.log.file_task_handler import LogMessages, LogMetadata, LogSourceInfo @@ -129,6 +130,64 @@ def getattr_nested(obj, item, default): return default +def _render_log_id(ti: TaskInstance | TaskInstanceKey, try_number: int, json_format: bool) -> str: + from airflow.models.taskinstance import TaskInstance, TaskInstanceKey + + with create_session() as session: + if isinstance(ti, TaskInstanceKey) or isinstance(ti, TaskInstance): + ti = _ensure_ti(ti, session) + dag_run = ti.get_dagrun(session=session) + if USE_PER_RUN_LOG_ID: + log_id_template = dag_run.get_log_template(session=session).elasticsearch_id + if TYPE_CHECKING: + assert ti.task + try: + dag = ti.task.dag + except AttributeError: # ti.task is not always set. + data_interval = (dag_run.data_interval_start, dag_run.data_interval_end) + else: + if TYPE_CHECKING: + assert dag is not None + # TODO: Task-SDK: Where should this function be? + data_interval = dag.get_run_data_interval(dag_run) # type: ignore[attr-defined] + if json_format: + data_interval_start = _clean_date(data_interval[0]) + data_interval_end = _clean_date(data_interval[1]) + logical_date = _clean_date(dag_run.logical_date) + else: + if data_interval[0]: + data_interval_start = data_interval[0].isoformat() + else: + data_interval_start = "" + if data_interval[1]: + data_interval_end = data_interval[1].isoformat() + else: + data_interval_end = "" + logical_date = dag_run.logical_date.isoformat() + return log_id_template.format( + dag_id=ti.dag_id, + task_id=ti.task_id, + run_id=getattr(ti, "run_id", ""), + data_interval_start=data_interval_start, + data_interval_end=data_interval_end, + logical_date=logical_date, + execution_date=logical_date, + try_number=try_number, + map_index=getattr(ti, "map_index", ""), + ) + + +def _clean_date(value: datetime | None) -> str: + """ + Clean up a date value so that it is safe to query in elasticsearch by removing reserved characters. + + https://www.elastic.co/guide/en/elasticsearch/reference/current/query-dsl-query-string-query.html#_reserved_characters + """ + if value is None: + return "" + return value.strftime("%Y_%m_%dT%H_%M_%S_%f") + + class ElasticsearchTaskHandler(FileTaskHandler, ExternalLoggingMixin, LoggingMixin): """ ElasticsearchTaskHandler is a python log handler that reads logs from Elasticsearch. @@ -164,8 +223,8 @@ def __init__( base_log_folder: str, end_of_log_mark: str, write_stdout: bool, - json_format: bool, json_fields: str, + json_format: bool = False, write_to_es: bool = False, target_index: str = "airflow-logs", host_field: str = "host", @@ -246,66 +305,6 @@ def format_url(host: str) -> str: return host - def _render_log_id(self, ti: TaskInstance | TaskInstanceKey, try_number: int) -> str: - from airflow.models.taskinstance import TaskInstanceKey - - with create_session() as session: - if isinstance(ti, TaskInstanceKey): - ti = _ensure_ti(ti, session) - dag_run = ti.get_dagrun(session=session) - if USE_PER_RUN_LOG_ID: - log_id_template = dag_run.get_log_template(session=session).elasticsearch_id - - if TYPE_CHECKING: - assert ti.task - try: - dag = ti.task.dag - except AttributeError: # ti.task is not always set. - data_interval = (dag_run.data_interval_start, dag_run.data_interval_end) - else: - if TYPE_CHECKING: - assert dag is not None - # TODO: Task-SDK: Where should this function be? - data_interval = dag.get_run_data_interval(dag_run) # type: ignore[attr-defined] - - if self.json_format: - data_interval_start = self._clean_date(data_interval[0]) - data_interval_end = self._clean_date(data_interval[1]) - logical_date = self._clean_date(dag_run.logical_date) - else: - if data_interval[0]: - data_interval_start = data_interval[0].isoformat() - else: - data_interval_start = "" - if data_interval[1]: - data_interval_end = data_interval[1].isoformat() - else: - data_interval_end = "" - logical_date = dag_run.logical_date.isoformat() - - return log_id_template.format( - dag_id=ti.dag_id, - task_id=ti.task_id, - run_id=getattr(ti, "run_id", ""), - data_interval_start=data_interval_start, - data_interval_end=data_interval_end, - logical_date=logical_date, - execution_date=logical_date, - try_number=try_number, - map_index=getattr(ti, "map_index", ""), - ) - - @staticmethod - def _clean_date(value: datetime | None) -> str: - """ - Clean up a date value so that it is safe to query in elasticsearch by removing reserved characters. - - https://www.elastic.co/guide/en/elasticsearch/reference/current/query-dsl-query-string-query.html#_reserved_characters - """ - if value is None: - return "" - return value.strftime("%Y_%m_%dT%H_%M_%S_%f") - def _read_grouped_logs(self): return True @@ -329,7 +328,7 @@ def _read( metadata["offset"] = 0 offset = metadata["offset"] - log_id = self._render_log_id(ti, try_number) + log_id = _render_log_id(ti, try_number, self.json_format) response = self.io._es_read(log_id, offset, ti) # TODO: Can we skip group logs by host ? if response is not None and response.hits: @@ -492,12 +491,10 @@ def set_context(self, ti: TaskInstance, *, identifier: str | None = None) -> Non "dag_id": str(ti.dag_id), "task_id": str(ti.task_id), date_key: ( - self._clean_date(ti.logical_date) - if AIRFLOW_V_3_0_PLUS - else self._clean_date(ti.execution_date) + _clean_date(ti.logical_date) if AIRFLOW_V_3_0_PLUS else _clean_date(ti.execution_date) ), "try_number": str(ti.try_number), - "log_id": self._render_log_id(ti, ti.try_number), + "log_id": _render_log_id(ti, ti.try_number, self.json_format), }, ) @@ -563,7 +560,7 @@ def get_external_log_url(self, task_instance: TaskInstance, try_number: int) -> :param try_number: task instance try_number to read logs from. :return: URL to the external log collection service """ - log_id = self._render_log_id(task_instance, try_number) + log_id = _render_log_id(task_instance, try_number, self.json_format) scheme = "" if "://" in self.frontend else "https://" return scheme + self.frontend.format(log_id=quote(log_id)) @@ -613,6 +610,7 @@ def _get_result(self, hit: dict[Any, Any], parent_class=None) -> Hit: @attrs.define(kw_only=True) class ElasticsearchRemoteLogIO(LoggingMixin): # noqa: D101 + json_format: bool = False write_stdout: bool = False delete_local_copy: bool = False host: str = "http://localhost:9200" @@ -643,6 +641,10 @@ def upload(self, path: os.PathLike | str, ti: RuntimeTI): else: local_loc = self.base_log_folder.joinpath(path) + # Convert the runtimeTI to the real TaskInstance that via fetching from DB + ti = TaskInstance.get_task_instance( + ti.dag_id, ti.run_id, ti.task_id, ti.map_index if ti.map_index is not None else -1 + ) # type: ignore[assignment] if local_loc.is_file() and self.write_stdout: # Intentionally construct the log_id and offset field log_lines = self._parse_raw_log(local_loc.read_text(), ti) @@ -660,13 +662,13 @@ def _parse_raw_log(self, log: str, ti: RuntimeTI) -> list[dict[str, Any]]: logs = log.split("\n") parsed_logs = [] offset = 1 + log_id = _render_log_id(ti, ti.try_number, self.json_format) # type: ignore[arg-type] for line in logs: # Make sure line is not empty if line.strip(): # construct log_id which is {dag_id}-{task_id}-{run_id}-{map_index}-{try_number} # also construct the offset field (default is 'offset') log_dict = json.loads(line) - log_id = f"{ti.dag_id}-{ti.task_id}-{ti.run_id}-{ti.map_index}-{ti.try_number}" log_dict.update({"log_id": log_id, self.offset_field: offset}) offset += 1 parsed_logs.append(log_dict) @@ -689,7 +691,7 @@ def _write_to_es(self, log_lines: list[dict[str, Any]]) -> bool: return False def read(self, relative_path: str, ti: RuntimeTI) -> tuple[LogSourceInfo, LogMessages]: - log_id = f"{ti.dag_id}-{ti.task_id}-{ti.run_id}-{ti.map_index}-{ti.try_number}" + log_id = _render_log_id(ti, ti.try_number, self.json_format) # type: ignore[arg-type] self.log.info("Reading log %s from Elasticsearch", log_id) offset = 0 response = self._es_read(log_id, offset, ti) diff --git a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py index bd48c3f66a9c8..363d8030e51ff 100644 --- a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py +++ b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py @@ -40,6 +40,8 @@ VALID_ES_CONFIG_KEYS, ElasticsearchRemoteLogIO, ElasticsearchTaskHandler, + _clean_date, + _render_log_id, get_es_kwargs_from_config, getattr_nested, ) @@ -75,7 +77,7 @@ class TestElasticsearchTaskHandler: TASK_ID = "task_for_testing_es_log_handler" LOGICAL_DATE = datetime(2016, 1, 1) LOG_ID = f"{DAG_ID}-{TASK_ID}-2016-01-01T00:00:00+00:00-1" - JSON_LOG_ID = f"{DAG_ID}-{TASK_ID}-{ElasticsearchTaskHandler._clean_date(LOGICAL_DATE)}-1" + JSON_LOG_ID = f"{DAG_ID}-{TASK_ID}-{_clean_date(LOGICAL_DATE)}-1" FILENAME_TEMPLATE = "{try_number}.log" @pytest.fixture @@ -751,13 +753,13 @@ def test_close_with_no_stream(self, ti): @pytest.mark.db_test def test_render_log_id(self, ti): - assert self.es_task_handler._render_log_id(ti, 1) == self.LOG_ID + assert _render_log_id(ti, 1, self.es_task_handler.json_format) == self.LOG_ID self.es_task_handler.json_format = True - assert self.es_task_handler._render_log_id(ti, 1) == self.JSON_LOG_ID + assert _render_log_id(ti, 1, self.es_task_handler.json_format) == self.JSON_LOG_ID def test_clean_date(self): - clean_logical_date = self.es_task_handler._clean_date(datetime(2016, 7, 8, 9, 10, 11, 12)) + clean_logical_date = _clean_date(datetime(2016, 7, 8, 9, 10, 11, 12)) assert clean_logical_date == "2016_07_08T09_10_11_000012" @pytest.mark.db_test @@ -1022,7 +1024,7 @@ def test_write_to_es(self, tmp_json_file, ti): offset = 1 expected_msg = ["start", "processing", "end"] - expected_log_id = f"{ti.dag_id}-{ti.task_id}-{ti.run_id}-{ti.map_index}-{ti.try_number}" + expected_log_id = _render_log_id(ti, 1, self.elasticsearch_io.json_format) assert res["hits"]["total"]["value"] == 3 for msg, hit in zip(expected_msg, res["hits"]["hits"]): assert hit["_index"] == "airflow-logs" @@ -1038,7 +1040,6 @@ def test_write_to_stdout(self, tmp_json_file, ti, capsys): captured = capsys.readouterr() stdout_lines = captured.out.strip().splitlines() - log_entries = [json.loads(line) for line in stdout_lines] assert log_entries[0]["message"] == "start" assert log_entries[1]["message"] == "processing" @@ -1081,7 +1082,7 @@ def test_read_es_log(self, write_to_es, ti): @patch("elasticsearch.Elasticsearch.count", return_value={"count": 0}) def test_read_with_missing_log(self, mocked_count, ti): log_source_info, log_messages = self.elasticsearch_io.read("", ti) - log_id = f"{ti.dag_id}-{ti.task_id}-{ti.run_id}-{ti.map_index}-{ti.try_number}" + log_id = _render_log_id(ti, ti.try_number, self.elasticsearch_io.json_format) assert log_source_info == [] assert f"*** Log {log_id} not found in Elasticsearch" in log_messages[0] mocked_count.assert_called_once() From a74d7dc4fa4e0c764274b90c755f105f67232e5e Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Sun, 31 Aug 2025 12:16:03 +0800 Subject: [PATCH 15/46] use hasattr to return hit field or message field, instead of relying on airflow version --- .../elasticsearch/log/es_task_handler.py | 27 ++++++++++--------- 1 file changed, 15 insertions(+), 12 deletions(-) diff --git a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py index d92c10c6fc3b4..39ba2f1e69d08 100644 --- a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py @@ -356,11 +356,9 @@ def _read( # # In Airflow 3.x, the "message" field is renamed to "event". # We check the correct attribute depending on the Airflow major version. - if AIRFLOW_V_3_0_PLUS: - end_mark_found = any(x[-1].event == self.end_of_log_mark for x in logs_by_host.values()) - else: - end_mark_found = any(x[-1].message == self.end_of_log_mark for x in logs_by_host.values()) - + end_mark_found = any( + self._get_log_message(x[-1]) == self.end_of_log_mark for x in logs_by_host.values() + ) if end_mark_found: metadata["end_of_log"] = True @@ -405,10 +403,9 @@ def concat_logs(hits: list[Hit]) -> str: # # In Airflow 3.x, the "message" field is renamed to "event". # We check the correct attribute depending on the Airflow major version. - if AIRFLOW_V_3_0_PLUS: - log_range = (len(hits) - 1) if hits[-1].event == self.end_of_log_mark else len(hits) - else: - log_range = (len(hits) - 1) if hits[-1].message == self.end_of_log_mark else len(hits) + log_range = ( + (len(hits) - 1) if self._get_log_message(hits[-1]) == self.end_of_log_mark else len(hits) + ) return "\n".join(self._format_msg(hits[i]) for i in range(log_range)) if logs_by_host: @@ -460,9 +457,7 @@ def _format_msg(self, hit: Hit): # # In Airflow 3.x, the "message" field is renamed to "event". # We check the correct attribute depending on the Airflow major version. - if AIRFLOW_V_3_0_PLUS: - return hit.event - return hit.message + return self._get_log_message(hit) def emit(self, record): if self.handler: @@ -607,6 +602,14 @@ def _get_result(self, hit: dict[Any, Any], parent_class=None) -> Hit: callback: type[Hit] | Callable[..., Any] = getattr(doc_class, "from_es", doc_class) return callback(hit) + def _get_log_message(self, hit: Hit) -> str: + """Get log message from hit, supporting both Airflow 2.x and 3.x formats.""" + if hasattr(hit, "event"): + return hit.event + if hasattr(hit, "message"): + return hit.message + return "" + @attrs.define(kw_only=True) class ElasticsearchRemoteLogIO(LoggingMixin): # noqa: D101 From f1562c02b748198b915c36cda7340a054de62a21 Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Sat, 6 Sep 2025 16:05:56 +0200 Subject: [PATCH 16/46] Move concat_logs under ESTaskHandler. Revise sample_log_response to have airflow 2 & airflow 3 responses. --- .../elasticsearch/log/es_task_handler.py | 55 +++------ .../log/elasticmock/fake_elasticsearch.py | 114 +++++++++++++++++- .../elasticsearch/log/test_es_task_handler.py | 19 +-- 3 files changed, 138 insertions(+), 50 deletions(-) diff --git a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py index 39ba2f1e69d08..7cdc807a13979 100644 --- a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py @@ -347,15 +347,6 @@ def _read( # have the log uploaded but will not be stored in elasticsearch. metadata["end_of_log"] = False if logs_by_host: - # In Airflow 2.x, the log record JSON has a "message" key, e.g.: - # { - # "message": "Dag name:dataset_consumes_1 queued_at:2025-08-12 15:05:57.703493+00:00", - # "offset": 1755011166339518208, - # "log_id": "dataset_consumes_1-consuming_1-manual__2025-08-12T15:05:57.691303+00:00--1-1" - # } - # - # In Airflow 3.x, the "message" field is renamed to "event". - # We check the correct attribute depending on the Airflow major version. end_mark_found = any( self._get_log_message(x[-1]) == self.end_of_log_mark for x in logs_by_host.values() ) @@ -391,23 +382,6 @@ def _read( if int(offset) != int(next_offset) or "last_log_timestamp" not in metadata: metadata["last_log_timestamp"] = str(cur_ts) - # If we hit the end of the log, remove the actual end_of_log message - # to prevent it from showing in the UI. - def concat_logs(hits: list[Hit]) -> str: - # In Airflow 2.x, the log record JSON has a "message" key, e.g.: - # { - # "message": "Dag name:dataset_consumes_1 queued_at:2025-08-12 15:05:57.703493+00:00", - # "offset": 1755011166339518208, - # "log_id": "dataset_consumes_1-consuming_1-manual__2025-08-12T15:05:57.691303+00:00--1-1" - # } - # - # In Airflow 3.x, the "message" field is renamed to "event". - # We check the correct attribute depending on the Airflow major version. - log_range = ( - (len(hits) - 1) if self._get_log_message(hits[-1]) == self.end_of_log_mark else len(hits) - ) - return "\n".join(self._format_msg(hits[i]) for i in range(log_range)) - if logs_by_host: if AIRFLOW_V_3_0_PLUS: from airflow.utils.log.file_task_handler import StructuredLogMessage @@ -430,7 +404,7 @@ def concat_logs(hits: list[Hit]) -> str: ] else: message = [ - (host, concat_logs(hits)) # type: ignore[misc] + (host, self.concat_logs(hits)) # type: ignore[misc] for host, hits in logs_by_host.items() ] else: @@ -448,15 +422,6 @@ def _format_msg(self, hit: Hit): ) # Just a safe-guard to preserve backwards-compatibility - # In Airflow 2.x, the log record JSON has a "message" key, e.g.: - # { - # "message": "Dag name:dataset_consumes_1 queued_at:2025-08-12 15:05:57.703493+00:00", - # "offset": 1755011166339518208, - # "log_id": "dataset_consumes_1-consuming_1-manual__2025-08-12T15:05:57.691303+00:00--1-1" - # } - # - # In Airflow 3.x, the "message" field is renamed to "event". - # We check the correct attribute depending on the Airflow major version. return self._get_log_message(hit) def emit(self, record): @@ -603,13 +568,29 @@ def _get_result(self, hit: dict[Any, Any], parent_class=None) -> Hit: return callback(hit) def _get_log_message(self, hit: Hit) -> str: - """Get log message from hit, supporting both Airflow 2.x and 3.x formats.""" + """ + Get log message from hit, supporting both Airflow 2.x and 3.x formats. + + In Airflow 2.x, the log record JSON has a "message" key, e.g.: + { + "message": "Dag name:dataset_consumes_1 queued_at:2025-08-12 15:05:57.703493+00:00", + "offset": 1755011166339518208, + "log_id": "dataset_consumes_1-consuming_1-manual__2025-08-12T15:05:57.691303+00:00--1-1" + } + + In Airflow 3.x, the "message" field is renamed to "event". + We check the correct attribute depending on the Airflow major version. + """ if hasattr(hit, "event"): return hit.event if hasattr(hit, "message"): return hit.message return "" + def concat_logs(self, hits: list[Hit]) -> str: + log_range = (len(hits) - 1) if self._get_log_message(hits[-1]) == self.end_of_log_mark else len(hits) + return "\n".join(self._format_msg(hits[i]) for i in range(log_range)) + @attrs.define(kw_only=True) class ElasticsearchRemoteLogIO(LoggingMixin): # noqa: D101 diff --git a/providers/elasticsearch/tests/unit/elasticsearch/log/elasticmock/fake_elasticsearch.py b/providers/elasticsearch/tests/unit/elasticsearch/log/elasticmock/fake_elasticsearch.py index cf08bd928feae..df101f2f918e4 100644 --- a/providers/elasticsearch/tests/unit/elasticsearch/log/elasticmock/fake_elasticsearch.py +++ b/providers/elasticsearch/tests/unit/elasticsearch/log/elasticmock/fake_elasticsearch.py @@ -80,7 +80,7 @@ def info(self, params=None): } @query_params() - def sample_log_response(self, headers=None, params=None): + def sample_airflow_2_log_response(self, headers=None, params=None): return { "_shards": {"failed": 0, "skipped": 0, "successful": 7, "total": 7}, "hits": { @@ -116,6 +116,114 @@ def sample_log_response(self, headers=None, params=None): "example_bash_operator.run_after_loop ", "task_id": "run_after_loop", "try_number": "1", + }, + "_type": "_doc", + }, + { + "_id": "qteZT4kBjAZqZnexVUxl", + "_index": ".ds-filebeat-8.8.2-2023.07.09-000001", + "_score": 2.482621, + "_source": { + "@timestamp": "2023-07-13T14:13:15.141Z", + "asctime": "2023-07-09T07:47:43.917+0000", + "container": {"id": "airflow"}, + "dag_id": "example_bash_operator", + "ecs": {"version": "8.0.0"}, + "execution_date": "2023_07_09T07_47_32_000000", + "filename": "taskinstance.py", + "input": {"type": "log"}, + "levelname": "INFO", + "lineno": 1347, + "log": { + "file": { + "path": "/opt/airflow/Documents/GitHub/airflow/logs/" + "dag_id=example_bash_operator" + "/run_id=run_run/task_id=run_after_loop/attempt=1.log" + }, + "offset": 988, + }, + "log.offset": 1688888863917961216, + "log_id": "example_bash_operator-run_after_loop-run_run--1-1", + "message": "Starting attempt 1 of 1", + "task_id": "run_after_loop", + "try_number": "1", + }, + "_type": "_doc", + }, + { + "_id": "v9eZT4kBjAZqZnexVUx2", + "_index": ".ds-filebeat-8.8.2-2023.07.09-000001", + "_score": 2.482621, + "_source": { + "@timestamp": "2023-07-13T14:13:15.143Z", + "asctime": "2023-07-09T07:47:43.928+0000", + "container": {"id": "airflow"}, + "dag_id": "example_bash_operator", + "ecs": {"version": "8.0.0"}, + "execution_date": "2023_07_09T07_47_32_000000", + "filename": "taskinstance.py", + "input": {"type": "log"}, + "levelname": "INFO", + "lineno": 1368, + "log": { + "file": { + "path": "/opt/airflow/Documents/GitHub/airflow/logs/" + "dag_id=example_bash_operator" + "/run_id=run_run/task_id=run_after_loop/attempt=1.log" + }, + "offset": 1372, + }, + "log.offset": 1688888863928218880, + "log_id": "example_bash_operator-run_after_loop-run_run--1-1", + "message": "Executing on 2023-07-09 " + "07:47:32+00:00", + "task_id": "run_after_loop", + "try_number": "1", + }, + "_type": "_doc", + }, + ], + "max_score": 2.482621, + "total": {"relation": "eq", "value": 36}, + }, + "timed_out": False, + "took": 7, + } + + @query_params() + def sample_airflow_3_log_response(self, headers=None, params=None): + return { + "_shards": {"failed": 0, "skipped": 0, "successful": 7, "total": 7}, + "hits": { + "hits": [ + { + "_id": "jdeZT4kBjAZqZnexVUxk", + "_index": ".ds-filebeat-8.8.2-2023.07.09-000001", + "_score": 2.482621, + "_source": { + "@timestamp": "2023-07-13T14:13:15.140Z", + "asctime": "2023-07-09T07:47:43.907+0000", + "container": {"id": "airflow"}, + "dag_id": "example_bash_operator", + "ecs": {"version": "8.0.0"}, + "execution_date": "2023_07_09T07_47_32_000000", + "filename": "taskinstance.py", + "input": {"type": "log"}, + "levelname": "INFO", + "lineno": 1144, + "log": { + "file": { + "path": "/opt/airflow/Documents/GitHub/airflow/logs/" + "dag_id=example_bash_operator'" + "/run_id=run_run/task_id=run_after_loop/attempt=1.log" + }, + "offset": 0, + }, + "log.offset": 1688888863907337472, + "log_id": "example_bash_operator-run_after_loop-run_run--1-1", + "task_id": "run_after_loop", + "try_number": "1", "event": "Dependencies all met for " "dep_context=non-requeueable deps " "ti= on 2023-07-09 " - "07:47:32+00:00", "task_id": "run_after_loop", "try_number": "1", "event": "Executing Date: Sat, 6 Sep 2025 20:18:58 +0200 Subject: [PATCH 17/46] Fix ruff. get log id before creating runtime instance --- .../elasticsearch/log/es_task_handler.py | 16 ++++++---------- 1 file changed, 6 insertions(+), 10 deletions(-) diff --git a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py index 44f5bb423a38b..e76ebafd57fb1 100644 --- a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py @@ -44,19 +44,14 @@ from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.models.dagrun import DagRun +from airflow.models.taskinstance import TaskInstance from airflow.providers.elasticsearch.log.es_json_formatter import ElasticsearchJSONFormatter -from airflow.providers.elasticsearch.log.es_response import ElasticSearchResponse, Hit +from airflow.providers.elasticsearch.log.es_response import ElasticSearchResponse, Hit, resolve_nested from airflow.providers.elasticsearch.version_compat import ( AIRFLOW_V_3_0_PLUS, AIRFLOW_V_3_1_PLUS, EsLogMsgType, ) -from airflow.models.taskinstance import TaskInstance -from airflow.providers.elasticsearch.log.es_json_formatter import ( - ElasticsearchJSONFormatter, -) -from airflow.providers.elasticsearch.log.es_response import ElasticSearchResponse, Hit, resolve_nested -from airflow.providers.elasticsearch.version_compat import AIRFLOW_V_3_0_PLUS, EsLogMsgType from airflow.utils import timezone from airflow.utils.log.file_task_handler import FileTaskHandler from airflow.utils.log.logging_mixin import ExternalLoggingMixin, LoggingMixin @@ -141,6 +136,7 @@ def getattr_nested(obj, item, default): except AttributeError: return default + def _render_log_id(ti: TaskInstance | TaskInstanceKey, try_number: int, json_format: bool) -> str: from airflow.models.taskinstance import TaskInstanceKey @@ -627,13 +623,14 @@ def upload(self, path: os.PathLike | str, ti: RuntimeTI): else: local_loc = self.base_log_folder.joinpath(path) + log_id = _render_log_id(ti, ti.try_number, self.json_format) # type: ignore[arg-type] # Convert the runtimeTI to the real TaskInstance that via fetching from DB ti = TaskInstance.get_task_instance( ti.dag_id, ti.run_id, ti.task_id, ti.map_index if ti.map_index is not None else -1 ) # type: ignore[assignment] if local_loc.is_file() and self.write_stdout: # Intentionally construct the log_id and offset field - log_lines = self._parse_raw_log(local_loc.read_text(), ti) + log_lines = self._parse_raw_log(local_loc.read_text(), log_id, ti) for line in log_lines: sys.stdout.write(json.dumps(line) + "\n") sys.stdout.flush() @@ -644,11 +641,10 @@ def upload(self, path: os.PathLike | str, ti: RuntimeTI): if success and self.delete_local_copy: shutil.rmtree(os.path.dirname(local_loc)) - def _parse_raw_log(self, log: str, ti: RuntimeTI) -> list[dict[str, Any]]: + def _parse_raw_log(self, log: str, log_id: str, ti: RuntimeTI) -> list[dict[str, Any]]: logs = log.split("\n") parsed_logs = [] offset = 1 - log_id = _render_log_id(ti, ti.try_number, self.json_format) # type: ignore[arg-type] for line in logs: # Make sure line is not empty if line.strip(): From 136ee8f41769846e381b763c465ca6868710e4f7 Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Sat, 6 Sep 2025 21:21:36 +0200 Subject: [PATCH 18/46] Fix failing CI --- .../airflow/providers/elasticsearch/log/es_task_handler.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py index e76ebafd57fb1..fd81054b17aed 100644 --- a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py @@ -630,18 +630,18 @@ def upload(self, path: os.PathLike | str, ti: RuntimeTI): ) # type: ignore[assignment] if local_loc.is_file() and self.write_stdout: # Intentionally construct the log_id and offset field - log_lines = self._parse_raw_log(local_loc.read_text(), log_id, ti) + log_lines = self._parse_raw_log(local_loc.read_text(), log_id) for line in log_lines: sys.stdout.write(json.dumps(line) + "\n") sys.stdout.flush() if local_loc.is_file() and self.write_to_es: - log_lines = self._parse_raw_log(local_loc.read_text(), ti) + log_lines = self._parse_raw_log(local_loc.read_text(), log_id) success = self._write_to_es(log_lines) if success and self.delete_local_copy: shutil.rmtree(os.path.dirname(local_loc)) - def _parse_raw_log(self, log: str, log_id: str, ti: RuntimeTI) -> list[dict[str, Any]]: + def _parse_raw_log(self, log: str, log_id: str) -> list[dict[str, Any]]: logs = log.split("\n") parsed_logs = [] offset = 1 From d86b92a7c7b6295efbde588641f72ba2ec0eacdf Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Sun, 14 Sep 2025 12:51:45 +0800 Subject: [PATCH 19/46] Use global session instead of creating a new session --- .../elasticsearch/log/es_task_handler.py | 22 +++++++++++-------- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py index fd81054b17aed..f88c653ca9b31 100644 --- a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py @@ -41,6 +41,7 @@ from elasticsearch.exceptions import NotFoundError import airflow.logging_config as alc +from airflow import settings from airflow.configuration import conf from airflow.exceptions import AirflowException from airflow.models.dagrun import DagRun @@ -56,7 +57,6 @@ from airflow.utils.log.file_task_handler import FileTaskHandler from airflow.utils.log.logging_mixin import ExternalLoggingMixin, LoggingMixin from airflow.utils.module_loading import import_string -from airflow.utils.session import create_session if AIRFLOW_V_3_1_PLUS: from airflow.sdk import timezone @@ -140,12 +140,11 @@ def getattr_nested(obj, item, default): def _render_log_id(ti: TaskInstance | TaskInstanceKey, try_number: int, json_format: bool) -> str: from airflow.models.taskinstance import TaskInstanceKey - with create_session() as session: - if isinstance(ti, TaskInstanceKey): - ti = _ensure_ti(ti, session) - dag_run = ti.get_dagrun(session=session) - if USE_PER_RUN_LOG_ID: - log_id_template = dag_run.get_log_template(session=session).elasticsearch_id + if isinstance(ti, TaskInstanceKey): + ti = _ensure_ti(ti, settings.Session) + dag_run = ti.get_dagrun(session=settings.Session) + if USE_PER_RUN_LOG_ID: + log_id_template = dag_run.get_log_template(session=settings.Session).elasticsearch_id if json_format: data_interval_start = _clean_date(dag_run.data_interval_start) @@ -623,13 +622,18 @@ def upload(self, path: os.PathLike | str, ti: RuntimeTI): else: local_loc = self.base_log_folder.joinpath(path) - log_id = _render_log_id(ti, ti.try_number, self.json_format) # type: ignore[arg-type] # Convert the runtimeTI to the real TaskInstance that via fetching from DB ti = TaskInstance.get_task_instance( - ti.dag_id, ti.run_id, ti.task_id, ti.map_index if ti.map_index is not None else -1 + ti.dag_id, + ti.run_id, + ti.task_id, + ti.map_index if ti.map_index is not None else -1, + session=settings.Session, ) # type: ignore[assignment] + log_id = _render_log_id(ti, ti.try_number, self.json_format) # type: ignore[arg-type] if local_loc.is_file() and self.write_stdout: # Intentionally construct the log_id and offset field + log_lines = self._parse_raw_log(local_loc.read_text(), log_id) for line in log_lines: sys.stdout.write(json.dumps(line) + "\n") From b0aede8878e89aa906d4ec92a01a4f5804d7d14e Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Sat, 20 Sep 2025 15:49:22 +0800 Subject: [PATCH 20/46] Do not fetch log_id_template from DB. Get it from config instead. --- .../airflow_local_settings.py | 2 + .../elasticsearch/log/es_task_handler.py | 49 ++++++------------- .../elasticsearch/log/test_es_task_handler.py | 23 ++++++--- 3 files changed, 31 insertions(+), 43 deletions(-) diff --git a/airflow-core/src/airflow/config_templates/airflow_local_settings.py b/airflow-core/src/airflow/config_templates/airflow_local_settings.py index 0f8891e4cd8c1..3e1535bedf694 100644 --- a/airflow-core/src/airflow/config_templates/airflow_local_settings.py +++ b/airflow-core/src/airflow/config_templates/airflow_local_settings.py @@ -287,6 +287,7 @@ def _default_conn_name_from(mod_path, hook_name): ELASTICSEARCH_TARGET_INDEX: str = conf.get_mandatory_value("elasticsearch", "TARGET_INDEX") ELASTICSEARCH_HOST_FIELD: str = conf.get_mandatory_value("elasticsearch", "HOST_FIELD") ELASTICSEARCH_OFFSET_FIELD: str = conf.get_mandatory_value("elasticsearch", "OFFSET_FIELD") + ELASTICSEARCH_LOG_ID_TEMPLATE: str = conf.get_mandatory_value("elasticsearch", "LOG_ID_TEMPLATE") REMOTE_TASK_LOG = ElasticsearchRemoteLogIO( host=ELASTICSEARCH_HOST, @@ -298,6 +299,7 @@ def _default_conn_name_from(mod_path, hook_name): base_log_folder=BASE_LOG_FOLDER, delete_local_copy=delete_local_copy, json_format=ELASTICSEARCH_JSON_FORMAT, + log_id_template=ELASTICSEARCH_LOG_ID_TEMPLATE, ) elif OPENSEARCH_HOST: diff --git a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py index e2ad173baf79c..6e7439b405105 100644 --- a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py @@ -137,38 +137,11 @@ def getattr_nested(obj, item, default): return default -def _render_log_id(ti: TaskInstance | TaskInstanceKey, try_number: int, json_format: bool) -> str: - from airflow.models.taskinstance import TaskInstanceKey - - if isinstance(ti, TaskInstanceKey): - ti = _ensure_ti(ti, settings.Session) - dag_run = ti.get_dagrun(session=settings.Session) - if USE_PER_RUN_LOG_ID: - log_id_template = dag_run.get_log_template(session=settings.Session).elasticsearch_id - - if json_format: - data_interval_start = _clean_date(dag_run.data_interval_start) - data_interval_end = _clean_date(dag_run.data_interval_end) - logical_date = _clean_date(dag_run.logical_date) - else: - if dag_run.data_interval_start: - data_interval_start = dag_run.data_interval_start.isoformat() - else: - data_interval_start = "" - if dag_run.data_interval_end: - data_interval_end = dag_run.data_interval_end.isoformat() - else: - data_interval_end = "" - logical_date = dag_run.logical_date.isoformat() - +def _render_log_id(log_id_template: str, ti: TaskInstance | TaskInstanceKey, try_number: int) -> str: return log_id_template.format( dag_id=ti.dag_id, task_id=ti.task_id, run_id=getattr(ti, "run_id", ""), - data_interval_start=data_interval_start, - data_interval_end=data_interval_end, - logical_date=logical_date, - execution_date=logical_date, try_number=try_number, map_index=getattr(ti, "map_index", ""), ) @@ -270,6 +243,11 @@ def __init__( self.handler: logging.FileHandler | logging.StreamHandler | None = None self._doc_type_map: dict[Any, Any] = {} self._doc_type: list[Any] = [] + self.log_id_template: str = conf.get( + "elasticsearch", + "log_id_template", + fallback="{dag_id}-{task_id}-{run_id}-{map_index}-{try_number}", + ) self.io = ElasticsearchRemoteLogIO( host=self.host, target_index=self.target_index, @@ -279,6 +257,7 @@ def __init__( host_field=self.host_field, base_log_folder=base_log_folder, delete_local_copy=self.delete_local_copy, + log_id_template=self.log_id_template, ) # Airflow 3 introduce REMOTE_TASK_LOG for handling remote logging # REMOTE_TASK_LOG should be explicitly set in airflow_local_settings.py when trying to use ESTaskHandler @@ -331,7 +310,7 @@ def _read( metadata["offset"] = 0 offset = metadata["offset"] - log_id = _render_log_id(ti, try_number, self.json_format) + log_id = _render_log_id(self.log_id_template, ti, try_number) response = self.io._es_read(log_id, offset, ti) # TODO: Can we skip group logs by host ? if response is not None and response.hits: @@ -340,7 +319,6 @@ def _read( else: logs_by_host = None next_offset = offset - # Ensure a string here. Large offset numbers will get JSON.parsed incorrectly # on the client. Sending as a string prevents this issue. # https://developer.mozilla.org/en-US/docs/Web/JavaScript/Reference/Global_Objects/Number/MAX_SAFE_INTEGER @@ -457,7 +435,7 @@ def set_context(self, ti: TaskInstance, *, identifier: str | None = None) -> Non _clean_date(ti.logical_date) if AIRFLOW_V_3_0_PLUS else _clean_date(ti.execution_date) ), "try_number": str(ti.try_number), - "log_id": _render_log_id(ti, ti.try_number, self.json_format), + "log_id": _render_log_id(self.log_id_template, ti, ti.try_number), }, ) @@ -523,7 +501,7 @@ def get_external_log_url(self, task_instance: TaskInstance, try_number: int) -> :param try_number: task instance try_number to read logs from. :return: URL to the external log collection service """ - log_id = _render_log_id(task_instance, try_number, self.json_format) + log_id = _render_log_id(self.log_id_template, task_instance, try_number) scheme = "" if "://" in self.frontend else "https://" return scheme + self.frontend.format(log_id=quote(log_id)) @@ -606,6 +584,7 @@ class ElasticsearchRemoteLogIO(LoggingMixin): # noqa: D101 offset_field: str = "offset" write_to_es: bool = False base_log_folder: Path = attrs.field(converter=Path) + log_id_template: str = "{dag_id}-{task_id}-{run_id}-{map_index}-{try_number}" processors = () @@ -636,7 +615,7 @@ def upload(self, path: os.PathLike | str, ti: RuntimeTI): ti.map_index if ti.map_index is not None else -1, session=settings.Session, ) # type: ignore[assignment] - log_id = _render_log_id(ti, ti.try_number, self.json_format) # type: ignore[arg-type] + log_id = _render_log_id(self.log_id_template, ti, ti.try_number) # type: ignore[arg-type] if local_loc.is_file() and self.write_stdout: # Intentionally construct the log_id and offset field @@ -682,8 +661,8 @@ def _write_to_es(self, log_lines: list[dict[str, Any]]) -> bool: self.log.exception("Unable to insert logs into Elasticsearch. Reason: %s", str(e)) return False - def read(self, relative_path: str, ti: RuntimeTI) -> tuple[LogSourceInfo, LogMessages]: - log_id = _render_log_id(ti, ti.try_number, self.json_format) # type: ignore[arg-type] + def read(self, _relative_path: str, ti: RuntimeTI) -> tuple[LogSourceInfo, LogMessages]: + log_id = _render_log_id(self.log_id_template, ti, ti.try_number) # type: ignore[arg-type] self.log.info("Reading log %s from Elasticsearch", log_id) offset = 0 response = self._es_read(log_id, offset, ti) diff --git a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py index 82bfb7469ac92..3ea523fe217eb 100644 --- a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py +++ b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py @@ -59,10 +59,11 @@ ES_PROVIDER_YAML_FILE = AIRFLOW_PROVIDERS_ROOT_PATH / "elasticsearch" / "provider.yaml" -def get_ti(dag_id, task_id, logical_date, create_task_instance): +def get_ti(dag_id, task_id, run_id, logical_date, create_task_instance): ti = create_task_instance( dag_id=dag_id, task_id=task_id, + run_id=run_id, logical_date=logical_date, dagrun_state=DagRunState.RUNNING, state=TaskInstanceState.RUNNING, @@ -75,8 +76,9 @@ def get_ti(dag_id, task_id, logical_date, create_task_instance): class TestElasticsearchTaskHandler: DAG_ID = "dag_for_testing_es_task_handler" TASK_ID = "task_for_testing_es_log_handler" + RUN_ID = "run_for_testing_es_log_handler" LOGICAL_DATE = datetime(2016, 1, 1) - LOG_ID = f"{DAG_ID}-{TASK_ID}-2016-01-01T00:00:00+00:00-1" + LOG_ID = f"{DAG_ID}-{TASK_ID}-{RUN_ID}--1-1" JSON_LOG_ID = f"{DAG_ID}-{TASK_ID}-{_clean_date(LOGICAL_DATE)}-1" FILENAME_TEMPLATE = "{try_number}.log" @@ -93,6 +95,7 @@ def ti(self, create_task_instance, create_log_template): yield get_ti( dag_id=self.DAG_ID, task_id=self.TASK_ID, + run_id=self.RUN_ID, logical_date=self.LOGICAL_DATE, create_task_instance=create_task_instance, ) @@ -310,9 +313,11 @@ def test_read_missing_logs(self, seconds, create_task_instance): When the log actually isn't there to be found, we only want to wait for 5 seconds. In this case we expect to receive a message of the form 'Log {log_id} not found in elasticsearch ...' """ + run_id = "wrong_run_id" ti = get_ti( self.DAG_ID, self.TASK_ID, + run_id, pendulum.instance(self.LOGICAL_DATE).add(days=1), # so logs are not found create_task_instance=create_task_instance, ) @@ -637,7 +642,7 @@ def test_read_with_json_format_with_custom_offset_and_host_fields(self, ti): self.body = { "message": self.test_message, "event": self.test_message, - "log_id": f"{self.DAG_ID}-{self.TASK_ID}-2016_01_01T00_00_00_000000-1", + "log_id": self.LOG_ID, "log": {"offset": 1}, "host": {"name": "somehostname"}, "asctime": "2020-12-24 19:25:00,962", @@ -756,10 +761,10 @@ def test_close_with_no_stream(self, ti): @pytest.mark.db_test def test_render_log_id(self, ti): - assert _render_log_id(ti, 1, self.es_task_handler.json_format) == self.LOG_ID + assert _render_log_id(self.es_task_handler.log_id_template, ti, 1) == self.LOG_ID self.es_task_handler.json_format = True - assert _render_log_id(ti, 1, self.es_task_handler.json_format) == self.JSON_LOG_ID + assert _render_log_id(self.es_task_handler.log_id_template, ti, 1) == self.LOG_ID def test_clean_date(self): clean_logical_date = _clean_date(datetime(2016, 7, 8, 9, 10, 11, 12)) @@ -773,7 +778,7 @@ def test_clean_date(self): ( True, "localhost:5601/{log_id}", - "https://localhost:5601/" + quote(JSON_LOG_ID), + "https://localhost:5601/" + quote(LOG_ID), ), ( False, @@ -953,6 +958,7 @@ def test_self_not_valid_arg(): class TestElasticsearchRemoteLogIO: DAG_ID = "dag_for_testing_es_log_handler" TASK_ID = "task_for_testing_es_log_handler" + RUN_ID = "run_for_testing_es_log_handler" LOGICAL_DATE = datetime(2016, 1, 1) FILENAME_TEMPLATE = "{try_number}.log" @@ -1001,6 +1007,7 @@ def ti(self, create_task_instance, create_log_template): yield get_ti( dag_id=self.DAG_ID, task_id=self.TASK_ID, + run_id=self.RUN_ID, logical_date=self.LOGICAL_DATE, create_task_instance=create_task_instance, ) @@ -1027,7 +1034,7 @@ def test_write_to_es(self, tmp_json_file, ti): offset = 1 expected_msg = ["start", "processing", "end"] - expected_log_id = _render_log_id(ti, 1, self.elasticsearch_io.json_format) + expected_log_id = _render_log_id(self.elasticsearch_io.log_id_template, ti, 1) assert res["hits"]["total"]["value"] == 3 for msg, hit in zip(expected_msg, res["hits"]["hits"]): assert hit["_index"] == "airflow-logs" @@ -1085,7 +1092,7 @@ def test_read_es_log(self, write_to_es, ti): @patch("elasticsearch.Elasticsearch.count", return_value={"count": 0}) def test_read_with_missing_log(self, mocked_count, ti): log_source_info, log_messages = self.elasticsearch_io.read("", ti) - log_id = _render_log_id(ti, ti.try_number, self.elasticsearch_io.json_format) + log_id = _render_log_id(self.elasticsearch_io.log_id_template, ti, ti.try_number) assert log_source_info == [] assert f"*** Log {log_id} not found in Elasticsearch" in log_messages[0] mocked_count.assert_called_once() From 98af510f77d99c5b519e0d1f0239f190f5d6c0b4 Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Sat, 20 Sep 2025 17:36:47 +0800 Subject: [PATCH 21/46] Remove DB session. Fix compatibility CI --- .../elasticsearch/log/es_task_handler.py | 39 +------------------ .../elasticsearch/log/test_es_task_handler.py | 8 +--- 2 files changed, 3 insertions(+), 44 deletions(-) diff --git a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py index 6e7439b405105..817133f7a6257 100644 --- a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py @@ -41,11 +41,8 @@ from elasticsearch.exceptions import NotFoundError import airflow.logging_config as alc -from airflow import settings from airflow.configuration import conf -from airflow.exceptions import AirflowException from airflow.models.dagrun import DagRun -from airflow.models.taskinstance import TaskInstance from airflow.providers.elasticsearch.log.es_json_formatter import ElasticsearchJSONFormatter from airflow.providers.elasticsearch.log.es_response import ElasticSearchResponse, Hit, resolve_nested from airflow.providers.elasticsearch.version_compat import ( @@ -66,7 +63,7 @@ if TYPE_CHECKING: from datetime import datetime - from airflow.models.taskinstance import TaskInstanceKey + from airflow.models.taskinstance import TaskInstance, TaskInstanceKey from airflow.sdk.types import RuntimeTaskInstanceProtocol as RuntimeTI from airflow.utils.log.file_task_handler import LogMessages, LogMetadata, LogSourceInfo @@ -96,32 +93,6 @@ def get_es_kwargs_from_config() -> dict[str, Any]: return kwargs_dict -def _ensure_ti(ti: TaskInstanceKey | TaskInstance, session) -> TaskInstance: - """ - Given TI | TIKey, return a TI object. - - Will raise exception if no TI is found in the database. - """ - from airflow.models.taskinstance import TaskInstance, TaskInstanceKey - - if not isinstance(ti, TaskInstanceKey): - return ti - val = ( - session.query(TaskInstance) - .filter( - TaskInstance.task_id == ti.task_id, - TaskInstance.dag_id == ti.dag_id, - TaskInstance.run_id == ti.run_id, - TaskInstance.map_index == ti.map_index, - ) - .one_or_none() - ) - if isinstance(val, TaskInstance): - val.try_number = ti.try_number - return val - raise AirflowException(f"Could not find TaskInstance for {ti}") - - def getattr_nested(obj, item, default): """ Get item from obj but return default if not found. @@ -607,14 +578,6 @@ def upload(self, path: os.PathLike | str, ti: RuntimeTI): else: local_loc = self.base_log_folder.joinpath(path) - # Convert the runtimeTI to the real TaskInstance that via fetching from DB - ti = TaskInstance.get_task_instance( - ti.dag_id, - ti.run_id, - ti.task_id, - ti.map_index if ti.map_index is not None else -1, - session=settings.Session, - ) # type: ignore[assignment] log_id = _render_log_id(self.log_id_template, ti, ti.try_number) # type: ignore[arg-type] if local_loc.is_file() and self.write_stdout: # Intentionally construct the log_id and offset field diff --git a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py index 3ea523fe217eb..9ba606c96e2e1 100644 --- a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py +++ b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py @@ -624,9 +624,7 @@ def test_read_with_json_format(self, ti): logs = list(logs) assert logs[2].event == self.test_message else: - assert ( - logs[0][0][1] == "[2020-12-24 19:25:00,962] {taskinstance.py:851} INFO - some random stuff - " - ) + assert logs[0][0][1] == self.test_message @pytest.mark.db_test def test_read_with_json_format_with_custom_offset_and_host_fields(self, ti): @@ -660,9 +658,7 @@ def test_read_with_json_format_with_custom_offset_and_host_fields(self, ti): logs = list(logs) assert logs[2].event == self.test_message else: - assert ( - logs[0][0][1] == "[2020-12-24 19:25:00,962] {taskinstance.py:851} INFO - some random stuff - " - ) + assert logs[0][0][1] == self.test_message @pytest.mark.db_test def test_read_with_custom_offset_and_host_fields(self, ti): From d456f2c4fcc6d2a3ab52ca6037c9318d9614d607 Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Sun, 21 Sep 2025 21:33:30 +0800 Subject: [PATCH 22/46] fix getting log_id_template_from config --- .../airflow/providers/elasticsearch/log/es_task_handler.py | 6 +++++- .../tests/unit/elasticsearch/log/test_es_task_handler.py | 4 +++- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py index 817133f7a6257..06b5f63a38b4f 100644 --- a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py @@ -555,7 +555,11 @@ class ElasticsearchRemoteLogIO(LoggingMixin): # noqa: D101 offset_field: str = "offset" write_to_es: bool = False base_log_folder: Path = attrs.field(converter=Path) - log_id_template: str = "{dag_id}-{task_id}-{run_id}-{map_index}-{try_number}" + log_id_template: str = conf.get( + "elasticsearch", + "log_id_template", + fallback="{dag_id}-{task_id}-{run_id}-{map_index}-{try_number}", + ) processors = () diff --git a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py index 9ba606c96e2e1..b6bea057ab15c 100644 --- a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py +++ b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py @@ -77,8 +77,10 @@ class TestElasticsearchTaskHandler: DAG_ID = "dag_for_testing_es_task_handler" TASK_ID = "task_for_testing_es_log_handler" RUN_ID = "run_for_testing_es_log_handler" + MAP_INDEX = -1 + TRY_NUM = 1 LOGICAL_DATE = datetime(2016, 1, 1) - LOG_ID = f"{DAG_ID}-{TASK_ID}-{RUN_ID}--1-1" + LOG_ID = f"{DAG_ID}-{TASK_ID}-{RUN_ID}-{MAP_INDEX}-{TRY_NUM}" JSON_LOG_ID = f"{DAG_ID}-{TASK_ID}-{_clean_date(LOGICAL_DATE)}-1" FILENAME_TEMPLATE = "{try_number}.log" From 1c797d110b3854a6f1c38c1e735031da775d6b03 Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Sun, 5 Oct 2025 16:18:52 +0800 Subject: [PATCH 23/46] fix timeout error when connecting to elasticsearch --- providers/elasticsearch/tests/conftest.py | 26 ++++++++++++++++++++++- 1 file changed, 25 insertions(+), 1 deletion(-) diff --git a/providers/elasticsearch/tests/conftest.py b/providers/elasticsearch/tests/conftest.py index f36dd9ad703b1..9aee5882a6932 100644 --- a/providers/elasticsearch/tests/conftest.py +++ b/providers/elasticsearch/tests/conftest.py @@ -16,14 +16,38 @@ # under the License. from __future__ import annotations +import time + import pytest +from elasticsearch import Elasticsearch from testcontainers.elasticsearch import ElasticSearchContainer @pytest.fixture(scope="session") def es_8_container_url(): with ElasticSearchContainer("docker.elastic.co/elasticsearch/elasticsearch:8.19.0") as es: - yield es.get_url() + url = es.get_url() + client = Elasticsearch( + url, + request_timeout=30, + retry_on_timeout=True, + max_retries=5, + ) + # Give the node time to be ready. In CI this can take a while. + deadline = time.time() + 120 # up to 2 minutes on slow runners + last_err = None + while time.time() < deadline: + try: + # Wait until at least yellow (all primaries assigned) + client.cluster.health(wait_for_status="yellow", timeout="5s") + break + except Exception as e: + last_err = e + time.sleep(1) + else: + raise RuntimeError(f"Elasticsearch did not become ready: {last_err}") + + yield url pytest_plugins = "tests_common.pytest_plugin" From f4a02dd990c366cbb33507924b4628435263564d Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Sun, 5 Oct 2025 21:27:28 +0800 Subject: [PATCH 24/46] Revert "fix timeout error when connecting to elasticsearch" This reverts commit 1c797d110b3854a6f1c38c1e735031da775d6b03. --- providers/elasticsearch/tests/conftest.py | 26 +---------------------- 1 file changed, 1 insertion(+), 25 deletions(-) diff --git a/providers/elasticsearch/tests/conftest.py b/providers/elasticsearch/tests/conftest.py index 9aee5882a6932..f36dd9ad703b1 100644 --- a/providers/elasticsearch/tests/conftest.py +++ b/providers/elasticsearch/tests/conftest.py @@ -16,38 +16,14 @@ # under the License. from __future__ import annotations -import time - import pytest -from elasticsearch import Elasticsearch from testcontainers.elasticsearch import ElasticSearchContainer @pytest.fixture(scope="session") def es_8_container_url(): with ElasticSearchContainer("docker.elastic.co/elasticsearch/elasticsearch:8.19.0") as es: - url = es.get_url() - client = Elasticsearch( - url, - request_timeout=30, - retry_on_timeout=True, - max_retries=5, - ) - # Give the node time to be ready. In CI this can take a while. - deadline = time.time() + 120 # up to 2 minutes on slow runners - last_err = None - while time.time() < deadline: - try: - # Wait until at least yellow (all primaries assigned) - client.cluster.health(wait_for_status="yellow", timeout="5s") - break - except Exception as e: - last_err = e - time.sleep(1) - else: - raise RuntimeError(f"Elasticsearch did not become ready: {last_err}") - - yield url + yield es.get_url() pytest_plugins = "tests_common.pytest_plugin" From f9f359aa0d36650b139b61630d4c1abb62958802 Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Sun, 5 Oct 2025 21:57:19 +0800 Subject: [PATCH 25/46] Set higher memory for testcontainer --- providers/elasticsearch/tests/conftest.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/providers/elasticsearch/tests/conftest.py b/providers/elasticsearch/tests/conftest.py index f36dd9ad703b1..14e90d878ce75 100644 --- a/providers/elasticsearch/tests/conftest.py +++ b/providers/elasticsearch/tests/conftest.py @@ -23,6 +23,8 @@ @pytest.fixture(scope="session") def es_8_container_url(): with ElasticSearchContainer("docker.elastic.co/elasticsearch/elasticsearch:8.19.0") as es: + es.with_env("discovery.type", "single-node") + es.with_env("ES_JAVA_OPTS", "-Xms512m -Xmx512m") yield es.get_url() From a01e7ad2c6e26f42d394400d0fc0f01aa0ec1604 Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Sun, 5 Oct 2025 22:55:05 +0800 Subject: [PATCH 26/46] Do not refresh all index. Extend timeout to 30 seconds --- providers/elasticsearch/tests/conftest.py | 2 -- .../unit/elasticsearch/log/test_es_task_handler.py | 12 ++++++++---- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/providers/elasticsearch/tests/conftest.py b/providers/elasticsearch/tests/conftest.py index 14e90d878ce75..f36dd9ad703b1 100644 --- a/providers/elasticsearch/tests/conftest.py +++ b/providers/elasticsearch/tests/conftest.py @@ -23,8 +23,6 @@ @pytest.fixture(scope="session") def es_8_container_url(): with ElasticSearchContainer("docker.elastic.co/elasticsearch/elasticsearch:8.19.0") as es: - es.with_env("discovery.type", "single-node") - es.with_env("ES_JAVA_OPTS", "-Xms512m -Xmx512m") yield es.get_url() diff --git a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py index 98265b58a12f1..8819873537070 100644 --- a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py +++ b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py @@ -1022,13 +1022,17 @@ def write_to_es(self, tmp_json_file, ti, unique_index): self.elasticsearch_io.target_index = unique_index self.elasticsearch_io.index_pattern = unique_index self.elasticsearch_io.upload(tmp_json_file, ti) - self.elasticsearch_io.client.indices.refresh(index="_all") + self.elasticsearch_io.client.indices.refresh(index=unique_index, request_timeout=60) def test_write_to_es(self, tmp_json_file, ti): self.elasticsearch_io.write_stdout = False self.elasticsearch_io.upload(tmp_json_file, ti) - self.elasticsearch_io.client.indices.refresh(index="_all") - res = self.elasticsearch_io.client.search(index="_all", query={"match_all": {}}) + self.elasticsearch_io.client.indices.refresh( + index=self.elasticsearch_io.target_index, request_timeout=60 + ) + res = self.elasticsearch_io.client.search( + index=self.elasticsearch_io.target_index, query={"match_all": {}} + ) offset = 1 expected_msg = ["start", "processing", "end"] @@ -1040,7 +1044,7 @@ def test_write_to_es(self, tmp_json_file, ti): assert hit["_source"]["offset"] == offset assert hit["_source"]["log_id"] == expected_log_id offset += 1 - self.elasticsearch_io.client.indices.delete(index="airflow-logs") + self.elasticsearch_io.client.indices.delete(index=self.elasticsearch_io.target_index) def test_write_to_stdout(self, tmp_json_file, ti, capsys): self.elasticsearch_io.write_to_es = False From 6dad32c844411fead7d91370cad36666ea3e863c Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Mon, 6 Oct 2025 19:01:58 +0800 Subject: [PATCH 27/46] Prove es to be write-ready before starting the test --- providers/elasticsearch/tests/conftest.py | 28 ++++++++++++++++++++++- 1 file changed, 27 insertions(+), 1 deletion(-) diff --git a/providers/elasticsearch/tests/conftest.py b/providers/elasticsearch/tests/conftest.py index f36dd9ad703b1..11ec42dfe0cd1 100644 --- a/providers/elasticsearch/tests/conftest.py +++ b/providers/elasticsearch/tests/conftest.py @@ -16,14 +16,40 @@ # under the License. from __future__ import annotations +import time + import pytest +from elasticsearch import Elasticsearch from testcontainers.elasticsearch import ElasticSearchContainer @pytest.fixture(scope="session") def es_8_container_url(): with ElasticSearchContainer("docker.elastic.co/elasticsearch/elasticsearch:8.19.0") as es: - yield es.get_url() + url = es.get_url() + + client = Elasticsearch(url, request_timeout=60, retry_on_timeout=True, max_retries=5) + client.cluster.health(wait_for_status="yellow", timeout="60s") + client.indices.create( + index="airflow-logs", + settings={"index": {"number_of_shards": 1, "number_of_replicas": 0}}, + ) + + end = time.time() + 120 + attempt, last = 0, None + while time.time() < end: + try: + client.index(index="airflow-logs", id=f"probe-{attempt}", document={"ok": True, "i": attempt}) + client.indices.refresh(index="airflow-logs", request_timeout=30, ignore_unavailable=True) + break + except Exception as e: + last = e + attempt += 1 + time.sleep(min(0.25 * attempt, 3.0)) + else: + raise RuntimeError(f"Elasticsearch write readiness failed: {last}") + + yield url pytest_plugins = "tests_common.pytest_plugin" From 4ae74d54c328d5f532cd05ad532c0eebe3b8399b Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Mon, 6 Oct 2025 20:28:52 +0800 Subject: [PATCH 28/46] Revert "Prove es to be write-ready before starting the test" This reverts commit 6dad32c844411fead7d91370cad36666ea3e863c. --- providers/elasticsearch/tests/conftest.py | 28 +---------------------- 1 file changed, 1 insertion(+), 27 deletions(-) diff --git a/providers/elasticsearch/tests/conftest.py b/providers/elasticsearch/tests/conftest.py index 11ec42dfe0cd1..f36dd9ad703b1 100644 --- a/providers/elasticsearch/tests/conftest.py +++ b/providers/elasticsearch/tests/conftest.py @@ -16,40 +16,14 @@ # under the License. from __future__ import annotations -import time - import pytest -from elasticsearch import Elasticsearch from testcontainers.elasticsearch import ElasticSearchContainer @pytest.fixture(scope="session") def es_8_container_url(): with ElasticSearchContainer("docker.elastic.co/elasticsearch/elasticsearch:8.19.0") as es: - url = es.get_url() - - client = Elasticsearch(url, request_timeout=60, retry_on_timeout=True, max_retries=5) - client.cluster.health(wait_for_status="yellow", timeout="60s") - client.indices.create( - index="airflow-logs", - settings={"index": {"number_of_shards": 1, "number_of_replicas": 0}}, - ) - - end = time.time() + 120 - attempt, last = 0, None - while time.time() < end: - try: - client.index(index="airflow-logs", id=f"probe-{attempt}", document={"ok": True, "i": attempt}) - client.indices.refresh(index="airflow-logs", request_timeout=30, ignore_unavailable=True) - break - except Exception as e: - last = e - attempt += 1 - time.sleep(min(0.25 * attempt, 3.0)) - else: - raise RuntimeError(f"Elasticsearch write readiness failed: {last}") - - yield url + yield es.get_url() pytest_plugins = "tests_common.pytest_plugin" From e8bd766d7ff5dcdf9eccc78b1a1ee4d62d0ef42d Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Mon, 6 Oct 2025 20:30:40 +0800 Subject: [PATCH 29/46] Use fileLock to prevent contention --- providers/elasticsearch/tests/conftest.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/providers/elasticsearch/tests/conftest.py b/providers/elasticsearch/tests/conftest.py index f36dd9ad703b1..1cc83ef71ab0a 100644 --- a/providers/elasticsearch/tests/conftest.py +++ b/providers/elasticsearch/tests/conftest.py @@ -17,13 +17,17 @@ from __future__ import annotations import pytest +from filelock import FileLock from testcontainers.elasticsearch import ElasticSearchContainer +LOCK_PATH = "/tmp/airflow_es_testcontainers.lock" + @pytest.fixture(scope="session") def es_8_container_url(): - with ElasticSearchContainer("docker.elastic.co/elasticsearch/elasticsearch:8.19.0") as es: - yield es.get_url() + with FileLock(LOCK_PATH, timeout=1200): + with ElasticSearchContainer("docker.elastic.co/elasticsearch/elasticsearch:8.19.0") as es: + yield es.get_url() pytest_plugins = "tests_common.pytest_plugin" From 6cc9cf8dfcc66f3195a69112c8c3233a795887d5 Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Mon, 6 Oct 2025 21:37:52 +0800 Subject: [PATCH 30/46] Increase client timeout to 60s with 5 retry --- providers/elasticsearch/tests/conftest.py | 8 ++------ .../tests/unit/elasticsearch/log/test_es_task_handler.py | 6 ++++++ 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/providers/elasticsearch/tests/conftest.py b/providers/elasticsearch/tests/conftest.py index 1cc83ef71ab0a..f36dd9ad703b1 100644 --- a/providers/elasticsearch/tests/conftest.py +++ b/providers/elasticsearch/tests/conftest.py @@ -17,17 +17,13 @@ from __future__ import annotations import pytest -from filelock import FileLock from testcontainers.elasticsearch import ElasticSearchContainer -LOCK_PATH = "/tmp/airflow_es_testcontainers.lock" - @pytest.fixture(scope="session") def es_8_container_url(): - with FileLock(LOCK_PATH, timeout=1200): - with ElasticSearchContainer("docker.elastic.co/elasticsearch/elasticsearch:8.19.0") as es: - yield es.get_url() + with ElasticSearchContainer("docker.elastic.co/elasticsearch/elasticsearch:8.19.0") as es: + yield es.get_url() pytest_plugins = "tests_common.pytest_plugin" diff --git a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py index 8819873537070..c3c669c671e72 100644 --- a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py +++ b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py @@ -1021,10 +1021,16 @@ def unique_index(self): def write_to_es(self, tmp_json_file, ti, unique_index): self.elasticsearch_io.target_index = unique_index self.elasticsearch_io.index_pattern = unique_index + self.elasticsearch_io.client = self.elasticsearch_io.client.options( + request_timeout=60, retry_on_timeout=True, max_retries=5 + ) self.elasticsearch_io.upload(tmp_json_file, ti) self.elasticsearch_io.client.indices.refresh(index=unique_index, request_timeout=60) def test_write_to_es(self, tmp_json_file, ti): + self.elasticsearch_io.client = self.elasticsearch_io.client.options( + request_timeout=60, retry_on_timeout=True, max_retries=5 + ) self.elasticsearch_io.write_stdout = False self.elasticsearch_io.upload(tmp_json_file, ti) self.elasticsearch_io.client.indices.refresh( From 9b2e05cbf2a36c90bc84ee5b4fcd1c354c872d6f Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Thu, 9 Oct 2025 10:26:56 +0800 Subject: [PATCH 31/46] Set TESTCONTAINERS_HOST_OVERRIDE to host.docker.internal --- providers/elasticsearch/tests/conftest.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/providers/elasticsearch/tests/conftest.py b/providers/elasticsearch/tests/conftest.py index f36dd9ad703b1..fff16ab38e9fa 100644 --- a/providers/elasticsearch/tests/conftest.py +++ b/providers/elasticsearch/tests/conftest.py @@ -16,9 +16,13 @@ # under the License. from __future__ import annotations +import os + import pytest from testcontainers.elasticsearch import ElasticSearchContainer +os.environ.setdefault("TESTCONTAINERS_HOST_OVERRIDE", "host.docker.internal") + @pytest.fixture(scope="session") def es_8_container_url(): From d2808a54b845ac2144b69d9537507d3a55c08a1d Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Thu, 9 Oct 2025 14:43:56 +0800 Subject: [PATCH 32/46] Revert "Set TESTCONTAINERS_HOST_OVERRIDE to host.docker.internal" This reverts commit 9b2e05cbf2a36c90bc84ee5b4fcd1c354c872d6f. --- providers/elasticsearch/tests/conftest.py | 4 ---- 1 file changed, 4 deletions(-) diff --git a/providers/elasticsearch/tests/conftest.py b/providers/elasticsearch/tests/conftest.py index fff16ab38e9fa..f36dd9ad703b1 100644 --- a/providers/elasticsearch/tests/conftest.py +++ b/providers/elasticsearch/tests/conftest.py @@ -16,13 +16,9 @@ # under the License. from __future__ import annotations -import os - import pytest from testcontainers.elasticsearch import ElasticSearchContainer -os.environ.setdefault("TESTCONTAINERS_HOST_OVERRIDE", "host.docker.internal") - @pytest.fixture(scope="session") def es_8_container_url(): From 52fcfd4753a8c58b8bc073585862e4b42b4e08dd Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Thu, 9 Oct 2025 16:30:55 +0800 Subject: [PATCH 33/46] Add BulkIndexError to debug timeout --- .../airflow/providers/elasticsearch/log/es_task_handler.py | 4 ++++ providers/elasticsearch/tests/conftest.py | 5 ++--- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py index 06b5f63a38b4f..8b0d9f5fe5bf8 100644 --- a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py @@ -624,6 +624,10 @@ def _write_to_es(self, log_lines: list[dict[str, Any]]) -> bool: try: _ = helpers.bulk(self.client, bulk_actions) return True + except helpers.BulkIndexError as bie: + self.log.exception("Bulk upload failed for %d log(s)", len(bie.errors)) + for error in bie.errors: + self.log.exception(error) except Exception as e: self.log.exception("Unable to insert logs into Elasticsearch. Reason: %s", str(e)) return False diff --git a/providers/elasticsearch/tests/conftest.py b/providers/elasticsearch/tests/conftest.py index f36dd9ad703b1..c17bfb7c8eb56 100644 --- a/providers/elasticsearch/tests/conftest.py +++ b/providers/elasticsearch/tests/conftest.py @@ -19,11 +19,10 @@ import pytest from testcontainers.elasticsearch import ElasticSearchContainer +pytest_plugins = "tests_common.pytest_plugin" + @pytest.fixture(scope="session") def es_8_container_url(): with ElasticSearchContainer("docker.elastic.co/elasticsearch/elasticsearch:8.19.0") as es: yield es.get_url() - - -pytest_plugins = "tests_common.pytest_plugin" From ce4a161681c3eea1cb0fcd308e516d8408982311 Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Thu, 9 Oct 2025 17:46:35 +0800 Subject: [PATCH 34/46] Fix mypy. Add TESTCONTAINER hostname override in docker compose --- .../src/airflow/providers/elasticsearch/log/es_task_handler.py | 1 + scripts/ci/docker-compose/base.yml | 1 + 2 files changed, 2 insertions(+) diff --git a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py index 8b0d9f5fe5bf8..1e8880df2603d 100644 --- a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py @@ -628,6 +628,7 @@ def _write_to_es(self, log_lines: list[dict[str, Any]]) -> bool: self.log.exception("Bulk upload failed for %d log(s)", len(bie.errors)) for error in bie.errors: self.log.exception(error) + return False except Exception as e: self.log.exception("Unable to insert logs into Elasticsearch. Reason: %s", str(e)) return False diff --git a/scripts/ci/docker-compose/base.yml b/scripts/ci/docker-compose/base.yml index abbb96e965495..880011acb8e2b 100644 --- a/scripts/ci/docker-compose/base.yml +++ b/scripts/ci/docker-compose/base.yml @@ -23,6 +23,7 @@ services: - USER=root - ADDITIONAL_PATH=~/.local/bin - KUBECONFIG=/files/.kube/config + - TESTCONTAINERS_HOST_OVERRIDE=host.docker.internal env_file: - _generated_docker_compose.env volumes: From f3ff086e0779c8a59e9106201ebd03b1b25cced8 Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Fri, 10 Oct 2025 14:58:27 +0800 Subject: [PATCH 35/46] revert docker-compose changes. Add mem_reservation for testcontainer. --- providers/elasticsearch/tests/conftest.py | 6 +++++- scripts/ci/docker-compose/base.yml | 1 - 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/providers/elasticsearch/tests/conftest.py b/providers/elasticsearch/tests/conftest.py index c17bfb7c8eb56..85014c84c7795 100644 --- a/providers/elasticsearch/tests/conftest.py +++ b/providers/elasticsearch/tests/conftest.py @@ -24,5 +24,9 @@ @pytest.fixture(scope="session") def es_8_container_url(): - with ElasticSearchContainer("docker.elastic.co/elasticsearch/elasticsearch:8.19.0") as es: + with ElasticSearchContainer( + "docker.elastic.co/elasticsearch/elasticsearch:8.19.0", + mem_reservation="512m", + nano_cpus=2_000_000_000, # 0.5 CPU; reduce to 100_000_000 for ~0.1 CPU and more slowdown + ) as es: yield es.get_url() diff --git a/scripts/ci/docker-compose/base.yml b/scripts/ci/docker-compose/base.yml index 880011acb8e2b..abbb96e965495 100644 --- a/scripts/ci/docker-compose/base.yml +++ b/scripts/ci/docker-compose/base.yml @@ -23,7 +23,6 @@ services: - USER=root - ADDITIONAL_PATH=~/.local/bin - KUBECONFIG=/files/.kube/config - - TESTCONTAINERS_HOST_OVERRIDE=host.docker.internal env_file: - _generated_docker_compose.env volumes: From 059906e0fd938c70e37bfb2c7dba32893c871618 Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Fri, 10 Oct 2025 18:30:34 +0800 Subject: [PATCH 36/46] Increase timeout to 120. Disable unnecessary modules of ES --- providers/elasticsearch/tests/conftest.py | 22 ++++++++++++++----- .../elasticsearch/log/test_es_task_handler.py | 8 +++---- 2 files changed, 21 insertions(+), 9 deletions(-) diff --git a/providers/elasticsearch/tests/conftest.py b/providers/elasticsearch/tests/conftest.py index 85014c84c7795..f699f8aab9fe7 100644 --- a/providers/elasticsearch/tests/conftest.py +++ b/providers/elasticsearch/tests/conftest.py @@ -24,9 +24,21 @@ @pytest.fixture(scope="session") def es_8_container_url(): - with ElasticSearchContainer( - "docker.elastic.co/elasticsearch/elasticsearch:8.19.0", - mem_reservation="512m", - nano_cpus=2_000_000_000, # 0.5 CPU; reduce to 100_000_000 for ~0.1 CPU and more slowdown - ) as es: + # with ElasticSearchContainer( + # "docker.elastic.co/elasticsearch/elasticsearch:8.19.0", + # # mem_reservation="512m", + # mem_limit="512m", + # nano_cpus=2_000_000_000, # 0.5 CPU; reduce to 100_000_000 for ~0.1 CPU and more slowdown + # ) as es: + es = ( + ElasticSearchContainer("docker.elastic.co/elasticsearch/elasticsearch:8.19.0", mem_reservation="512m") + .with_env("discovery.type", "single-node") + .with_env("xpack.security.enabled", "false") + .with_env("xpack.ml.enabled", "false") + .with_env("xpack.watcher.enabled", "false") + .with_env("xpack.monitoring.collection.enabled", "false") + .with_env("ingest.geoip.downloader.enabled", "false") + .with_env("ES_JAVA_OPTS", "-Xms512m -Xmx512m") + ) + with es: yield es.get_url() diff --git a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py index c3c669c671e72..dfd8493ccd70f 100644 --- a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py +++ b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py @@ -1022,19 +1022,19 @@ def write_to_es(self, tmp_json_file, ti, unique_index): self.elasticsearch_io.target_index = unique_index self.elasticsearch_io.index_pattern = unique_index self.elasticsearch_io.client = self.elasticsearch_io.client.options( - request_timeout=60, retry_on_timeout=True, max_retries=5 + request_timeout=120, retry_on_timeout=True, max_retries=5 ) self.elasticsearch_io.upload(tmp_json_file, ti) - self.elasticsearch_io.client.indices.refresh(index=unique_index, request_timeout=60) + self.elasticsearch_io.client.indices.refresh(index=unique_index, request_timeout=120) def test_write_to_es(self, tmp_json_file, ti): self.elasticsearch_io.client = self.elasticsearch_io.client.options( - request_timeout=60, retry_on_timeout=True, max_retries=5 + request_timeout=120, retry_on_timeout=True, max_retries=5 ) self.elasticsearch_io.write_stdout = False self.elasticsearch_io.upload(tmp_json_file, ti) self.elasticsearch_io.client.indices.refresh( - index=self.elasticsearch_io.target_index, request_timeout=60 + index=self.elasticsearch_io.target_index, request_timeout=120 ) res = self.elasticsearch_io.client.search( index=self.elasticsearch_io.target_index, query={"match_all": {}} From 132922d40af863df2c4dba34e2244f61bc54b50f Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Fri, 10 Oct 2025 20:40:24 +0800 Subject: [PATCH 37/46] Extend pytest execution and setup timeout to 300 --- providers/elasticsearch/tests/conftest.py | 18 +----------------- .../elasticsearch/log/test_es_task_handler.py | 4 ++++ 2 files changed, 5 insertions(+), 17 deletions(-) diff --git a/providers/elasticsearch/tests/conftest.py b/providers/elasticsearch/tests/conftest.py index f699f8aab9fe7..c17bfb7c8eb56 100644 --- a/providers/elasticsearch/tests/conftest.py +++ b/providers/elasticsearch/tests/conftest.py @@ -24,21 +24,5 @@ @pytest.fixture(scope="session") def es_8_container_url(): - # with ElasticSearchContainer( - # "docker.elastic.co/elasticsearch/elasticsearch:8.19.0", - # # mem_reservation="512m", - # mem_limit="512m", - # nano_cpus=2_000_000_000, # 0.5 CPU; reduce to 100_000_000 for ~0.1 CPU and more slowdown - # ) as es: - es = ( - ElasticSearchContainer("docker.elastic.co/elasticsearch/elasticsearch:8.19.0", mem_reservation="512m") - .with_env("discovery.type", "single-node") - .with_env("xpack.security.enabled", "false") - .with_env("xpack.ml.enabled", "false") - .with_env("xpack.watcher.enabled", "false") - .with_env("xpack.monitoring.collection.enabled", "false") - .with_env("ingest.geoip.downloader.enabled", "false") - .with_env("ES_JAVA_OPTS", "-Xms512m -Xmx512m") - ) - with es: + with ElasticSearchContainer("docker.elastic.co/elasticsearch/elasticsearch:8.19.0") as es: yield es.get_url() diff --git a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py index dfd8493ccd70f..b2cab4bb93969 100644 --- a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py +++ b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py @@ -1027,6 +1027,8 @@ def write_to_es(self, tmp_json_file, ti, unique_index): self.elasticsearch_io.upload(tmp_json_file, ti) self.elasticsearch_io.client.indices.refresh(index=unique_index, request_timeout=120) + @pytest.mark.setup_timeout(300) + @pytest.mark.execution_timeout(300) def test_write_to_es(self, tmp_json_file, ti): self.elasticsearch_io.client = self.elasticsearch_io.client.options( request_timeout=120, retry_on_timeout=True, max_retries=5 @@ -1082,6 +1084,8 @@ def test_raw_log_should_contain_log_id_and_offset(self, tmp_json_file, ti): assert "log_id" in json_log_line assert "offset" in json_log_line + @pytest.mark.setup_timeout(300) + @pytest.mark.execution_timeout(300) @patch( "airflow.providers.elasticsearch.log.es_task_handler.TASK_LOG_FIELDS", ["message"], From f822f8636306124c7a69607878c493acae1918aa Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Fri, 10 Oct 2025 22:28:56 +0800 Subject: [PATCH 38/46] Combine read write es into one test. Pre-create index and wait for shard to be ready --- providers/elasticsearch/tests/conftest.py | 38 +++++++++++-- .../elasticsearch/log/test_es_task_handler.py | 54 +++++-------------- 2 files changed, 48 insertions(+), 44 deletions(-) diff --git a/providers/elasticsearch/tests/conftest.py b/providers/elasticsearch/tests/conftest.py index c17bfb7c8eb56..e0a874e7e79b6 100644 --- a/providers/elasticsearch/tests/conftest.py +++ b/providers/elasticsearch/tests/conftest.py @@ -17,12 +17,44 @@ from __future__ import annotations import pytest +from elasticsearch import Elasticsearch from testcontainers.elasticsearch import ElasticSearchContainer pytest_plugins = "tests_common.pytest_plugin" +def _wait_for_cluster_ready(es: Elasticsearch, timeout_s: int = 120) -> None: + es.cluster.health(wait_for_status="yellow", timeout=f"{timeout_s}s") + + +def _ensure_index(es: Elasticsearch, index: str, timeout_s: int = 120) -> None: + if not es.indices.exists(index=index): + es.indices.create( + index=index, + settings={ + "index": { + "number_of_shards": 1, + "number_of_replicas": 0, + } + }, + ) + # Wait until the index primary is active + es.cluster.health(index=index, wait_for_status="yellow", timeout=f"{timeout_s}s") + + @pytest.fixture(scope="session") -def es_8_container_url(): - with ElasticSearchContainer("docker.elastic.co/elasticsearch/elasticsearch:8.19.0") as es: - yield es.get_url() +def es_8_container_url() -> str: + es = ElasticSearchContainer("docker.elastic.co/elasticsearch/elasticsearch:8.19.0").with_env( + "discovery.type", "single-node" + ) + with es: + url = es.get_url() + client = Elasticsearch( + url, + request_timeout=120, + retry_on_timeout=True, + max_retries=5, + ) + _wait_for_cluster_ready(client, timeout_s=120) + _ensure_index(client, "airflow-logs", timeout_s=120) + yield url diff --git a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py index b2cab4bb93969..3e6e6f1435511 100644 --- a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py +++ b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py @@ -1017,19 +1017,13 @@ def unique_index(self): """Generate a unique index name for each test.""" return f"airflow-logs-{uuid.uuid4()}" - @pytest.fixture - def write_to_es(self, tmp_json_file, ti, unique_index): - self.elasticsearch_io.target_index = unique_index - self.elasticsearch_io.index_pattern = unique_index - self.elasticsearch_io.client = self.elasticsearch_io.client.options( - request_timeout=120, retry_on_timeout=True, max_retries=5 - ) - self.elasticsearch_io.upload(tmp_json_file, ti) - self.elasticsearch_io.client.indices.refresh(index=unique_index, request_timeout=120) - @pytest.mark.setup_timeout(300) @pytest.mark.execution_timeout(300) - def test_write_to_es(self, tmp_json_file, ti): + @patch( + "airflow.providers.elasticsearch.log.es_task_handler.TASK_LOG_FIELDS", + ["message"], + ) + def test_read_write_to_es(self, tmp_json_file, ti): self.elasticsearch_io.client = self.elasticsearch_io.client.options( request_timeout=120, retry_on_timeout=True, max_retries=5 ) @@ -1038,21 +1032,16 @@ def test_write_to_es(self, tmp_json_file, ti): self.elasticsearch_io.client.indices.refresh( index=self.elasticsearch_io.target_index, request_timeout=120 ) - res = self.elasticsearch_io.client.search( - index=self.elasticsearch_io.target_index, query={"match_all": {}} - ) + log_source_info, log_messages = self.elasticsearch_io.read("", ti) + assert log_source_info[0] == self.elasticsearch_8_url + assert len(log_messages) == 3 - offset = 1 expected_msg = ["start", "processing", "end"] - expected_log_id = _render_log_id(self.elasticsearch_io.log_id_template, ti, 1) - assert res["hits"]["total"]["value"] == 3 - for msg, hit in zip(expected_msg, res["hits"]["hits"]): - assert hit["_index"] == "airflow-logs" - assert hit["_source"]["message"] == msg - assert hit["_source"]["offset"] == offset - assert hit["_source"]["log_id"] == expected_log_id - offset += 1 - self.elasticsearch_io.client.indices.delete(index=self.elasticsearch_io.target_index) + for msg, log_message in zip(expected_msg, log_messages): + print(f"msg: {msg}, log_message: {log_message}") + json_log = json.loads(log_message) + assert "message" in json_log + assert json_log["message"] == msg def test_write_to_stdout(self, tmp_json_file, ti, capsys): self.elasticsearch_io.write_to_es = False @@ -1084,23 +1073,6 @@ def test_raw_log_should_contain_log_id_and_offset(self, tmp_json_file, ti): assert "log_id" in json_log_line assert "offset" in json_log_line - @pytest.mark.setup_timeout(300) - @pytest.mark.execution_timeout(300) - @patch( - "airflow.providers.elasticsearch.log.es_task_handler.TASK_LOG_FIELDS", - ["message"], - ) - def test_read_es_log(self, write_to_es, ti): - log_source_info, log_messages = self.elasticsearch_io.read("", ti) - assert log_source_info[0] == self.elasticsearch_8_url - assert len(log_messages) == 3 - - expected_msg = ["start", "processing", "end"] - for msg, log_message in zip(expected_msg, log_messages): - json_log = json.loads(log_message) - assert "message" in json_log - assert json_log["message"] == msg - @patch("elasticsearch.Elasticsearch.count", return_value={"count": 0}) def test_read_with_missing_log(self, mocked_count, ti): log_source_info, log_messages = self.elasticsearch_io.read("", ti) From e4731ed6b39052a7354492ba261bf88c6bcddb9b Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Mon, 13 Oct 2025 19:05:48 +0800 Subject: [PATCH 39/46] Print container stats and indices to debug --- providers/elasticsearch/tests/conftest.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/providers/elasticsearch/tests/conftest.py b/providers/elasticsearch/tests/conftest.py index e0a874e7e79b6..0857f071141c5 100644 --- a/providers/elasticsearch/tests/conftest.py +++ b/providers/elasticsearch/tests/conftest.py @@ -55,6 +55,10 @@ def es_8_container_url() -> str: retry_on_timeout=True, max_retries=5, ) + stats = client.cluster.stats() + indices = stats.get("indices", {}) + print(f"stats: {stats})") + print(f"indices: {indices})") _wait_for_cluster_ready(client, timeout_s=120) _ensure_index(client, "airflow-logs", timeout_s=120) yield url From aec737430487a1988df1111894bb093c22cbc5ac Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Mon, 13 Oct 2025 21:06:02 +0800 Subject: [PATCH 40/46] Disable disk-based allocation of shard --- providers/elasticsearch/tests/conftest.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/providers/elasticsearch/tests/conftest.py b/providers/elasticsearch/tests/conftest.py index 0857f071141c5..adb3f7b0c0979 100644 --- a/providers/elasticsearch/tests/conftest.py +++ b/providers/elasticsearch/tests/conftest.py @@ -44,8 +44,10 @@ def _ensure_index(es: Elasticsearch, index: str, timeout_s: int = 120) -> None: @pytest.fixture(scope="session") def es_8_container_url() -> str: - es = ElasticSearchContainer("docker.elastic.co/elasticsearch/elasticsearch:8.19.0").with_env( - "discovery.type", "single-node" + es = ( + ElasticSearchContainer("docker.elastic.co/elasticsearch/elasticsearch:8.19.0") + .with_env("discovery.type", "single-node") + .with_env("cluster.routing.allocation.disk.threshold_enabled", "false") ) with es: url = es.get_url() From 1ad3376e62cf6305a79a9931e143e48f6ebdad91 Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Mon, 13 Oct 2025 22:34:04 +0800 Subject: [PATCH 41/46] Remove debug lines --- providers/elasticsearch/tests/conftest.py | 4 ---- 1 file changed, 4 deletions(-) diff --git a/providers/elasticsearch/tests/conftest.py b/providers/elasticsearch/tests/conftest.py index adb3f7b0c0979..92b08d2ec0e0d 100644 --- a/providers/elasticsearch/tests/conftest.py +++ b/providers/elasticsearch/tests/conftest.py @@ -57,10 +57,6 @@ def es_8_container_url() -> str: retry_on_timeout=True, max_retries=5, ) - stats = client.cluster.stats() - indices = stats.get("indices", {}) - print(f"stats: {stats})") - print(f"indices: {indices})") _wait_for_cluster_ready(client, timeout_s=120) _ensure_index(client, "airflow-logs", timeout_s=120) yield url From b5d7e01bce7e61b32fe8b5249bba0c1478025564 Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Thu, 16 Oct 2025 23:20:22 +0800 Subject: [PATCH 42/46] Fix infinite read --- .../src/airflow/providers/elasticsearch/log/es_task_handler.py | 1 + 1 file changed, 1 insertion(+) diff --git a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py index 1e8880df2603d..867e53f075a39 100644 --- a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py @@ -361,6 +361,7 @@ def _read( ] else: message = [] + metadata["end_of_log"] = True return message, metadata def _format_msg(self, hit: Hit): From 765e7bcc2ae122bf42e538a587600d46d6e4b315 Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Fri, 17 Oct 2025 10:33:37 +0800 Subject: [PATCH 43/46] Fix failing CI --- .../unit/elasticsearch/log/test_es_task_handler.py | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py index 3e6e6f1435511..e48fe1379132b 100644 --- a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py +++ b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py @@ -293,7 +293,7 @@ def test_read_with_patterns_no_match(self, ti): metadata = metadatas[0] assert metadata["offset"] == "0" - assert not metadata["end_of_log"] + assert metadata["end_of_log"] # last_log_timestamp won't change if no log lines read. assert timezone.parse(metadata["last_log_timestamp"]) == ts @@ -335,7 +335,7 @@ def test_read_missing_logs(self, seconds, create_task_instance): else: # we've "waited" less than 5 seconds so it should not be "end of log" and should be no log message assert logs == [] - assert metadatas["end_of_log"] is False + assert metadatas["end_of_log"] is True assert metadatas["offset"] == "0" assert timezone.parse(metadatas["last_log_timestamp"]) == ts else: @@ -447,7 +447,7 @@ def test_read_nonexistent_log(self, ti): metadata = metadatas[0] assert metadata["offset"] == "0" - assert not metadata["end_of_log"] + assert metadata["end_of_log"] # last_log_timestamp won't change if no log lines read. assert timezone.parse(metadata["last_log_timestamp"]) == ts @@ -455,6 +455,7 @@ def test_read_nonexistent_log(self, ti): def test_read_with_empty_metadata(self, ti): ts = pendulum.now() logs, metadatas = self.es_task_handler.read(ti, 1, {}) + print(f"metadatas: {metadatas}") if AIRFLOW_V_3_0_PLUS: logs = list(logs) assert logs[0].event == "::group::Log message source details" @@ -470,7 +471,7 @@ def test_read_with_empty_metadata(self, ti): assert self.test_message == logs[0][0][-1] metadata = metadatas[0] - + print(f"metadatas: {metadatas}") assert not metadata["end_of_log"] # offset should be initialized to 0 if not provided. assert metadata["offset"] == "1" @@ -492,7 +493,7 @@ def test_read_with_empty_metadata(self, ti): metadata = metadatas[0] - assert not metadata["end_of_log"] + assert metadata["end_of_log"] # offset should be initialized to 0 if not provided. assert metadata["offset"] == "0" # last_log_timestamp will be initialized using log reading time From 18d39a6d87ae21380177ac2ee401d658ad1ac660 Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Fri, 17 Oct 2025 20:37:22 +0800 Subject: [PATCH 44/46] Fix failing CI --- .../tests/unit/elasticsearch/log/test_es_task_handler.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py index e48fe1379132b..02ac65105e5b3 100644 --- a/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py +++ b/providers/elasticsearch/tests/unit/elasticsearch/log/test_es_task_handler.py @@ -351,7 +351,7 @@ def test_read_missing_logs(self, seconds, create_task_instance): # we've "waited" less than 5 seconds so it should not be "end of log" and should be no log message assert len(logs[0]) == 0 assert logs == [[]] - assert metadatas[0]["end_of_log"] is False + assert metadatas[0]["end_of_log"] is True assert len(logs) == len(metadatas) assert metadatas[0]["offset"] == "0" assert timezone.parse(metadatas[0]["last_log_timestamp"]) == ts From d9b2444282582bae9849bfa049c851f25114d92f Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Sun, 19 Oct 2025 09:44:47 +0800 Subject: [PATCH 45/46] Update default helm value file --- chart/values.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/chart/values.yaml b/chart/values.yaml index 85afe04b2f620..9469eabdbc957 100644 --- a/chart/values.yaml +++ b/chart/values.yaml @@ -2918,7 +2918,7 @@ config: run_duration: 41460 elasticsearch: json_format: 'True' - log_id_template: "{dag_id}_{task_id}_{execution_date}_{try_number}" + log_id_template: "{dag_id}-{task_id}-{run_id}-{map_index}-{try_number}" elasticsearch_configs: max_retries: 3 timeout: 30 From 6a0e3558251707be989bc3d93419c27c3d783b7f Mon Sep 17 00:00:00 2001 From: Owen Leung Date: Wed, 29 Oct 2025 14:23:19 +0800 Subject: [PATCH 46/46] Fix failing ruff --- .../providers/elasticsearch/log/es_task_handler.py | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py index d5cbd9a061707..c1db73d22c8bc 100644 --- a/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/providers/elasticsearch/src/airflow/providers/elasticsearch/log/es_task_handler.py @@ -43,16 +43,9 @@ import airflow.logging_config as alc from airflow.configuration import conf from airflow.models.dagrun import DagRun -from airflow.providers.common.compat.sdk import timezone from airflow.providers.elasticsearch.log.es_json_formatter import ElasticsearchJSONFormatter -from airflow.providers.elasticsearch.log.es_response import ElasticSearchResponse, Hit -from airflow.providers.elasticsearch.version_compat import AIRFLOW_V_3_0_PLUS from airflow.providers.elasticsearch.log.es_response import ElasticSearchResponse, Hit, resolve_nested -from airflow.providers.elasticsearch.version_compat import ( - AIRFLOW_V_3_0_PLUS, - AIRFLOW_V_3_1_PLUS, - EsLogMsgType, -) +from airflow.providers.elasticsearch.version_compat import AIRFLOW_V_3_0_PLUS from airflow.utils import timezone from airflow.utils.log.file_task_handler import FileTaskHandler from airflow.utils.log.logging_mixin import ExternalLoggingMixin, LoggingMixin