You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2022/07/26 23:59:00 UTC

[GitHub] [airflow] cloventt opened a new issue, #25325: "Maximum recursion depth exceeded" when configuring task logs to go to both the task logger and console for a Kubernetes worker

cloventt opened a new issue, #25325:
URL: https://github.com/apache/airflow/issues/25325

   ### Apache Airflow version
   
   2.3.3 (latest released)
   
   ### What happened
   
   I am trying to configure my Kubernetes task workers to log task logs to their STDOUT, so that they can be tailed by the UI while tasks are running. In addition I want log files to be sent to S3, so I have set `logging.remote_logging` to `True` and `logging.remote_base_log_folder` to a location in an S3 bucket. By default, Kubernetes worker tasks don't seem to log to their console, and so logs for running workers cannot be tailed via the Airflow web UI.
   
   To do this, I load this custom logging config:
   
   ```python
   from copy import deepcopy
   # import the default logging configuration
   from airflow.config_templates.airflow_local_settings import DEFAULT_LOGGING_CONFIG
   
   LOGGING_CONFIG = deepcopy(DEFAULT_LOGGING_CONFIG)
   
   # this line adds the "secondary_s3_task_handler" as a handler to airflow.task
   LOGGING_CONFIG['loggers']['airflow.task']['handlers'] = ["console", "task"]
   ```
   
   I expect this to cause the Airflow worker to log to console while it is running so that the web UI can tail the pod stdout to get live logs, and to then also write the log file to S3 when the task has completed. Instead, the worker pod instantly crashes with the following stack trace:
   
   ```
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/cli/commands/task_command.py", line 241, in _run_task_by_local_task_job
       run_job.run()
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/jobs/base_job.py", line 244, in run
       self._execute()
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/jobs/local_task_job.py", line 91, in _execute
       if not self.task_instance.check_and_change_state_before_execution(
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/utils/session.py", line 71, in wrapper
       return func(*args, session=session, **kwargs)
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/models/taskinstance.py", line 1331, in check_and_change_state_before_execution
       if not self.are_dependencies_met(
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/utils/session.py", line 68, in wrapper
       return func(*args, **kwargs)
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/models/taskinstance.py", line 1179, in are_dependencies_met
       verbose_aware_logger("Dependencies all met for %s", self)
     File "/usr/local/lib/python3.9/logging/__init__.py", line 1446, in info
       self._log(INFO, msg, args, **kwargs)
     File "/usr/local/lib/python3.9/logging/__init__.py", line 1589, in _log
       self.handle(record)
     File "/usr/local/lib/python3.9/logging/__init__.py", line 1599, in handle
       self.callHandlers(record)
     File "/usr/local/lib/python3.9/logging/__init__.py", line 1661, in callHandlers
       hdlr.handle(record)
     File "/usr/local/lib/python3.9/logging/__init__.py", line 952, in handle
       self.emit(record)
     File "/usr/local/lib/python3.9/logging/__init__.py", line 1086, in emit
       stream.write(msg + self.terminator)
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/utils/log/logging_mixin.py", line 127, in write
       self.flush()
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/utils/log/logging_mixin.py", line 134, in flush
       self._propagate_log(buf)
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/utils/log/logging_mixin.py", line 115, in _propagate_log
       self.logger.log(self.level, remove_escape_codes(message))
   
    < .... a lot of recursion later ... >
   
     File "/usr/local/lib/python3.9/logging/__init__.py", line 1512, in log
       self._log(level, msg, args, **kwargs)
     File "/usr/local/lib/python3.9/logging/__init__.py", line 1589, in _log
       self.handle(record)
     File "/usr/local/lib/python3.9/logging/__init__.py", line 1599, in handle
       self.callHandlers(record)
     File "/usr/local/lib/python3.9/logging/__init__.py", line 1661, in callHandlers
       hdlr.handle(record)
     File "/usr/local/lib/python3.9/logging/__init__.py", line 952, in handle
       self.emit(record)
     File "/usr/local/lib/python3.9/logging/__init__.py", line 1086, in emit
       stream.write(msg + self.terminator)
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/utils/log/logging_mixin.py", line 127, in write
       self.flush()
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/utils/log/logging_mixin.py", line 134, in flush
       self._propagate_log(buf)
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/utils/log/logging_mixin.py", line 115, in _propagate_log
       self.logger.log(self.level, remove_escape_codes(message))
     File "/usr/local/lib/python3.9/logging/__init__.py", line 1512, in log
       self._log(level, msg, args, **kwargs)
     File "/usr/local/lib/python3.9/logging/__init__.py", line 1589, in _log
       self.handle(record)
     File "/usr/local/lib/python3.9/logging/__init__.py", line 1598, in handle
       if (not self.disabled) and self.filter(record):
     File "/usr/local/lib/python3.9/logging/__init__.py", line 806, in filter
       result = f.filter(record)
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/utils/log/secrets_masker.py", line 170, in filter
       record.__dict__[k] = self.redact(v)
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/utils/log/secrets_masker.py", line 239, in redact
       return self._redact(item, name, depth=0)
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/utils/log/secrets_masker.py", line 226, in _redact
       repr(item),
   RecursionError: maximum recursion depth exceeded while getting the repr of an object
   [2022-07-26 23:37:00,872] {connection.py:424} ERROR - Unable to retrieve connection from secrets backend (MetastoreBackend). Checking subsequent secrets backend.
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/models/connection.py", line 420, in get_connection_from_secrets
       conn = secrets_backend.get_connection(conn_id=conn_id)
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/utils/session.py", line 70, in wrapper
       with create_session() as session:
     File "/usr/local/lib/python3.9/contextlib.py", line 119, in __enter__
       return next(self.gen)
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/utils/session.py", line 29, in create_session
       raise RuntimeError("Session must be set before!")
   RuntimeError: Session must be set before!
   [2022-07-26 23:37:00,872] {s3_task_handler.py:185} ERROR - Could not verify previous log to append
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/providers/amazon/aws/log/s3_task_handler.py", line 181, in s3_write
       if append and self.s3_log_exists(remote_log_location):
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/providers/amazon/aws/log/s3_task_handler.py", line 147, in s3_log_exists
       return self.hook.check_for_key(remote_log_location)
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/providers/amazon/aws/hooks/s3.py", line 59, in wrapper
       connection = self.get_connection(self.aws_conn_id)
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/hooks/base.py", line 67, in get_connection
       conn = Connection.get_connection_from_secrets(conn_id)
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/models/connection.py", line 430, in get_connection_from_secrets
       raise AirflowNotFoundException(f"The conn_id `{conn_id}` isn't defined")
   airflow.exceptions.AirflowNotFoundException: The conn_id `aws_logging` isn't defined
   [2022-07-26 23:37:00,873] {connection.py:424} ERROR - Unable to retrieve connection from secrets backend (MetastoreBackend). Checking subsequent secrets backend.
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/models/connection.py", line 420, in get_connection_from_secrets
       conn = secrets_backend.get_connection(conn_id=conn_id)
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/utils/session.py", line 70, in wrapper
       with create_session() as session:
     File "/usr/local/lib/python3.9/contextlib.py", line 119, in __enter__
       return next(self.gen)
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/utils/session.py", line 29, in create_session
       raise RuntimeError("Session must be set before!")
   RuntimeError: Session must be set before!
   [2022-07-26 23:37:00,873] {s3_task_handler.py:201} WARNING - Failed attempt to write logs to s3://<s3 log location>.log, will retry
   [2022-07-26 23:37:00,873] {connection.py:424} ERROR - Unable to retrieve connection from secrets backend (MetastoreBackend). Checking subsequent secrets backend.
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/models/connection.py", line 420, in get_connection_from_secrets
       conn = secrets_backend.get_connection(conn_id=conn_id)
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/utils/session.py", line 70, in wrapper
       with create_session() as session:
     File "/usr/local/lib/python3.9/contextlib.py", line 119, in __enter__
       return next(self.gen)
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/utils/session.py", line 29, in create_session
       raise RuntimeError("Session must be set before!")
   RuntimeError: Session must be set before!
   [2022-07-26 23:37:00,873] {s3_task_handler.py:203} ERROR - Could not write logs to s3:<s3 location>
   Traceback (most recent call last):
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/providers/amazon/aws/log/s3_task_handler.py", line 192, in s3_write
       self.hook.load_string(
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/providers/amazon/aws/hooks/s3.py", line 59, in wrapper
       connection = self.get_connection(self.aws_conn_id)
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/hooks/base.py", line 67, in get_connection
       conn = Connection.get_connection_from_secrets(conn_id)
     File "/home/airflow/.local/lib/python3.9/site-packages/airflow/models/connection.py", line 430, in get_connection_from_secrets
       raise AirflowNotFoundException(f"The conn_id `{conn_id}` isn't defined")
   airflow.exceptions.AirflowNotFoundException: The conn_id `aws_logging` isn't defined
   
   ```
   
   Is there some other configuration I've missed that should cause Kubernetes worker pods to log to STDOUT as well as to file?
   
   ### What you think should happen instead
   
   The worker pod should log to console and to the file that will be uploaded to S3, without an infinite recursion happening.
   
   ### How to reproduce
   
   Configure Airflow with the KubernetesExecutor and S3 Remote logging, then set the task to log to both `task` and `console` as in the example config above.
   
   ### Operating System
   
   linux
   
   ### Versions of Apache Airflow Providers
   
   _No response_
   
   ### Deployment
   
   Other
   
   ### Deployment details
   
   Deployed on Amazon EKS without Helm
   ```
   Server Version: version.Info{Major:"1", Minor:"21+", GitVersion:"v1.21.12-eks-a64ea69", GitCommit:"d4336843ba36120e9ed1491fddff5f2fec33eb77", GitTreeState:"clean", BuildDate:"2022-05-12T18:29:27Z", GoVersion:"go1.16.15", Compiler:"gc", Platform:"linux/amd64"}
   ```
   
   ### Anything else
   
   _No response_
   
   ### Are you willing to submit PR?
   
   - [X] Yes I am willing to submit a PR!
   
   ### Code of Conduct
   
   - [X] I agree to follow this project's [Code of Conduct](https://github.com/apache/airflow/blob/main/CODE_OF_CONDUCT.md)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] potiuk closed issue #25325: "Maximum recursion depth exceeded" when configuring task logs to go to both the task logger and console for a Kubernetes worker

Posted by GitBox <gi...@apache.org>.
potiuk closed issue #25325: "Maximum recursion depth exceeded" when configuring task logs to go to both the task logger and console for a Kubernetes worker
URL: https://github.com/apache/airflow/issues/25325


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [airflow] boring-cyborg[bot] commented on issue #25325: "Maximum recursion depth exceeded" when configuring task logs to go to both the task logger and console for a Kubernetes worker

Posted by GitBox <gi...@apache.org>.
boring-cyborg[bot] commented on issue #25325:
URL: https://github.com/apache/airflow/issues/25325#issuecomment-1196110356

   Thanks for opening your first issue here! Be sure to follow the issue template!
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org